You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by um...@apache.org on 2018/08/12 10:21:44 UTC

[01/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. [Forced Update!]

Repository: hadoop
Updated Branches:
  refs/heads/HDFS-10285 85405a669 -> 3ac07b720 (forced update)


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/5ce332dc
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/5ce332dc
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/5ce332dc

Branch: refs/heads/HDFS-10285
Commit: 5ce332dc9a072f8850ab71ba16898faf8e866c06
Parents: 6fe6c54
Author: Uma Maheswara Rao G <um...@intel.com>
Authored: Mon May 22 21:39:43 2017 -0700
Committer: Uma Maheswara Rao Gangumalla <um...@apache.org>
Committed: Sun Aug 12 03:05:59 2018 -0700

----------------------------------------------------------------------
 .../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/5ce332dc/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 0df58bf..1dbee96 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(
@@ -193,29 +190,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();
@@ -477,71 +451,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/5ce332dc/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/5ce332dc/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 cd2662d..3c6f837 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
@@ -297,7 +297,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/5ce332dc/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 4e4096b..1a06105 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
@@ -1418,22 +1418,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/5ce332dc/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 f895d86..16ebe18 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
@@ -2262,10 +2262,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();
   }
 
   /**
@@ -7855,6 +7857,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/5ce332dc/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/5ce332dc/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/5ce332dc/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/5ce332dc/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


[44/50] [abbrv] hadoop git commit: HDFS-13076: [SPS]: Cleanup work for HDFS-10285 merge. Contributed by Rakesh R.

Posted by um...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/39ed3a66/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/IntraSPSNameNodeBlockMoveTaskHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/IntraSPSNameNodeBlockMoveTaskHandler.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/IntraSPSNameNodeBlockMoveTaskHandler.java
deleted file mode 100644
index d6e92d2..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/IntraSPSNameNodeBlockMoveTaskHandler.java
+++ /dev/null
@@ -1,63 +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.sps;
-
-import java.io.IOException;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
-import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
-import org.apache.hadoop.hdfs.server.namenode.Namesystem;
-import org.apache.hadoop.hdfs.server.protocol.BlockStorageMovementCommand.BlockMovingInfo;
-
-/**
- * This class handles the internal SPS block movements. This will assign block
- * movement tasks to target datanode descriptors.
- */
-@InterfaceAudience.Private
-public class IntraSPSNameNodeBlockMoveTaskHandler
-    implements BlockMoveTaskHandler {
-
-  private BlockManager blockManager;
-  private Namesystem namesystem;
-
-  public IntraSPSNameNodeBlockMoveTaskHandler(BlockManager blockManager,
-      Namesystem namesytem) {
-    this.blockManager = blockManager;
-    this.namesystem = namesytem;
-  }
-
-  @Override
-  public void submitMoveTask(BlockMovingInfo blkMovingInfo) throws IOException {
-    namesystem.readLock();
-    try {
-      DatanodeDescriptor dn = blockManager.getDatanodeManager()
-          .getDatanode(blkMovingInfo.getTarget().getDatanodeUuid());
-      if (dn == null) {
-        throw new IOException("Failed to schedule block movement task:"
-            + blkMovingInfo + " as target datanode: "
-            + blkMovingInfo.getTarget() + " doesn't exists");
-      }
-      dn.incrementBlocksScheduled(blkMovingInfo.getTargetStorageType());
-      dn.addBlocksToMoveStorage(blkMovingInfo);
-    } finally {
-      namesystem.readUnlock();
-    }
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/39ed3a66/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/IntraSPSNameNodeContext.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/IntraSPSNameNodeContext.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/IntraSPSNameNodeContext.java
deleted file mode 100644
index 2bf4810..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/IntraSPSNameNodeContext.java
+++ /dev/null
@@ -1,189 +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.sps;
-
-import static org.apache.hadoop.hdfs.server.common.HdfsServerConstants.XATTR_SATISFY_STORAGE_POLICY;
-
-import java.io.IOException;
-import java.util.Arrays;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.fs.ParentNotDirectoryException;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.UnresolvedLinkException;
-import org.apache.hadoop.hdfs.DFSUtilClient;
-import org.apache.hadoop.hdfs.protocol.Block;
-import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
-import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
-import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
-import org.apache.hadoop.hdfs.server.namenode.INode;
-import org.apache.hadoop.hdfs.server.namenode.Namesystem;
-import org.apache.hadoop.hdfs.server.protocol.BlockStorageMovementCommand.BlockMovingInfo;
-import org.apache.hadoop.hdfs.server.protocol.DatanodeStorageReport;
-import org.apache.hadoop.hdfs.server.namenode.sps.StoragePolicySatisfier.DatanodeMap;
-import org.apache.hadoop.net.NetworkTopology;
-import org.apache.hadoop.security.AccessControlException;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * This class is the Namenode implementation for analyzing the file blocks which
- * are expecting to change its storages and assigning the block storage
- * movements to satisfy the storage policy.
- */
-@InterfaceAudience.Private
-public class IntraSPSNameNodeContext implements Context {
-  private static final Logger LOG = LoggerFactory
-      .getLogger(IntraSPSNameNodeContext.class);
-
-  private final Namesystem namesystem;
-  private final BlockManager blockManager;
-
-  private SPSService service;
-  private final FileCollector fileCollector;
-  private final BlockMoveTaskHandler blockMoveTaskHandler;
-
-  public IntraSPSNameNodeContext(Namesystem namesystem,
-      BlockManager blockManager, SPSService service) {
-    this.namesystem = namesystem;
-    this.blockManager = blockManager;
-    this.service = service;
-    fileCollector = new IntraSPSNameNodeFileIdCollector(
-        namesystem.getFSDirectory(), service);
-    blockMoveTaskHandler = new IntraSPSNameNodeBlockMoveTaskHandler(
-        blockManager, namesystem);
-  }
-
-  @Override
-  public int getNumLiveDataNodes() {
-    return blockManager.getDatanodeManager().getNumLiveDataNodes();
-  }
-
-  /**
-   * @return object containing information regarding the file.
-   */
-  @Override
-  public HdfsFileStatus getFileInfo(long inodeID) throws IOException {
-    Path filePath = DFSUtilClient.makePathFromFileId(inodeID);
-    return namesystem.getFileInfo(filePath.toString(), true, true);
-  }
-
-  @Override
-  public DatanodeStorageReport[] getLiveDatanodeStorageReport()
-      throws IOException {
-    namesystem.readLock();
-    try {
-      return blockManager.getDatanodeManager()
-          .getDatanodeStorageReport(DatanodeReportType.LIVE);
-    } finally {
-      namesystem.readUnlock();
-    }
-  }
-
-  @Override
-  public boolean isFileExist(long inodeId) {
-    return namesystem.getFSDirectory().getInode(inodeId) != null;
-  }
-
-  @Override
-  public void removeSPSHint(long inodeId) throws IOException {
-    this.namesystem.removeXattr(inodeId, XATTR_SATISFY_STORAGE_POLICY);
-  }
-
-  @Override
-  public boolean isRunning() {
-    return namesystem.isRunning() && service.isRunning();
-  }
-
-  @Override
-  public boolean isInSafeMode() {
-    return namesystem.isInSafeMode();
-  }
-
-  @Override
-  public boolean isMoverRunning() {
-    String moverId = HdfsServerConstants.MOVER_ID_PATH.toString();
-    return namesystem.isFileOpenedForWrite(moverId);
-  }
-
-  @Override
-  public void addDropPreviousSPSWorkAtDNs() {
-    namesystem.readLock();
-    try {
-      blockManager.getDatanodeManager().addDropSPSWorkCommandsToAllDNs();
-    } finally {
-      namesystem.readUnlock();
-    }
-  }
-
-  @Override
-  public BlockStoragePolicy getStoragePolicy(byte policyID) {
-    return blockManager.getStoragePolicy(policyID);
-  }
-
-  @Override
-  public NetworkTopology getNetworkTopology(DatanodeMap datanodeMap) {
-    return blockManager.getDatanodeManager().getNetworkTopology();
-  }
-
-  @Override
-  public long getFileID(String path) throws UnresolvedLinkException,
-      AccessControlException, ParentNotDirectoryException {
-    namesystem.readLock();
-    try {
-      INode inode = namesystem.getFSDirectory().getINode(path);
-      return inode == null ? -1 : inode.getId();
-    } finally {
-      namesystem.readUnlock();
-    }
-  }
-
-  @Override
-  public Long getNextSPSPath() {
-    return blockManager.getSPSManager().getNextPathId();
-  }
-
-  @Override
-  public void removeSPSPathId(long trackId) {
-    blockManager.getSPSManager().removePathId(trackId);
-  }
-
-  @Override
-  public void removeAllSPSPathIds() {
-    blockManager.getSPSManager().removeAllPathIds();
-  }
-
-  @Override
-  public void scanAndCollectFiles(long filePath)
-      throws IOException, InterruptedException {
-    fileCollector.scanAndCollectFiles(filePath);
-  }
-
-  @Override
-  public void submitMoveTask(BlockMovingInfo blkMovingInfo) throws IOException {
-    blockMoveTaskHandler.submitMoveTask(blkMovingInfo);
-  }
-
-  @Override
-  public void notifyMovementTriedBlocks(Block[] moveAttemptFinishedBlks) {
-    LOG.info("Movement attempted blocks: {}",
-        Arrays.asList(moveAttemptFinishedBlks));
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/39ed3a66/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/IntraSPSNameNodeFileIdCollector.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/IntraSPSNameNodeFileIdCollector.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/IntraSPSNameNodeFileIdCollector.java
deleted file mode 100644
index 0473b9d..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/IntraSPSNameNodeFileIdCollector.java
+++ /dev/null
@@ -1,185 +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.sps;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.hdfs.DFSConfigKeys;
-import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
-import org.apache.hadoop.hdfs.server.namenode.FSDirectory;
-import org.apache.hadoop.hdfs.server.namenode.FSTreeTraverser;
-import org.apache.hadoop.hdfs.server.namenode.INode;
-
-/**
- * A specific implementation for scanning the directory with Namenode internal
- * Inode structure and collects the file ids under the given directory ID.
- */
-@InterfaceAudience.Private
-public class IntraSPSNameNodeFileIdCollector extends FSTreeTraverser
-    implements FileCollector {
-  private int maxQueueLimitToScan;
-  private final SPSService service;
-
-  private int remainingCapacity = 0;
-
-  private List<ItemInfo> currentBatch;
-
-  public IntraSPSNameNodeFileIdCollector(FSDirectory dir,
-      SPSService service) {
-    super(dir, service.getConf());
-    this.service = service;
-    this.maxQueueLimitToScan = service.getConf().getInt(
-        DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_QUEUE_LIMIT_KEY,
-        DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_QUEUE_LIMIT_DEFAULT);
-    currentBatch = new ArrayList<>(maxQueueLimitToScan);
-  }
-
-  @Override
-  protected boolean processFileInode(INode inode, TraverseInfo traverseInfo)
-      throws IOException, InterruptedException {
-    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 shouldSubmitCurrentBatch() {
-    return remainingCapacity <= 0;
-  }
-
-  @Override
-  protected void checkINodeReady(long startId) throws IOException {
-    // SPS work won't be scheduled if NN is in standby. So, skipping NN
-    // standby check.
-    return;
-  }
-
-  @Override
-  protected void submitCurrentBatch(Long startId)
-      throws IOException, InterruptedException {
-    // Add current child's to queue
-    service.addAllFilesToProcess(startId,
-        currentBatch, false);
-    currentBatch.clear();
-  }
-
-  @Override
-  protected void throttle() throws InterruptedException {
-    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;
-  }
-
-  @Override
-  protected void checkPauseForTesting() throws InterruptedException {
-    // Nothing to do
-  }
-
-  @Override
-  public void scanAndCollectFiles(final long startINodeId)
-      throws IOException, InterruptedException {
-    FSDirectory fsd = getFSDirectory();
-    INode startInode = fsd.getInode(startINodeId);
-    if (startInode != null) {
-      remainingCapacity = remainingCapacity();
-      if (remainingCapacity == 0) {
-        throttle();
-      }
-      if (startInode.isFile()) {
-        currentBatch
-            .add(new ItemInfo(startInode.getId(), startInode.getId()));
-      } else {
-        readLock();
-        // NOTE: this lock will not be held for full directory scanning. It is
-        // basically a sliced locking. Once it collects a batch size( at max the
-        // size of maxQueueLimitToScan (default 1000)) file ids, then it will
-        // unlock and submits the current batch to SPSService. Once
-        // service.processingQueueSize() shows empty slots, then lock will be
-        // re-acquired and scan will be resumed. This logic was re-used from
-        // EDEK feature.
-        try {
-          traverseDir(startInode.asDirectory(), startINodeId,
-              HdfsFileStatus.EMPTY_NAME, new SPSTraverseInfo(startINodeId));
-        } finally {
-          readUnlock();
-        }
-      }
-      // Mark startInode traverse is done, this is last-batch
-      service.addAllFilesToProcess(startInode.getId(), currentBatch, true);
-      currentBatch.clear();
-    }
-  }
-
-  /**
-   * Returns queue remaining capacity.
-   */
-  public synchronized int remainingCapacity() {
-    int size = service.processingQueueSize();
-    int remainingSize = 0;
-    if (size < maxQueueLimitToScan) {
-      remainingSize = maxQueueLimitToScan - size;
-    }
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("SPS processing Q -> maximum capacity:{}, current size:{},"
-          + " remaining size:{}", maxQueueLimitToScan, size, remainingSize);
-    }
-    return remainingSize;
-  }
-
-  class SPSTraverseInfo extends TraverseInfo {
-    private long startId;
-
-    SPSTraverseInfo(long startId) {
-      this.startId = startId;
-    }
-
-    public long getStartId() {
-      return startId;
-    }
-  }
-
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/39ed3a66/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/SPSService.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/SPSService.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/SPSService.java
index 86634d8..a62dd93 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/SPSService.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/SPSService.java
@@ -102,11 +102,6 @@ public interface SPSService {
   int processingQueueSize();
 
   /**
-   * Clear inodeId present in the processing queue.
-   */
-  void clearQueue(long spsPath);
-
-  /**
    * @return the configuration.
    */
   Configuration getConf();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/39ed3a66/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/StoragePolicySatisfier.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/StoragePolicySatisfier.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/StoragePolicySatisfier.java
index 4af6c8f..7ebd23d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/StoragePolicySatisfier.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/StoragePolicySatisfier.java
@@ -43,14 +43,12 @@ import org.apache.hadoop.hdfs.protocol.DatanodeInfoWithStorage;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.StoragePolicySatisfierMode;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants.StoragePolicySatisfyPathStatus;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.HdfsLocatedFileStatus;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock;
 import org.apache.hadoop.hdfs.server.balancer.Matcher;
-import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.namenode.ErasureCodingPolicyManager;
 import org.apache.hadoop.hdfs.server.protocol.BlockStorageMovementCommand.BlockMovingInfo;
 import org.apache.hadoop.hdfs.util.StripedBlockUtil;
@@ -159,15 +157,6 @@ public class StoragePolicySatisfier implements SPSService, Runnable {
           serviceMode);
       return;
     }
-    if (serviceMode == StoragePolicySatisfierMode.INTERNAL
-        && ctxt.isMoverRunning()) {
-      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 "
           + "start it.", StringUtils.toLowerCase(serviceMode.toString()));
@@ -177,9 +166,6 @@ public class StoragePolicySatisfier implements SPSService, Runnable {
     }
 
     isRunning = true;
-    // 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();
@@ -201,7 +187,6 @@ public class StoragePolicySatisfier implements SPSService, Runnable {
     this.storageMovementsMonitor.stop();
     if (forceStop) {
       storageMovementNeeded.clearQueuesWithNotification();
-      addDropSPSWorkCommandsToAllDNs();
     } else {
       LOG.info("Stopping StoragePolicySatisfier.");
     }
@@ -234,14 +219,6 @@ public class StoragePolicySatisfier implements SPSService, Runnable {
     return isRunning;
   }
 
-  /**
-   * Adding drop commands to all datanodes to stop performing the satisfier
-   * block movements, if any.
-   */
-  private void addDropSPSWorkCommandsToAllDNs() {
-    ctxt.addDropPreviousSPSWorkAtDNs();
-  }
-
   @Override
   public void run() {
     while (isRunning) {
@@ -1101,13 +1078,6 @@ public class StoragePolicySatisfier implements SPSService, Runnable {
   }
 
   /**
-   * Clear queues for given track id.
-   */
-  public void clearQueue(long trackId) {
-    storageMovementNeeded.clearQueue(trackId);
-  }
-
-  /**
    * This class contains information of an attempted blocks and its last
    * attempted or reported time stamp. This is used by
    * {@link BlockStorageMovementAttemptedItems#storageMovementAttemptedItems}.
@@ -1158,20 +1128,6 @@ public class StoragePolicySatisfier implements SPSService, Runnable {
     }
   }
 
-  /**
-   * Returns sps invoked path status. This method is used by internal satisfy
-   * storage policy service.
-   *
-   * @param path
-   *          sps path
-   * @return storage policy satisfy path status
-   * @throws IOException
-   */
-  public StoragePolicySatisfyPathStatus checkStoragePolicySatisfyPathStatus(
-      String path) throws IOException {
-    return storageMovementNeeded.getStatus(ctxt.getFileID(path));
-  }
-
   @Override
   public void addFileToProcess(ItemInfo trackInfo, boolean scanCompleted) {
     storageMovementNeeded.add(trackInfo, scanCompleted);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/39ed3a66/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/StoragePolicySatisfyManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/StoragePolicySatisfyManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/StoragePolicySatisfyManager.java
index 0507d6b..074eab6 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/StoragePolicySatisfyManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/StoragePolicySatisfyManager.java
@@ -18,30 +18,27 @@
 package org.apache.hadoop.hdfs.server.namenode.sps;
 
 import java.io.IOException;
+import java.util.Iterator;
 import java.util.LinkedList;
 import java.util.Queue;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.StoragePolicySatisfierMode;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants.StoragePolicySatisfyPathStatus;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.namenode.Namesystem;
 import org.apache.hadoop.hdfs.server.sps.ExternalStoragePolicySatisfier;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import com.google.common.annotations.VisibleForTesting;
+
 /**
  * This manages satisfy storage policy invoked path ids and expose methods to
- * process these path ids. It maintains sps mode(INTERNAL/EXTERNAL/NONE)
+ * process these path ids. It maintains sps mode(EXTERNAL/NONE)
  * configured by the administrator.
  *
  * <p>
- * If the configured mode is {@link StoragePolicySatisfierMode.INTERNAL}, then
- * it will start internal sps daemon service inside namenode and process sps
- * invoked path ids to satisfy the storage policy.
- *
- * <p>
  * If the configured mode is {@link StoragePolicySatisfierMode.EXTERNAL}, then
  * it won't do anything, just maintains the sps invoked path ids. Administrator
  * requires to start external sps service explicitly, to fetch the sps invoked
@@ -66,10 +63,9 @@ public class StoragePolicySatisfyManager {
   private final Queue<Long> pathsToBeTraveresed;
   private final int outstandingPathsLimit;
   private final Namesystem namesystem;
-  private final BlockManager blkMgr;
 
-  public StoragePolicySatisfyManager(Configuration conf, Namesystem namesystem,
-      BlockManager blkMgr) {
+  public StoragePolicySatisfyManager(Configuration conf,
+      Namesystem namesystem) {
     // StoragePolicySatisfier(SPS) configs
     storagePolicyEnabled = conf.getBoolean(
         DFSConfigKeys.DFS_STORAGE_POLICY_ENABLED_KEY,
@@ -82,21 +78,16 @@ public class StoragePolicySatisfyManager {
         DFSConfigKeys.DFS_SPS_MAX_OUTSTANDING_PATHS_DEFAULT);
     mode = StoragePolicySatisfierMode.fromString(modeVal);
     pathsToBeTraveresed = new LinkedList<Long>();
+    this.namesystem = namesystem;
     // instantiate SPS service by just keeps config reference and not starting
     // any supporting threads.
     spsService = new StoragePolicySatisfier(conf);
-    this.namesystem = namesystem;
-    this.blkMgr = blkMgr;
   }
 
   /**
    * This function will do following logic based on the configured sps mode:
    *
    * <p>
-   * If the configured mode is {@link StoragePolicySatisfierMode.INTERNAL}, then
-   * starts internal daemon service inside namenode.
-   *
-   * <p>
    * If the configured mode is {@link StoragePolicySatisfierMode.EXTERNAL}, then
    * it won't do anything. Administrator requires to start external sps service
    * explicitly.
@@ -113,17 +104,6 @@ public class StoragePolicySatisfyManager {
     }
 
     switch (mode) {
-    case INTERNAL:
-      if (spsService.isRunning()) {
-        LOG.info("Storage policy satisfier is already running"
-            + " as internal daemon service inside namenode.");
-        return;
-      }
-      // starts internal daemon service inside namenode
-      spsService.init(
-          new IntraSPSNameNodeContext(namesystem, blkMgr, spsService));
-      spsService.start(false, mode);
-      break;
     case EXTERNAL:
       LOG.info("Storage policy satisfier is configured as external, "
           + "please start external sps service explicitly to satisfy policy");
@@ -141,10 +121,6 @@ public class StoragePolicySatisfyManager {
    * This function will do following logic based on the configured sps mode:
    *
    * <p>
-   * If the configured mode is {@link StoragePolicySatisfierMode.INTERNAL}, then
-   * stops internal daemon service inside namenode.
-   *
-   * <p>
    * If the configured mode is {@link StoragePolicySatisfierMode.EXTERNAL}, then
    * it won't do anything. Administrator requires to stop external sps service
    * explicitly, if needed.
@@ -162,16 +138,6 @@ public class StoragePolicySatisfyManager {
     }
 
     switch (mode) {
-    case INTERNAL:
-      removeAllPathIds();
-      if (!spsService.isRunning()) {
-        LOG.info("Internal storage policy satisfier daemon service"
-            + " is not running");
-        return;
-      }
-      // stops internal daemon service running inside namenode
-      spsService.stop(false);
-      break;
     case EXTERNAL:
       removeAllPathIds();
       if (LOG.isDebugEnabled()) {
@@ -194,11 +160,8 @@ public class StoragePolicySatisfyManager {
   }
 
   /**
-   * Sets new sps mode. If the new mode is internal, then it will start internal
-   * sps service inside namenode. If the new mode is external, then stops
-   * internal sps service running(if any) inside namenode. If the new mode is
-   * none, then it will disable the sps feature completely by clearing all
-   * queued up sps path's hint.
+   * Sets new sps mode. If the new mode is none, then it will disable the sps
+   * feature completely by clearing all queued up sps path's hint.
    */
   public void changeModeEvent(StoragePolicySatisfierMode newMode) {
     if (!storagePolicyEnabled) {
@@ -212,16 +175,6 @@ public class StoragePolicySatisfyManager {
     }
 
     switch (newMode) {
-    case INTERNAL:
-      if (spsService.isRunning()) {
-        LOG.info("Storage policy satisfier is already running as {} mode.",
-            mode);
-        return;
-      }
-      spsService.init(new IntraSPSNameNodeContext(this.namesystem, this.blkMgr,
-          spsService));
-      spsService.start(true, newMode);
-      break;
     case EXTERNAL:
       if (mode == newMode) {
         LOG.info("Storage policy satisfier is already in mode:{},"
@@ -238,7 +191,7 @@ public class StoragePolicySatisfyManager {
       }
       LOG.info("Disabling StoragePolicySatisfier, mode:{}", newMode);
       spsService.stop(true);
-      removeAllPathIds();
+      clearPathIds();
       break;
     default:
       if (LOG.isDebugEnabled()) {
@@ -252,77 +205,15 @@ public class StoragePolicySatisfyManager {
   }
 
   /**
-   * This function will do following logic based on the configured sps mode:
-   *
-   * <p>
-   * If the configured mode is {@link StoragePolicySatisfierMode.INTERNAL}, then
-   * timed wait to stop internal storage policy satisfier daemon threads.
-   *
-   * <p>
-   * If the configured mode is {@link StoragePolicySatisfierMode.EXTERNAL}, then
-   * it won't do anything, just ignore it.
-   *
-   * <p>
-   * If the configured mode is {@link StoragePolicySatisfierMode.NONE}, then the
-   * service is disabled. It won't do any action, just ignore it.
-   */
-  public void stopGracefully() {
-    switch (mode) {
-    case INTERNAL:
-      spsService.stopGracefully();
-      break;
-    case EXTERNAL:
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("Ignoring, StoragePolicySatisfier feature is running"
-            + " outside namenode");
-      }
-      break;
-    case NONE:
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("Ignoring, StoragePolicySatisfier feature is disabled");
-      }
-      break;
-    default:
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("Invalid mode:{}", mode);
-      }
-      break;
-    }
-  }
-
-  /**
    * @return true if the internal storage policy satisfier daemon is running,
    *         false otherwise.
    */
-  public boolean isInternalSatisfierRunning() {
+  @VisibleForTesting
+  public boolean isSatisfierRunning() {
     return spsService.isRunning();
   }
 
   /**
-   * @return internal SPS service instance.
-   */
-  public SPSService getInternalSPSService() {
-    return this.spsService;
-  }
-
-  /**
-   * @return status Storage policy satisfy status of the path. It is supported
-   *         only for the internal sps daemon service.
-   * @throws IOException
-   *           if the Satisfier is not running inside namenode.
-   */
-  public StoragePolicySatisfyPathStatus checkStoragePolicySatisfyPathStatus(
-      String path) throws IOException {
-    if (mode != StoragePolicySatisfierMode.INTERNAL) {
-      LOG.debug("Satisfier is not running inside namenode, so status "
-          + "can't be returned.");
-      throw new IOException("Satisfier is not running inside namenode, "
-          + "so status can't be returned.");
-    }
-    return spsService.checkStoragePolicySatisfyPathStatus(path);
-  }
-
-  /**
    * @return the next SPS path id, on which path users has invoked to satisfy
    *         storages.
    */
@@ -348,10 +239,22 @@ public class StoragePolicySatisfyManager {
 
   /**
    * Removes the SPS path id from the list of sps paths.
+   *
+   * @throws IOException
    */
-  public void removePathId(long trackId) {
+  private void clearPathIds(){
     synchronized (pathsToBeTraveresed) {
-      pathsToBeTraveresed.remove(trackId);
+      Iterator<Long> iterator = pathsToBeTraveresed.iterator();
+      while (iterator.hasNext()) {
+        Long trackId = iterator.next();
+        try {
+          namesystem.removeXattr(trackId,
+              HdfsServerConstants.XATTR_SATISFY_STORAGE_POLICY);
+        } catch (IOException e) {
+          LOG.debug("Failed to remove sps xatttr!", e);
+        }
+        iterator.remove();
+      }
     }
   }
 
@@ -374,12 +277,11 @@ public class StoragePolicySatisfyManager {
   }
 
   /**
-   * @return true if sps is configured as an internal service or external
+   * @return true if sps is configured as an external
    *         service, false otherwise.
    */
   public boolean isEnabled() {
-    return mode == StoragePolicySatisfierMode.INTERNAL
-        || mode == StoragePolicySatisfierMode.EXTERNAL;
+    return mode == StoragePolicySatisfierMode.EXTERNAL;
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/39ed3a66/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/sps/ExternalSPSContext.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/sps/ExternalSPSContext.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/sps/ExternalSPSContext.java
index 189bc2b..3293035 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/sps/ExternalSPSContext.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/sps/ExternalSPSContext.java
@@ -150,11 +150,6 @@ public class ExternalSPSContext implements Context {
   }
 
   @Override
-  public void addDropPreviousSPSWorkAtDNs() {
-    // Nothing todo
-  }
-
-  @Override
   public void removeSPSHint(long inodeId) throws IOException {
     Path filePath = DFSUtilClient.makePathFromFileId(inodeId);
     nnc.getDistributedFileSystem().removeXAttr(filePath,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/39ed3a66/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/sps/ExternalStoragePolicySatisfier.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/sps/ExternalStoragePolicySatisfier.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/sps/ExternalStoragePolicySatisfier.java
index af90f0d..8e19a7c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/sps/ExternalStoragePolicySatisfier.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/sps/ExternalStoragePolicySatisfier.java
@@ -68,15 +68,6 @@ public final class ExternalStoragePolicySatisfier {
       StoragePolicySatisfier sps = new StoragePolicySatisfier(spsConf);
       nnc = getNameNodeConnector(spsConf);
 
-      boolean spsRunning;
-      spsRunning = nnc.getDistributedFileSystem().getClient()
-          .isInternalSatisfierRunning();
-      if (spsRunning) {
-        throw new RuntimeException(
-            "Startup failed due to StoragePolicySatisfier"
-                + " running inside Namenode.");
-      }
-
       ExternalSPSContext context = new ExternalSPSContext(sps, nnc);
       sps.init(context);
       sps.start(true, StoragePolicySatisfierMode.EXTERNAL);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/39ed3a66/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 d8392fa..e02208c 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,7 +26,6 @@ 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;
@@ -34,7 +33,6 @@ 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;
@@ -297,88 +295,6 @@ 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 ...");
-      boolean running = true;
-      while (running) {
-        StoragePolicySatisfyPathStatus status = dfs.getClient()
-            .checkStoragePolicySatisfyPathStatus(path);
-        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;
-        }
-
-        try {
-          Thread.sleep(10000);
-        } catch (InterruptedException e) {
-        }
-      }
-      System.out.println(" done");
-    }
-  }
-
-  /**
-   * Command to check storage policy satisfier status running internal(inside)
-   * Namenode.
-   */
-  private static class IsInternalSatisfierRunningCommand
-      implements AdminHelper.Command {
-    @Override
-    public String getName() {
-      return "-isInternalSatisfierRunning";
-    }
-
-    @Override
-    public String getShortUsage() {
-      return "[" + getName() + "]\n";
-    }
-
-    @Override
-    public String getLongUsage() {
-      return getShortUsage() + "\n"
-          + "Check the status of Storage Policy Statisfier"
-          + " running inside Namenode.\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().isInternalSatisfierRunning()){
-          System.out.println("yes");
-        }else{
-          System.out.println("no");
-        }
       } catch (Exception e) {
         System.err.println(AdminHelper.prettifyException(e));
         return 2;
@@ -438,7 +354,6 @@ public class StoragePolicyAdmin extends Configured implements Tool {
       new SetStoragePolicyCommand(),
       new GetStoragePolicyCommand(),
       new UnsetStoragePolicyCommand(),
-      new SatisfyStoragePolicyCommand(),
-      new IsInternalSatisfierRunningCommand()
+      new SatisfyStoragePolicyCommand()
   };
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/39ed3a66/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 baf7ec7..4a8f9f0 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/DatanodeProtocol.proto
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/DatanodeProtocol.proto
@@ -60,8 +60,6 @@ message DatanodeCommandProto {
     NullDatanodeCommand = 7;
     BlockIdCommand = 8;
     BlockECReconstructionCommand = 9;
-    BlockStorageMovementCommand = 10;
-    DropSPSWorkCommand = 11;
   }
 
   required Type cmdType = 1;    // Type of the command
@@ -76,8 +74,6 @@ message DatanodeCommandProto {
   optional RegisterCommandProto registerCmd = 7;
   optional BlockIdCommandProto blkIdCmd = 8;
   optional BlockECReconstructionCommandProto blkECReconstructionCmd = 9;
-  optional BlockStorageMovementCommandProto blkStorageMovementCmd = 10;
-  optional DropSPSWorkCommandProto dropSPSWorkCmd = 11;
 }
 
 /**
@@ -158,32 +154,6 @@ message BlockECReconstructionCommandProto {
   repeated BlockECReconstructionInfoProto blockECReconstructioninfo = 1;
 }
 
- /**
- * Block storage movement command
- */
-message BlockStorageMovementCommandProto {
-  required string blockPoolId = 1;
-  repeated BlockMovingInfoProto blockMovingInfo = 2;
-}
-
-/**
- * Instruct datanode to drop SPS work queues
- */
-message DropSPSWorkCommandProto {
-  // void
-}
-
-/**
- * Block storage movement information
- */
-message BlockMovingInfoProto {
-  required BlockProto block = 1;
-  required DatanodeInfoProto sourceDnInfo = 2;
-  required DatanodeInfoProto targetDnInfo = 3;
-  required StorageTypeProto sourceStorageType = 4;
-  required StorageTypeProto targetStorageType = 5;
-}
-
 /**
  * registration - Information of the datanode registering with the namenode
  */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/39ed3a66/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 0b533c2..f720d0d 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
@@ -4501,8 +4501,7 @@
   <name>dfs.storage.policy.satisfier.mode</name>
   <value>none</value>
   <description>
-    Following values are supported - internal, external, none.
-    If internal, StoragePolicySatisfier will be enabled and started along with active namenode.
+    Following values are supported - external, none.
     If external, StoragePolicySatisfier will be enabled and started as an independent service outside namenode.
     If none, StoragePolicySatisfier is disabled.
     By default, StoragePolicySatisfier is disabled.
@@ -4561,17 +4560,6 @@
 </property>
 
 <property>
-  <name>dfs.storage.policy.satisfier.low.max-streams.preference</name>
-  <value>true</value>
-  <description>
-    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 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>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/39ed3a66/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 5872ef8..3789779 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/ArchivalStorage.md
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/ArchivalStorage.md
@@ -106,9 +106,9 @@ 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. If there are any failures in movement, the SPS will re-attempt by sending new block movement tasks.
+The SPS tool running external to namenode 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 as internal service to Namenode or as an external service outside Namenode or disabled dynamically without restarting the Namenode.
+SPS can be enabled as an external service outside Namenode or 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)
 
@@ -125,8 +125,8 @@ Detailed design documentation can be found at [Storage Policy Satisfier(SPS) (HD
 
 ####Configurations:
 
-*   **dfs.storage.policy.satisfier.mode** - Used to enable(internal service inside NN or external service outside NN) or disable SPS.
-   Following string values are supported - `internal`, `external`, `none`. Configuring `internal` or `external` value represents SPS is enable and `none` to disable.
+*   **dfs.storage.policy.satisfier.mode** - Used to enable external service outside NN or disable SPS.
+   Following string values are supported - `external`, `none`. Configuring `external` value represents SPS is enable and `none` to disable.
    The default value is `none`.
 
 *   **dfs.storage.policy.satisfier.recheck.timeout.millis** - A timeout to re-check the processed block storage movement
@@ -218,25 +218,17 @@ Schedule blocks to move based on file's/directory's current storage policy.
 
 * Command:
 
-        hdfs storagepolicies -satisfyStoragePolicy [-w] -path <path>
+        hdfs storagepolicies -satisfyStoragePolicy -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/>FAILURE : Few blocks failed to move.<br/>NOT_AVAILABLE - Status not available. |
 
-### SPS Running Status
 
-Check the running status of Storage Policy Satisfier service in namenode. If it is running, return 'yes'. Otherwise return 'no'.
-
-* Command:
-
-        hdfs storagepolicies -isInternalSatisfierRunning
-
-### Enable(internal service inside NN or external service outside NN) or Disable SPS without restarting Namenode
-If administrator wants to switch modes of SPS feature while Namenode is running, first he/she needs to update the desired value(internal or external or none) for the configuration item `dfs.storage.policy.satisfier.mode` in configuration file (`hdfs-site.xml`) and then run the following Namenode reconfig command
+### Enable external service outside NN or Disable SPS without restarting Namenode
+If administrator wants to switch modes of SPS feature while Namenode is running, first he/she needs to update the desired value(external or none) for the configuration item `dfs.storage.policy.satisfier.mode` in configuration file (`hdfs-site.xml`) and then run the following Namenode reconfig command
 
 * Command:
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/39ed3a66/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 bab37e4..fb4616a 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
@@ -59,6 +59,7 @@ import java.security.NoSuchAlgorithmException;
 import java.security.PrivilegedExceptionAction;
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.Collection;
 import java.util.EnumSet;
 import java.util.HashMap;
 import java.util.HashSet;
@@ -139,6 +140,7 @@ import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenSecretManager;
 import org.apache.hadoop.hdfs.security.token.block.ExportedBlockKeys;
+import org.apache.hadoop.hdfs.server.balancer.NameNodeConnector;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManagerTestUtil;
@@ -165,6 +167,7 @@ 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.namenode.sps.StoragePolicySatisfier;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocol;
@@ -193,6 +196,7 @@ import org.apache.hadoop.util.Time;
 import org.apache.hadoop.util.Tool;
 import org.apache.hadoop.util.VersionInfo;
 import org.apache.log4j.Level;
+import org.junit.Assert;
 import org.junit.Assume;
 import org.apache.hadoop.util.ToolRunner;
 
@@ -2491,4 +2495,40 @@ public class DFSTestUtil {
       }
     }, 100, timeout);
   }
+
+  /**
+   * Get namenode connector using the given configuration and file path.
+   *
+   * @param conf
+   *          hdfs configuration
+   * @param filePath
+   *          file path
+   * @param namenodeCount
+   *          number of namenodes
+   * @param createMoverPath
+   *          create move path flag to skip the path creation
+   * @return Namenode connector.
+   * @throws IOException
+   */
+  public static NameNodeConnector getNameNodeConnector(Configuration conf,
+      Path filePath, int namenodeCount, boolean createMoverPath)
+          throws IOException {
+    final Collection<URI> namenodes = DFSUtil.getInternalNsRpcUris(conf);
+    Assert.assertEquals(namenodeCount, namenodes.size());
+    NameNodeConnector.checkOtherInstanceRunning(createMoverPath);
+    while (true) {
+      try {
+        final List<NameNodeConnector> nncs = NameNodeConnector
+            .newNameNodeConnectors(namenodes,
+                StoragePolicySatisfier.class.getSimpleName(),
+                filePath, conf,
+                NameNodeConnector.DEFAULT_MAX_IDLE_ITERATIONS);
+        return nncs.get(0);
+      } catch (IOException e) {
+        LOG.warn("Failed to connect with namenode", e);
+        // Ignore
+      }
+    }
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/39ed3a66/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 d0c3a83..4863ca1 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
@@ -123,8 +123,6 @@ 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, null))
-        .when(mockDn).getStoragePolicySatisfyWorker();
 
     // Set up a simulated dataset with our fake BP
     mockFSDataset = Mockito.spy(new SimulatedFSDataset(null, conf));
@@ -378,8 +376,6 @@ 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, null))
-        .when(mockDn).getStoragePolicySatisfyWorker();
     final AtomicInteger count = new AtomicInteger();
     Mockito.doAnswer(new Answer<Void>() {
       @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/39ed3a66/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
deleted file mode 100644
index 51d3254..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestStoragePolicySatisfyWorker.java
+++ /dev/null
@@ -1,241 +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.datanode;
-
-import java.io.IOException;
-import java.net.InetSocketAddress;
-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.Block;
-import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants.StoragePolicySatisfierMode;
-import org.apache.hadoop.hdfs.protocol.LocatedBlock;
-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;
-
-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 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);
-    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);
-    conf.set(DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_MODE_KEY,
-        StoragePolicySatisfierMode.INTERNAL.toString());
-    // Reduced refresh cycle to update latest datanodes.
-    conf.setLong(DFSConfigKeys.DFS_SPS_DATANODE_CACHE_REFRESH_INTERVAL_MS,
-        1000);
-  }
-
-  @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 {
-    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");
-
-    dfs.satisfyStoragePolicy(new Path(file));
-
-    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());
-
-    SimpleBlocksMovementsStatusHandler handler =
-        new SimpleBlocksMovementsStatusHandler();
-    StoragePolicySatisfyWorker worker = new StoragePolicySatisfyWorker(conf,
-        src, handler);
-    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);
-      worker.processBlockMovingTasks(cluster.getNamesystem().getBlockPoolId(),
-          blockMovingInfos);
-      waitForBlockMovementCompletion(handler, 1, 30000);
-    } finally {
-      worker.stop();
-    }
-  }
-
-  private void waitForBlockMovementCompletion(
-      final SimpleBlocksMovementsStatusHandler handler,
-      int expectedFinishedItemsCount, int timeout) throws Exception {
-    GenericTestUtils.waitFor(new Supplier<Boolean>() {
-      @Override
-      public Boolean get() {
-        List<Block> completedBlocks = handler.getMoveAttemptFinishedBlocks();
-        int finishedCount = completedBlocks.size();
-        LOG.info("Block movement completed count={}, expected={} and actual={}",
-            completedBlocks.size(), expectedFinishedItemsCount, finishedCount);
-        return expectedFinishedItemsCount == finishedCount;
-      }
-    }, 100, timeout);
-  }
-
-  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);
-  }
-
-  private BlockMovingInfo prepareBlockMovingInfo(Block block,
-      DatanodeInfo src, DatanodeInfo destin, StorageType storageType,
-      StorageType targetStorageType) {
-    return new BlockMovingInfo(block, src, destin, storageType,
-        targetStorageType);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/39ed3a66/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 900dcdb..200178d 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
@@ -678,7 +678,7 @@ public class TestMover {
   public void testMoveWhenStoragePolicySatisfierIsRunning() throws Exception {
     final Configuration conf = new HdfsConfiguration();
     conf.set(DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_MODE_KEY,
-        StoragePolicySatisfierMode.INTERNAL.toString());
+        StoragePolicySatisfierMode.EXTERNAL.toString());
     final MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
         .numDataNodes(3)
         .storageTypes(
@@ -686,6 +686,9 @@ public class TestMover {
                 {StorageType.DISK}}).build();
     try {
       cluster.waitActive();
+      // Simulate External sps by creating #getNameNodeConnector instance.
+      DFSTestUtil.getNameNodeConnector(conf, HdfsServerConstants.MOVER_ID_PATH,
+          1, true);
       final DistributedFileSystem dfs = cluster.getFileSystem();
       final String file = "/testMoveWhenStoragePolicySatisfierIsRunning";
       // write to DISK
@@ -697,7 +700,7 @@ public class TestMover {
       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();
+      int exitcode = ExitStatus.IO_EXCEPTION.getExitCode();
       Assert.assertEquals("Exit code should be " + exitcode, exitcode, rc);
     } finally {
       cluster.shutdown();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/39ed3a66/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 ee0b2e6..0a1b129 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
@@ -248,17 +248,17 @@ public class TestNameNodeReconfigure {
 
     // enable SPS internally by keeping DFS_STORAGE_POLICY_ENABLED_KEY
     nameNode.reconfigureProperty(DFS_STORAGE_POLICY_SATISFIER_MODE_KEY,
-        StoragePolicySatisfierMode.INTERNAL.toString());
+        StoragePolicySatisfierMode.EXTERNAL.toString());
 
     // Since DFS_STORAGE_POLICY_ENABLED_KEY is disabled, SPS can't be enabled.
     assertNull("SPS shouldn't start as "
         + DFSConfigKeys.DFS_STORAGE_POLICY_ENABLED_KEY + " is disabled",
             nameNode.getNamesystem().getBlockManager().getSPSManager());
     verifySPSEnabled(nameNode, DFS_STORAGE_POLICY_SATISFIER_MODE_KEY,
-        StoragePolicySatisfierMode.INTERNAL, false);
+        StoragePolicySatisfierMode.EXTERNAL, false);
 
     assertEquals(DFS_STORAGE_POLICY_SATISFIER_MODE_KEY + " has wrong value",
-        StoragePolicySatisfierMode.INTERNAL.toString(), nameNode.getConf()
+        StoragePolicySatisfierMode.EXTERNAL.toString(), nameNode.getConf()
             .get(DFS_STORAGE_POLICY_SATISFIER_MODE_KEY,
             DFS_STORAGE_POLICY_SATISFIER_MODE_DEFAULT));
   }
@@ -285,12 +285,6 @@ public class TestNameNodeReconfigure {
           e.getCause());
     }
 
-    // enable internal SPS
-    nameNode.reconfigureProperty(DFS_STORAGE_POLICY_SATISFIER_MODE_KEY,
-        StoragePolicySatisfierMode.INTERNAL.toString());
-    verifySPSEnabled(nameNode, DFS_STORAGE_POLICY_SATISFIER_MODE_KEY,
-        StoragePolicySatisfierMode.INTERNAL, true);
-
     // disable SPS
     nameNode.reconfigureProperty(DFS_STORAGE_POLICY_SATISFIER_MODE_KEY,
         StoragePolicySatisfierMode.NONE.toString());
@@ -302,7 +296,7 @@ public class TestNameNodeReconfigure {
         StoragePolicySatisfierMode.EXTERNAL.toString());
     assertEquals(DFS_STORAGE_POLICY_SATISFIER_MODE_KEY + " has wrong value",
         false, nameNode.getNamesystem().getBlockManager().getSPSManager()
-            .isInternalSatisfierRunning());
+            .isSatisfierRunning());
     assertEquals(DFS_STORAGE_POLICY_SATISFIER_MODE_KEY + " has wrong value",
         StoragePolicySatisfierMode.EXTERNAL.toString(),
         nameNode.getConf().get(DFS_STORAGE_POLICY_SATISFIER_MODE_KEY,
@@ -337,27 +331,15 @@ public class TestNameNodeReconfigure {
               + " by admin. Seek for an admin help to enable it "
               + "or use Mover tool.", e);
     }
-
-    // start internal
-    nameNode.reconfigureProperty(DFS_STORAGE_POLICY_SATISFIER_MODE_KEY,
-        StoragePolicySatisfierMode.INTERNAL.toString());
-    assertEquals(DFS_STORAGE_POLICY_SATISFIER_MODE_KEY + " has wrong value",
-        true, nameNode.getNamesystem().getBlockManager().getSPSManager()
-            .isInternalSatisfierRunning());
-    assertEquals(DFS_STORAGE_POLICY_SATISFIER_MODE_KEY + " has wrong value",
-        StoragePolicySatisfierMode.INTERNAL.toString(),
-        nameNode.getConf().get(DFS_STORAGE_POLICY_SATISFIER_MODE_KEY,
-            DFS_STORAGE_POLICY_SATISFIER_MODE_DEFAULT));
   }
 
   void verifySPSEnabled(final NameNode nameNode, String property,
       StoragePolicySatisfierMode expected, boolean isSatisfierRunning) {
     StoragePolicySatisfyManager spsMgr = nameNode
             .getNamesystem().getBlockManager().getSPSManager();
-    boolean isInternalSatisfierRunning = spsMgr != null
-        ? spsMgr.isInternalSatisfierRunning() : false;
-    assertEquals(property + " has wrong value", isSatisfierRunning,
-        isInternalSatisfierRunning);
+    boolean isSPSRunning = spsMgr != null ? spsMgr.isSatisfierRunning()
+        : false;
+    assertEquals(property + " has wrong value", isSPSRunning, isSPSRunning);
     String actual = nameNode.getConf().get(property,
         DFS_STORAGE_POLICY_SATISFIER_MODE_DEFAULT);
     assertEquals(property + " has wrong value", expected,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/39ed3a66/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 e079471..2ad8640 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
@@ -29,7 +29,11 @@ import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.MiniDFSCluster.DataNodeProperties;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.StoragePolicySatisfierMode;
 import org.apache.hadoop.hdfs.MiniDFSNNTopology;
+import org.apache.hadoop.hdfs.server.balancer.NameNodeConnector;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.namenode.ha.HATestUtil;
+import org.apache.hadoop.hdfs.server.namenode.sps.StoragePolicySatisfier;
+import org.apache.hadoop.hdfs.server.sps.ExternalSPSContext;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.junit.Test;
 
@@ -45,11 +49,13 @@ import static org.junit.Assert.*;
  * Test persistence of satisfying files/directories.
  */
 public class TestPersistentStoragePolicySatisfier {
-
   private static Configuration conf;
 
   private static MiniDFSCluster cluster;
   private static DistributedFileSystem fs;
+  private NameNodeConnector nnc;
+  private StoragePolicySatisfier sps;
+  private ExternalSPSContext ctxt;
 
   private static Path testFile =
       new Path("/testFile");
@@ -65,7 +71,6 @@ public class TestPersistentStoragePolicySatisfier {
   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.DISK, StorageType.ARCHIVE, StorageType.SSD},
@@ -104,7 +109,7 @@ public class TestPersistentStoragePolicySatisfier {
         DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_RECHECK_TIMEOUT_MILLIS_KEY,
         "3000");
     conf.set(DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_MODE_KEY,
-        StoragePolicySatisfierMode.INTERNAL.toString());
+        StoragePolicySatisfierMode.EXTERNAL.toString());
     // Reduced refresh cycle to update latest datanodes.
     conf.setLong(DFSConfigKeys.DFS_SPS_DATANODE_CACHE_REFRESH_INTERVAL_MS,
         1000);
@@ -124,6 +129,14 @@ public class TestPersistentStoragePolicySatisfier {
     } else {
       fs = cluster.getFileSystem();
     }
+    nnc = DFSTestUtil.getNameNodeConnector(conf,
+        HdfsServerConstants.MOVER_ID_PATH, 1, false);
+
+    sps = new StoragePolicySatisfier(conf);
+    ctxt = new ExternalSPSContext(sps, nnc);
+
+    sps.init(ctxt);
+    sps.start(true, StoragePolicySatisfierMode.EXTERNAL);
 
     createTestFiles(fs, replication);
   }
@@ -158,6 +171,9 @@ public class TestPersistentStoragePolicySatisfier {
       cluster.shutdown(true);
       cluster = null;
     }
+    if (sps != null) {
+      sps.stopGracefully();
+    }
   }
 
   /**
@@ -203,49 +219,6 @@ public class TestPersistentStoragePolicySatisfier {
   }
 
   /**
-   * 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, new HdfsConfiguration());
-
-      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.DISK, 1, timeout, fs);
-      DFSTestUtil.waitExpectedStorageType(
-          childFileName, StorageType.ARCHIVE, 2, 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.
@@ -282,63 +255,6 @@ public class TestPersistentStoragePolicySatisfier {
   }
 
   /**
-   * 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 {
-    MiniDFSCluster haCluster = null;
-    try {
-      conf = new HdfsConfiguration();
-      conf.set(DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_MODE_KEY,
-          StoragePolicySatisfierMode.INTERNAL.toString());
-      // Reduced refresh cycle to update latest datanodes.
-      conf.setLong(DFSConfigKeys.DFS_SPS_DATANODE_CACHE_REFRESH_INTERVAL_MS,
-          1000);
-      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 {
-      if(fs != null) {
-        fs.close();
-        fs = null;
-      }
-      if(haCluster != null) {
-        haCluster.shutdown(true);
-        haCluster = null;
-      }
-    }
-  }
-
-  /**
    * 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
@@ -388,7 +304,7 @@ public class TestPersistentStoragePolicySatisfier {
    * 3. make sure sps xattr is removed.
    * @throws Exception
    */
-  @Test(timeout = 300000)
+  @Test(timeout = 300000000)
   public void testDropSPS() throws Exception {
     try {
       clusterSetUp();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/39ed3a66/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 0cadc83..cf04db0 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,11 +17,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.List;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.ReconfigurationException;
@@ -32,24 +28,15 @@ import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.MiniDFSNNTopology;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.StoragePolicySatisfierMode;
-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.Test;
-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);
 
   private final Configuration config = new HdfsConfiguration();
   private static final int DEFAULT_BLOCK_SIZE = 1024;
@@ -67,7 +54,7 @@ public class TestStoragePolicySatisfierWithHA {
   private void createCluster() throws IOException {
     config.setLong("dfs.block.size", DEFAULT_BLOCK_SIZE);
     config.set(DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_MODE_KEY,
-        StoragePolicySatisfierMode.INTERNAL.toString());
+        StoragePolicySatisfierMode.EXTERNAL.toString());
     // Reduced refresh cycle to update latest datanodes.
     config.setLong(DFSConfigKeys.DFS_SPS_DATANODE_CACHE_REFRESH_INTERVAL_MS,
         1000);
@@ -101,50 +88,19 @@ public class TestStoragePolicySatisfierWithHA {
   public void testWhenNNHAStateChanges() throws IOException {
     try {
       createCluster();
-      boolean running;
-
-      dfs = cluster.getFileSystem(1);
-
-      try {
-        dfs.getClient().isInternalSatisfierRunning();
-        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);
-      dfs = cluster.getFileSystem(0);
-      running = dfs.getClient().isInternalSatisfierRunning();
-      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 {
-        dfs.getClient().isInternalSatisfierRunning();
-        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);
-        }
-      }
-
+      cluster.waitActive();
       try {
         cluster.getNameNode(0).reconfigurePropertyImpl(
             DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_MODE_KEY,
-            StoragePolicySatisfierMode.EXTERNAL.toString());
+            StoragePolicySatisfierMode.NONE.toString());
         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_MODE_KEY
-            + " from 'INTERNAL' to 'EXTERNAL'", e);
+            + " from 'EXTERNAL' to 'NONE'", e);
         GenericTestUtils.assertExceptionContains(
             "Enabling or disabling storage policy satisfier service on "
                 + "standby NameNode is not allowed", e.getCause());
@@ -153,104 +109,4 @@ 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


[45/50] [abbrv] hadoop git commit: HDFS-13076: [SPS]: Cleanup work for HDFS-10285 merge. Contributed by Rakesh R.

Posted by um...@apache.org.
HDFS-13076: [SPS]: Cleanup work for HDFS-10285 merge. 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/39ed3a66
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/39ed3a66
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/39ed3a66

Branch: refs/heads/HDFS-10285
Commit: 39ed3a66dbb01383ed16b141183fc48bfd2e613d
Parents: dfcb331
Author: Uma Maheswara Rao G <um...@intel.com>
Authored: Mon Jul 23 16:05:35 2018 -0700
Committer: Uma Maheswara Rao Gangumalla <um...@apache.org>
Committed: Sun Aug 12 03:06:06 2018 -0700

----------------------------------------------------------------------
 .../java/org/apache/hadoop/hdfs/DFSClient.java  |   26 -
 .../hadoop/hdfs/protocol/ClientProtocol.java    |   29 -
 .../hadoop/hdfs/protocol/HdfsConstants.java     |   40 -
 .../ClientNamenodeProtocolTranslatorPB.java     |   36 -
 .../hadoop/hdfs/protocolPB/PBHelperClient.java  |   37 -
 .../src/main/proto/ClientNamenodeProtocol.proto |   26 -
 .../federation/router/RouterRpcServer.java      |   14 -
 .../org/apache/hadoop/hdfs/DFSConfigKeys.java   |    4 -
 ...tNamenodeProtocolServerSideTranslatorPB.java |   39 -
 .../apache/hadoop/hdfs/protocolPB/PBHelper.java |   99 -
 .../hadoop/hdfs/server/balancer/ExitStatus.java |    3 +-
 .../server/blockmanagement/BlockManager.java    |   21 +-
 .../blockmanagement/DatanodeDescriptor.java     |   68 -
 .../server/blockmanagement/DatanodeManager.java |   94 +-
 .../hdfs/server/datanode/BPOfferService.java    |   12 -
 .../hadoop/hdfs/server/datanode/DataNode.java   |   12 -
 .../datanode/StoragePolicySatisfyWorker.java    |  217 ---
 .../apache/hadoop/hdfs/server/mover/Mover.java  |   21 -
 .../hdfs/server/namenode/FSDirXAttrOp.java      |   12 -
 .../hdfs/server/namenode/FSNamesystem.java      |    8 +-
 .../hdfs/server/namenode/NameNodeRpcServer.java |   36 -
 .../sps/BlockStorageMovementNeeded.java         |  121 +-
 .../hdfs/server/namenode/sps/Context.java       |    5 -
 .../IntraSPSNameNodeBlockMoveTaskHandler.java   |   63 -
 .../namenode/sps/IntraSPSNameNodeContext.java   |  189 --
 .../sps/IntraSPSNameNodeFileIdCollector.java    |  185 --
 .../hdfs/server/namenode/sps/SPSService.java    |    5 -
 .../namenode/sps/StoragePolicySatisfier.java    |   44 -
 .../sps/StoragePolicySatisfyManager.java        |  156 +-
 .../hdfs/server/sps/ExternalSPSContext.java     |    5 -
 .../sps/ExternalStoragePolicySatisfier.java     |    9 -
 .../hadoop/hdfs/tools/StoragePolicyAdmin.java   |   87 +-
 .../src/main/proto/DatanodeProtocol.proto       |   30 -
 .../src/main/resources/hdfs-default.xml         |   14 +-
 .../src/site/markdown/ArchivalStorage.md        |   22 +-
 .../org/apache/hadoop/hdfs/DFSTestUtil.java     |   40 +
 .../server/datanode/TestBPOfferService.java     |    4 -
 .../TestStoragePolicySatisfyWorker.java         |  241 ---
 .../hadoop/hdfs/server/mover/TestMover.java     |    7 +-
 .../namenode/TestNameNodeReconfigure.java       |   32 +-
 .../TestPersistentStoragePolicySatisfier.java   |  124 +-
 .../TestStoragePolicySatisfierWithHA.java       |  152 +-
 .../TestBlockStorageMovementAttemptedItems.java |    3 +-
 .../sps/TestStoragePolicySatisfier.java         | 1825 ------------------
 ...stStoragePolicySatisfierWithStripedFile.java |   87 +-
 .../sps/TestExternalStoragePolicySatisfier.java | 1433 +++++++++++++-
 .../hdfs/tools/TestStoragePolicyCommands.java   |    2 +-
 .../TestStoragePolicySatisfyAdminCommands.java  |   56 +-
 48 files changed, 1517 insertions(+), 4278 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/39ed3a66/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 b6f9bdd..adbb133 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,7 +123,6 @@ 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.HdfsLocatedFileStatus;
@@ -3110,10 +3109,6 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
     }
   }
 
-  public boolean isInternalSatisfierRunning() throws IOException {
-    return namenode.isInternalSatisfierRunning();
-  }
-
   Tracer getTracer() {
     return tracer;
   }
@@ -3170,25 +3165,4 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
     checkOpen();
     return new OpenFilesIterator(namenode, tracer, openFilesTypes, path);
   }
-
-  /**
-   * 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/39ed3a66/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 5c51c22..e8c881b 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
@@ -45,7 +45,6 @@ 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.OpenFilesIterator.OpenFilesType;
-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;
@@ -1757,32 +1756,4 @@ public interface ClientProtocol {
    */
   @AtMostOnce
   void satisfyStoragePolicy(String path) throws IOException;
-
-  /**
-   * Check if internal StoragePolicySatisfier is running.
-   * @return true if internal StoragePolicySatisfier is running
-   * @throws IOException
-   */
-  @Idempotent
-  boolean isInternalSatisfierRunning() 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/39ed3a66/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 ab48dcd..6de186a 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
@@ -134,12 +134,6 @@ public final class HdfsConstants {
   public enum StoragePolicySatisfierMode {
 
     /**
-     * This mode represents that SPS service is running inside Namenode and can
-     * accept any SPS call request.
-     */
-    INTERNAL,
-
-    /**
      * This mode represents that SPS service is running outside Namenode as an
      * external service and can accept any SPS call request.
      */
@@ -166,40 +160,6 @@ public final class HdfsConstants {
     }
   }
 
-
-  /**
-   * 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,
-
-    /**
-     * Few blocks failed to move and the path is still not
-     * fully satisfied the storage policy.
-     */
-    FAILURE,
-
-    /**
-     * Status not available.
-     */
-    NOT_AVAILABLE
-  }
-
   public enum RollingUpgradeAction {
     QUERY, PREPARE, FINALIZE;
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/39ed3a66/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 683ccca..e4bca51 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,7 +70,6 @@ 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.HdfsLocatedFileStatus;
 import org.apache.hadoop.hdfs.protocol.LastBlockWithStatus;
@@ -101,8 +100,6 @@ 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;
@@ -150,8 +147,6 @@ 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.IsInternalSatisfierRunningRequestProto;
-import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.IsInternalSatisfierRunningResponseProto;
 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;
@@ -301,9 +296,6 @@ public class ClientNamenodeProtocolTranslatorPB implements
   private final static GetErasureCodingCodecsRequestProto
       VOID_GET_EC_CODEC_REQUEST = GetErasureCodingCodecsRequestProto
       .newBuilder().build();
-  private final static IsInternalSatisfierRunningRequestProto
-      VOID_IS_SPS_RUNNING_REQUEST = IsInternalSatisfierRunningRequestProto
-      .newBuilder().build();
 
 
   public ClientNamenodeProtocolTranslatorPB(ClientNamenodeProtocolPB proxy) {
@@ -1912,18 +1904,6 @@ public class ClientNamenodeProtocolTranslatorPB implements
   }
 
   @Override
-  public boolean isInternalSatisfierRunning() throws IOException {
-    try {
-      IsInternalSatisfierRunningResponseProto rep =
-          rpcProxy.isInternalSatisfierRunning(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();
@@ -1977,20 +1957,4 @@ 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/39ed3a66/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 7770e31..4a5a493 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
@@ -130,7 +130,6 @@ 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;
@@ -3399,40 +3398,4 @@ public class PBHelperClient {
     }
     return typeProtos;
   }
-
-  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 FAILURE:
-      return StoragePolicySatisfyPathStatus.FAILURE;
-    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 FAILURE:
-      return HdfsConstants.StoragePolicySatisfyPathStatus.FAILURE;
-    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/39ed3a66/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 e8e3a58..49ea3f3 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
@@ -838,28 +838,6 @@ message SatisfyStoragePolicyResponseProto {
 
 }
 
-message IsInternalSatisfierRunningRequestProto {  // no parameters
-}
-
-message IsInternalSatisfierRunningResponseProto {
-  required bool running = 1;
-}
-
-message CheckStoragePolicySatisfyPathStatusRequestProto {  // no parameters
-  required string src = 1;
-}
-
-message CheckStoragePolicySatisfyPathStatusResponseProto {
-  enum StoragePolicySatisfyPathStatus {
-    PENDING = 0;
-    IN_PROGRESS = 1;
-    SUCCESS = 2;
-    FAILURE = 3;
-    NOT_AVAILABLE = 4;
-  }
-  required StoragePolicySatisfyPathStatus status = 1;
-}
-
 service ClientNamenodeProtocol {
   rpc getBlockLocations(GetBlockLocationsRequestProto)
       returns(GetBlockLocationsResponseProto);
@@ -1048,8 +1026,4 @@ service ClientNamenodeProtocol {
       returns(ListOpenFilesResponseProto);
   rpc satisfyStoragePolicy(SatisfyStoragePolicyRequestProto)
       returns(SatisfyStoragePolicyResponseProto);
-  rpc isInternalSatisfierRunning(IsInternalSatisfierRunningRequestProto)
-      returns(IsInternalSatisfierRunningResponseProto);
-  rpc checkStoragePolicySatisfyPathStatus(CheckStoragePolicySatisfyPathStatusRequestProto)
-      returns(CheckStoragePolicySatisfyPathStatusResponseProto);
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/39ed3a66/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcServer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcServer.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcServer.java
index 36645c9..29f32a6 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcServer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcServer.java
@@ -87,7 +87,6 @@ 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.HdfsLocatedFileStatus;
 import org.apache.hadoop.hdfs.protocol.LastBlockWithStatus;
@@ -2498,19 +2497,6 @@ public class RouterRpcServer extends AbstractService
   }
 
   @Override
-  public boolean isInternalSatisfierRunning() throws IOException {
-    checkOperation(OperationCategory.READ, false);
-    return false;
-  }
-
-  @Override
-  public StoragePolicySatisfyPathStatus checkStoragePolicySatisfyPathStatus(
-      String path) throws IOException {
-    checkOperation(OperationCategory.READ, false);
-    return StoragePolicySatisfyPathStatus.NOT_AVAILABLE;
-  }
-
-  @Override
   public Long getNextSPSPath() throws IOException {
     checkOperation(OperationCategory.READ, false);
     // not supported

http://git-wip-us.apache.org/repos/asf/hadoop/blob/39ed3a66/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 cf383d0..5ed35b8 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
@@ -639,10 +639,6 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
       "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_LOW_MAX_STREAMS_PREFERENCE_DEFAULT =
-      true;
   public static final String DFS_SPS_MAX_OUTSTANDING_PATHS_KEY =
       "dfs.storage.policy.satisfier.max.outstanding.paths";
   public static final int DFS_SPS_MAX_OUTSTANDING_PATHS_DEFAULT = 10000;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/39ed3a66/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 b0816cb..e51529e 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,7 +48,6 @@ 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;
@@ -86,8 +85,6 @@ 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;
@@ -162,8 +159,6 @@ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.IsFile
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.IsFileClosedResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.UpgradeStatusRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.UpgradeStatusResponseProto;
-import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.IsInternalSatisfierRunningRequestProto;
-import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.IsInternalSatisfierRunningResponseProto;
 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;
@@ -1865,22 +1860,6 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
   }
 
   @Override
-  public IsInternalSatisfierRunningResponseProto
-      isInternalSatisfierRunning(RpcController controller,
-      IsInternalSatisfierRunningRequestProto req)
-      throws ServiceException {
-    try {
-      boolean ret = server.isInternalSatisfierRunning();
-      IsInternalSatisfierRunningResponseProto.Builder builder =
-          IsInternalSatisfierRunningResponseProto.newBuilder();
-      builder.setRunning(ret);
-      return builder.build();
-    } catch (IOException e) {
-      throw new ServiceException(e);
-    }
-  }
-
-  @Override
   public GetQuotaUsageResponseProto getQuotaUsage(
       RpcController controller, GetQuotaUsageRequestProto req)
       throws ServiceException {
@@ -1925,22 +1904,4 @@ 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/39ed3a66/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 f51f839..ac01348 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
@@ -42,11 +42,9 @@ 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;
-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;
@@ -56,11 +54,9 @@ 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.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.ProvidedStorageLocationProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.StorageUuidsProto;
@@ -102,8 +98,6 @@ 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;
@@ -111,7 +105,6 @@ 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;
@@ -143,10 +136,6 @@ 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 */
@@ -480,10 +469,6 @@ public class PBHelper {
       return PBHelper.convert(proto.getBlkIdCmd());
     case BlockECReconstructionCommand:
       return PBHelper.convert(proto.getBlkECReconstructionCmd());
-    case BlockStorageMovementCommand:
-      return PBHelper.convert(proto.getBlkStorageMovementCmd());
-    case DropSPSWorkCommand:
-      return DROP_SPS_WORK_CMD;
     default:
       return null;
     }
@@ -618,15 +603,6 @@ 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_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);
@@ -1148,79 +1124,4 @@ public class PBHelper {
 
     return new FileRegion(block, providedStorageLocation);
   }
-
-  private static BlockStorageMovementCommandProto convert(
-      BlockStorageMovementCommand blkStorageMovementCmd) {
-    BlockStorageMovementCommandProto.Builder builder =
-        BlockStorageMovementCommandProto.newBuilder();
-
-    builder.setBlockPoolId(blkStorageMovementCmd.getBlockPoolId());
-    Collection<BlockMovingInfo> blockMovingInfos = blkStorageMovementCmd
-        .getBlockMovingTasks();
-    for (BlockMovingInfo blkMovingInfo : blockMovingInfos) {
-      builder.addBlockMovingInfo(convertBlockMovingInfo(blkMovingInfo));
-    }
-    return builder.build();
-  }
-
-  private static BlockMovingInfoProto convertBlockMovingInfo(
-      BlockMovingInfo blkMovingInfo) {
-    BlockMovingInfoProto.Builder builder = BlockMovingInfoProto
-        .newBuilder();
-    builder.setBlock(PBHelperClient.convert(blkMovingInfo.getBlock()));
-
-    DatanodeInfo sourceDnInfo = blkMovingInfo.getSource();
-    builder.setSourceDnInfo(PBHelperClient.convert(sourceDnInfo));
-
-    DatanodeInfo targetDnInfo = blkMovingInfo.getTarget();
-    builder.setTargetDnInfo(PBHelperClient.convert(targetDnInfo));
-
-    StorageType sourceStorageType = blkMovingInfo.getSourceStorageType();
-    builder.setSourceStorageType(
-        PBHelperClient.convertStorageType(sourceStorageType));
-
-    StorageType targetStorageType = blkMovingInfo.getTargetStorageType();
-    builder.setTargetStorageType(
-        PBHelperClient.convertStorageType(targetStorageType));
-
-    return builder.build();
-  }
-
-  private static DatanodeCommand convert(
-      BlockStorageMovementCommandProto blkStorageMovementCmdProto) {
-    Collection<BlockMovingInfo> blockMovingInfos = new ArrayList<>();
-    List<BlockMovingInfoProto> blkSPSatisfyList =
-        blkStorageMovementCmdProto.getBlockMovingInfoList();
-    for (BlockMovingInfoProto blkSPSatisfy : blkSPSatisfyList) {
-      blockMovingInfos.add(convertBlockMovingInfo(blkSPSatisfy));
-    }
-    return new BlockStorageMovementCommand(
-        DatanodeProtocol.DNA_BLOCK_STORAGE_MOVEMENT,
-        blkStorageMovementCmdProto.getBlockPoolId(), blockMovingInfos);
-  }
-
-  private static BlockMovingInfo convertBlockMovingInfo(
-      BlockMovingInfoProto blockStorageMovingInfoProto) {
-    BlockProto blockProto = blockStorageMovingInfoProto.getBlock();
-    Block block = PBHelperClient.convert(blockProto);
-
-    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/39ed3a66/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 5cf4204..6bf2986 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,8 +30,7 @@ public enum ExitStatus {
   IO_EXCEPTION(-4),
   ILLEGAL_ARGUMENTS(-5),
   INTERRUPTED(-6),
-  UNFINALIZED_UPGRADE(-7),
-  SKIPPED_DUE_TO_SPS(-8);
+  UNFINALIZED_UPGRADE(-7);
 
   private final int code;
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/39ed3a66/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 bb63f2a..87bd155 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
@@ -93,7 +93,6 @@ import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.namenode.Namesystem;
 import org.apache.hadoop.hdfs.server.namenode.ha.HAContext;
 import org.apache.hadoop.hdfs.server.namenode.metrics.NameNodeMetrics;
-import org.apache.hadoop.hdfs.server.namenode.sps.SPSService;
 import org.apache.hadoop.hdfs.server.namenode.sps.StoragePolicySatisfyManager;
 import org.apache.hadoop.hdfs.server.protocol.BlockCommand;
 import org.apache.hadoop.hdfs.server.protocol.BlockReportContext;
@@ -719,9 +718,6 @@ public class BlockManager implements BlockStatsMXBean {
     datanodeManager.close();
     pendingReconstruction.stop();
     blocksMap.close();
-    if (getSPSManager() != null) {
-      getSPSManager().stopGracefully();
-    }
   }
 
   /** @return the datanodeManager */
@@ -3889,21 +3885,6 @@ public class BlockManager implements BlockStatsMXBean {
     }
     processAndHandleReportedBlock(storageInfo, block, ReplicaState.FINALIZED,
         delHintNode);
-
-    // notify SPS about the reported block
-    notifyStorageMovementAttemptFinishedBlk(storageInfo, block);
-  }
-
-  private void notifyStorageMovementAttemptFinishedBlk(
-      DatanodeStorageInfo storageInfo, Block block) {
-    if (getSPSManager() != null) {
-      SPSService sps = getSPSManager().getInternalSPSService();
-      if (sps.isRunning()) {
-        sps.notifyStorageMovementAttemptFinishedBlk(
-            storageInfo.getDatanodeDescriptor(), storageInfo.getStorageType(),
-            block);
-      }
-    }
   }
   
   private void processAndHandleReportedBlock(
@@ -5088,7 +5069,7 @@ public class BlockManager implements BlockStatsMXBean {
       LOG.info("Storage policy satisfier is disabled");
       return false;
     }
-    spsManager = new StoragePolicySatisfyManager(conf, namesystem, this);
+    spsManager = new StoragePolicySatisfyManager(conf, namesystem);
     return true;
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/39ed3a66/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 24b948c..9c96f16 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,7 +43,6 @@ 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.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;
@@ -207,14 +206,6 @@ public class DatanodeDescriptor extends DatanodeInfo {
   private final LightWeightHashSet<Block> invalidateBlocks =
       new LightWeightHashSet<>();
 
-  /**
-   * A queue of blocks corresponding to trackID for moving its storage
-   * placements by this datanode.
-   */
-  private final BlockQueue<BlockMovingInfo> storageMovementBlocks =
-      new BlockQueue<>();
-  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
    * in case of errors (e.g. datanode does not report if an error occurs
@@ -369,7 +360,6 @@ public class DatanodeDescriptor extends DatanodeInfo {
     this.pendingCached.clear();
     this.cached.clear();
     this.pendingUncached.clear();
-    this.storageMovementBlocks.clear();
   }
 
   public int numBlocks() {
@@ -1075,62 +1065,4 @@ public class DatanodeDescriptor extends DatanodeInfo {
     }
     return false;
   }
-
-  /**
-   * Add the block infos which needs to move its storage locations.
-   *
-   * @param blkMovingInfo
-   *          - storage mismatched block info
-   */
-  public void addBlocksToMoveStorage(BlockMovingInfo blkMovingInfo) {
-    storageMovementBlocks.offer(blkMovingInfo);
-    BlockManager.LOG
-        .debug("Adding block move task " + blkMovingInfo + " to " + getName()
-            + ", current queue size is " + storageMovementBlocks.size());
-  }
-
-  /**
-   * Return the number of blocks queued up for movement.
-   */
-  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 or null if
-   *         there is no block infos to move.
-   */
-  public BlockMovingInfo[] getBlocksToMoveStorages(int numBlocksToMoveTasks) {
-    List<BlockMovingInfo> blockMovingInfos = storageMovementBlocks
-        .poll(numBlocksToMoveTasks);
-    if (blockMovingInfos == null || blockMovingInfos.size() <= 0) {
-      return null;
-    }
-    BlockMovingInfo[] blkMoveArray = new BlockMovingInfo[blockMovingInfos
-        .size()];
-    return blockMovingInfos.toArray(blkMoveArray);
-  }
-
-  /**
-   * 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/39ed3a66/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 3542864..4173f48 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
@@ -49,7 +49,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;
@@ -210,8 +209,6 @@ public class DatanodeManager {
    */
   private final long timeBetweenResendingCachingDirectivesMs;
 
-  private final boolean blocksToMoveLowPriority;
-
   DatanodeManager(final BlockManager blockManager, final Namesystem namesystem,
       final Configuration conf) throws IOException {
     this.namesystem = namesystem;
@@ -336,12 +333,6 @@ 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
-    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,
@@ -1101,19 +1092,6 @@ 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 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
           if(this.rejectUnresolvedTopologyDN) {
             nodeS.setNetworkLocation(resolveNetworkLocation(nodeS));
@@ -1691,47 +1669,18 @@ 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, erasure-coded block queues and block storage movement
-    // queues.
+    // replication and erasure-coded block queues.
     int totalReplicateBlocks = nodeinfo.getNumberOfReplicateBlocks();
     int totalECBlocks = nodeinfo.getNumberOfBlocksToBeErasureCoded();
-    int totalBlocksToMove = nodeinfo.getNumberOfBlocksToMoveStorages();
     int totalBlocks = totalReplicateBlocks + totalECBlocks;
-    if (totalBlocks > 0 || totalBlocksToMove > 0) {
-      int numReplicationTasks = 0;
-      int numECTasks = 0;
-      int numBlocksToMoveTasks = 0;
-      // 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 (!blocksToMoveLowPriority) {
-        // 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 (totalBlocks > 0) {
+      int numReplicationTasks = (int) Math.ceil(
+          (double) (totalReplicateBlocks * maxTransfers) / totalBlocks);
+      int numECTasks = (int) Math.ceil(
+          (double) (totalECBlocks * maxTransfers) / totalBlocks);
       if (LOG.isDebugEnabled()) {
         LOG.debug("Pending replication tasks: " + numReplicationTasks
-            + " erasure-coded tasks: " + numECTasks + " blocks to move tasks: "
-            + numBlocksToMoveTasks);
+            + " erasure-coded tasks: " + numECTasks);
       }
       // check pending replication tasks
       List<BlockTargetPair> pendingList = nodeinfo.getReplicationCommand(
@@ -1747,23 +1696,6 @@ 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
@@ -2037,18 +1969,6 @@ public class DatanodeManager {
   }
 
   /**
-   * 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);
-      }
-    }
-  }
-
-  /**
    * Generates datanode reports for the given report type.
    *
    * @param type

http://git-wip-us.apache.org/repos/asf/hadoop/blob/39ed3a66/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 1656b16..a25f6a9 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
@@ -795,16 +795,6 @@ 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.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());
     }
@@ -835,8 +825,6 @@ class BPOfferService {
     case DatanodeProtocol.DNA_CACHE:
     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/39ed3a66/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 21af33f..aa044f9 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
@@ -386,7 +386,6 @@ 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()
@@ -1426,9 +1425,6 @@ public class DataNode extends ReconfigurableBase
 
     ecWorker = new ErasureCodingWorker(getConf(), this);
     blockRecoveryWorker = new BlockRecoveryWorker(this);
-    storagePolicySatisfyWorker =
-        new StoragePolicySatisfyWorker(getConf(), this, null);
-    storagePolicySatisfyWorker.start();
 
     blockPoolManager = new BlockPoolManager(this);
     blockPoolManager.refreshNamenodes(getConf());
@@ -1981,10 +1977,6 @@ public class DataNode extends ReconfigurableBase
       }
     }
 
-    // stop storagePolicySatisfyWorker
-    if (storagePolicySatisfyWorker != null) {
-      storagePolicySatisfyWorker.stop();
-    }
     List<BPOfferService> bposArray = (this.blockPoolManager == null)
         ? new ArrayList<BPOfferService>()
         : this.blockPoolManager.getAllNamenodeThreads();
@@ -3624,8 +3616,4 @@ public class DataNode extends ReconfigurableBase
     }
     return this.diskBalancer;
   }
-
-  StoragePolicySatisfyWorker getStoragePolicySatisfyWorker() {
-    return storagePolicySatisfyWorker;
-  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/39ed3a66/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
deleted file mode 100644
index 0157205..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/StoragePolicySatisfyWorker.java
+++ /dev/null
@@ -1,217 +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.datanode;
-
-import java.io.IOException;
-import java.util.Collection;
-import java.util.EnumSet;
-import java.util.concurrent.Callable;
-import java.util.concurrent.CompletionService;
-import java.util.concurrent.ExecutorCompletionService;
-import java.util.concurrent.ExecutorService;
-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.ExtendedBlock;
-import org.apache.hadoop.hdfs.protocol.datatransfer.sasl.DataEncryptionKeyFactory;
-import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
-import org.apache.hadoop.hdfs.server.common.sps.BlockDispatcher;
-import org.apache.hadoop.hdfs.server.common.sps.BlockMovementAttemptFinished;
-import org.apache.hadoop.hdfs.server.common.sps.BlockMovementStatus;
-import org.apache.hadoop.hdfs.server.common.sps.BlockStorageMovementTracker;
-import org.apache.hadoop.hdfs.server.common.sps.BlocksMovementsStatusHandler;
-import org.apache.hadoop.hdfs.server.protocol.BlockStorageMovementCommand.BlockMovingInfo;
-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 moverThreads;
-  private final ExecutorService moveExecutor;
-  private final CompletionService<BlockMovementAttemptFinished>
-      moverCompletionService;
-  private final BlockStorageMovementTracker movementTracker;
-  private Daemon movementTrackerThread;
-  private final BlockDispatcher blkDispatcher;
-
-  public StoragePolicySatisfyWorker(Configuration conf, DataNode datanode,
-      BlocksMovementsStatusHandler handler) {
-    this.datanode = datanode;
-    // Defaulting to 10. This is to minimize the number of move ops.
-    moverThreads = conf.getInt(DFSConfigKeys.DFS_MOVER_MOVERTHREADS_KEY, 10);
-    moveExecutor = initializeBlockMoverThreadPool(moverThreads);
-    moverCompletionService = new ExecutorCompletionService<>(moveExecutor);
-    movementTracker = new BlockStorageMovementTracker(moverCompletionService,
-        handler);
-    movementTrackerThread = new Daemon(movementTracker);
-    movementTrackerThread.setName("BlockStorageMovementTracker");
-    DNConf dnConf = datanode.getDnConf();
-    int ioFileBufferSize = DFSUtilClient.getIoFileBufferSize(conf);
-    blkDispatcher = new BlockDispatcher(dnConf.getSocketTimeout(),
-        ioFileBufferSize, dnConf.getConnectToDnViaHostname());
-  }
-
-  /**
-   * Start StoragePolicySatisfyWorker, which will start block movement tracker
-   * thread to track the completion of block movements.
-   */
-  void start() {
-    movementTrackerThread.start();
-  }
-
-  /**
-   * Stop StoragePolicySatisfyWorker, which will terminate executor service and
-   * stop block movement tracker thread.
-   */
-  void stop() {
-    movementTracker.stopTracking();
-    movementTrackerThread.interrupt();
-    moveExecutor.shutdown();
-    try {
-      moveExecutor.awaitTermination(500, TimeUnit.MILLISECONDS);
-    } catch (InterruptedException e) {
-      LOG.error("Interrupted while waiting for mover thread to terminate", e);
-    }
-  }
-
-  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;
-          }
-        });
-
-    moverThreadPool.allowCoreThreadTimeOut(true);
-    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.
-   *
-   * @param blockPoolID block pool identifier
-   *
-   * @param blockMovingInfos
-   *          list of blocks to be moved
-   */
-  public void processBlockMovingTasks(final String blockPoolID,
-      final Collection<BlockMovingInfo> blockMovingInfos) {
-    LOG.debug("Received BlockMovingTasks {}", blockMovingInfos);
-    for (BlockMovingInfo blkMovingInfo : blockMovingInfos) {
-      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);
-      moverCompletionService.submit(blockMovingTask);
-    }
-  }
-
-  /**
-   * 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 final String blockPoolID;
-    private final BlockMovingInfo blkMovingInfo;
-
-    BlockMovingTask(String blockPoolID, BlockMovingInfo blkMovInfo) {
-      this.blockPoolID = blockPoolID;
-      this.blkMovingInfo = blkMovInfo;
-    }
-
-    @Override
-    public BlockMovementAttemptFinished call() {
-      BlockMovementStatus status = moveBlock();
-      return new BlockMovementAttemptFinished(blkMovingInfo.getBlock(),
-          blkMovingInfo.getSource(), blkMovingInfo.getTarget(),
-          blkMovingInfo.getTargetStorageType(), status);
-    }
-
-    private BlockMovementStatus moveBlock() {
-      datanode.incrementXmitsInProgress();
-      ExtendedBlock eb = new ExtendedBlock(blockPoolID,
-          blkMovingInfo.getBlock());
-      try {
-        Token<BlockTokenIdentifier> accessToken = datanode.getBlockAccessToken(
-            eb, EnumSet.of(BlockTokenIdentifier.AccessMode.WRITE),
-            new StorageType[]{blkMovingInfo.getTargetStorageType()},
-            new String[0]);
-        DataEncryptionKeyFactory keyFactory = datanode
-            .getDataEncryptionKeyFactoryForBlock(eb);
-
-        return blkDispatcher.moveBlock(blkMovingInfo,
-            datanode.getSaslClient(), eb, datanode.newSocket(),
-            keyFactory, accessToken);
-      } catch (IOException e) {
-        // TODO: handle failure retries
-        LOG.warn(
-            "Failed to move block:{} from src:{} to destin:{} to satisfy "
-                + "storageType:{}",
-            blkMovingInfo.getBlock(), blkMovingInfo.getSource(),
-            blkMovingInfo.getTarget(), blkMovingInfo.getTargetStorageType(), e);
-        return BlockMovementStatus.DN_BLK_STORAGE_MOVEMENT_FAILURE;
-      } finally {
-        datanode.decrementXmitsInProgress();
-      }
-    }
-  }
-
-  /**
-   * 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.");
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/39ed3a66/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 af5ab2d..c7a53e1 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,8 +48,6 @@ 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;
 import org.apache.hadoop.security.SecurityUtil;
@@ -658,25 +656,6 @@ public class Mover {
           final Mover m = new Mover(nnc, conf, retryCount,
               excludedPinnedBlocks);
 
-          boolean spsRunning;
-          try {
-            spsRunning = nnc.getDistributedFileSystem().getClient()
-                .isInternalSatisfierRunning();
-          } 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"
-                + " service running inside namenode. 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/39ed3a66/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 3b68979..1cb414d 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
@@ -32,7 +32,6 @@ import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ReencryptionInfoProto;
 import org.apache.hadoop.hdfs.protocolPB.PBHelperClient;
 import org.apache.hadoop.hdfs.server.namenode.FSDirectory.DirOp;
-import org.apache.hadoop.hdfs.server.namenode.sps.StoragePolicySatisfyManager;
 import org.apache.hadoop.security.AccessControlException;
 
 import java.io.FileNotFoundException;
@@ -207,17 +206,6 @@ 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))) {
-          StoragePolicySatisfyManager spsManager =
-              fsd.getBlockManager().getSPSManager();
-          if (spsManager != null) {
-            spsManager.getInternalSPSService().clearQueue(inode.getId());
-          }
-          break;
-        }
-      }
       XAttrStorage.updateINodeXAttrs(inode, newXAttrs, snapshotId);
       return removedXAttrs;
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/39ed3a66/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 afe9092..7bc9ecc 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
@@ -209,7 +209,6 @@ import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.ReencryptAction;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants.StoragePolicySatisfierMode;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.LastBlockWithStatus;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
@@ -1363,9 +1362,6 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
         // Don't want to keep replication queues when not in Active.
         blockManager.clearQueues();
         blockManager.setInitializedReplQueues(false);
-        if (blockManager.getSPSManager() != null) {
-          blockManager.getSPSManager().stopGracefully();
-        }
       }
     } finally {
       writeUnlock("stopActiveServices");
@@ -2275,9 +2271,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     }
     // checks sps status
     boolean disabled = (blockManager.getSPSManager() == null);
-    if (disabled || (blockManager
-        .getSPSManager().getMode() == StoragePolicySatisfierMode.INTERNAL
-        && !blockManager.getSPSManager().isInternalSatisfierRunning())) {
+    if (disabled) {
       throw new UnsupportedActionException(
           "Cannot request to satisfy storage policy "
               + "when storage policy satisfier feature has been disabled"

http://git-wip-us.apache.org/repos/asf/hadoop/blob/39ed3a66/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 2f3325f..318f801 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
@@ -111,7 +111,6 @@ 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.StoragePolicySatisfierMode;
-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;
@@ -2534,41 +2533,6 @@ public class NameNodeRpcServer implements NamenodeProtocols {
   }
 
   @Override
-  public boolean isInternalSatisfierRunning() throws IOException {
-    checkNNStartup();
-    String operationName = "isInternalSatisfierRunning";
-    namesystem.checkSuperuserPrivilege(operationName);
-    if (nn.isStandbyState()) {
-      throw new StandbyException("Not supported by Standby Namenode.");
-    }
-    StoragePolicySatisfyManager spsMgr =
-        namesystem.getBlockManager().getSPSManager();
-    boolean isInternalSatisfierRunning = (spsMgr != null
-        ? spsMgr.isInternalSatisfierRunning() : false);
-    namesystem.logAuditEvent(true, operationName, null);
-    return isInternalSatisfierRunning;
-  }
-
-  @Override
-  public StoragePolicySatisfyPathStatus checkStoragePolicySatisfyPathStatus(
-      String path) throws IOException {
-    checkNNStartup();
-    if (nn.isStandbyState()) {
-      throw new StandbyException("Not supported by Standby Namenode.");
-    }
-    if (namesystem.getBlockManager().getSPSManager() == null) {
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("Satisfier is not running inside namenode, so status "
-            + "can't be returned.");
-      }
-      throw new IOException("Satisfier is not running inside namenode, "
-          + "so status can't be returned.");
-    }
-    return namesystem.getBlockManager().getSPSManager()
-        .checkStoragePolicySatisfyPathStatus(path);
-  }
-
-  @Override
   public Long getNextSPSPath() throws IOException {
     checkNNStartup();
     String operationName = "getNextSPSPath";

http://git-wip-us.apache.org/repos/asf/hadoop/blob/39ed3a66/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/BlockStorageMovementNeeded.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/BlockStorageMovementNeeded.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/BlockStorageMovementNeeded.java
index c95dcda..b990bc5 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/BlockStorageMovementNeeded.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/BlockStorageMovementNeeded.java
@@ -23,14 +23,10 @@ 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.util.Daemon;
-import org.apache.hadoop.util.Time;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -60,9 +56,6 @@ public class BlockStorageMovementNeeded {
   private final Map<Long, DirPendingWorkInfo> pendingWorkForDirectory =
       new HashMap<>();
 
-  private final Map<Long, StoragePolicySatisfyPathStatusInfo> spsStatus =
-      new ConcurrentHashMap<>();
-
   private final Context ctxt;
 
   private Daemon pathIdCollector;
@@ -86,9 +79,6 @@ public class BlockStorageMovementNeeded {
    *          - track info for satisfy the policy
    */
   public synchronized void add(ItemInfo trackInfo) {
-    spsStatus.put(trackInfo.getFile(),
-        new StoragePolicySatisfyPathStatusInfo(
-            StoragePolicySatisfyPathStatus.IN_PROGRESS));
     storageMovementNeeded.add(trackInfo);
   }
 
@@ -129,7 +119,7 @@ public class BlockStorageMovementNeeded {
     if (itemInfo.getStartPath() == itemInfo.getFile()) {
       return;
     }
-    updatePendingDirScanStats(itemInfo.getFile(), 1, scanCompleted);
+    updatePendingDirScanStats(itemInfo.getStartPath(), 1, scanCompleted);
   }
 
   private void updatePendingDirScanStats(long startPath, int numScannedFiles,
@@ -181,7 +171,6 @@ public class BlockStorageMovementNeeded {
       if (!ctxt.isFileExist(startId)) {
         // directory deleted just remove it.
         this.pendingWorkForDirectory.remove(startId);
-        updateStatus(startId, isSuccess);
       } else {
         DirPendingWorkInfo pendingWork = pendingWorkForDirectory.get(startId);
         if (pendingWork != null) {
@@ -189,17 +178,13 @@ public class BlockStorageMovementNeeded {
           if (pendingWork.isDirWorkDone()) {
             ctxt.removeSPSHint(startId);
             pendingWorkForDirectory.remove(startId);
-            pendingWork.setFailure(!isSuccess);
-            updateStatus(startId, pendingWork.isPolicySatisfied());
           }
-          pendingWork.setFailure(isSuccess);
         }
       }
     } else {
       // Remove xAttr if trackID doesn't exist in
       // storageMovementAttemptedItems or file policy satisfied.
       ctxt.removeSPSHint(trackInfo.getFile());
-      updateStatus(trackInfo.getFile(), isSuccess);
     }
   }
 
@@ -216,24 +201,6 @@ public class BlockStorageMovementNeeded {
   }
 
   /**
-   * Mark inode status as SUCCESS in map.
-   */
-  private void updateStatus(long startId, boolean isSuccess){
-    StoragePolicySatisfyPathStatusInfo spsStatusInfo =
-        spsStatus.get(startId);
-    if (spsStatusInfo == null) {
-      spsStatusInfo = new StoragePolicySatisfyPathStatusInfo();
-      spsStatus.put(startId, spsStatusInfo);
-    }
-
-    if (isSuccess) {
-      spsStatusInfo.setSuccess();
-    } else {
-      spsStatusInfo.setFailure();
-    }
-  }
-
-  /**
    * Clean all the movements in spsDirsToBeTraveresed/storageMovementNeeded
    * and notify to clean up required resources.
    * @throws IOException
@@ -277,7 +244,6 @@ public class BlockStorageMovementNeeded {
     @Override
     public void run() {
       LOG.info("Starting SPSPathIdProcessor!.");
-      long lastStatusCleanTime = 0;
       Long startINode = null;
       while (ctxt.isRunning()) {
         try {
@@ -289,9 +255,6 @@ public class BlockStorageMovementNeeded {
               // Waiting for SPS path
               Thread.sleep(3000);
             } else {
-              spsStatus.put(startINode,
-                  new StoragePolicySatisfyPathStatusInfo(
-                      StoragePolicySatisfyPathStatus.IN_PROGRESS));
               ctxt.scanAndCollectFiles(startINode);
               // check if directory was empty and no child added to queue
               DirPendingWorkInfo dirPendingWorkInfo =
@@ -300,15 +263,8 @@ public class BlockStorageMovementNeeded {
                   && dirPendingWorkInfo.isDirWorkDone()) {
                 ctxt.removeSPSHint(startINode);
                 pendingWorkForDirectory.remove(startINode);
-                updateStatus(startINode, true);
               }
             }
-            //Clear the SPS status if status is in SUCCESS more than 5 min.
-            if (Time.monotonicNow()
-                - lastStatusCleanTime > statusClearanceElapsedTimeMs) {
-              lastStatusCleanTime = Time.monotonicNow();
-              cleanSPSStatus();
-            }
             startINode = null; // Current inode successfully scanned.
           }
         } catch (Throwable t) {
@@ -328,16 +284,6 @@ 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();
-        }
-      }
-    }
   }
 
   /**
@@ -347,7 +293,6 @@ public class BlockStorageMovementNeeded {
 
     private int pendingWorkCount = 0;
     private boolean fullyScanned = false;
-    private boolean success = true;
 
     /**
      * Increment the pending work count for directory.
@@ -378,20 +323,6 @@ 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 activate() {
@@ -406,56 +337,6 @@ public class BlockStorageMovementNeeded {
     }
   }
 
-  /**
-   * 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 void setFailure() {
-      this.status = StoragePolicySatisfyPathStatus.FAILURE;
-      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
-          || StoragePolicySatisfyPathStatus.FAILURE == 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) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/39ed3a66/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/Context.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/Context.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/Context.java
index d538374..afa5a50 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/Context.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/Context.java
@@ -94,11 +94,6 @@ public interface Context {
   BlockStoragePolicy getStoragePolicy(byte policyId);
 
   /**
-   * Drop the SPS work in case if any previous work queued up.
-   */
-  void addDropPreviousSPSWorkAtDNs();
-
-  /**
    * Remove the hint which was added to track SPS call.
    *
    * @param spsPath


---------------------------------------------------------------------
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-13057: [SPS]: Revisit configurations to make SPS service modes internal/external/none. Contributed by Rakesh R.

Posted by um...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/3b83110d/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/sps/TestStoragePolicySatisfierWithStripedFile.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/sps/TestStoragePolicySatisfierWithStripedFile.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/sps/TestStoragePolicySatisfierWithStripedFile.java
index 0e3a5a3..2257608 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/sps/TestStoragePolicySatisfierWithStripedFile.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/sps/TestStoragePolicySatisfierWithStripedFile.java
@@ -41,6 +41,7 @@ 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.protocol.HdfsConstants.StoragePolicySatisfierMode;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.junit.Assert;
@@ -103,8 +104,8 @@ public class TestStoragePolicySatisfierWithStripedFile {
     }
 
     final Configuration conf = new HdfsConfiguration();
-    conf.setBoolean(DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_ENABLED_KEY,
-        true);
+    conf.set(DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_MODE_KEY,
+        StoragePolicySatisfierMode.INTERNAL.toString());
     initConfWithStripe(conf, defaultStripeBlockSize);
     final MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
         .numDataNodes(numOfDatanodes)
@@ -216,8 +217,8 @@ public class TestStoragePolicySatisfierWithStripedFile {
     }
 
     final Configuration conf = new HdfsConfiguration();
-    conf.setBoolean(DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_ENABLED_KEY,
-        true);
+    conf.set(DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_MODE_KEY,
+        StoragePolicySatisfierMode.INTERNAL.toString());
     initConfWithStripe(conf, defaultStripeBlockSize);
     final MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
         .numDataNodes(numOfDatanodes)
@@ -328,8 +329,8 @@ public class TestStoragePolicySatisfierWithStripedFile {
     conf.set(DFSConfigKeys
         .DFS_STORAGE_POLICY_SATISFIER_RECHECK_TIMEOUT_MILLIS_KEY,
         "3000");
-    conf.setBoolean(DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_ENABLED_KEY,
-        true);
+    conf.set(DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_MODE_KEY,
+        StoragePolicySatisfierMode.INTERNAL.toString());
     initConfWithStripe(conf, defaultStripeBlockSize);
     final MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
         .numDataNodes(numOfDatanodes)
@@ -420,8 +421,8 @@ public class TestStoragePolicySatisfierWithStripedFile {
     }
 
     final Configuration conf = new HdfsConfiguration();
-    conf.setBoolean(DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_ENABLED_KEY,
-        true);
+    conf.set(DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_MODE_KEY,
+        StoragePolicySatisfierMode.INTERNAL.toString());
     initConfWithStripe(conf, defaultStripeBlockSize);
     final MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
         .numDataNodes(numOfDatanodes)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3b83110d/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/sps/TestExternalStoragePolicySatisfier.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/sps/TestExternalStoragePolicySatisfier.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/sps/TestExternalStoragePolicySatisfier.java
index 9a401bd..42b04da 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/sps/TestExternalStoragePolicySatisfier.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/sps/TestExternalStoragePolicySatisfier.java
@@ -31,6 +31,7 @@ import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants.StoragePolicySatisfierMode;
 import org.apache.hadoop.hdfs.server.balancer.NameNodeConnector;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.namenode.sps.BlockMovementListener;
@@ -54,12 +55,19 @@ public class TestExternalStoragePolicySatisfier
       new StorageType[][]{{StorageType.DISK, StorageType.DISK},
           {StorageType.DISK, StorageType.DISK},
           {StorageType.DISK, StorageType.DISK}};
+  private NameNodeConnector nnc;
+
+  @Override
+  public void setUp() {
+    super.setUp();
+
+    getConf().set(DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_MODE_KEY,
+        StoragePolicySatisfierMode.EXTERNAL.toString());
+  }
 
   @Override
   public void createCluster() throws IOException {
     getConf().setLong("dfs.block.size", DEFAULT_BLOCK_SIZE);
-    getConf().setBoolean(DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_ENABLED_KEY,
-        true);
     setCluster(startCluster(getConf(), allDiskTypes, NUM_OF_DATANODES,
         STORAGES_PER_DATANODE, CAPACITY));
     getFS();
@@ -80,35 +88,75 @@ public class TestExternalStoragePolicySatisfier
         .numDataNodes(numberOfDatanodes).storagesPerDatanode(storagesPerDn)
         .storageTypes(storageTypes).storageCapacities(capacities).build();
     cluster.waitActive();
-    if (conf.getBoolean(DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_ENABLED_KEY,
-        false)) {
-      BlockManager blkMgr = cluster.getNameNode().getNamesystem()
-          .getBlockManager();
-      SPSService spsService = blkMgr.getSPSService();
-      spsService.stopGracefully();
-
-      IntraSPSNameNodeContext context = new IntraSPSNameNodeContext(
-          cluster.getNameNode().getNamesystem(),
-          blkMgr, blkMgr.getSPSService());
-      ExternalBlockMovementListener blkMoveListener =
-          new ExternalBlockMovementListener();
-      ExternalSPSBlockMoveTaskHandler externalHandler =
-          new ExternalSPSBlockMoveTaskHandler(conf, getNameNodeConnector(conf),
-              blkMgr.getSPSService());
-      externalHandler.init();
-      spsService.init(context,
-          new ExternalSPSFileIDCollector(context, blkMgr.getSPSService(), 5),
-          externalHandler,
-          blkMoveListener);
-      spsService.start(true);
-    }
+
+    nnc = getNameNodeConnector(getConf());
+
+    BlockManager blkMgr = cluster.getNameNode().getNamesystem()
+        .getBlockManager();
+    SPSService spsService = blkMgr.getSPSService();
+    spsService.stopGracefully();
+
+    // TODO: Since External is not fully implemented, just used INTERNAL now.
+    // Need to set External context here.
+    IntraSPSNameNodeContext context = new IntraSPSNameNodeContext(
+        cluster.getNameNode().getNamesystem(), blkMgr, blkMgr.getSPSService()) {
+      public boolean isRunning() {
+        return true;
+      };
+    };
+    ExternalBlockMovementListener blkMoveListener =
+        new ExternalBlockMovementListener();
+    ExternalSPSBlockMoveTaskHandler externalHandler =
+        new ExternalSPSBlockMoveTaskHandler(conf, nnc,
+            blkMgr.getSPSService());
+    externalHandler.init();
+    spsService.init(context,
+        new ExternalSPSFileIDCollector(context, blkMgr.getSPSService()),
+        externalHandler,
+        blkMoveListener);
+    spsService.start(true, StoragePolicySatisfierMode.EXTERNAL);
     return cluster;
   }
 
+  public void restartNamenode() throws IOException{
+    BlockManager blkMgr = getCluster().getNameNode().getNamesystem()
+        .getBlockManager();
+    SPSService spsService = blkMgr.getSPSService();
+    spsService.stopGracefully();
+
+    getCluster().restartNameNodes();
+    getCluster().waitActive();
+    blkMgr = getCluster().getNameNode().getNamesystem()
+        .getBlockManager();
+    spsService = blkMgr.getSPSService();
+    spsService.stopGracefully();
+
+    // TODO: Since External is not fully implemented, just used INTERNAL now.
+    // Need to set External context here.
+    IntraSPSNameNodeContext context = new IntraSPSNameNodeContext(
+        getCluster().getNameNode().getNamesystem(), blkMgr,
+        blkMgr.getSPSService()) {
+      public boolean isRunning() {
+        return true;
+      };
+    };
+    ExternalBlockMovementListener blkMoveListener =
+        new ExternalBlockMovementListener();
+    ExternalSPSBlockMoveTaskHandler externalHandler =
+        new ExternalSPSBlockMoveTaskHandler(getConf(), nnc,
+            blkMgr.getSPSService());
+    externalHandler.init();
+    spsService.init(context,
+        new ExternalSPSFileIDCollector(context, blkMgr.getSPSService()),
+        externalHandler,
+        blkMoveListener);
+    spsService.start(true, StoragePolicySatisfierMode.EXTERNAL);
+  }
+
   @Override
   public FileIdCollector createFileIdCollector(StoragePolicySatisfier sps,
       Context ctxt) {
-    return new ExternalSPSFileIDCollector(ctxt, sps, 5);
+    return new ExternalSPSFileIDCollector(ctxt, sps);
   }
 
   private class ExternalBlockMovementListener implements BlockMovementListener {
@@ -148,4 +196,18 @@ public class TestExternalStoragePolicySatisfier
       + " So, ignoring it.")
   public void testBatchProcessingForSPSDirectory() throws Exception {
   }
+
+  /**
+   * Status won't be supported for external SPS, now. So, ignoring it.
+   */
+  @Ignore("Status is not supported for external SPS. So, ignoring it.")
+  public void testStoragePolicySatisfyPathStatus() throws Exception {
+  }
+
+  /**
+   * Status won't be supported for external SPS, now. So, ignoring it.
+   */
+  @Ignore("Status is not supported for external SPS. So, ignoring it.")
+  public void testMaxRetryForFailedBlock() throws Exception {
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3b83110d/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 7e0663d..28838a6 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
@@ -29,6 +29,7 @@ import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants.StoragePolicySatisfierMode;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
 import org.junit.After;
 import org.junit.Before;
@@ -48,8 +49,8 @@ public class TestStoragePolicyCommands {
   @Before
   public void clusterSetUp() throws IOException, URISyntaxException {
     conf = new HdfsConfiguration();
-    conf.setBoolean(DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_ENABLED_KEY,
-        true);
+    conf.set(DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_MODE_KEY,
+        StoragePolicySatisfierMode.INTERNAL.toString());
     StorageType[][] newtypes = new StorageType[][] {
         {StorageType.ARCHIVE, StorageType.DISK}};
     cluster = new MiniDFSCluster.Builder(conf).numDataNodes(REPL)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3b83110d/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
index 856c3ec..6a30c03 100644
--- 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
@@ -28,6 +28,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.protocol.HdfsConstants.StoragePolicySatisfierMode;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
@@ -46,8 +47,8 @@ public class TestStoragePolicySatisfyAdminCommands {
   @Before
   public void clusterSetUp() throws IOException, URISyntaxException {
     conf = new HdfsConfiguration();
-    conf.setBoolean(DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_ENABLED_KEY,
-        true);
+    conf.set(DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_MODE_KEY,
+        StoragePolicySatisfierMode.INTERNAL.toString());
     StorageType[][] newtypes = new StorageType[][] {
         {StorageType.ARCHIVE, StorageType.DISK}};
     cluster = new MiniDFSCluster.Builder(conf).numDataNodes(REPL)
@@ -94,16 +95,17 @@ public class TestStoragePolicySatisfyAdminCommands {
     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");
+    DFSTestUtil.toolRun(admin, "-isInternalSatisfierRunning", 0, "yes");
 
     cluster.getNameNode().reconfigureProperty(
-        DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_ENABLED_KEY, "false");
+        DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_MODE_KEY,
+        StoragePolicySatisfierMode.NONE.toString());
     cluster.waitActive();
 
-    DFSTestUtil.toolRun(admin, "-isSatisfierRunning", 0, "no");
+    DFSTestUtil.toolRun(admin, "-isInternalSatisfierRunning", 0, "no");
 
     // Test with unnecessary args
-    DFSTestUtil.toolRun(admin, "-isSatisfierRunning status", 1,
+    DFSTestUtil.toolRun(admin, "-isInternalSatisfierRunning 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


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

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

Branch: refs/heads/HDFS-10285
Commit: c561cb316e365ef674784cd6cf0b12c0fbc271a3
Parents: 9b83f94
Author: Surendra Singh Lilhore <su...@apache.org>
Authored: Wed Nov 15 20:22:27 2017 +0530
Committer: Uma Maheswara Rao Gangumalla <um...@apache.org>
Committed: Sun Aug 12 03:06:02 2018 -0700

----------------------------------------------------------------------
 .../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/c561cb31/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 190a1c6..aabcdd9 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
@@ -150,6 +150,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/c561cb31/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 9281bff..7770e31 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
@@ -3409,6 +3409,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:
@@ -3425,6 +3427,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/c561cb31/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 1de13ca..933a19a 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
@@ -854,7 +854,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/c561cb31/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 d577e4c..f1a59d3 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
@@ -634,10 +634,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/c561cb31/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 b1db377..6aab5e9 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,
@@ -1699,11 +1699,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/c561cb31/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/c561cb31/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/c561cb31/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/c561cb31/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 b0be914..5990a5f 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
@@ -7869,6 +7869,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/c561cb31/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/c561cb31/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/c561cb31/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 2fbdcc9..897ae9a 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
@@ -4557,12 +4557,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/c561cb31/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 cf17e99..15f5d2c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/ArchivalStorage.md
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/ArchivalStorage.md
@@ -224,7 +224,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/c561cb31/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/c561cb31/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


[43/50] [abbrv] hadoop git commit: HDFS-13076: [SPS]: Cleanup work for HDFS-10285 merge. Contributed by Rakesh R.

Posted by um...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/39ed3a66/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/sps/TestBlockStorageMovementAttemptedItems.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/sps/TestBlockStorageMovementAttemptedItems.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/sps/TestBlockStorageMovementAttemptedItems.java
index f85769f..f48521b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/sps/TestBlockStorageMovementAttemptedItems.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/sps/TestBlockStorageMovementAttemptedItems.java
@@ -34,6 +34,7 @@ import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.server.namenode.sps.StoragePolicySatisfier.StorageTypeNodePair;
+import org.apache.hadoop.hdfs.server.sps.ExternalSPSContext;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
@@ -52,7 +53,7 @@ public class TestBlockStorageMovementAttemptedItems {
   @Before
   public void setup() throws Exception {
     Configuration config = new HdfsConfiguration();
-    Context ctxt = Mockito.mock(IntraSPSNameNodeContext.class);
+    Context ctxt = Mockito.mock(ExternalSPSContext.class);
     SPSService sps = new StoragePolicySatisfier(config);
     Mockito.when(ctxt.isRunning()).thenReturn(true);
     Mockito.when(ctxt.isInSafeMode()).thenReturn(false);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/39ed3a66/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/sps/TestStoragePolicySatisfier.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/sps/TestStoragePolicySatisfier.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/sps/TestStoragePolicySatisfier.java
deleted file mode 100644
index ec5307b..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/sps/TestStoragePolicySatisfier.java
+++ /dev/null
@@ -1,1825 +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.sps;
-
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_STORAGE_POLICY_ENABLED_KEY;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_QUEUE_LIMIT_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 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;
-import java.net.InetSocketAddress;
-import java.util.ArrayList;
-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;
-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.MiniDFSCluster.DataNodeProperties;
-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.Block;
-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.StoragePolicySatisfierMode;
-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;
-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.hdfs.server.namenode.FSDirectory;
-import org.apache.hadoop.hdfs.server.namenode.FSEditLog;
-import org.apache.hadoop.hdfs.server.namenode.FSTreeTraverser;
-import org.apache.hadoop.hdfs.server.namenode.INode;
-import org.apache.hadoop.test.GenericTestUtils;
-import org.apache.hadoop.test.GenericTestUtils.LogCapturer;
-import org.junit.After;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Test;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import org.slf4j.event.Level;
-
-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 {
-
-  {
-    GenericTestUtils.setLogLevel(
-        getLogger(FSTreeTraverser.class), Level.DEBUG);
-  }
-
-  private static final String ONE_SSD = "ONE_SSD";
-  private static final String COLD = "COLD";
-  protected static final Logger LOG =
-      LoggerFactory.getLogger(TestStoragePolicySatisfier.class);
-  private Configuration config = null;
-  private StorageType[][] allDiskTypes =
-      new StorageType[][]{{StorageType.DISK, StorageType.DISK},
-          {StorageType.DISK, StorageType.DISK},
-          {StorageType.DISK, StorageType.DISK}};
-  private MiniDFSCluster hdfsCluster = null;
-  private DistributedFileSystem dfs = null;
-  public static final int NUM_OF_DATANODES = 3;
-  public static final int STORAGES_PER_DATANODE = 2;
-  public static final long CAPACITY = 2 * 256 * 1024 * 1024;
-  public static final String FILE = "/testMoveToSatisfyStoragePolicy";
-  public static final int DEFAULT_BLOCK_SIZE = 1024;
-
-  /**
-   * Sets hdfs cluster.
-   */
-  public void setCluster(MiniDFSCluster cluster) {
-    this.hdfsCluster = cluster;
-  }
-
-  /**
-   * @return conf.
-   */
-  public Configuration getConf() {
-    return this.config;
-  }
-
-  /**
-   * @return hdfs cluster.
-   */
-  public MiniDFSCluster getCluster() {
-    return hdfsCluster;
-  }
-
-  /**
-   * Gets distributed file system.
-   *
-   * @throws IOException
-   */
-  public DistributedFileSystem getFS() throws IOException {
-    this.dfs = hdfsCluster.getFileSystem();
-    return this.dfs;
-  }
-
-  @After
-  public void shutdownCluster() {
-    if (hdfsCluster != null) {
-      hdfsCluster.shutdown();
-    }
-  }
-
-  public void createCluster() throws IOException {
-    config.setLong("dfs.block.size", DEFAULT_BLOCK_SIZE);
-    hdfsCluster = startCluster(config, allDiskTypes, NUM_OF_DATANODES,
-        STORAGES_PER_DATANODE, CAPACITY);
-    getFS();
-    writeContent(FILE);
-  }
-
-  @Before
-  public void setUp() {
-    config = new HdfsConfiguration();
-    config.set(DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_MODE_KEY,
-        StoragePolicySatisfierMode.INTERNAL.toString());
-    // Most of the tests are restarting DNs and NN. So, reduced refresh cycle to
-    // update latest datanodes.
-    config.setLong(DFSConfigKeys.DFS_SPS_DATANODE_CACHE_REFRESH_INTERVAL_MS,
-        3000);
-  }
-
-  @Test(timeout = 300000)
-  public void testWhenStoragePolicySetToCOLD()
-      throws Exception {
-
-    try {
-      createCluster();
-      doTestWhenStoragePolicySetToCOLD();
-    } finally {
-      shutdownCluster();
-    }
-  }
-
-  private void doTestWhenStoragePolicySetToCOLD() throws Exception {
-    // 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, NUM_OF_DATANODES, newtypes,
-        STORAGES_PER_DATANODE, CAPACITY, hdfsCluster);
-
-    hdfsCluster.triggerHeartbeats();
-    dfs.satisfyStoragePolicy(new Path(FILE));
-    // Wait till namenode notified about the block location details
-    DFSTestUtil.waitExpectedStorageType(FILE, StorageType.ARCHIVE, 3, 35000,
-        dfs);
-  }
-
-  @Test(timeout = 300000)
-  public void testWhenStoragePolicySetToALLSSD()
-      throws Exception {
-    try {
-      createCluster();
-      // Change policy to ALL_SSD
-      dfs.setStoragePolicy(new Path(FILE), "ALL_SSD");
-
-      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, NUM_OF_DATANODES, newtypes,
-          STORAGES_PER_DATANODE, CAPACITY, hdfsCluster);
-      dfs.satisfyStoragePolicy(new Path(FILE));
-      hdfsCluster.triggerHeartbeats();
-      // Wait till StorgePolicySatisfier Identified that block to move to SSD
-      // areas
-      DFSTestUtil.waitExpectedStorageType(FILE, StorageType.SSD, 3, 30000, dfs);
-    } finally {
-      shutdownCluster();
-    }
-  }
-
-  @Test(timeout = 300000)
-  public void testWhenStoragePolicySetToONESSD()
-      throws Exception {
-    try {
-      createCluster();
-      // Change policy to ONE_SSD
-      dfs.setStoragePolicy(new Path(FILE), ONE_SSD);
-
-      StorageType[][] newtypes =
-          new StorageType[][]{{StorageType.SSD, StorageType.DISK}};
-
-      // Making sure SDD based nodes added to cluster. Adding SSD based
-      // datanodes.
-      startAdditionalDNs(config, 1, NUM_OF_DATANODES, newtypes,
-          STORAGES_PER_DATANODE, CAPACITY, hdfsCluster);
-      dfs.satisfyStoragePolicy(new Path(FILE));
-      hdfsCluster.triggerHeartbeats();
-      // Wait till StorgePolicySatisfier Identified that block to move to SSD
-      // areas
-      DFSTestUtil.waitExpectedStorageType(FILE, StorageType.SSD, 1, 30000, dfs);
-      DFSTestUtil.waitExpectedStorageType(FILE, StorageType.DISK, 2, 30000,
-          dfs);
-    } finally {
-      shutdownCluster();
-    }
-  }
-
-  /**
-   * Tests to verify that the block storage movement report will be propagated
-   * to Namenode via datanode heartbeat.
-   */
-  @Test(timeout = 300000)
-  public void testBlksStorageMovementAttemptFinishedReport() throws Exception {
-    try {
-      createCluster();
-      // Change policy to ONE_SSD
-      dfs.setStoragePolicy(new Path(FILE), ONE_SSD);
-
-      StorageType[][] newtypes =
-          new StorageType[][]{{StorageType.SSD, StorageType.DISK}};
-
-      // Making sure SDD based nodes added to cluster. Adding SSD based
-      // datanodes.
-      startAdditionalDNs(config, 1, NUM_OF_DATANODES, newtypes,
-          STORAGES_PER_DATANODE, CAPACITY, hdfsCluster);
-      dfs.satisfyStoragePolicy(new Path(FILE));
-      hdfsCluster.triggerHeartbeats();
-
-      // Wait till the block is moved to SSD areas
-      DFSTestUtil.waitExpectedStorageType(FILE, StorageType.SSD, 1, 30000, dfs);
-      DFSTestUtil.waitExpectedStorageType(FILE, StorageType.DISK, 2, 30000,
-          dfs);
-
-      waitForBlocksMovementAttemptReport(1, 30000);
-    } finally {
-      shutdownCluster();
-    }
-  }
-
-  /**
-   * 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 {
-    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);
-      }
-      // Change policy to ONE_SSD
-      for (String fileName : files) {
-        dfs.setStoragePolicy(new Path(fileName), ONE_SSD);
-        dfs.satisfyStoragePolicy(new Path(fileName));
-      }
-
-      StorageType[][] newtypes =
-          new StorageType[][]{{StorageType.SSD, StorageType.DISK}};
-
-      // Making sure SDD based nodes added to cluster. Adding SSD based
-      // datanodes.
-      startAdditionalDNs(config, 1, NUM_OF_DATANODES, newtypes,
-          STORAGES_PER_DATANODE, CAPACITY, hdfsCluster);
-      hdfsCluster.triggerHeartbeats();
-
-      for (String fileName : files) {
-        // Wait till the block is moved to SSD areas
-        DFSTestUtil.waitExpectedStorageType(
-            fileName, StorageType.SSD, 1, 30000, dfs);
-        DFSTestUtil.waitExpectedStorageType(
-            fileName, StorageType.DISK, 2, 30000, dfs);
-      }
-
-      waitForBlocksMovementAttemptReport(files.size(), 30000);
-    } finally {
-      shutdownCluster();
-    }
-  }
-
-  /**
-   * Tests to verify hdfsAdmin.satisfyStoragePolicy works well for file.
-   * @throws Exception
-   */
-  @Test(timeout = 300000)
-  public void testSatisfyFileWithHdfsAdmin() throws Exception {
-    try {
-      createCluster();
-      HdfsAdmin hdfsAdmin =
-          new HdfsAdmin(FileSystem.getDefaultUri(config), config);
-      // Change policy to COLD
-      dfs.setStoragePolicy(new Path(FILE), COLD);
-
-      StorageType[][] newtypes =
-          new StorageType[][]{{StorageType.DISK, StorageType.ARCHIVE},
-              {StorageType.DISK, StorageType.ARCHIVE},
-              {StorageType.DISK, StorageType.ARCHIVE}};
-      startAdditionalDNs(config, 3, NUM_OF_DATANODES, newtypes,
-          STORAGES_PER_DATANODE, CAPACITY, hdfsCluster);
-
-      hdfsAdmin.satisfyStoragePolicy(new Path(FILE));
-
-      hdfsCluster.triggerHeartbeats();
-      // Wait till namenode notified about the block location details
-      DFSTestUtil.waitExpectedStorageType(FILE, StorageType.ARCHIVE, 3, 30000,
-          dfs);
-    } finally {
-      shutdownCluster();
-    }
-  }
-
-  /**
-   * Tests to verify hdfsAdmin.satisfyStoragePolicy works well for dir.
-   * @throws Exception
-   */
-  @Test(timeout = 300000)
-  public void testSatisfyDirWithHdfsAdmin() throws Exception {
-    try {
-      createCluster();
-      HdfsAdmin hdfsAdmin =
-          new HdfsAdmin(FileSystem.getDefaultUri(config), config);
-      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, NUM_OF_DATANODES, newtypes,
-          STORAGES_PER_DATANODE, CAPACITY, hdfsCluster);
-
-      hdfsAdmin.satisfyStoragePolicy(new Path(subDir));
-
-      hdfsCluster.triggerHeartbeats();
-
-      // take effect for the file in the directory.
-      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.
-      DFSTestUtil.waitExpectedStorageType(
-          subFile2, StorageType.SSD, 1, 30000, dfs);
-      DFSTestUtil.waitExpectedStorageType(
-          subFile2, StorageType.DISK, 2, 30000, dfs);
-    } finally {
-      shutdownCluster();
-    }
-  }
-
-  /**
-   * Tests to verify hdfsAdmin.satisfyStoragePolicy exceptions.
-   * @throws Exception
-   */
-  @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);
-      restartNamenode();
-      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) {
-        GenericTestUtils.assertExceptionContains(String.format(
-            "Failed to satisfy storage policy since %s is set to false.",
-            DFS_STORAGE_POLICY_ENABLED_KEY), e);
-      }
-
-      hdfsCluster.getConfiguration(0).
-          setBoolean(DFSConfigKeys.DFS_STORAGE_POLICY_ENABLED_KEY, true);
-      restartNamenode();
-
-      hdfsAdmin = new HdfsAdmin(FileSystem.getDefaultUri(config), config);
-      try {
-        hdfsAdmin.satisfyStoragePolicy(new Path(nonExistingFile));
-        Assert.fail("Should throw FileNotFoundException for " +
-            nonExistingFile);
-      } catch (FileNotFoundException e) {
-
-      }
-
-      try {
-        hdfsAdmin.satisfyStoragePolicy(new Path(FILE));
-        hdfsAdmin.satisfyStoragePolicy(new Path(FILE));
-      } catch (Exception e) {
-        Assert.fail(String.format("Allow to invoke mutlipe times "
-            + "#satisfyStoragePolicy() api for a path %s , internally just "
-            + "skipping addtion to satisfy movement queue.", FILE));
-      }
-    } finally {
-      shutdownCluster();
-    }
-  }
-
-  /**
-   * 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 {
-      createCluster();
-      // Change policy to COLD
-      dfs.setStoragePolicy(new Path(FILE), COLD);
-
-      StorageType[][] newtypes =
-          new StorageType[][]{{StorageType.ARCHIVE, StorageType.ARCHIVE}};
-
-      // Adding ARCHIVE based datanodes.
-      startAdditionalDNs(config, 1, NUM_OF_DATANODES, newtypes,
-          STORAGES_PER_DATANODE, CAPACITY, hdfsCluster);
-
-      dfs.satisfyStoragePolicy(new Path(FILE));
-      hdfsCluster.triggerHeartbeats();
-      // Wait till StorgePolicySatisfier identified that block to move to
-      // ARCHIVE area.
-      DFSTestUtil.waitExpectedStorageType(FILE, StorageType.ARCHIVE, 1, 30000,
-          dfs);
-      DFSTestUtil.waitExpectedStorageType(FILE, StorageType.DISK, 2, 30000,
-          dfs);
-
-      waitForBlocksMovementAttemptReport(1, 30000);
-    } finally {
-      shutdownCluster();
-    }
-  }
-
-  /**
-   * 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 {
-      createCluster();
-      // Change policy to COLD
-      dfs.setStoragePolicy(new Path(FILE), COLD);
-
-      StorageType[][] newtypes =
-          new StorageType[][]{{StorageType.DISK, StorageType.DISK}};
-      // Adding DISK based datanodes
-      startAdditionalDNs(config, 1, NUM_OF_DATANODES, newtypes,
-          STORAGES_PER_DATANODE, CAPACITY, hdfsCluster);
-
-      dfs.satisfyStoragePolicy(new Path(FILE));
-      hdfsCluster.triggerHeartbeats();
-
-      // 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);
-      // Since there is no target node the item will get timed out and then
-      // re-attempted.
-      waitForAttemptedItems(1, 30000);
-    } finally {
-      shutdownCluster();
-    }
-  }
-
-  /**
-   * Tests to verify that SPS should not start when a Mover instance
-   * is running.
-   */
-  @Test(timeout = 300000)
-  public void testWhenMoverIsAlreadyRunningBeforeStoragePolicySatisfier()
-      throws Exception {
-    boolean running;
-    FSDataOutputStream out = null;
-    try {
-      createCluster();
-      // Stop SPS
-      hdfsCluster.getNameNode().reconfigureProperty(
-          DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_MODE_KEY,
-          StoragePolicySatisfierMode.NONE.toString());
-      running = hdfsCluster.getFileSystem()
-          .getClient().isInternalSatisfierRunning();
-      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().reconfigureProperty(
-          DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_MODE_KEY,
-          StoragePolicySatisfierMode.INTERNAL.toString());
-
-      running = hdfsCluster.getFileSystem()
-          .getClient().isInternalSatisfierRunning();
-      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().reconfigureProperty(
-          DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_MODE_KEY,
-          StoragePolicySatisfierMode.INTERNAL.toString());
-      running = hdfsCluster.getFileSystem()
-          .getClient().isInternalSatisfierRunning();
-      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_MODE_KEY, e);
-    } finally {
-      if (out != null) {
-        out.close();
-      }
-      shutdownCluster();
-    }
-  }
-
-  /**
-   * 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 the case by creating MOVER_ID file
-      DFSTestUtil.createFile(hdfsCluster.getFileSystem(),
-          HdfsServerConstants.MOVER_ID_PATH, 0, (short) 1, 0);
-      restartNamenode();
-      boolean running = hdfsCluster.getFileSystem()
-          .getClient().isInternalSatisfierRunning();
-      Assert.assertTrue("SPS should be running as "
-          + "no Mover really running", running);
-    } finally {
-      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 {
-    try{
-      config.setBoolean(DFSConfigKeys.DFS_DATANODE_BLOCK_PINNING_ENABLED, true);
-      hdfsCluster = startCluster(config, allDiskTypes, 3, 2, CAPACITY);
-
-      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);
-
-      StorageType[][] newtypes =
-          new StorageType[][]{{StorageType.ARCHIVE, StorageType.ARCHIVE},
-              {StorageType.ARCHIVE, StorageType.ARCHIVE},
-              {StorageType.ARCHIVE, StorageType.ARCHIVE}};
-      // Adding DISK based datanodes
-      startAdditionalDNs(config, 3, NUM_OF_DATANODES, newtypes,
-          STORAGES_PER_DATANODE, CAPACITY, hdfsCluster);
-
-      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);
-      waitForBlocksMovementAttemptReport(1, 30000);
-      DFSTestUtil.waitExpectedStorageType(
-          file1, StorageType.ARCHIVE, 1, 30000, dfs);
-      DFSTestUtil.waitExpectedStorageType(
-          file1, StorageType.DISK, 2, 30000, dfs);
-    } finally {
-      shutdownCluster();
-    }
-  }
-
-  /**
-   * 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,
-          STORAGES_PER_DATANODE, CAPACITY);
-      dfs = hdfsCluster.getFileSystem();
-      writeContent(FILE, (short) 5);
-
-      // Change policy to COLD
-      dfs.setStoragePolicy(new Path(FILE), COLD);
-
-      dfs.satisfyStoragePolicy(new Path(FILE));
-      hdfsCluster.triggerHeartbeats();
-      // Wait till StorgePolicySatisfier identified that block to move to
-      // ARCHIVE area.
-      DFSTestUtil.waitExpectedStorageType(FILE, StorageType.ARCHIVE, 2, 30000,
-          dfs);
-      DFSTestUtil.waitExpectedStorageType(FILE, StorageType.DISK, 3, 30000,
-          dfs);
-
-      waitForBlocksMovementAttemptReport(1, 30000);
-    } finally {
-      shutdownCluster();
-    }
-  }
-
-  /**
-   * 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, NUM_OF_DATANODES,
-          STORAGES_PER_DATANODE, CAPACITY);
-      dfs = hdfsCluster.getFileSystem();
-      writeContent(FILE);
-
-      // Change policy to ONE_SSD
-      dfs.setStoragePolicy(new Path(FILE), ONE_SSD);
-
-      dfs.satisfyStoragePolicy(new Path(FILE));
-      hdfsCluster.triggerHeartbeats();
-      DFSTestUtil.waitExpectedStorageType(FILE, StorageType.SSD, 1, 30000, dfs);
-      DFSTestUtil.waitExpectedStorageType(FILE, StorageType.DISK, 2, 30000,
-          dfs);
-
-    } 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,
-          STORAGES_PER_DATANODE, CAPACITY);
-      dfs = hdfsCluster.getFileSystem();
-      writeContent(FILE);
-
-      // Change policy to WARM
-      dfs.setStoragePolicy(new Path(FILE), "WARM");
-      dfs.satisfyStoragePolicy(new Path(FILE));
-      hdfsCluster.triggerHeartbeats();
-
-      DFSTestUtil.waitExpectedStorageType(FILE, StorageType.DISK, 1, 30000,
-          dfs);
-      DFSTestUtil.waitExpectedStorageType(FILE, StorageType.ARCHIVE, 2, 30000,
-          dfs);
-    } finally {
-      shutdownCluster();
-    }
-  }
-
-  /**
-   * 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,
-          STORAGES_PER_DATANODE, 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.
-   */
-  @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, NUM_OF_DATANODES,
-          STORAGES_PER_DATANODE, dnCapacity);
-      dfs = hdfsCluster.getFileSystem();
-      writeContent(FILE);
-
-      // Change policy to ONE_SSD
-      dfs.setStoragePolicy(new Path(FILE), ONE_SSD);
-      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);
-      }
-      dfs.satisfyStoragePolicy(new Path(FILE));
-
-      // 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(1));
-      }
-
-      // 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();
-    }
-  }
-
-  /**
-   * 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 =
-        StripedFileTestUtil.getDefaultECPolicy().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,
-          STORAGES_PER_DATANODE, CAPACITY);
-      dfs = hdfsCluster.getFileSystem();
-      dfs.enableErasureCodingPolicy(
-          StripedFileTestUtil.getDefaultECPolicy().getName());
-
-      // 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,
-          StripedFileTestUtil.getDefaultECPolicy().getName());
-
-      // write file to fooDir
-      final String testFile = "/foo/bar";
-      long fileLen = 20 * defaultStripedBlockSize;
-      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();
-    }
-  }
-
-  /**
-   * 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 {
-    try {
-      hdfsCluster = startCluster(config, allDiskTypes, NUM_OF_DATANODES,
-          STORAGES_PER_DATANODE, CAPACITY);
-      hdfsCluster.waitActive();
-      DistributedFileSystem fs = hdfsCluster.getFileSystem();
-      Path filePath = new Path("/zeroSizeFile");
-      DFSTestUtil.createFile(fs, filePath, 0, (short) 1, 0);
-      FSEditLog editlog = hdfsCluster.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 = hdfsCluster.getNameNode().getNamesystem().getFSDirectory()
-          .getINode(filePath.toString());
-      Assert.assertTrue("XAttrFeature should be null for file",
-          inode.getXAttrFeature() == null);
-    } finally {
-      shutdownCluster();
-    }
-  }
-
-  /**
-   * 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 {
-    try {
-      config.set(DFSConfigKeys
-          .DFS_STORAGE_POLICY_SATISFIER_RECHECK_TIMEOUT_MILLIS_KEY,
-          "3000");
-      config.set(DFSConfigKeys
-          .DFS_STORAGE_POLICY_SATISFIER_SELF_RETRY_TIMEOUT_MILLIS_KEY,
-          "5000");
-      StorageType[][] newtypes = new StorageType[][] {
-          {StorageType.ARCHIVE, StorageType.DISK},
-          {StorageType.ARCHIVE, StorageType.DISK},
-          {StorageType.ARCHIVE, StorageType.DISK}};
-      hdfsCluster = startCluster(config, newtypes, 3, 2, CAPACITY);
-      hdfsCluster.waitActive();
-      DistributedFileSystem fs = hdfsCluster.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(hdfsCluster.stopDataNode(0));
-      list.add(hdfsCluster.stopDataNode(0));
-      list.add(hdfsCluster.stopDataNode(0));
-      restartNamenode();
-      hdfsCluster.restartDataNode(list.get(0), false);
-      hdfsCluster.restartDataNode(list.get(1), false);
-      hdfsCluster.waitActive();
-      fs.satisfyStoragePolicy(filePath);
-      DFSTestUtil.waitExpectedStorageType(filePath.toString(),
-          StorageType.ARCHIVE, 2, 30000, hdfsCluster.getFileSystem());
-      hdfsCluster.restartDataNode(list.get(2), false);
-      DFSTestUtil.waitExpectedStorageType(filePath.toString(),
-          StorageType.ARCHIVE, 3, 30000, hdfsCluster.getFileSystem());
-    } finally {
-      shutdownCluster();
-    }
-  }
-
-  /**
-   * 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 {
-    try {
-      config.set(DFSConfigKeys
-          .DFS_STORAGE_POLICY_SATISFIER_RECHECK_TIMEOUT_MILLIS_KEY,
-          "3000");
-      config.set(DFSConfigKeys
-          .DFS_STORAGE_POLICY_SATISFIER_SELF_RETRY_TIMEOUT_MILLIS_KEY,
-          "5000");
-      StorageType[][] newtypes = new StorageType[][] {
-          {StorageType.ARCHIVE, StorageType.DISK},
-          {StorageType.ARCHIVE, StorageType.DISK},
-          {StorageType.ARCHIVE, StorageType.DISK},
-          {StorageType.ARCHIVE, StorageType.DISK},
-          {StorageType.ARCHIVE, StorageType.DISK}};
-      hdfsCluster = startCluster(config, newtypes, 5, 2, CAPACITY);
-      hdfsCluster.waitActive();
-      DistributedFileSystem fs = hdfsCluster.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, 60000, hdfsCluster.getFileSystem());
-      assertFalse("Log output does not contain expected log message: ",
-          logs.getOutput().contains("some of the blocks are low redundant"));
-    } finally {
-      shutdownCluster();
-    }
-  }
-
-  /**
-   * Test SPS for empty directory, xAttr should be removed.
-   */
-  @Test(timeout = 300000)
-  public void testSPSForEmptyDirectory() throws IOException, TimeoutException,
-      InterruptedException {
-    try {
-      hdfsCluster = startCluster(config, allDiskTypes, NUM_OF_DATANODES,
-          STORAGES_PER_DATANODE, CAPACITY);
-      hdfsCluster.waitActive();
-      DistributedFileSystem fs = hdfsCluster.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, hdfsCluster.getNamesystem(), 30000);
-    } finally {
-      shutdownCluster();
-    }
-  }
-
-  /**
-   * Test SPS for not exist directory.
-   */
-  @Test(timeout = 300000)
-  public void testSPSForNonExistDirectory() throws Exception {
-    try {
-      hdfsCluster = startCluster(config, allDiskTypes, NUM_OF_DATANODES,
-          STORAGES_PER_DATANODE, CAPACITY);
-      hdfsCluster.waitActive();
-      DistributedFileSystem fs = hdfsCluster.getFileSystem();
-      Path emptyDir = new Path("/emptyDir");
-      try {
-        fs.satisfyStoragePolicy(emptyDir);
-        fail("FileNotFoundException should throw");
-      } catch (FileNotFoundException e) {
-        // nothing to do
-      }
-    } finally {
-      shutdownCluster();
-    }
-  }
-
-  /**
-   * Test SPS for directory tree which doesn't have files.
-   */
-  @Test(timeout = 300000)
-  public void testSPSWithDirectoryTreeWithoutFile() throws Exception {
-    try {
-      hdfsCluster = startCluster(config, allDiskTypes, NUM_OF_DATANODES,
-          STORAGES_PER_DATANODE, CAPACITY);
-      hdfsCluster.waitActive();
-      // Create directories
-      /*
-       *                   root
-       *                    |
-       *           A--------C--------D
-       *                    |
-       *               G----H----I
-       *                    |
-       *                    O
-       */
-      DistributedFileSystem fs = hdfsCluster.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, hdfsCluster.getNamesystem(), 30000);
-    } finally {
-      shutdownCluster();
-    }
-  }
-
-  /**
-   * 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);
-      hdfsCluster = startCluster(config, diskTypes, diskTypes.length,
-          STORAGES_PER_DATANODE, 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 = 3000000)
-  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);
-      // Set queue max capacity
-      config.setInt(DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_QUEUE_LIMIT_KEY,
-          5);
-      hdfsCluster = startCluster(config, diskTypes, diskTypes.length,
-          STORAGES_PER_DATANODE, 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);
-      }
-      waitForBlocksMovementAttemptReport(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(timeout = 300000)
-  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);
-    config.setInt(DFS_STORAGE_POLICY_SATISFIER_QUEUE_LIMIT_KEY, 10);
-    hdfsCluster = startCluster(config, diskTypes, diskTypes.length,
-        STORAGES_PER_DATANODE, 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 = new StoragePolicySatisfier(config);
-    Context ctxt = new IntraSPSNameNodeContext(
-        hdfsCluster.getNamesystem(),
-        hdfsCluster.getNamesystem().getBlockManager(), sps) {
-      @Override
-      public boolean isInSafeMode() {
-        return false;
-      }
-
-      @Override
-      public boolean isRunning() {
-        return true;
-      }
-    };
-
-    sps.init(ctxt);
-    sps.getStorageMovementQueue().activate();
-
-    INode rootINode = fsDir.getINode("/root");
-    hdfsCluster.getNamesystem().getBlockManager().getSPSManager()
-        .addPathId(rootINode.getId());
-
-    //Wait for thread to reach U.
-    Thread.sleep(1000);
-    dfs.delete(new Path("/root/D/L"), true);
-
-
-    assertTraversal(expectedTraverseOrder, fsDir, sps);
-    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(timeout = 300000)
-  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);
-    config.setInt(DFS_STORAGE_POLICY_SATISFIER_QUEUE_LIMIT_KEY, 10);
-    hdfsCluster = startCluster(config, diskTypes, diskTypes.length,
-        STORAGES_PER_DATANODE, 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();
-
-    // 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 = new StoragePolicySatisfier(config);
-    Context ctxt = new IntraSPSNameNodeContext(
-        hdfsCluster.getNamesystem(),
-        hdfsCluster.getNamesystem().getBlockManager(), sps) {
-      @Override
-      public boolean isInSafeMode() {
-        return false;
-      }
-
-      @Override
-      public boolean isRunning() {
-        return true;
-      }
-    };
-    sps.init(ctxt);
-    sps.getStorageMovementQueue().activate();
-
-    INode rootINode = fsDir.getINode("/root");
-    hdfsCluster.getNamesystem().getBlockManager().getSPSManager()
-        .addPathId(rootINode.getId());
-
-    // Wait for thread to reach U.
-    Thread.sleep(1000);
-
-    dfs.delete(new Path("/root/D/L"), true);
-
-    assertTraversal(expectedTraverseOrder, fsDir, sps);
-    dfs.delete(new Path("/root"), true);
-  }
-
-  private void assertTraversal(List<String> expectedTraverseOrder,
-      FSDirectory fsDir, StoragePolicySatisfier sps)
-          throws InterruptedException {
-    // Remove 10 element and make queue free, So other traversing will start.
-    for (int i = 0; i < 10; i++) {
-      String path = expectedTraverseOrder.remove(0);
-      ItemInfo itemInfo = sps.getStorageMovementQueue().get();
-      if (itemInfo == null) {
-        continue;
-      }
-      Long trackId = itemInfo.getFile();
-      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) {
-      ItemInfo itemInfo = sps.getStorageMovementQueue().get();
-      if (itemInfo == null) {
-        continue;
-      }
-      Long trackId = itemInfo.getFile();
-      INode inode = fsDir.getInode(trackId);
-      assertTrue("Failed to traverse tree, expected " + path + " but got "
-          + inode.getFullPathName(), path.equals(inode.getFullPathName()));
-    }
-  }
-
-  /**
-   * 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.set(DFSConfigKeys
-          .DFS_STORAGE_POLICY_SATISFIER_RECHECK_TIMEOUT_MILLIS_KEY,
-          "3000");
-      config.set(DFSConfigKeys
-          .DFS_STORAGE_POLICY_SATISFIER_SELF_RETRY_TIMEOUT_MILLIS_KEY,
-          "5000");
-      config.setBoolean(DFSConfigKeys
-          .DFS_STORAGE_POLICY_SATISFIER_LOW_MAX_STREAMS_PREFERENCE_KEY,
-          false);
-
-      StorageType[][] storagetypes = new StorageType[][] {
-          {StorageType.ARCHIVE, StorageType.DISK},
-          {StorageType.ARCHIVE, StorageType.DISK}};
-
-      hdfsCluster = startCluster(config, storagetypes, 2, 2, CAPACITY);
-      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, NUM_OF_DATANODES, newtypes,
-          STORAGES_PER_DATANODE, 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, 60000, 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();
-    }
-  }
-
-  @Test(timeout = 300000)
-  public void testStoragePolicySatisfyPathStatus() throws Exception {
-    try {
-      config.set(DFSConfigKeys
-          .DFS_STORAGE_POLICY_SATISFIER_RECHECK_TIMEOUT_MILLIS_KEY,
-          "3000");
-      config.setBoolean(DFSConfigKeys
-          .DFS_STORAGE_POLICY_SATISFIER_LOW_MAX_STREAMS_PREFERENCE_KEY,
-          false);
-
-      StorageType[][] storagetypes = new StorageType[][] {
-          {StorageType.ARCHIVE, StorageType.DISK},
-          {StorageType.ARCHIVE, StorageType.DISK}};
-      hdfsCluster = startCluster(config, storagetypes, 2, 2, CAPACITY);
-      hdfsCluster.waitActive();
-      // BlockStorageMovementNeeded.setStatusClearanceElapsedTimeMs(200000);
-      dfs = hdfsCluster.getFileSystem();
-      Path filePath = new Path("/file");
-      DFSTestUtil.createFile(dfs, filePath, 1024, (short) 2,
-            0);
-      dfs.setStoragePolicy(filePath, "COLD");
-      dfs.satisfyStoragePolicy(filePath);
-      Thread.sleep(3000);
-      StoragePolicySatisfyPathStatus status = dfs.getClient()
-          .checkStoragePolicySatisfyPathStatus(filePath.toString());
-      Assert.assertTrue(
-          "Status should be IN_PROGRESS/SUCCESS, but status is " + status,
-          StoragePolicySatisfyPathStatus.IN_PROGRESS.equals(status)
-              || StoragePolicySatisfyPathStatus.SUCCESS.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);
-      BlockStorageMovementNeeded.setStatusClearanceElapsedTimeMs(1000);
-      // 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();
-    }
-  }
-
-  @Test(timeout = 300000)
-  public void testMaxRetryForFailedBlock() throws Exception {
-    try {
-      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 = startCluster(config, storagetypes, 2, 2, CAPACITY);
-      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
-    /*
-     *                           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();
-    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());
-      InternalDataNodeTestUtils.mockDatanodeBlkPinning(dn, true);
-      favoredNodesCount--;
-      if (favoredNodesCount <= 0) {
-        break; // marked favoredNodesCount number of pinned block location
-      }
-    }
-    return file1;
-  }
-
-  public void waitForAttemptedItems(long expectedBlkMovAttemptedCount,
-      int timeout) throws TimeoutException, InterruptedException {
-    BlockManager blockManager = hdfsCluster.getNamesystem().getBlockManager();
-    final StoragePolicySatisfier sps =
-        (StoragePolicySatisfier) blockManager.getSPSManager()
-        .getInternalSPSService();
-    GenericTestUtils.waitFor(new Supplier<Boolean>() {
-      @Override
-      public Boolean get() {
-        LOG.info("expectedAttemptedItemsCount={} actualAttemptedItemsCount={}",
-            expectedBlkMovAttemptedCount,
-            ((BlockStorageMovementAttemptedItems) (sps
-                .getAttemptedItemsMonitor())).getAttemptedItemsCount());
-        return ((BlockStorageMovementAttemptedItems) (sps
-            .getAttemptedItemsMonitor()))
-            .getAttemptedItemsCount() == expectedBlkMovAttemptedCount;
-      }
-    }, 100, timeout);
-  }
-
-  public void waitForBlocksMovementAttemptReport(
-      long expectedMovementFinishedBlocksCount, int timeout)
-          throws TimeoutException, InterruptedException {
-    BlockManager blockManager = hdfsCluster.getNamesystem().getBlockManager();
-    final StoragePolicySatisfier sps =
-        (StoragePolicySatisfier) blockManager.getSPSManager()
-        .getInternalSPSService();
-    GenericTestUtils.waitFor(new Supplier<Boolean>() {
-      @Override
-      public Boolean get() {
-        int actualCount = ((BlockStorageMovementAttemptedItems) (sps
-            .getAttemptedItemsMonitor())).getAttemptedItemsCount();
-        LOG.info("MovementFinishedBlocks: expectedCount={} actualCount={}",
-            expectedMovementFinishedBlocksCount, actualCount);
-        return actualCount
-            >= expectedMovementFinishedBlocksCount;
-      }
-    }, 100, timeout);
-  }
-
-  public 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),
-        replicatonFactor);
-    for (int i = 0; i < 1024; i++) {
-      out.write(i);
-    }
-    out.close();
-  }
-
-  private void startAdditionalDNs(final Configuration conf,
-      int newNodesRequired, int existingNodesNum, StorageType[][] newTypes,
-      int storagesPerDn, long nodeCapacity, final MiniDFSCluster cluster)
-          throws IOException {
-    long[][] capacities;
-    existingNodesNum += newNodesRequired;
-    capacities = new long[newNodesRequired][storagesPerDn];
-    for (int i = 0; i < newNodesRequired; i++) {
-      for (int j = 0; j < storagesPerDn; j++) {
-        capacities[i][j] = nodeCapacity;
-      }
-    }
-
-    cluster.startDataNodes(conf, newNodesRequired, newTypes, true, null, null,
-        null, capacities, null, false, false, false, null);
-    cluster.triggerHeartbeats();
-  }
-
-  public 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;
-  }
-
-  public void restartNamenode() throws IOException {
-    hdfsCluster.restartNameNodes();
-    hdfsCluster.waitActive();
-  }
-
-  /**
-   * Implementation of listener callback, where it collects all the sps move
-   * attempted blocks for assertion.
-   */
-  public static final class ExternalBlockMovementListener
-      implements BlockMovementListener {
-
-    private List<Block> actualBlockMovements = new ArrayList<>();
-
-    @Override
-    public void notifyMovementTriedBlocks(Block[] moveAttemptFinishedBlks) {
-      for (Block block : moveAttemptFinishedBlks) {
-        actualBlockMovements.add(block);
-      }
-      LOG.info("Movement attempted blocks:{}", actualBlockMovements);
-    }
-
-    public List<Block> getActualBlockMovements() {
-      return actualBlockMovements;
-    }
-
-    public void clear() {
-      actualBlockMovements.clear();
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/39ed3a66/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/sps/TestStoragePolicySatisfierWithStripedFile.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/sps/TestStoragePolicySatisfierWithStripedFile.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/sps/TestStoragePolicySatisfierWithStripedFile.java
index 8a25a5e..250e54b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/sps/TestStoragePolicySatisfierWithStripedFile.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/sps/TestStoragePolicySatisfierWithStripedFile.java
@@ -42,7 +42,9 @@ 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.protocol.HdfsConstants.StoragePolicySatisfierMode;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
+import org.apache.hadoop.hdfs.server.balancer.NameNodeConnector;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
+import org.apache.hadoop.hdfs.server.sps.ExternalSPSContext;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.junit.Assert;
 import org.junit.Before;
@@ -70,6 +72,9 @@ public class TestStoragePolicySatisfierWithStripedFile {
   private int cellSize;
   private int defaultStripeBlockSize;
   private Configuration conf;
+  private StoragePolicySatisfier sps;
+  private ExternalSPSContext ctxt;
+  private NameNodeConnector nnc;
 
   private ErasureCodingPolicy getEcPolicy() {
     return StripedFileTestUtil.getDefaultECPolicy();
@@ -87,7 +92,7 @@ public class TestStoragePolicySatisfierWithStripedFile {
     defaultStripeBlockSize = cellSize * stripesPerBlock;
     conf = new HdfsConfiguration();
     conf.set(DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_MODE_KEY,
-        StoragePolicySatisfierMode.INTERNAL.toString());
+        StoragePolicySatisfierMode.EXTERNAL.toString());
     // Reduced refresh cycle to update latest datanodes.
     conf.setLong(DFSConfigKeys.DFS_SPS_DATANODE_CACHE_REFRESH_INTERVAL_MS,
         1000);
@@ -102,8 +107,8 @@ public class TestStoragePolicySatisfierWithStripedFile {
    */
   @Test(timeout = 300000)
   public void testMoverWithFullStripe() throws Exception {
-    // start 10 datanodes
-    int numOfDatanodes = 10;
+    // start 11 datanodes
+    int numOfDatanodes = 11;
     int storagesPerDatanode = 2;
     long capacity = 20 * defaultStripeBlockSize;
     long[][] capacities = new long[numOfDatanodes][storagesPerDatanode];
@@ -122,6 +127,7 @@ public class TestStoragePolicySatisfierWithStripedFile {
             {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},
@@ -133,7 +139,7 @@ public class TestStoragePolicySatisfierWithStripedFile {
     HdfsAdmin hdfsAdmin = new HdfsAdmin(FileSystem.getDefaultUri(conf), conf);
     try {
       cluster.waitActive();
-
+      startSPS();
       DistributedFileSystem dfs = cluster.getFileSystem();
       dfs.enableErasureCodingPolicy(
           StripedFileTestUtil.getDefaultECPolicy().getName());
@@ -189,12 +195,12 @@ public class TestStoragePolicySatisfierWithStripedFile {
       LOG.info("Sets storage policy to COLD and invoked satisfyStoragePolicy");
       cluster.triggerHeartbeats();
 
-      waitForBlocksMovementAttemptReport(cluster, 9, 60000);
       // verify storage types and locations
       waitExpectedStorageType(cluster, fooFile, fileLen, StorageType.ARCHIVE, 9,
           9, 60000);
     } finally {
       cluster.shutdown();
+      sps.stopGracefully();
     }
   }
 
@@ -213,7 +219,7 @@ public class TestStoragePolicySatisfierWithStripedFile {
   public void testWhenOnlyFewTargetNodesAreAvailableToSatisfyStoragePolicy()
       throws Exception {
     // start 10 datanodes
-    int numOfDatanodes = 10;
+    int numOfDatanodes = 11;
     int storagesPerDatanode = 2;
     long capacity = 20 * defaultStripeBlockSize;
     long[][] capacities = new long[numOfDatanodes][storagesPerDatanode];
@@ -234,6 +240,7 @@ public class TestStoragePolicySatisfierWithStripedFile {
             {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}})
@@ -243,7 +250,7 @@ public class TestStoragePolicySatisfierWithStripedFile {
     HdfsAdmin hdfsAdmin = new HdfsAdmin(FileSystem.getDefaultUri(conf), conf);
     try {
       cluster.waitActive();
-
+      startSPS();
       DistributedFileSystem dfs = cluster.getFileSystem();
       dfs.enableErasureCodingPolicy(
           StripedFileTestUtil.getDefaultECPolicy().getName());
@@ -271,6 +278,7 @@ public class TestStoragePolicySatisfierWithStripedFile {
           Assert.assertEquals(StorageType.DISK, type);
         }
       }
+      Thread.sleep(5000);
       StripedFileTestUtil.verifyLocatedStripedBlocks(locatedBlocks,
           dataBlocks + parityBlocks);
 
@@ -296,13 +304,13 @@ public class TestStoragePolicySatisfierWithStripedFile {
       LOG.info("Sets storage policy to COLD and invoked satisfyStoragePolicy");
       cluster.triggerHeartbeats();
 
-      waitForBlocksMovementAttemptReport(cluster, 5, 60000);
-      waitForAttemptedItems(cluster, 1, 30000);
+      waitForAttemptedItems(1, 30000);
       // verify storage types and locations.
       waitExpectedStorageType(cluster, fooFile, fileLen, StorageType.ARCHIVE, 5,
           9, 60000);
     } finally {
       cluster.shutdown();
+      sps.stopGracefully();
     }
   }
 
@@ -352,6 +360,7 @@ public class TestStoragePolicySatisfierWithStripedFile {
         .build();
     try {
       cluster.waitActive();
+      startSPS();
       DistributedFileSystem fs = cluster.getFileSystem();
       fs.enableErasureCodingPolicy(
           StripedFileTestUtil.getDefaultECPolicy().getName());
@@ -393,6 +402,7 @@ public class TestStoragePolicySatisfierWithStripedFile {
           StorageType.ARCHIVE, 9, 9, 60000);
     } finally {
       cluster.shutdown();
+      sps.stopGracefully();
     }
   }
 
@@ -444,6 +454,7 @@ public class TestStoragePolicySatisfierWithStripedFile {
     HdfsAdmin hdfsAdmin = new HdfsAdmin(FileSystem.getDefaultUri(conf), conf);
     try {
       cluster.waitActive();
+      startSPS();
       DistributedFileSystem dfs = cluster.getFileSystem();
       dfs.enableErasureCodingPolicy(
           StripedFileTestUtil.getDefaultECPolicy().getName());
@@ -481,35 +492,25 @@ public class TestStoragePolicySatisfierWithStripedFile {
       LOG.info("Sets storage policy to COLD and invoked satisfyStoragePolicy");
       cluster.triggerHeartbeats();
 
-      waitForAttemptedItems(cluster, 1, 30000);
+      waitForAttemptedItems(1, 30000);
       // verify storage types and locations.
       waitExpectedStorageType(cluster, fooFile, fileLen, StorageType.DISK, 9, 9,
           60000);
-      waitForAttemptedItems(cluster, 1, 30000);
+      waitForAttemptedItems(1, 30000);
     } finally {
       cluster.shutdown();
+      sps.stopGracefully();
     }
   }
 
-  private void waitForAttemptedItems(MiniDFSCluster cluster,
-      long expectedBlkMovAttemptedCount, int timeout)
-          throws TimeoutException, InterruptedException {
-    BlockManager blockManager = cluster.getNamesystem().getBlockManager();
-    final StoragePolicySatisfier sps =
-        (StoragePolicySatisfier) blockManager
-        .getSPSManager().getInternalSPSService();
-    GenericTestUtils.waitFor(new Supplier<Boolean>() {
-      @Override
-      public Boolean get() {
-        LOG.info("expectedAttemptedItemsCount={} actualAttemptedItemsCount={}",
-            expectedBlkMovAttemptedCount,
-            ((BlockStorageMovementAttemptedItems) sps
-                .getAttemptedItemsMonitor()).getAttemptedItemsCount());
-        return ((BlockStorageMovementAttemptedItems) sps
-            .getAttemptedItemsMonitor())
-                .getAttemptedItemsCount() == expectedBlkMovAttemptedCount;
-      }
-    }, 100, timeout);
+  private void startSPS() throws IOException {
+    nnc = DFSTestUtil.getNameNodeConnector(conf,
+        HdfsServerConstants.MOVER_ID_PATH, 1, false);
+
+    sps = new StoragePolicySatisfier(conf);
+    ctxt = new ExternalSPSContext(sps, nnc);
+    sps.init(ctxt);
+    sps.start(true, StoragePolicySatisfierMode.EXTERNAL);
   }
 
   private static void initConfWithStripe(Configuration conf,
@@ -562,24 +563,18 @@ public class TestStoragePolicySatisfierWithStripedFile {
     }, 100, timeout);
   }
 
-  // Check whether the block movement attempt report has been arrived at the
-  // Namenode(SPS).
-  private void waitForBlocksMovementAttemptReport(MiniDFSCluster cluster,
-      long expectedMoveFinishedBlks, int timeout)
-          throws TimeoutException, InterruptedException {
-    BlockManager blockManager = cluster.getNamesystem().getBlockManager();
-    final StoragePolicySatisfier sps =
-        (StoragePolicySatisfier) blockManager.getSPSManager()
-        .getInternalSPSService();
+  private void waitForAttemptedItems(long expectedBlkMovAttemptedCount,
+      int timeout) throws TimeoutException, InterruptedException {
     GenericTestUtils.waitFor(new Supplier<Boolean>() {
       @Override
       public Boolean get() {
-        int actualCount = ((BlockStorageMovementAttemptedItems) (sps
-            .getAttemptedItemsMonitor())).getMovementFinishedBlocksCount();
-        LOG.info("MovementFinishedBlocks: expectedCount={} actualCount={}",
-            expectedMoveFinishedBlks,
-            actualCount);
-        return actualCount >= expectedMoveFinishedBlks;
+        LOG.info("expectedAttemptedItemsCount={} actualAttemptedItemsCount={}",
+            expectedBlkMovAttemptedCount,
+            ((BlockStorageMovementAttemptedItems) (sps
+                .getAttemptedItemsMonitor())).getAttemptedItemsCount());
+        return ((BlockStorageMovementAttemptedItems) (sps
+            .getAttemptedItemsMonitor()))
+                .getAttemptedItemsCount() == expectedBlkMovAttemptedCount;
       }
     }, 100, timeout);
   }


---------------------------------------------------------------------
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-13025. [SPS]: Implement a mechanism to scan the files for external SPS. Contributed by Uma Maheswara Rao G.

Posted by um...@apache.org.
HDFS-13025. [SPS]: Implement a mechanism to scan the files for external SPS. 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/3159b39c
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/3159b39c
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/3159b39c

Branch: refs/heads/HDFS-10285
Commit: 3159b39cf8ef704835325263154fb1a1cecc109d
Parents: 8d4f74e
Author: Rakesh Radhakrishnan <ra...@apache.org>
Authored: Tue Jan 23 20:09:26 2018 +0530
Committer: Uma Maheswara Rao Gangumalla <um...@apache.org>
Committed: Sun Aug 12 03:06:03 2018 -0700

----------------------------------------------------------------------
 .../sps/BlockStorageMovementNeeded.java         |  70 +++-
 .../hdfs/server/namenode/sps/Context.java       |   8 +
 .../IntraSPSNameNodeBlockMoveTaskHandler.java   |   2 +
 .../namenode/sps/IntraSPSNameNodeContext.java   |   7 +
 .../sps/IntraSPSNameNodeFileIdCollector.java    |   6 +-
 .../hdfs/server/namenode/sps/SPSService.java    |  10 +-
 .../namenode/sps/StoragePolicySatisfier.java    |   8 +-
 .../server/sps/ExternalSPSFileIDCollector.java  | 156 +++++++++
 .../hadoop/hdfs/server/sps/package-info.java    |  28 ++
 .../sps/TestStoragePolicySatisfier.java         | 323 ++++++++++---------
 .../sps/TestExternalStoragePolicySatisfier.java | 108 +++++++
 11 files changed, 556 insertions(+), 170 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/3159b39c/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/BlockStorageMovementNeeded.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/BlockStorageMovementNeeded.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/BlockStorageMovementNeeded.java
index 39a0051..b141502 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/BlockStorageMovementNeeded.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/BlockStorageMovementNeeded.java
@@ -97,23 +97,53 @@ public class BlockStorageMovementNeeded {
   }
 
   /**
-   * Add the itemInfo to tracking list for which storage movement
-   * expected if necessary.
+   * Add the itemInfo list to tracking list for which storage movement expected
+   * if necessary.
+   *
    * @param startId
-   *            - start id
+   *          - start id
    * @param itemInfoList
-   *            - List of child in the directory
+   *          - List of child in the directory
+   * @param scanCompleted
+   *          -Indicates whether the start id directory has no more elements to
+   *          scan.
    */
   @VisibleForTesting
-  public synchronized void addAll(long startId,
-      List<ItemInfo> itemInfoList, boolean scanCompleted) {
+  public synchronized void addAll(long startId, List<ItemInfo> itemInfoList,
+      boolean scanCompleted) {
     storageMovementNeeded.addAll(itemInfoList);
+    updatePendingDirScanStats(startId, itemInfoList.size(), scanCompleted);
+  }
+
+  /**
+   * Add the itemInfo to tracking list for which storage movement expected if
+   * necessary.
+   *
+   * @param itemInfoList
+   *          - List of child in the directory
+   * @param scanCompleted
+   *          -Indicates whether the ItemInfo start id directory has no more
+   *          elements to scan.
+   */
+  @VisibleForTesting
+  public synchronized void add(ItemInfo itemInfo, boolean scanCompleted) {
+    storageMovementNeeded.add(itemInfo);
+    // This represents sps start id is file, so no need to update pending dir
+    // stats.
+    if (itemInfo.getStartId() == itemInfo.getFileId()) {
+      return;
+    }
+    updatePendingDirScanStats(itemInfo.getStartId(), 1, scanCompleted);
+  }
+
+  private void updatePendingDirScanStats(long startId, int numScannedFiles,
+      boolean scanCompleted) {
     DirPendingWorkInfo pendingWork = pendingWorkForDirectory.get(startId);
     if (pendingWork == null) {
       pendingWork = new DirPendingWorkInfo();
       pendingWorkForDirectory.put(startId, pendingWork);
     }
-    pendingWork.addPendingWorkCount(itemInfoList.size());
+    pendingWork.addPendingWorkCount(numScannedFiles);
     if (scanCompleted) {
       pendingWork.markScanCompleted();
     }
@@ -250,13 +280,15 @@ public class BlockStorageMovementNeeded {
 
     @Override
     public void run() {
-      LOG.info("Starting FileInodeIdCollector!.");
+      LOG.info("Starting SPSPathIdProcessor!.");
       long lastStatusCleanTime = 0;
+      Long startINodeId = null;
       while (ctxt.isRunning()) {
-        LOG.info("Running FileInodeIdCollector!.");
         try {
           if (!ctxt.isInSafeMode()) {
-            Long startINodeId = ctxt.getNextSPSPathId();
+            if (startINodeId == null) {
+              startINodeId = ctxt.getNextSPSPathId();
+            } // else same id will be retried
             if (startINodeId == null) {
               // Waiting for SPS path
               Thread.sleep(3000);
@@ -281,9 +313,18 @@ public class BlockStorageMovementNeeded {
               lastStatusCleanTime = Time.monotonicNow();
               cleanSpsStatus();
             }
+            startINodeId = null; // Current inode id successfully scanned.
           }
         } catch (Throwable t) {
-          LOG.warn("Exception while loading inodes to satisfy the policy", t);
+          String reClass = t.getClass().getName();
+          if (InterruptedException.class.getName().equals(reClass)) {
+            LOG.info("SPSPathIdProcessor thread is interrupted. Stopping..");
+            Thread.currentThread().interrupt();
+            break;
+          }
+          LOG.warn("Exception while scanning file inodes to satisfy the policy",
+              t);
+          // TODO: may be we should retry the current inode id?
         }
       }
     }
@@ -426,4 +467,11 @@ public class BlockStorageMovementNeeded {
   public static long getStatusClearanceElapsedTimeMs() {
     return statusClearanceElapsedTimeMs;
   }
+
+  public void markScanCompletedForDir(Long inodeId) {
+    DirPendingWorkInfo pendingWork = pendingWorkForDirectory.get(inodeId);
+    if (pendingWork != null) {
+      pendingWork.markScanCompleted();
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3159b39c/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/Context.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/Context.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/Context.java
index b7053b9..f103dfe 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/Context.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/Context.java
@@ -167,4 +167,12 @@ public interface Context {
    */
   void removeAllSPSPathIds();
 
+  /**
+   * Gets the file path for a given inode id.
+   *
+   * @param inodeId
+   *          - path inode id.
+   */
+  String getFilePath(Long inodeId);
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3159b39c/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/IntraSPSNameNodeBlockMoveTaskHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/IntraSPSNameNodeBlockMoveTaskHandler.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/IntraSPSNameNodeBlockMoveTaskHandler.java
index 1da4af9..b27e8c9 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/IntraSPSNameNodeBlockMoveTaskHandler.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/IntraSPSNameNodeBlockMoveTaskHandler.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.hdfs.server.namenode.sps;
 
 import java.io.IOException;
 
+import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
 import org.apache.hadoop.hdfs.server.namenode.Namesystem;
@@ -29,6 +30,7 @@ import org.apache.hadoop.hdfs.server.protocol.BlockStorageMovementCommand.BlockM
  * This class handles the internal SPS block movements. This will assign block
  * movement tasks to target datanode descriptors.
  */
+@InterfaceAudience.Private
 public class IntraSPSNameNodeBlockMoveTaskHandler
     implements BlockMoveTaskHandler {
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3159b39c/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/IntraSPSNameNodeContext.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/IntraSPSNameNodeContext.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/IntraSPSNameNodeContext.java
index cef26ed..aed684a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/IntraSPSNameNodeContext.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/IntraSPSNameNodeContext.java
@@ -22,6 +22,7 @@ import static org.apache.hadoop.hdfs.server.common.HdfsServerConstants.XATTR_SAT
 import java.io.IOException;
 
 import org.apache.commons.lang.StringUtils;
+import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.fs.ParentNotDirectoryException;
 import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.fs.UnresolvedLinkException;
@@ -46,6 +47,7 @@ import org.slf4j.LoggerFactory;
  * are expecting to change its storages and assigning the block storage
  * movements to satisfy the storage policy.
  */
+@InterfaceAudience.Private
 public class IntraSPSNameNodeContext implements Context {
   private static final Logger LOG = LoggerFactory
       .getLogger(IntraSPSNameNodeContext.class);
@@ -195,4 +197,9 @@ public class IntraSPSNameNodeContext implements Context {
   public void removeAllSPSPathIds() {
     blockManager.removeAllSPSPathIds();
   }
+
+  @Override
+  public String getFilePath(Long inodeId) {
+    return namesystem.getFilePath(inodeId);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3159b39c/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/IntraSPSNameNodeFileIdCollector.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/IntraSPSNameNodeFileIdCollector.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/IntraSPSNameNodeFileIdCollector.java
index c6834c1..f7cd754 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/IntraSPSNameNodeFileIdCollector.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/IntraSPSNameNodeFileIdCollector.java
@@ -22,6 +22,7 @@ import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
 
+import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.server.namenode.FSDirectory;
@@ -32,6 +33,7 @@ import org.apache.hadoop.hdfs.server.namenode.INode;
  * A specific implementation for scanning the directory with Namenode internal
  * Inode structure and collects the file ids under the given directory ID.
  */
+@InterfaceAudience.Private
 public class IntraSPSNameNodeFileIdCollector extends FSTreeTraverser
     implements FileIdCollector {
   private int maxQueueLimitToScan;
@@ -131,12 +133,12 @@ public class IntraSPSNameNodeFileIdCollector extends FSTreeTraverser
       } else {
 
         readLock();
-        // NOTE: this lock will not be held until full directory scanning. It is
+        // NOTE: this lock will not be held for full directory scanning. It is
         // basically a sliced locking. Once it collects a batch size( at max the
         // size of maxQueueLimitToScan (default 1000)) file ids, then it will
         // unlock and submits the current batch to SPSService. Once
         // service.processingQueueSize() shows empty slots, then lock will be
-        // resumed and scan also will be resumed. This logic was re-used from
+        // re-acquired and scan will be resumed. This logic was re-used from
         // EDEK feature.
         try {
           traverseDir(startInode.asDirectory(), startINodeId,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3159b39c/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/SPSService.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/SPSService.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/SPSService.java
index 6d85ea6..d74e391 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/SPSService.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/SPSService.java
@@ -80,7 +80,7 @@ public interface SPSService {
    *
    * @param itemInfo
    */
-  void addFileIdToProcess(ItemInfo itemInfo);
+  void addFileIdToProcess(ItemInfo itemInfo, boolean scanCompleted);
 
   /**
    * Adds all the Item information(file id etc) to processing queue.
@@ -104,4 +104,12 @@ public interface SPSService {
    * @return the configuration.
    */
   Configuration getConf();
+
+  /**
+   * Marks the scanning of directory if finished.
+   *
+   * @param inodeId
+   *          - directory inode id.
+   */
+  void markScanCompletedForPath(Long inodeId);
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3159b39c/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/StoragePolicySatisfier.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/StoragePolicySatisfier.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/StoragePolicySatisfier.java
index 28c1372..aafdc65 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/StoragePolicySatisfier.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/StoragePolicySatisfier.java
@@ -563,7 +563,6 @@ public class StoragePolicySatisfier implements SPSService, Runnable {
                 chosenTarget.storageType, blockMovingInfos);
           }
           expected.remove(chosenTarget.storageType);
-          // TODO: We can increment scheduled block count for this node?
         }
       }
       // To avoid choosing this excludeNodes as targets later
@@ -924,7 +923,7 @@ public class StoragePolicySatisfier implements SPSService, Runnable {
   }
 
   @Override
-  public void addFileIdToProcess(ItemInfo trackInfo) {
+  public void addFileIdToProcess(ItemInfo trackInfo, boolean scanCompleted) {
     storageMovementNeeded.add(trackInfo);
   }
 
@@ -948,4 +947,9 @@ public class StoragePolicySatisfier implements SPSService, Runnable {
   public BlockStorageMovementNeeded getStorageMovementQueue() {
     return storageMovementNeeded;
   }
+
+  @Override
+  public void markScanCompletedForPath(Long inodeId) {
+    getStorageMovementQueue().markScanCompletedForDir(inodeId);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3159b39c/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/sps/ExternalSPSFileIDCollector.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/sps/ExternalSPSFileIDCollector.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/sps/ExternalSPSFileIDCollector.java
new file mode 100644
index 0000000..597a7d3
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/sps/ExternalSPSFileIDCollector.java
@@ -0,0 +1,156 @@
+package org.apache.hadoop.hdfs.server.sps;
+/**
+ * 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.
+ */
+
+import java.io.IOException;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
+import org.apache.hadoop.hdfs.protocol.DirectoryListing;
+import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
+import org.apache.hadoop.hdfs.server.namenode.sps.Context;
+import org.apache.hadoop.hdfs.server.namenode.sps.FileIdCollector;
+import org.apache.hadoop.hdfs.server.namenode.sps.ItemInfo;
+import org.apache.hadoop.hdfs.server.namenode.sps.SPSService;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * This class is to scan the paths recursively. If file is directory, then it
+ * will scan for files recursively. If the file is non directory, then it will
+ * just submit the same file to process.
+ */
+@InterfaceAudience.Private
+public class ExternalSPSFileIDCollector implements FileIdCollector {
+  public static final Logger LOG =
+      LoggerFactory.getLogger(ExternalSPSFileIDCollector.class);
+  private Context cxt;
+  private DistributedFileSystem dfs;
+  private SPSService service;
+  private int maxQueueLimitToScan;
+
+  public ExternalSPSFileIDCollector(Context cxt, SPSService service,
+      int batchSize) {
+    this.cxt = cxt;
+    this.service = service;
+    this.maxQueueLimitToScan = service.getConf().getInt(
+        DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_QUEUE_LIMIT_KEY,
+        DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_QUEUE_LIMIT_DEFAULT);
+    try {
+      // TODO: probably we could get this dfs from external context? but this is
+      // too specific to external.
+      dfs = getFS(service.getConf());
+    } catch (IOException e) {
+      LOG.error("Unable to get the filesystem. Make sure Namenode running and "
+          + "configured namenode address is correct.", e);
+    }
+  }
+
+  private DistributedFileSystem getFS(Configuration conf) throws IOException {
+    return (DistributedFileSystem) FileSystem
+        .get(FileSystem.getDefaultUri(conf), conf);
+  }
+
+  /**
+   * Recursively scan the given path and add the file info to SPS service for
+   * processing.
+   */
+  private void processPath(long startID, String fullPath) {
+    for (byte[] lastReturnedName = HdfsFileStatus.EMPTY_NAME;;) {
+      final DirectoryListing children;
+      try {
+        children = dfs.getClient().listPaths(fullPath, lastReturnedName, false);
+      } catch (IOException e) {
+        LOG.warn("Failed to list directory " + fullPath
+            + ". Ignore the directory and continue.", e);
+        return;
+      }
+      if (children == null) {
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("The scanning start dir/sub dir " + fullPath
+              + " does not have childrens.");
+        }
+        return;
+      }
+
+      for (HdfsFileStatus child : children.getPartialListing()) {
+        if (child.isFile()) {
+          service.addFileIdToProcess(new ItemInfo(startID, child.getFileId()),
+              false);
+          checkProcessingQueuesFree();
+        } else {
+          String fullPathStr = child.getFullName(fullPath);
+          if (child.isDirectory()) {
+            if (!fullPathStr.endsWith(Path.SEPARATOR)) {
+              fullPathStr = fullPathStr + Path.SEPARATOR;
+            }
+            processPath(startID, fullPathStr);
+          }
+        }
+      }
+
+      if (children.hasMore()) {
+        lastReturnedName = children.getLastName();
+      } else {
+        return;
+      }
+    }
+  }
+
+  private void checkProcessingQueuesFree() {
+    int remainingCapacity = remainingCapacity();
+    // wait for queue to be free
+    while (remainingCapacity <= 0) {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Waiting for storageMovementNeeded queue to be free!");
+      }
+      try {
+        Thread.sleep(5000);
+      } catch (InterruptedException e) {
+        Thread.currentThread().interrupt();
+      }
+      remainingCapacity = remainingCapacity();
+    }
+  }
+
+  /**
+   * Returns queue remaining capacity.
+   */
+  public int remainingCapacity() {
+    int size = service.processingQueueSize();
+    if (size >= maxQueueLimitToScan) {
+      return 0;
+    } else {
+      return (maxQueueLimitToScan - size);
+    }
+  }
+
+  @Override
+  public void scanAndCollectFileIds(Long inodeId) throws IOException {
+    if (dfs == null) {
+      dfs = getFS(service.getConf());
+    }
+    processPath(inodeId, cxt.getFilePath(inodeId));
+    service.markScanCompletedForPath(inodeId);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3159b39c/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/sps/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/sps/package-info.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/sps/package-info.java
new file mode 100644
index 0000000..f705df2
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/sps/package-info.java
@@ -0,0 +1,28 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+/**
+ * This package provides a mechanism for satisfying the storage policy of a
+ * path.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+package org.apache.hadoop.hdfs.server.sps;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3159b39c/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/sps/TestStoragePolicySatisfier.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/sps/TestStoragePolicySatisfier.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/sps/TestStoragePolicySatisfier.java
index 9354044..e0bf410 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/sps/TestStoragePolicySatisfier.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/sps/TestStoragePolicySatisfier.java
@@ -71,6 +71,7 @@ import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.test.GenericTestUtils.LogCapturer;
 import org.junit.After;
 import org.junit.Assert;
+import org.junit.Before;
 import org.junit.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -93,18 +94,41 @@ public class TestStoragePolicySatisfier {
   private static final String COLD = "COLD";
   private static final Logger LOG =
       LoggerFactory.getLogger(TestStoragePolicySatisfier.class);
-  private final Configuration config = new HdfsConfiguration();
+  private Configuration config = null;
   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 dfs = null;
-  private static final int DEFAULT_BLOCK_SIZE = 1024;
+  public static final int NUM_OF_DATANODES = 3;
+  public static final int STORAGES_PER_DATANODE = 2;
+  public static final long CAPACITY = 2 * 256 * 1024 * 1024;
+  public static final String FILE = "/testMoveWhenStoragePolicyNotSatisfying";
+  public static final int DEFAULT_BLOCK_SIZE = 1024;
+
+  /**
+   * Sets hdfs cluster.
+   */
+  public void setCluster(MiniDFSCluster cluster) {
+    this.hdfsCluster = cluster;
+  }
+
+  /**
+   * @return conf.
+   */
+  public Configuration getConf() {
+    return this.config;
+  }
+
+  /**
+   * Gets distributed file system.
+   *
+   * @throws IOException
+   */
+  public void getFS() throws IOException {
+    this.dfs = hdfsCluster.getFileSystem();
+  }
 
   @After
   public void shutdownCluster() {
@@ -113,14 +137,19 @@ public class TestStoragePolicySatisfier {
     }
   }
 
-  private void createCluster() throws IOException {
+  public 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();
-    writeContent(file);
+    hdfsCluster = startCluster(config, allDiskTypes, NUM_OF_DATANODES,
+        STORAGES_PER_DATANODE, CAPACITY);
+    getFS();
+    writeContent(FILE);
+  }
+
+  @Before
+  public void setUp() {
+    config = new HdfsConfiguration();
   }
 
   @Test(timeout = 300000)
@@ -137,19 +166,19 @@ public class TestStoragePolicySatisfier {
 
   private void doTestWhenStoragePolicySetToCOLD() throws Exception {
     // 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}};
-    startAdditionalDNs(config, 3, numOfDatanodes, newtypes,
-        storagesPerDatanode, capacity, hdfsCluster);
+    startAdditionalDNs(config, 3, NUM_OF_DATANODES, newtypes,
+        STORAGES_PER_DATANODE, CAPACITY, hdfsCluster);
 
     hdfsCluster.triggerHeartbeats();
-    dfs.satisfyStoragePolicy(new Path(file));
+    dfs.satisfyStoragePolicy(new Path(FILE));
     // Wait till namenode notified about the block location details
-    DFSTestUtil.waitExpectedStorageType(file, StorageType.ARCHIVE, 3, 35000,
+    DFSTestUtil.waitExpectedStorageType(FILE, StorageType.ARCHIVE, 3, 35000,
         dfs);
   }
 
@@ -159,7 +188,7 @@ public class TestStoragePolicySatisfier {
     try {
       createCluster();
       // Change policy to ALL_SSD
-      dfs.setStoragePolicy(new Path(file), "ALL_SSD");
+      dfs.setStoragePolicy(new Path(FILE), "ALL_SSD");
 
       StorageType[][] newtypes =
           new StorageType[][]{{StorageType.SSD, StorageType.DISK},
@@ -168,14 +197,13 @@ public class TestStoragePolicySatisfier {
 
       // Making sure SDD based nodes added to cluster. Adding SSD based
       // datanodes.
-      startAdditionalDNs(config, 3, numOfDatanodes, newtypes,
-          storagesPerDatanode, capacity, hdfsCluster);
-      dfs.satisfyStoragePolicy(new Path(file));
+      startAdditionalDNs(config, 3, NUM_OF_DATANODES, newtypes,
+          STORAGES_PER_DATANODE, CAPACITY, hdfsCluster);
+      dfs.satisfyStoragePolicy(new Path(FILE));
       hdfsCluster.triggerHeartbeats();
       // Wait till StorgePolicySatisfier Identified that block to move to SSD
       // areas
-      DFSTestUtil.waitExpectedStorageType(
-          file, StorageType.SSD, 3, 30000, dfs);
+      DFSTestUtil.waitExpectedStorageType(FILE, StorageType.SSD, 3, 30000, dfs);
     } finally {
       shutdownCluster();
     }
@@ -187,23 +215,22 @@ public class TestStoragePolicySatisfier {
     try {
       createCluster();
       // Change policy to ONE_SSD
-      dfs.setStoragePolicy(new Path(file), ONE_SSD);
+      dfs.setStoragePolicy(new Path(FILE), ONE_SSD);
 
       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);
-      dfs.satisfyStoragePolicy(new Path(file));
+      startAdditionalDNs(config, 1, NUM_OF_DATANODES, newtypes,
+          STORAGES_PER_DATANODE, CAPACITY, hdfsCluster);
+      dfs.satisfyStoragePolicy(new Path(FILE));
       hdfsCluster.triggerHeartbeats();
       // Wait till StorgePolicySatisfier Identified that block to move to SSD
       // areas
-      DFSTestUtil.waitExpectedStorageType(
-          file, StorageType.SSD, 1, 30000, dfs);
-      DFSTestUtil.waitExpectedStorageType(
-          file, StorageType.DISK, 2, 30000, dfs);
+      DFSTestUtil.waitExpectedStorageType(FILE, StorageType.SSD, 1, 30000, dfs);
+      DFSTestUtil.waitExpectedStorageType(FILE, StorageType.DISK, 2, 30000,
+          dfs);
     } finally {
       shutdownCluster();
     }
@@ -218,23 +245,22 @@ public class TestStoragePolicySatisfier {
     try {
       createCluster();
       // Change policy to ONE_SSD
-      dfs.setStoragePolicy(new Path(file), ONE_SSD);
+      dfs.setStoragePolicy(new Path(FILE), ONE_SSD);
 
       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);
-      dfs.satisfyStoragePolicy(new Path(file));
+      startAdditionalDNs(config, 1, NUM_OF_DATANODES, newtypes,
+          STORAGES_PER_DATANODE, CAPACITY, hdfsCluster);
+      dfs.satisfyStoragePolicy(new Path(FILE));
       hdfsCluster.triggerHeartbeats();
 
       // Wait till the block is moved to SSD areas
-      DFSTestUtil.waitExpectedStorageType(
-          file, StorageType.SSD, 1, 30000, dfs);
-      DFSTestUtil.waitExpectedStorageType(
-          file, StorageType.DISK, 2, 30000, dfs);
+      DFSTestUtil.waitExpectedStorageType(FILE, StorageType.SSD, 1, 30000, dfs);
+      DFSTestUtil.waitExpectedStorageType(FILE, StorageType.DISK, 2, 30000,
+          dfs);
 
       waitForBlocksMovementAttemptReport(1, 30000);
     } finally {
@@ -251,7 +277,7 @@ public class TestStoragePolicySatisfier {
     try {
       createCluster();
       List<String> files = new ArrayList<>();
-      files.add(file);
+      files.add(FILE);
 
       // Creates 4 more files. Send all of them for satisfying the storage
       // policy together.
@@ -271,8 +297,8 @@ public class TestStoragePolicySatisfier {
 
       // Making sure SDD based nodes added to cluster. Adding SSD based
       // datanodes.
-      startAdditionalDNs(config, 1, numOfDatanodes, newtypes,
-          storagesPerDatanode, capacity, hdfsCluster);
+      startAdditionalDNs(config, 1, NUM_OF_DATANODES, newtypes,
+          STORAGES_PER_DATANODE, CAPACITY, hdfsCluster);
       hdfsCluster.triggerHeartbeats();
 
       for (String fileName : files) {
@@ -300,21 +326,21 @@ 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.DISK, StorageType.ARCHIVE},
               {StorageType.DISK, StorageType.ARCHIVE},
               {StorageType.DISK, StorageType.ARCHIVE}};
-      startAdditionalDNs(config, 3, numOfDatanodes, newtypes,
-          storagesPerDatanode, capacity, hdfsCluster);
+      startAdditionalDNs(config, 3, NUM_OF_DATANODES, newtypes,
+          STORAGES_PER_DATANODE, CAPACITY, hdfsCluster);
 
-      hdfsAdmin.satisfyStoragePolicy(new Path(file));
+      hdfsAdmin.satisfyStoragePolicy(new Path(FILE));
 
       hdfsCluster.triggerHeartbeats();
       // Wait till namenode notified about the block location details
-      DFSTestUtil.waitExpectedStorageType(
-          file, StorageType.ARCHIVE, 3, 30000, dfs);
+      DFSTestUtil.waitExpectedStorageType(FILE, StorageType.ARCHIVE, 3, 30000,
+          dfs);
     } finally {
       shutdownCluster();
     }
@@ -344,8 +370,8 @@ public class TestStoragePolicySatisfier {
 
       StorageType[][] newtypes =
           new StorageType[][]{{StorageType.SSD, StorageType.DISK}};
-      startAdditionalDNs(config, 1, numOfDatanodes, newtypes,
-          storagesPerDatanode, capacity, hdfsCluster);
+      startAdditionalDNs(config, 1, NUM_OF_DATANODES, newtypes,
+          STORAGES_PER_DATANODE, CAPACITY, hdfsCluster);
 
       hdfsAdmin.satisfyStoragePolicy(new Path(subDir));
 
@@ -384,11 +410,11 @@ public class TestStoragePolicySatisfier {
           new HdfsAdmin(FileSystem.getDefaultUri(config), config);
 
       try {
-        hdfsAdmin.satisfyStoragePolicy(new Path(file));
+        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));
+            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.",
@@ -409,17 +435,17 @@ public class TestStoragePolicySatisfier {
       }
 
       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));
+        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);
+                + "satisfying storage policy.", FILE),
+            e);
       }
     } finally {
       shutdownCluster();
@@ -446,23 +472,23 @@ public class TestStoragePolicySatisfier {
     try {
       createCluster();
       // Change policy to COLD
-      dfs.setStoragePolicy(new Path(file), COLD);
+      dfs.setStoragePolicy(new Path(FILE), COLD);
 
       StorageType[][] newtypes =
           new StorageType[][]{{StorageType.ARCHIVE, StorageType.ARCHIVE}};
 
       // Adding ARCHIVE based datanodes.
-      startAdditionalDNs(config, 1, numOfDatanodes, newtypes,
-          storagesPerDatanode, capacity, hdfsCluster);
+      startAdditionalDNs(config, 1, NUM_OF_DATANODES, newtypes,
+          STORAGES_PER_DATANODE, CAPACITY, hdfsCluster);
 
-      dfs.satisfyStoragePolicy(new Path(file));
+      dfs.satisfyStoragePolicy(new Path(FILE));
       hdfsCluster.triggerHeartbeats();
       // Wait till StorgePolicySatisfier identified that block to move to
       // ARCHIVE area.
-      DFSTestUtil.waitExpectedStorageType(
-          file, StorageType.ARCHIVE, 1, 30000, dfs);
-      DFSTestUtil.waitExpectedStorageType(
-          file, StorageType.DISK, 2, 30000, dfs);
+      DFSTestUtil.waitExpectedStorageType(FILE, StorageType.ARCHIVE, 1, 30000,
+          dfs);
+      DFSTestUtil.waitExpectedStorageType(FILE, StorageType.DISK, 2, 30000,
+          dfs);
 
       waitForBlocksMovementAttemptReport(1, 30000);
     } finally {
@@ -489,22 +515,22 @@ public class TestStoragePolicySatisfier {
     try {
       createCluster();
       // Change policy to COLD
-      dfs.setStoragePolicy(new Path(file), COLD);
+      dfs.setStoragePolicy(new Path(FILE), COLD);
 
       StorageType[][] newtypes =
           new StorageType[][]{{StorageType.DISK, StorageType.DISK}};
       // Adding DISK based datanodes
-      startAdditionalDNs(config, 1, numOfDatanodes, newtypes,
-          storagesPerDatanode, capacity, hdfsCluster);
+      startAdditionalDNs(config, 1, NUM_OF_DATANODES, newtypes,
+          STORAGES_PER_DATANODE, CAPACITY, hdfsCluster);
 
-      dfs.satisfyStoragePolicy(new Path(file));
+      dfs.satisfyStoragePolicy(new Path(FILE));
       hdfsCluster.triggerHeartbeats();
 
       // 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);
+      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);
@@ -628,8 +654,8 @@ public class TestStoragePolicySatisfier {
               {StorageType.ARCHIVE, StorageType.ARCHIVE},
               {StorageType.ARCHIVE, StorageType.ARCHIVE}};
       // Adding DISK based datanodes
-      startAdditionalDNs(config, 3, numOfDatanodes, newtypes,
-          storagesPerDatanode, capacity, hdfsCluster);
+      startAdditionalDNs(config, 3, NUM_OF_DATANODES, newtypes,
+          STORAGES_PER_DATANODE, CAPACITY, hdfsCluster);
 
       dfs.satisfyStoragePolicy(new Path(file1));
       hdfsCluster.triggerHeartbeats();
@@ -682,21 +708,21 @@ public class TestStoragePolicySatisfier {
               {StorageType.DISK, StorageType.DISK},
               {StorageType.DISK, StorageType.ARCHIVE}};
       hdfsCluster = startCluster(config, allDiskTypes, numOfDns,
-          storagesPerDatanode, capacity);
+          STORAGES_PER_DATANODE, CAPACITY);
       dfs = hdfsCluster.getFileSystem();
-      writeContent(file, (short) 5);
+      writeContent(FILE, (short) 5);
 
       // Change policy to COLD
-      dfs.setStoragePolicy(new Path(file), COLD);
+      dfs.setStoragePolicy(new Path(FILE), COLD);
 
-      dfs.satisfyStoragePolicy(new Path(file));
+      dfs.satisfyStoragePolicy(new Path(FILE));
       hdfsCluster.triggerHeartbeats();
       // Wait till StorgePolicySatisfier identified that block to move to
       // ARCHIVE area.
-      DFSTestUtil.waitExpectedStorageType(
-          file, StorageType.ARCHIVE, 2, 30000, dfs);
-      DFSTestUtil.waitExpectedStorageType(
-          file, StorageType.DISK, 3, 30000, dfs);
+      DFSTestUtil.waitExpectedStorageType(FILE, StorageType.ARCHIVE, 2, 30000,
+          dfs);
+      DFSTestUtil.waitExpectedStorageType(FILE, StorageType.DISK, 3, 30000,
+          dfs);
 
       waitForBlocksMovementAttemptReport(1, 30000);
     } finally {
@@ -720,20 +746,19 @@ public class TestStoragePolicySatisfier {
     config.setBoolean(DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_ENABLED_KEY,
         true);
     try {
-      hdfsCluster = startCluster(config, diskTypes, numOfDatanodes,
-          storagesPerDatanode, capacity);
+      hdfsCluster = startCluster(config, diskTypes, NUM_OF_DATANODES,
+          STORAGES_PER_DATANODE, CAPACITY);
       dfs = hdfsCluster.getFileSystem();
-      writeContent(file);
+      writeContent(FILE);
 
       // Change policy to ONE_SSD
-      dfs.setStoragePolicy(new Path(file), ONE_SSD);
+      dfs.setStoragePolicy(new Path(FILE), ONE_SSD);
 
-      dfs.satisfyStoragePolicy(new Path(file));
+      dfs.satisfyStoragePolicy(new Path(FILE));
       hdfsCluster.triggerHeartbeats();
-      DFSTestUtil.waitExpectedStorageType(
-          file, StorageType.SSD, 1, 30000, dfs);
-      DFSTestUtil.waitExpectedStorageType(
-          file, StorageType.DISK, 2, 30000, dfs);
+      DFSTestUtil.waitExpectedStorageType(FILE, StorageType.SSD, 1, 30000, dfs);
+      DFSTestUtil.waitExpectedStorageType(FILE, StorageType.DISK, 2, 30000,
+          dfs);
 
     } finally {
       shutdownCluster();
@@ -760,19 +785,19 @@ public class TestStoragePolicySatisfier {
         true);
     try {
       hdfsCluster = startCluster(config, diskTypes, diskTypes.length,
-          storagesPerDatanode, capacity);
+          STORAGES_PER_DATANODE, CAPACITY);
       dfs = hdfsCluster.getFileSystem();
-      writeContent(file);
+      writeContent(FILE);
 
       // Change policy to WARM
-      dfs.setStoragePolicy(new Path(file), "WARM");
-      dfs.satisfyStoragePolicy(new Path(file));
+      dfs.setStoragePolicy(new Path(FILE), "WARM");
+      dfs.satisfyStoragePolicy(new Path(FILE));
       hdfsCluster.triggerHeartbeats();
 
-      DFSTestUtil.waitExpectedStorageType(
-          file, StorageType.DISK, 1, 30000, dfs);
-      DFSTestUtil.waitExpectedStorageType(
-          file, StorageType.ARCHIVE, 2, 30000, dfs);
+      DFSTestUtil.waitExpectedStorageType(FILE, StorageType.DISK, 1, 30000,
+          dfs);
+      DFSTestUtil.waitExpectedStorageType(FILE, StorageType.ARCHIVE, 2, 30000,
+          dfs);
     } finally {
       shutdownCluster();
     }
@@ -794,31 +819,31 @@ public class TestStoragePolicySatisfier {
       config.setBoolean(DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_ENABLED_KEY,
           true);
       hdfsCluster = startCluster(config, diskTypes, diskTypes.length,
-          storagesPerDatanode, capacity);
+          STORAGES_PER_DATANODE, CAPACITY);
       dfs = hdfsCluster.getFileSystem();
       // 1. Write two replica on disk
-      DFSTestUtil.createFile(dfs, new Path(file), DEFAULT_BLOCK_SIZE,
+      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");
+      dfs.setStoragePolicy(new Path(FILE), "COLD");
 
       // 3.Change replication factor to 3.
-      dfs.setReplication(new Path(file), (short) 3);
+      dfs.setReplication(new Path(FILE), (short) 3);
 
-      DFSTestUtil
-          .waitExpectedStorageType(file, StorageType.DISK, 2, 30000, dfs);
-      DFSTestUtil.waitExpectedStorageType(file, StorageType.ARCHIVE, 1, 30000,
+      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");
+      dfs.setStoragePolicy(new Path(FILE), "HOT");
 
       // 4. Satisfy the policy.
-      dfs.satisfyStoragePolicy(new Path(file));
+      dfs.satisfyStoragePolicy(new Path(FILE));
 
       // 5. Block should move successfully .
-      DFSTestUtil
-          .waitExpectedStorageType(file, StorageType.DISK, 3, 30000, dfs);
+      DFSTestUtil.waitExpectedStorageType(FILE, StorageType.DISK, 3, 30000,
+          dfs);
     } finally {
       shutdownCluster();
     }
@@ -840,13 +865,13 @@ public class TestStoragePolicySatisfier {
         true);
     long dnCapacity = 1024 * DEFAULT_BLOCK_SIZE + (2 * DEFAULT_BLOCK_SIZE - 1);
     try {
-      hdfsCluster = startCluster(config, diskTypes, numOfDatanodes,
-          storagesPerDatanode, dnCapacity);
+      hdfsCluster = startCluster(config, diskTypes, NUM_OF_DATANODES,
+          STORAGES_PER_DATANODE, dnCapacity);
       dfs = hdfsCluster.getFileSystem();
-      writeContent(file);
+      writeContent(FILE);
 
       // Change policy to ONE_SSD
-      dfs.setStoragePolicy(new Path(file), ONE_SSD);
+      dfs.setStoragePolicy(new Path(FILE), ONE_SSD);
       Path filePath = new Path("/testChooseInSameDatanode");
       final FSDataOutputStream out =
           dfs.create(filePath, false, 100, (short) 1, 2 * DEFAULT_BLOCK_SIZE);
@@ -869,7 +894,7 @@ public class TestStoragePolicySatisfier {
       for (DataNode dataNode : dataNodes) {
         DataNodeTestUtils.setHeartbeatsDisabledForTests(dataNode, true);
       }
-      dfs.satisfyStoragePolicy(new Path(file));
+      dfs.satisfyStoragePolicy(new Path(FILE));
 
       // Wait for items to be processed
       waitForAttemptedItems(1, 30000);
@@ -887,9 +912,9 @@ public class TestStoragePolicySatisfier {
       }
       hdfsCluster.triggerHeartbeats();
 
-      DFSTestUtil.waitExpectedStorageType(file, StorageType.DISK, 3, 30000,
+      DFSTestUtil.waitExpectedStorageType(FILE, StorageType.DISK, 3, 30000,
           dfs);
-      DFSTestUtil.waitExpectedStorageType(file, StorageType.SSD, 0, 30000, dfs);
+      DFSTestUtil.waitExpectedStorageType(FILE, StorageType.SSD, 0, 30000, dfs);
     } finally {
       shutdownCluster();
     }
@@ -928,7 +953,7 @@ public class TestStoragePolicySatisfier {
         true);
     try {
       hdfsCluster = startCluster(config, diskTypes, diskTypes.length,
-          storagesPerDatanode, capacity);
+          STORAGES_PER_DATANODE, CAPACITY);
       dfs = hdfsCluster.getFileSystem();
       dfs.enableErasureCodingPolicy(
           StripedFileTestUtil.getDefaultECPolicy().getName());
@@ -1029,8 +1054,7 @@ public class TestStoragePolicySatisfier {
           {StorageType.ARCHIVE, StorageType.DISK},
           {StorageType.ARCHIVE, StorageType.DISK},
           {StorageType.ARCHIVE, StorageType.DISK}};
-      cluster = new MiniDFSCluster.Builder(conf).numDataNodes(3)
-          .storageTypes(newtypes).build();
+      cluster = startCluster(conf, newtypes, 3, 2, CAPACITY);
       cluster.waitActive();
       DistributedFileSystem fs = cluster.getFileSystem();
       Path filePath = new Path("/zeroSizeFile");
@@ -1211,7 +1235,7 @@ public class TestStoragePolicySatisfier {
       config.setBoolean(DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_ENABLED_KEY,
           true);
       hdfsCluster = startCluster(config, diskTypes, diskTypes.length,
-          storagesPerDatanode, capacity);
+          STORAGES_PER_DATANODE, CAPACITY);
       dfs = hdfsCluster.getFileSystem();
       createDirectoryTree(dfs);
 
@@ -1245,7 +1269,7 @@ public class TestStoragePolicySatisfier {
       config.setInt(DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_QUEUE_LIMIT_KEY,
           5);
       hdfsCluster = startCluster(config, diskTypes, diskTypes.length,
-          storagesPerDatanode, capacity);
+          STORAGES_PER_DATANODE, CAPACITY);
       dfs = hdfsCluster.getFileSystem();
       createDirectoryTree(dfs);
       List<String> files = getDFSListOfTree();
@@ -1284,7 +1308,7 @@ public class TestStoragePolicySatisfier {
     config.setLong("dfs.block.size", DEFAULT_BLOCK_SIZE);
     config.setInt(DFS_STORAGE_POLICY_SATISFIER_QUEUE_LIMIT_KEY, 10);
     hdfsCluster = startCluster(config, diskTypes, diskTypes.length,
-        storagesPerDatanode, capacity);
+        STORAGES_PER_DATANODE, CAPACITY);
     dfs = hdfsCluster.getFileSystem();
     createDirectoryTree(dfs);
 
@@ -1312,8 +1336,7 @@ public class TestStoragePolicySatisfier {
       }
     };
 
-    FileIdCollector fileIDCollector =
-        new IntraSPSNameNodeFileIdCollector(fsDir, sps);
+    FileIdCollector fileIDCollector = createFileIdCollector(sps, ctxt);
     sps.init(ctxt, fileIDCollector, null);
     sps.getStorageMovementQueue().activate();
 
@@ -1323,31 +1346,20 @@ public class TestStoragePolicySatisfier {
 
     //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 = sps.getStorageMovementQueue().get().getFileId();
-      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 = sps.getStorageMovementQueue().get().getFileId();
-      INode inode = fsDir.getInode(trackId);
-      assertTrue("Failed to traverse tree, expected " + path + " but got "
-          + inode.getFullPathName(), path.equals(inode.getFullPathName()));
-    }
+    assertTraversal(expectedTraverseOrder, fsDir, sps);
     dfs.delete(new Path("/root"), true);
   }
 
+  public FileIdCollector createFileIdCollector(StoragePolicySatisfier sps,
+      Context ctxt) {
+    FileIdCollector fileIDCollector = new IntraSPSNameNodeFileIdCollector(
+        hdfsCluster.getNamesystem().getFSDirectory(), sps);
+    return fileIDCollector;
+  }
+
   /**
    *  Test traverse when root parent got deleted.
    *  1. Delete L when traversing Q
@@ -1362,7 +1374,7 @@ public class TestStoragePolicySatisfier {
     config.setLong("dfs.block.size", DEFAULT_BLOCK_SIZE);
     config.setInt(DFS_STORAGE_POLICY_SATISFIER_QUEUE_LIMIT_KEY, 10);
     hdfsCluster = startCluster(config, diskTypes, diskTypes.length,
-        storagesPerDatanode, capacity);
+        STORAGES_PER_DATANODE, CAPACITY);
     dfs = hdfsCluster.getFileSystem();
     createDirectoryTree(dfs);
 
@@ -1378,7 +1390,6 @@ public class TestStoragePolicySatisfier {
 
     // 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 = new StoragePolicySatisfier(config);
     StoragePolicySatisfier sps = new StoragePolicySatisfier(config);
     Context ctxt = new IntraSPSNameNodeContext(hdfsCluster.getNamesystem(),
         hdfsCluster.getNamesystem().getBlockManager(), sps) {
@@ -1392,9 +1403,7 @@ public class TestStoragePolicySatisfier {
         return true;
       }
     };
-
-    FileIdCollector fileIDCollector =
-        new IntraSPSNameNodeFileIdCollector(fsDir, sps);
+    FileIdCollector fileIDCollector = createFileIdCollector(sps, ctxt);
     sps.init(ctxt, fileIDCollector, null);
     sps.getStorageMovementQueue().activate();
 
@@ -1407,6 +1416,13 @@ public class TestStoragePolicySatisfier {
 
     dfs.delete(new Path("/root/D/L"), true);
 
+    assertTraversal(expectedTraverseOrder, fsDir, sps);
+    dfs.delete(new Path("/root"), true);
+  }
+
+  private void assertTraversal(List<String> expectedTraverseOrder,
+      FSDirectory fsDir, StoragePolicySatisfier sps)
+          throws InterruptedException {
     // Remove 10 element and make queue free, So other traversing will start.
     for (int i = 0; i < 10; i++) {
       String path = expectedTraverseOrder.remove(0);
@@ -1426,7 +1442,6 @@ public class TestStoragePolicySatisfier {
       assertTrue("Failed to traverse tree, expected " + path + " but got "
           + inode.getFullPathName(), path.equals(inode.getFullPathName()));
     }
-    dfs.delete(new Path("/root"), true);
   }
 
   /**
@@ -1473,8 +1488,8 @@ public class TestStoragePolicySatisfier {
       StorageType[][] newtypes =
           new StorageType[][]{{StorageType.DISK, StorageType.SSD},
               {StorageType.DISK, StorageType.SSD}};
-      startAdditionalDNs(config, 2, numOfDatanodes, newtypes,
-          storagesPerDatanode, capacity, hdfsCluster);
+      startAdditionalDNs(config, 2, NUM_OF_DATANODES, newtypes,
+          STORAGES_PER_DATANODE, CAPACITY, hdfsCluster);
 
       // increase replication factor to 4 for the first 10 files and thus
       // initiate replica tasks
@@ -1772,7 +1787,7 @@ public class TestStoragePolicySatisfier {
     }, 100, timeout);
   }
 
-  private void writeContent(final String fileName) throws IOException {
+  public void writeContent(final String fileName) throws IOException {
     writeContent(fileName, (short) 3);
   }
 
@@ -1805,7 +1820,7 @@ public class TestStoragePolicySatisfier {
     cluster.triggerHeartbeats();
   }
 
-  private MiniDFSCluster startCluster(final Configuration conf,
+  public MiniDFSCluster startCluster(final Configuration conf,
       StorageType[][] storageTypes, int numberOfDatanodes, int storagesPerDn,
       long nodeCapacity) throws IOException {
     long[][] capacities = new long[numberOfDatanodes][storagesPerDn];

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3159b39c/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/sps/TestExternalStoragePolicySatisfier.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/sps/TestExternalStoragePolicySatisfier.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/sps/TestExternalStoragePolicySatisfier.java
new file mode 100644
index 0000000..3ced34e
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/sps/TestExternalStoragePolicySatisfier.java
@@ -0,0 +1,108 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.sps;
+
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.StorageType;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.server.namenode.sps.Context;
+import org.apache.hadoop.hdfs.server.namenode.sps.FileIdCollector;
+import org.apache.hadoop.hdfs.server.namenode.sps.IntraSPSNameNodeBlockMoveTaskHandler;
+import org.apache.hadoop.hdfs.server.namenode.sps.IntraSPSNameNodeContext;
+import org.apache.hadoop.hdfs.server.namenode.sps.SPSService;
+import org.apache.hadoop.hdfs.server.namenode.sps.StoragePolicySatisfier;
+import org.apache.hadoop.hdfs.server.namenode.sps.TestStoragePolicySatisfier;
+import org.junit.Ignore;
+
+/**
+ * Tests the external sps service plugins.
+ */
+public class TestExternalStoragePolicySatisfier
+    extends TestStoragePolicySatisfier {
+  private StorageType[][] allDiskTypes =
+      new StorageType[][]{{StorageType.DISK, StorageType.DISK},
+          {StorageType.DISK, StorageType.DISK},
+          {StorageType.DISK, StorageType.DISK}};
+
+  @Override
+  public void createCluster() throws IOException {
+    getConf().setLong("dfs.block.size", DEFAULT_BLOCK_SIZE);
+    getConf().setBoolean(DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_ENABLED_KEY,
+        true);
+    setCluster(startCluster(getConf(), allDiskTypes, NUM_OF_DATANODES,
+        STORAGES_PER_DATANODE, CAPACITY));
+    getFS();
+    writeContent(FILE);
+  }
+
+  @Override
+  public 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();
+    if (conf.getBoolean(DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_ENABLED_KEY,
+        false)) {
+      SPSService spsService = cluster.getNameNode().getNamesystem()
+          .getBlockManager().getSPSService();
+      spsService.stopGracefully();
+
+      IntraSPSNameNodeContext context = new IntraSPSNameNodeContext(
+          cluster.getNameNode().getNamesystem(),
+          cluster.getNameNode().getNamesystem().getBlockManager(), cluster
+              .getNameNode().getNamesystem().getBlockManager().getSPSService());
+
+      spsService.init(context,
+          new ExternalSPSFileIDCollector(context,
+              cluster.getNameNode().getNamesystem().getBlockManager()
+                  .getSPSService(),
+              5),
+          new IntraSPSNameNodeBlockMoveTaskHandler(
+              cluster.getNameNode().getNamesystem().getBlockManager(),
+              cluster.getNameNode().getNamesystem()));
+      spsService.start(true);
+    }
+    return cluster;
+  }
+
+  @Override
+  public FileIdCollector createFileIdCollector(StoragePolicySatisfier sps,
+      Context ctxt) {
+    return new ExternalSPSFileIDCollector(ctxt, sps, 5);
+  }
+
+  /**
+   * This test need not run as external scan is not a batch based scanning right
+   * now.
+   */
+  @Ignore("ExternalFileIdCollector is not batch based right now."
+      + " So, ignoring it.")
+  public void testBatchProcessingForSPSDirectory() throws Exception {
+  }
+}


---------------------------------------------------------------------
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-13077. [SPS]: Fix review comments of external storage policy satisfier. Contributed by Rakesh R.

Posted by um...@apache.org.
HDFS-13077. [SPS]: Fix review comments of external storage policy satisfier. 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/d3de4fb2
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/d3de4fb2
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/d3de4fb2

Branch: refs/heads/HDFS-10285
Commit: d3de4fb2a084cbadab8ef91f11aa7732d3b0f308
Parents: 5845c36
Author: Surendra Singh Lilhore <su...@apache.org>
Authored: Mon Jan 29 23:59:55 2018 +0530
Committer: Uma Maheswara Rao Gangumalla <um...@apache.org>
Committed: Sun Aug 12 03:06:04 2018 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/hdfs/DFSConfigKeys.java   |  14 +-
 .../server/blockmanagement/BlockManager.java    |  33 +++-
 .../namenode/FSDirSatisfyStoragePolicyOp.java   |  15 ++
 .../hdfs/server/namenode/FSNamesystem.java      |  41 ++--
 .../hdfs/server/namenode/NameNodeRpcServer.java |  11 ++
 .../hdfs/server/namenode/sps/SPSPathIds.java    |   8 +-
 .../namenode/sps/StoragePolicySatisfier.java    |   6 +-
 .../hdfs/server/sps/ExternalSPSContext.java     |   4 +
 .../sps/ExternalStoragePolicySatisfier.java     |  30 ++-
 .../sps/TestStoragePolicySatisfier.java         |   7 +-
 .../sps/TestExternalStoragePolicySatisfier.java | 195 ++++++++++++++++++-
 11 files changed, 323 insertions(+), 41 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/d3de4fb2/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 bf29d14..b354d64 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
@@ -614,7 +614,7 @@ 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
+  // StoragePolicySatisfier (SPS) related configurations
   public static final String  DFS_STORAGE_POLICY_SATISFIER_MODE_KEY =
       "dfs.storage.policy.satisfier.mode";
   public static final String DFS_STORAGE_POLICY_SATISFIER_MODE_DEFAULT =
@@ -643,6 +643,18 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
       "dfs.storage.policy.satisfier.low.max-streams.preference";
   public static final boolean DFS_STORAGE_POLICY_SATISFIER_LOW_MAX_STREAMS_PREFERENCE_DEFAULT =
       true;
+  public static final String DFS_SPS_MAX_OUTSTANDING_PATHS_KEY =
+      "dfs.storage.policy.satisfier.max.outstanding.paths";
+  public static final int DFS_SPS_MAX_OUTSTANDING_PATHS_DEFAULT = 10000;
+
+  // SPS keytab configurations, by default it is disabled.
+  public static final String  DFS_SPS_ADDRESS_KEY =
+      "dfs.storage.policy.satisfier.address";
+  public static final String  DFS_SPS_ADDRESS_DEFAULT= "0.0.0.0:0";
+  public static final String  DFS_SPS_KEYTAB_FILE_KEY =
+      "dfs.storage.policy.satisfier.keytab.file";
+  public static final String  DFS_SPS_KERBEROS_PRINCIPAL_KEY =
+      "dfs.storage.policy.satisfier.kerberos.principal";
 
   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/d3de4fb2/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 4ea64a3..9205910 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
@@ -439,6 +439,7 @@ public class BlockManager implements BlockStatsMXBean {
   private final boolean storagePolicyEnabled;
   private StoragePolicySatisfierMode spsMode;
   private SPSPathIds spsPaths;
+  private final int spsOutstandingPathsLimit;
 
   /** Minimum live replicas needed for the datanode to be transitioned
    * from ENTERING_MAINTENANCE to IN_MAINTENANCE.
@@ -478,14 +479,16 @@ public class BlockManager implements BlockStatsMXBean {
         DFSConfigKeys.DFS_NAMENODE_RECONSTRUCTION_PENDING_TIMEOUT_SEC_KEY,
         DFSConfigKeys.DFS_NAMENODE_RECONSTRUCTION_PENDING_TIMEOUT_SEC_DEFAULT)
         * 1000L);
-
+    // StoragePolicySatisfier(SPS) configs
     storagePolicyEnabled =
         conf.getBoolean(DFSConfigKeys.DFS_STORAGE_POLICY_ENABLED_KEY,
             DFSConfigKeys.DFS_STORAGE_POLICY_ENABLED_DEFAULT);
-    String spsModeVal =
-        conf.get(
-            DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_MODE_KEY,
-            DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_MODE_DEFAULT);
+    String spsModeVal = conf.get(
+        DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_MODE_KEY,
+        DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_MODE_DEFAULT);
+    spsOutstandingPathsLimit = conf.getInt(
+        DFSConfigKeys.DFS_SPS_MAX_OUTSTANDING_PATHS_KEY,
+        DFSConfigKeys.DFS_SPS_MAX_OUTSTANDING_PATHS_DEFAULT);
     spsMode = StoragePolicySatisfierMode.fromString(spsModeVal);
     spsPaths = new SPSPathIds();
     sps = new StoragePolicySatisfier(conf);
@@ -5188,6 +5191,12 @@ public class BlockManager implements BlockStatsMXBean {
    */
   public StoragePolicySatisfyPathStatus checkStoragePolicySatisfyPathStatus(
       String path) throws IOException {
+    if (spsMode != StoragePolicySatisfierMode.INTERNAL) {
+      LOG.debug("Satisfier is not running inside namenode, so status "
+          + "can't be returned.");
+      throw new IOException("Satisfier is not running inside namenode, "
+          + "so status can't be returned.");
+    }
     return sps.checkStoragePolicySatisfyPathStatus(path);
   }
 
@@ -5207,6 +5216,20 @@ public class BlockManager implements BlockStatsMXBean {
   }
 
   /**
+   * Verify that satisfier queue limit exceeds allowed outstanding limit.
+   */
+  public void verifyOutstandingSPSPathQLimit() throws IOException {
+    long size = spsPaths.size();
+    // Checking that the SPS call Q exceeds the allowed limit.
+    if (spsOutstandingPathsLimit - size <= 0) {
+      LOG.debug("Satisifer Q - outstanding limit:{}, current size:{}",
+          spsOutstandingPathsLimit, size);
+      throw new IOException("Outstanding satisfier queue limit: "
+          + spsOutstandingPathsLimit + " exceeded, try later!");
+    }
+  }
+
+  /**
    * Removes the SPS path id from the list of sps paths.
    */
   public void removeSPSPathId(long trackId) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d3de4fb2/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 eed6e52..5ffd6e8 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
@@ -45,6 +45,21 @@ final class FSDirSatisfyStoragePolicyOp {
   private FSDirSatisfyStoragePolicyOp() {
   }
 
+  /**
+   * Satisfy storage policy function which will add the entry to SPS call queue
+   * and will perform satisfaction async way.
+   *
+   * @param fsd
+   *          fs directory
+   * @param bm
+   *          block manager
+   * @param src
+   *          source path
+   * @param logRetryCache
+   *          whether to record RPC ids in editlog for retry cache rebuilding
+   * @return file status info
+   * @throws IOException
+   */
   static FileStatus satisfyStoragePolicy(FSDirectory fsd, BlockManager bm,
       String src, boolean logRetryCache) throws IOException {
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d3de4fb2/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 c93791b..5dfec25 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
@@ -2253,28 +2253,12 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       throws IOException {
     final String operationName = "satisfyStoragePolicy";
     FileStatus auditStat;
+    validateStoragePolicySatisfy();
     checkOperation(OperationCategory.WRITE);
     writeLock();
     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.isSPSEnabled()
-          || (blockManager.getSPSMode() == StoragePolicySatisfierMode.INTERNAL
-              && !blockManager.getStoragePolicySatisfier().isRunning())) {
-        throw new UnsupportedActionException(
-            "Cannot request to satisfy storage policy "
-                + "when storage policy satisfier feature has been disabled"
-                + " by admin. Seek for an admin help to enable it "
-                + "or use Mover tool.");
-      }
       auditStat = FSDirSatisfyStoragePolicyOp.satisfyStoragePolicy(
           dir, blockManager, src, logRetryCache);
     } catch (AccessControlException e) {
@@ -2287,6 +2271,29 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     logAuditEvent(true, operationName, src, null, auditStat);
   }
 
+  private void validateStoragePolicySatisfy()
+      throws UnsupportedActionException, IOException {
+    // 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));
+    }
+    // checks sps status
+    if (!blockManager.isSPSEnabled()
+        || (blockManager.getSPSMode() == StoragePolicySatisfierMode.INTERNAL
+            && !blockManager.getStoragePolicySatisfier().isRunning())) {
+      throw new UnsupportedActionException(
+          "Cannot request to satisfy storage policy "
+              + "when storage policy satisfier feature has been disabled"
+              + " by admin. Seek for an admin help to enable it "
+              + "or use Mover tool.");
+    }
+    // checks SPS Q has many outstanding requests.
+    blockManager.verifyOutstandingSPSPathQLimit();
+  }
+
   /**
    * unset storage policy set for a given file or a directory.
    *

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d3de4fb2/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 0e50965..d74dc9e 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
@@ -110,6 +110,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.StoragePolicySatisfierMode;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.StoragePolicySatisfyPathStatus;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
@@ -2578,6 +2579,16 @@ public class NameNodeRpcServer implements NamenodeProtocols {
     if (nn.isStandbyState()) {
       throw new StandbyException("Not supported by Standby Namenode.");
     }
+    // Check that internal SPS service is running
+    if (namesystem.getBlockManager()
+        .getSPSMode() == StoragePolicySatisfierMode.INTERNAL
+        && namesystem.getBlockManager().getSPSService().isRunning()) {
+      LOG.debug("SPS service is internally enabled and running inside "
+          + "namenode, so external SPS is not allowed to fetch the path Ids");
+      throw new IOException("SPS service is internally enabled and running"
+          + " inside namenode, so external SPS is not allowed to fetch"
+          + " the path Ids");
+    }
     return namesystem.getBlockManager().getNextSPSPathId();
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d3de4fb2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/SPSPathIds.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/SPSPathIds.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/SPSPathIds.java
index e0f4999..6c0f8b2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/SPSPathIds.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/SPSPathIds.java
@@ -31,7 +31,6 @@ import org.apache.hadoop.classification.InterfaceStability;
 public class SPSPathIds {
 
   // List of pending dir to satisfy the policy
-  // TODO: Make this bounded queue.
   private final Queue<Long> spsDirsToBeTraveresed = new LinkedList<Long>();
 
   /**
@@ -61,4 +60,11 @@ public class SPSPathIds {
   public synchronized Long pollNext() {
     return spsDirsToBeTraveresed.poll();
   }
+
+  /**
+   * @return the size of the queue.
+   */
+  public synchronized long size() {
+    return spsDirsToBeTraveresed.size();
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d3de4fb2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/StoragePolicySatisfier.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/StoragePolicySatisfier.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/StoragePolicySatisfier.java
index 4ddfe2e..87faced 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/StoragePolicySatisfier.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/StoragePolicySatisfier.java
@@ -91,7 +91,7 @@ public class StoragePolicySatisfier implements SPSService, Runnable {
   private int blockMovementMaxRetry;
   private Context ctxt;
   private BlockMoveTaskHandler blockMoveTaskHandler;
-  private Configuration conf;
+  private final Configuration conf;
 
   public StoragePolicySatisfier(Configuration conf) {
     this.conf = conf;
@@ -441,8 +441,8 @@ public class StoragePolicySatisfier implements SPSService, Runnable {
             liveDns, ecPolicy);
         if (blocksPaired) {
           status = BlocksMovingAnalysis.Status.BLOCKS_TARGETS_PAIRED;
-        } else
-          if (status != BlocksMovingAnalysis.Status.BLOCKS_TARGETS_PAIRED) {
+        } else if (status !=
+            BlocksMovingAnalysis.Status.BLOCKS_TARGETS_PAIRED) {
           // Check if the previous block was successfully paired. Here the
           // status will set to NO_BLOCKS_TARGETS_PAIRED only when none of the
           // blocks of a file found its eligible targets to satisfy the storage

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d3de4fb2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/sps/ExternalSPSContext.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/sps/ExternalSPSContext.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/sps/ExternalSPSContext.java
index e3b3bbb..c309209 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/sps/ExternalSPSContext.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/sps/ExternalSPSContext.java
@@ -175,6 +175,10 @@ public class ExternalSPSContext implements Context {
   @Override
   public boolean checkDNSpaceForScheduling(DatanodeInfo dn, StorageType type,
       long estimatedSize) {
+    // TODO: Instead of calling namenode for checking the available space, it
+    // can be optimized by maintaining local cache of datanode storage report
+    // and do the computations. This local cache can be refreshed per file or
+    // periodic fashion.
     try {
       return nnc.getNNProtocolConnection().checkDNSpaceForScheduling(dn, type,
           estimatedSize);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d3de4fb2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/sps/ExternalStoragePolicySatisfier.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/sps/ExternalStoragePolicySatisfier.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/sps/ExternalStoragePolicySatisfier.java
index c64abc3..59935b6 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/sps/ExternalStoragePolicySatisfier.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/sps/ExternalStoragePolicySatisfier.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.hdfs.server.sps;
 import static org.apache.hadoop.util.ExitUtil.terminate;
 
 import java.io.IOException;
+import java.net.InetSocketAddress;
 import java.net.URI;
 import java.util.ArrayList;
 import java.util.Collection;
@@ -28,6 +29,7 @@ import java.util.List;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.protocol.Block;
@@ -36,6 +38,9 @@ import org.apache.hadoop.hdfs.server.balancer.NameNodeConnector;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.namenode.sps.BlockMovementListener;
 import org.apache.hadoop.hdfs.server.namenode.sps.StoragePolicySatisfier;
+import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.security.SecurityUtil;
+import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.util.StringUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -44,20 +49,25 @@ import org.slf4j.LoggerFactory;
  * This class starts and runs external SPS service.
  */
 @InterfaceAudience.Private
-public class ExternalStoragePolicySatisfier {
+public final class ExternalStoragePolicySatisfier {
   public static final Logger LOG = LoggerFactory
       .getLogger(ExternalStoragePolicySatisfier.class);
 
+  private ExternalStoragePolicySatisfier() {
+    // This is just a class to start and run external sps.
+  }
+
   /**
    * Main method to start SPS service.
    */
-  public static void main(String args[]) throws Exception {
+  public static void main(String[] args) throws Exception {
     NameNodeConnector nnc = null;
     try {
       StringUtils.startupShutdownMessage(StoragePolicySatisfier.class, args,
           LOG);
       HdfsConfiguration spsConf = new HdfsConfiguration();
-      //TODO : login with SPS keytab
+      // login with SPS keytab
+      secureLogin(spsConf);
       StoragePolicySatisfier sps = new StoragePolicySatisfier(spsConf);
       nnc = getNameNodeConnector(spsConf);
 
@@ -92,6 +102,18 @@ public class ExternalStoragePolicySatisfier {
     }
   }
 
+  private static void secureLogin(Configuration conf)
+      throws IOException {
+    UserGroupInformation.setConfiguration(conf);
+    String addr = conf.get(DFSConfigKeys.DFS_SPS_ADDRESS_KEY,
+        DFSConfigKeys.DFS_SPS_ADDRESS_DEFAULT);
+    InetSocketAddress socAddr = NetUtils.createSocketAddr(addr, 0,
+        DFSConfigKeys.DFS_SPS_ADDRESS_KEY);
+    SecurityUtil.login(conf, DFSConfigKeys.DFS_SPS_KEYTAB_FILE_KEY,
+        DFSConfigKeys.DFS_SPS_KERBEROS_PRINCIPAL_KEY,
+        socAddr.getHostName());
+  }
+
   private static NameNodeConnector getNameNodeConnector(Configuration conf)
       throws IOException, InterruptedException {
     final Collection<URI> namenodes = DFSUtil.getInternalNsRpcUris(conf);
@@ -100,7 +122,7 @@ public class ExternalStoragePolicySatisfier {
       try {
         final List<NameNodeConnector> nncs = NameNodeConnector
             .newNameNodeConnectors(namenodes,
-                StoragePolicySatisfier.class.getSimpleName(),
+                ExternalStoragePolicySatisfier.class.getSimpleName(),
                 externalSPSPathId, conf,
                 NameNodeConnector.DEFAULT_MAX_IDLE_ITERATIONS);
         return nncs.get(0);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d3de4fb2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/sps/TestStoragePolicySatisfier.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/sps/TestStoragePolicySatisfier.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/sps/TestStoragePolicySatisfier.java
index 135d996..9e0a39f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/sps/TestStoragePolicySatisfier.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/sps/TestStoragePolicySatisfier.java
@@ -134,8 +134,9 @@ public class TestStoragePolicySatisfier {
    *
    * @throws IOException
    */
-  public void getFS() throws IOException {
+  public DistributedFileSystem getFS() throws IOException {
     this.dfs = hdfsCluster.getFileSystem();
+    return this.dfs;
   }
 
   @After
@@ -423,9 +424,9 @@ public class TestStoragePolicySatisfier {
                 + "for %s since %s is set to false.",
             FILE, DFS_STORAGE_POLICY_ENABLED_KEY));
       } catch (IOException e) {
-        Assert.assertTrue(e.getMessage().contains(String.format(
+        GenericTestUtils.assertExceptionContains(String.format(
             "Failed to satisfy storage policy since %s is set to false.",
-            DFS_STORAGE_POLICY_ENABLED_KEY)));
+            DFS_STORAGE_POLICY_ENABLED_KEY), e);
       }
 
       hdfsCluster.getConfiguration(0).

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d3de4fb2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/sps/TestExternalStoragePolicySatisfier.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/sps/TestExternalStoragePolicySatisfier.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/sps/TestExternalStoragePolicySatisfier.java
index febc2ea..15a4271 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/sps/TestExternalStoragePolicySatisfier.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/sps/TestExternalStoragePolicySatisfier.java
@@ -17,17 +17,40 @@
  */
 package org.apache.hadoop.hdfs.server.sps;
 
+import static org.apache.hadoop.fs.CommonConfigurationKeys.IPC_CLIENT_CONNECT_MAX_RETRIES_ON_SASL_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BLOCK_ACCESS_TOKEN_ENABLE_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_HTTPS_KEYSTORE_RESOURCE_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_HTTPS_ADDRESS_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_KERBEROS_PRINCIPAL_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_KEYTAB_FILE_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_HTTP_POLICY_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_HTTPS_ADDRESS_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_KERBEROS_PRINCIPAL_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_KEYTAB_FILE_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_SERVER_HTTPS_KEYSTORE_RESOURCE_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_SPS_ADDRESS_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_SPS_KERBEROS_PRINCIPAL_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_SPS_KEYTAB_FILE_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_SPS_MAX_OUTSTANDING_PATHS_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_WEB_AUTHENTICATION_KERBEROS_PRINCIPAL_KEY;
+import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_DATA_TRANSFER_PROTECTION_KEY;
+
+import java.io.File;
 import java.io.IOException;
 import java.net.URI;
+import java.security.PrivilegedExceptionAction;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.List;
+import java.util.Properties;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSUtil;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.StoragePolicySatisfierMode;
@@ -39,8 +62,17 @@ import org.apache.hadoop.hdfs.server.namenode.sps.FileIdCollector;
 import org.apache.hadoop.hdfs.server.namenode.sps.SPSService;
 import org.apache.hadoop.hdfs.server.namenode.sps.StoragePolicySatisfier;
 import org.apache.hadoop.hdfs.server.namenode.sps.TestStoragePolicySatisfier;
+import org.apache.hadoop.http.HttpConfig;
+import org.apache.hadoop.minikdc.MiniKdc;
+import org.apache.hadoop.security.SecurityUtil;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.authentication.util.KerberosName;
+import org.apache.hadoop.security.ssl.KeyStoreTestUtil;
+import org.apache.hadoop.test.GenericTestUtils;
+import org.junit.After;
 import org.junit.Assert;
 import org.junit.Ignore;
+import org.junit.Test;
 
 /**
  * Tests the external sps service plugins.
@@ -52,6 +84,18 @@ public class TestExternalStoragePolicySatisfier
           {StorageType.DISK, StorageType.DISK},
           {StorageType.DISK, StorageType.DISK}};
   private NameNodeConnector nnc;
+  private File keytabFile;
+  private String principal;
+  private MiniKdc kdc;
+  private File baseDir;
+
+  @After
+  public void destroy() throws Exception {
+    if (kdc != null) {
+      kdc.stop();
+      FileUtil.fullyDelete(baseDir);
+    }
+  }
 
   @Override
   public void setUp() {
@@ -178,20 +222,157 @@ public class TestExternalStoragePolicySatisfier
     }
   }
 
+  private void initSecureConf(Configuration conf) throws Exception {
+    String username = "externalSPS";
+    baseDir = GenericTestUtils
+        .getTestDir(TestExternalStoragePolicySatisfier.class.getSimpleName());
+    FileUtil.fullyDelete(baseDir);
+    Assert.assertTrue(baseDir.mkdirs());
+
+    Properties kdcConf = MiniKdc.createConf();
+    kdc = new MiniKdc(kdcConf, baseDir);
+    kdc.start();
+
+    SecurityUtil.setAuthenticationMethod(
+        UserGroupInformation.AuthenticationMethod.KERBEROS, conf);
+    UserGroupInformation.setConfiguration(conf);
+    KerberosName.resetDefaultRealm();
+    Assert.assertTrue("Expected configuration to enable security",
+        UserGroupInformation.isSecurityEnabled());
+
+    keytabFile = new File(baseDir, username + ".keytab");
+    String keytab = keytabFile.getAbsolutePath();
+    // Windows will not reverse name lookup "127.0.0.1" to "localhost".
+    String krbInstance = Path.WINDOWS ? "127.0.0.1" : "localhost";
+    principal = username + "/" + krbInstance + "@" + kdc.getRealm();
+    String spnegoPrincipal = "HTTP/" + krbInstance + "@" + kdc.getRealm();
+    kdc.createPrincipal(keytabFile, username, username + "/" + krbInstance,
+        "HTTP/" + krbInstance);
+
+    conf.set(DFS_NAMENODE_KERBEROS_PRINCIPAL_KEY, principal);
+    conf.set(DFS_NAMENODE_KEYTAB_FILE_KEY, keytab);
+    conf.set(DFS_DATANODE_KERBEROS_PRINCIPAL_KEY, principal);
+    conf.set(DFS_DATANODE_KEYTAB_FILE_KEY, keytab);
+    conf.set(DFS_WEB_AUTHENTICATION_KERBEROS_PRINCIPAL_KEY, spnegoPrincipal);
+    conf.setBoolean(DFS_BLOCK_ACCESS_TOKEN_ENABLE_KEY, true);
+    conf.set(DFS_DATA_TRANSFER_PROTECTION_KEY, "authentication");
+    conf.set(DFS_HTTP_POLICY_KEY, HttpConfig.Policy.HTTPS_ONLY.name());
+    conf.set(DFS_NAMENODE_HTTPS_ADDRESS_KEY, "localhost:0");
+    conf.set(DFS_DATANODE_HTTPS_ADDRESS_KEY, "localhost:0");
+    conf.setInt(IPC_CLIENT_CONNECT_MAX_RETRIES_ON_SASL_KEY, 10);
+
+    conf.set(DFS_SPS_ADDRESS_KEY, "localhost:0");
+    conf.set(DFS_SPS_KEYTAB_FILE_KEY, keytab);
+    conf.set(DFS_SPS_KERBEROS_PRINCIPAL_KEY, principal);
+
+    String keystoresDir = baseDir.getAbsolutePath();
+    String sslConfDir = KeyStoreTestUtil
+        .getClasspathDir(TestExternalStoragePolicySatisfier.class);
+    KeyStoreTestUtil.setupSSLConfig(keystoresDir, sslConfDir, conf, false);
+
+    conf.set(DFS_CLIENT_HTTPS_KEYSTORE_RESOURCE_KEY,
+        KeyStoreTestUtil.getClientSSLConfigFileName());
+    conf.set(DFS_SERVER_HTTPS_KEYSTORE_RESOURCE_KEY,
+        KeyStoreTestUtil.getServerSSLConfigFileName());
+
+    conf.setInt(DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_KEY, DEFAULT_BLOCK_SIZE);
+    conf.setLong(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1L);
+  }
+
   /**
-   * This test need not run as external scan is not a batch based scanning right
-   * now.
+   * Test SPS runs fine when logging in with a keytab in kerberized env. Reusing
+   * testWhenStoragePolicySetToALLSSD here for basic functionality testing.
    */
-  @Ignore("ExternalFileIdCollector is not batch based right now."
-      + " So, ignoring it.")
-  public void testBatchProcessingForSPSDirectory() throws Exception {
+  @Test(timeout = 300000)
+  public void testWithKeytabs() throws Exception {
+    try {
+      initSecureConf(getConf());
+      final UserGroupInformation ugi = UserGroupInformation
+          .loginUserFromKeytabAndReturnUGI(principal,
+              keytabFile.getAbsolutePath());
+      ugi.doAs(new PrivilegedExceptionAction<Void>() {
+        @Override
+        public Void run() throws Exception {
+          // verify that sps runs Ok.
+          testWhenStoragePolicySetToALLSSD();
+          // verify that UGI was logged in using keytab.
+          Assert.assertTrue(UserGroupInformation.isLoginKeytabBased());
+          return null;
+        }
+      });
+    } finally {
+      // Reset UGI so that other tests are not affected.
+      UserGroupInformation.reset();
+      UserGroupInformation.setConfiguration(new Configuration());
+    }
   }
 
   /**
-   * Status won't be supported for external SPS, now. So, ignoring it.
+   * Test verifies that SPS call will throw exception if the call Q exceeds
+   * OutstandingQueueLimit value.
+   *
+   * @throws Exception
    */
-  @Ignore("Status is not supported for external SPS. So, ignoring it.")
+  @Test(timeout = 300000)
+  public void testOutstandingQueueLimitExceeds() throws Exception {
+    try {
+      getConf().setInt(DFS_SPS_MAX_OUTSTANDING_PATHS_KEY, 3);
+      createCluster();
+      List<String> files = new ArrayList<>();
+      files.add(FILE);
+      DistributedFileSystem fs = getFS();
+      BlockManager blkMgr = getCluster().getNameNode().getNamesystem()
+          .getBlockManager();
+      SPSService spsService = blkMgr.getSPSService();
+      spsService.stopGracefully(); // stops SPS
+
+      // Creates 4 more files. Send all of them for satisfying the storage
+      // policy together.
+      for (int i = 0; i < 3; i++) {
+        String file1 = "/testOutstandingQueueLimitExceeds_" + i;
+        files.add(file1);
+        writeContent(file1);
+        fs.satisfyStoragePolicy(new Path(file1));
+      }
+      String fileExceeds = "/testOutstandingQueueLimitExceeds_" + 4;
+      files.add(fileExceeds);
+      writeContent(fileExceeds);
+      try {
+        fs.satisfyStoragePolicy(new Path(fileExceeds));
+        Assert.fail("Should throw exception as it exceeds "
+            + "outstanding SPS call Q limit");
+      } catch (IOException ioe) {
+        GenericTestUtils.assertExceptionContains(
+            "Outstanding satisfier queue limit: 3 exceeded, try later!", ioe);
+      }
+    } finally {
+      shutdownCluster();
+    }
+  }
+
+  /**
+   * Test verifies status check when Satisfier is not running inside namenode.
+   */
+  @Test(timeout = 90000)
   public void testStoragePolicySatisfyPathStatus() throws Exception {
+    createCluster();
+    DistributedFileSystem fs = getFS();
+    try {
+      fs.getClient().checkStoragePolicySatisfyPathStatus(FILE);
+      Assert.fail("Should throw exception as SPS is not running inside NN!");
+    } catch (IOException e) {
+      GenericTestUtils.assertExceptionContains("Satisfier is not running"
+          + " inside namenode, so status can't be returned.", e);
+    }
+  }
+
+  /**
+   * This test need not run as external scan is not a batch based scanning right
+   * now.
+   */
+  @Ignore("ExternalFileIdCollector is not batch based right now."
+      + " So, ignoring it.")
+  public void testBatchProcessingForSPSDirectory() throws Exception {
   }
 
   /**


---------------------------------------------------------------------
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-13110: [SPS]: Reduce the number of APIs in NamenodeProtocol used by external satisfier. Contributed by Rakesh R.

Posted by um...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/8467ec24/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/sps/ExternalSPSFileIDCollector.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/sps/ExternalSPSFileIDCollector.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/sps/ExternalSPSFileIDCollector.java
deleted file mode 100644
index ff277ba..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/sps/ExternalSPSFileIDCollector.java
+++ /dev/null
@@ -1,174 +0,0 @@
-package org.apache.hadoop.hdfs.server.sps;
-/**
- * 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.
- */
-
-import java.io.IOException;
-import java.util.ArrayList;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hdfs.DFSConfigKeys;
-import org.apache.hadoop.hdfs.DistributedFileSystem;
-import org.apache.hadoop.hdfs.protocol.DirectoryListing;
-import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
-import org.apache.hadoop.hdfs.server.namenode.sps.Context;
-import org.apache.hadoop.hdfs.server.namenode.sps.FileIdCollector;
-import org.apache.hadoop.hdfs.server.namenode.sps.ItemInfo;
-import org.apache.hadoop.hdfs.server.namenode.sps.SPSService;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * This class is to scan the paths recursively. If file is directory, then it
- * will scan for files recursively. If the file is non directory, then it will
- * just submit the same file to process.
- */
-@InterfaceAudience.Private
-public class ExternalSPSFileIDCollector implements FileIdCollector {
-  public static final Logger LOG =
-      LoggerFactory.getLogger(ExternalSPSFileIDCollector.class);
-  private Context cxt;
-  private DistributedFileSystem dfs;
-  private SPSService service;
-  private int maxQueueLimitToScan;
-
-  public ExternalSPSFileIDCollector(Context cxt, SPSService service) {
-    this.cxt = cxt;
-    this.service = service;
-    this.maxQueueLimitToScan = service.getConf().getInt(
-        DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_QUEUE_LIMIT_KEY,
-        DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_QUEUE_LIMIT_DEFAULT);
-    try {
-      // TODO: probably we could get this dfs from external context? but this is
-      // too specific to external.
-      dfs = getFS(service.getConf());
-    } catch (IOException e) {
-      LOG.error("Unable to get the filesystem. Make sure Namenode running and "
-          + "configured namenode address is correct.", e);
-    }
-  }
-
-  private DistributedFileSystem getFS(Configuration conf) throws IOException {
-    return (DistributedFileSystem) FileSystem
-        .get(FileSystem.getDefaultUri(conf), conf);
-  }
-
-  /**
-   * Recursively scan the given path and add the file info to SPS service for
-   * processing.
-   */
-  private long processPath(long startID, String fullPath) {
-    long pendingWorkCount = 0; // to be satisfied file counter
-    for (byte[] lastReturnedName = HdfsFileStatus.EMPTY_NAME;;) {
-      final DirectoryListing children;
-      try {
-        children = dfs.getClient().listPaths(fullPath, lastReturnedName, false);
-      } catch (IOException e) {
-        LOG.warn("Failed to list directory " + fullPath
-            + ". Ignore the directory and continue.", e);
-        return pendingWorkCount;
-      }
-      if (children == null) {
-        if (LOG.isDebugEnabled()) {
-          LOG.debug("The scanning start dir/sub dir " + fullPath
-              + " does not have childrens.");
-        }
-        return pendingWorkCount;
-      }
-
-      for (HdfsFileStatus child : children.getPartialListing()) {
-        if (child.isFile()) {
-          service.addFileIdToProcess(new ItemInfo(startID, child.getFileId()),
-              false);
-          checkProcessingQueuesFree();
-          pendingWorkCount++; // increment to be satisfied file count
-        } else {
-          String fullPathStr = child.getFullName(fullPath);
-          if (child.isDirectory()) {
-            if (!fullPathStr.endsWith(Path.SEPARATOR)) {
-              fullPathStr = fullPathStr + Path.SEPARATOR;
-            }
-            pendingWorkCount += processPath(startID, fullPathStr);
-          }
-        }
-      }
-
-      if (children.hasMore()) {
-        lastReturnedName = children.getLastName();
-      } else {
-        return pendingWorkCount;
-      }
-    }
-  }
-
-  private void checkProcessingQueuesFree() {
-    int remainingCapacity = remainingCapacity();
-    // wait for queue to be free
-    while (remainingCapacity <= 0) {
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("Waiting for storageMovementNeeded queue to be free!");
-      }
-      try {
-        Thread.sleep(5000);
-      } catch (InterruptedException e) {
-        Thread.currentThread().interrupt();
-      }
-      remainingCapacity = remainingCapacity();
-    }
-  }
-
-  /**
-   * Returns queue remaining capacity.
-   */
-  public int remainingCapacity() {
-    int size = service.processingQueueSize();
-    int remainingSize = 0;
-    if (size < maxQueueLimitToScan) {
-      remainingSize = maxQueueLimitToScan - size;
-    }
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("SPS processing Q -> maximum capacity:{}, current size:{},"
-          + " remaining size:{}", maxQueueLimitToScan, size, remainingSize);
-    }
-    return remainingSize;
-  }
-
-  @Override
-  public void scanAndCollectFileIds(Long inodeId) throws IOException {
-    if (dfs == null) {
-      dfs = getFS(service.getConf());
-    }
-    long pendingSatisfyItemsCount = processPath(inodeId,
-        cxt.getFilePath(inodeId));
-    // Check whether the given path contains any item to be tracked
-    // or the no to be satisfied paths. In case of empty list, add the given
-    // inodeId to the 'pendingWorkForDirectory' with empty list so that later
-    // SPSPathIdProcessor#run function will remove the SPS hint considering that
-    // this path is already satisfied the storage policy.
-    if (pendingSatisfyItemsCount <= 0) {
-      LOG.debug("There is no pending items to satisfy the given path "
-          + "inodeId:{}", inodeId);
-      service.addAllFileIdsToProcess(inodeId, new ArrayList<>(), true);
-    } else {
-      service.markScanCompletedForPath(inodeId);
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8467ec24/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/sps/ExternalSPSFilePathCollector.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/sps/ExternalSPSFilePathCollector.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/sps/ExternalSPSFilePathCollector.java
new file mode 100644
index 0000000..9435475
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/sps/ExternalSPSFilePathCollector.java
@@ -0,0 +1,172 @@
+package org.apache.hadoop.hdfs.server.sps;
+/**
+ * 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.
+ */
+
+import java.io.IOException;
+import java.util.ArrayList;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
+import org.apache.hadoop.hdfs.protocol.DirectoryListing;
+import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
+import org.apache.hadoop.hdfs.server.namenode.sps.FileCollector;
+import org.apache.hadoop.hdfs.server.namenode.sps.ItemInfo;
+import org.apache.hadoop.hdfs.server.namenode.sps.SPSService;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * This class is to scan the paths recursively. If file is directory, then it
+ * will scan for files recursively. If the file is non directory, then it will
+ * just submit the same file to process. This will use file string path
+ * representation.
+ */
+@InterfaceAudience.Private
+public class ExternalSPSFilePathCollector implements FileCollector <String>{
+  public static final Logger LOG =
+      LoggerFactory.getLogger(ExternalSPSFilePathCollector.class);
+  private DistributedFileSystem dfs;
+  private SPSService<String> service;
+  private int maxQueueLimitToScan;
+
+  public ExternalSPSFilePathCollector(SPSService<String> service) {
+    this.service = service;
+    this.maxQueueLimitToScan = service.getConf().getInt(
+        DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_QUEUE_LIMIT_KEY,
+        DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_QUEUE_LIMIT_DEFAULT);
+    try {
+      // TODO: probably we could get this dfs from external context? but this is
+      // too specific to external.
+      dfs = getFS(service.getConf());
+    } catch (IOException e) {
+      LOG.error("Unable to get the filesystem. Make sure Namenode running and "
+          + "configured namenode address is correct.", e);
+    }
+  }
+
+  private DistributedFileSystem getFS(Configuration conf) throws IOException {
+    return (DistributedFileSystem) FileSystem
+        .get(FileSystem.getDefaultUri(conf), conf);
+  }
+
+  /**
+   * Recursively scan the given path and add the file info to SPS service for
+   * processing.
+   */
+  private long processPath(String startID, String childPath) {
+    long pendingWorkCount = 0; // to be satisfied file counter
+    for (byte[] lastReturnedName = HdfsFileStatus.EMPTY_NAME;;) {
+      final DirectoryListing children;
+      try {
+        children = dfs.getClient().listPaths(childPath, lastReturnedName,
+            false);
+      } catch (IOException e) {
+        LOG.warn("Failed to list directory " + childPath
+            + ". Ignore the directory and continue.", e);
+        return pendingWorkCount;
+      }
+      if (children == null) {
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("The scanning start dir/sub dir " + childPath
+              + " does not have childrens.");
+        }
+        return pendingWorkCount;
+      }
+
+      for (HdfsFileStatus child : children.getPartialListing()) {
+        String childFullPath = child.getFullName(childPath);
+        if (child.isFile()) {
+          service.addFileToProcess(
+              new ItemInfo<String>(startID, childFullPath), false);
+          checkProcessingQueuesFree();
+          pendingWorkCount++; // increment to be satisfied file count
+        } else {
+          if (child.isDirectory()) {
+            if (!childFullPath.endsWith(Path.SEPARATOR)) {
+              childFullPath = childFullPath + Path.SEPARATOR;
+            }
+            pendingWorkCount += processPath(startID, childFullPath);
+          }
+        }
+      }
+
+      if (children.hasMore()) {
+        lastReturnedName = children.getLastName();
+      } else {
+        return pendingWorkCount;
+      }
+    }
+  }
+
+  private void checkProcessingQueuesFree() {
+    int remainingCapacity = remainingCapacity();
+    // wait for queue to be free
+    while (remainingCapacity <= 0) {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Waiting for storageMovementNeeded queue to be free!");
+      }
+      try {
+        Thread.sleep(5000);
+      } catch (InterruptedException e) {
+        Thread.currentThread().interrupt();
+      }
+      remainingCapacity = remainingCapacity();
+    }
+  }
+
+  /**
+   * Returns queue remaining capacity.
+   */
+  public int remainingCapacity() {
+    int size = service.processingQueueSize();
+    int remainingSize = 0;
+    if (size < maxQueueLimitToScan) {
+      remainingSize = maxQueueLimitToScan - size;
+    }
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("SPS processing Q -> maximum capacity:{}, current size:{},"
+          + " remaining size:{}", maxQueueLimitToScan, size, remainingSize);
+    }
+    return remainingSize;
+  }
+
+  @Override
+  public void scanAndCollectFiles(String path) throws IOException {
+    if (dfs == null) {
+      dfs = getFS(service.getConf());
+    }
+    long pendingSatisfyItemsCount = processPath(path, path);
+    // Check whether the given path contains any item to be tracked
+    // or the no to be satisfied paths. In case of empty list, add the given
+    // inodeId to the 'pendingWorkForDirectory' with empty list so that later
+    // SPSPathIdProcessor#run function will remove the SPS hint considering that
+    // this path is already satisfied the storage policy.
+    if (pendingSatisfyItemsCount <= 0) {
+      LOG.debug("There is no pending items to satisfy the given path "
+          + "inodeId:{}", path);
+      service.addAllFilesToProcess(path, new ArrayList<>(), true);
+    } else {
+      service.markScanCompletedForPath(path);
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8467ec24/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/sps/ExternalStoragePolicySatisfier.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/sps/ExternalStoragePolicySatisfier.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/sps/ExternalStoragePolicySatisfier.java
index 33448db..6fc35ea 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/sps/ExternalStoragePolicySatisfier.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/sps/ExternalStoragePolicySatisfier.java
@@ -68,7 +68,8 @@ public final class ExternalStoragePolicySatisfier {
       HdfsConfiguration spsConf = new HdfsConfiguration();
       // login with SPS keytab
       secureLogin(spsConf);
-      StoragePolicySatisfier sps = new StoragePolicySatisfier(spsConf);
+      StoragePolicySatisfier<String> sps = new StoragePolicySatisfier<String>(
+          spsConf);
       nnc = getNameNodeConnector(spsConf);
 
       boolean spsRunning;
@@ -86,8 +87,8 @@ public final class ExternalStoragePolicySatisfier {
       ExternalSPSBlockMoveTaskHandler externalHandler =
           new ExternalSPSBlockMoveTaskHandler(spsConf, nnc, sps);
       externalHandler.init();
-      sps.init(context, new ExternalSPSFileIDCollector(context, sps),
-          externalHandler, blkMoveListener);
+      sps.init(context, new ExternalSPSFilePathCollector(sps), externalHandler,
+          blkMoveListener);
       sps.start(true, StoragePolicySatisfierMode.EXTERNAL);
       if (sps != null) {
         sps.join();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8467ec24/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/NamenodeProtocol.proto
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/NamenodeProtocol.proto b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/NamenodeProtocol.proto
index b0e900d..b137f2f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/NamenodeProtocol.proto
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/NamenodeProtocol.proto
@@ -214,11 +214,11 @@ message GetFilePathResponseProto {
   required string srcPath = 1;
 }
 
-message GetNextSPSPathIdRequestProto {
+message GetNextSPSPathRequestProto {
 }
 
-message GetNextSPSPathIdResponseProto {
-  optional uint64 fileId = 1;
+message GetNextSPSPathResponseProto {
+  optional string spsPath = 1;
 }
 
 message CheckDNSpaceRequestProto {
@@ -322,26 +322,15 @@ service NamenodeProtocolService {
       returns (IsRollingUpgradeResponseProto);
 
   /**
-   * Return the corresponding file path for give file id
+   * Return the sps path from namenode
    */
-  rpc getFilePath(GetFilePathRequestProto)
-      returns (GetFilePathResponseProto);
+  rpc getNextSPSPath(GetNextSPSPathRequestProto)
+      returns (GetNextSPSPathResponseProto);
 
   /**
-   * Return the sps path id from namenode
-   */
-  rpc getNextSPSPathId(GetNextSPSPathIdRequestProto)
-      returns (GetNextSPSPathIdResponseProto);
-
-  /**
-   * Return the sps path id from namenode
+   * Verifies whether the given Datanode has the enough estimated size with
+   * given storage type for scheduling the block movement.
    */
   rpc checkDNSpaceForScheduling(CheckDNSpaceRequestProto)
       returns (CheckDNSpaceResponseProto);
-
-  /**
-   * check whether given file id has low redundancy blocks.
-   */
-  rpc hasLowRedundancyBlocks(HasLowRedundancyBlocksRequestProto)
-      returns (HasLowRedundancyBlocksResponseProto);
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8467ec24/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/sps/TestBlockStorageMovementAttemptedItems.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/sps/TestBlockStorageMovementAttemptedItems.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/sps/TestBlockStorageMovementAttemptedItems.java
index 4097339..29af885 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/sps/TestBlockStorageMovementAttemptedItems.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/sps/TestBlockStorageMovementAttemptedItems.java
@@ -40,22 +40,21 @@ import org.mockito.Mockito;
  */
 public class TestBlockStorageMovementAttemptedItems {
 
-  private BlockStorageMovementAttemptedItems bsmAttemptedItems = null;
-  private BlockStorageMovementNeeded unsatisfiedStorageMovementFiles = null;
+  private BlockStorageMovementAttemptedItems<Long> bsmAttemptedItems;
+  private BlockStorageMovementNeeded<Long> unsatisfiedStorageMovementFiles;
   private final int selfRetryTimeout = 500;
 
   @Before
   public void setup() throws Exception {
     Configuration config = new HdfsConfiguration();
-    Context ctxt = Mockito.mock(Context.class);
-    SPSService sps = Mockito.mock(StoragePolicySatisfier.class);
-    Mockito.when(sps.getConf()).thenReturn(config);
+    Context<Long> ctxt = Mockito.mock(IntraSPSNameNodeContext.class);
+    SPSService<Long> sps = new StoragePolicySatisfier<Long>(config);
     Mockito.when(ctxt.isRunning()).thenReturn(true);
     Mockito.when(ctxt.isInSafeMode()).thenReturn(false);
     Mockito.when(ctxt.isFileExist(Mockito.anyLong())).thenReturn(true);
     unsatisfiedStorageMovementFiles =
-        new BlockStorageMovementNeeded(ctxt, null);
-    bsmAttemptedItems = new BlockStorageMovementAttemptedItems(sps,
+        new BlockStorageMovementNeeded<Long>(ctxt, null);
+    bsmAttemptedItems = new BlockStorageMovementAttemptedItems<Long>(sps,
         unsatisfiedStorageMovementFiles, null);
   }
 
@@ -72,9 +71,9 @@ public class TestBlockStorageMovementAttemptedItems {
     long stopTime = monotonicNow() + (retryTimeout * 2);
     boolean isItemFound = false;
     while (monotonicNow() < (stopTime)) {
-      ItemInfo ele = null;
+      ItemInfo<Long> ele = null;
       while ((ele = unsatisfiedStorageMovementFiles.get()) != null) {
-        if (item == ele.getFileId()) {
+        if (item == ele.getFile()) {
           isItemFound = true;
           break;
         }
@@ -97,7 +96,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, 0));
+    bsmAttemptedItems.add(new AttemptedItemInfo<Long>(0L, 0L, 0L, blocks, 0));
     Block[] blockArray = new Block[blocks.size()];
     blocks.toArray(blockArray);
     bsmAttemptedItems.notifyMovementTriedBlocks(blockArray);
@@ -114,7 +113,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, 0));
+    bsmAttemptedItems.add(new AttemptedItemInfo<Long>(0L, 0L, 0L, blocks, 0));
     assertEquals("Shouldn't receive result", 0,
         bsmAttemptedItems.getMovementFinishedBlocksCount());
     assertEquals("Item doesn't exist in the attempted list", 1,
@@ -135,7 +134,7 @@ public class TestBlockStorageMovementAttemptedItems {
     blocks.add(new Block(5678L));
     Long trackID = 0L;
     bsmAttemptedItems
-        .add(new AttemptedItemInfo(trackID, trackID, 0L, blocks, 0));
+        .add(new AttemptedItemInfo<Long>(trackID, trackID, 0L, blocks, 0));
     Block[] blksMovementReport = new Block[1];
     blksMovementReport[0] = new Block(item);
     bsmAttemptedItems.notifyMovementTriedBlocks(blksMovementReport);
@@ -160,7 +159,7 @@ public class TestBlockStorageMovementAttemptedItems {
     List<Block> blocks = new ArrayList<>();
     blocks.add(new Block(item));
     bsmAttemptedItems
-        .add(new AttemptedItemInfo(trackID, trackID, 0L, blocks, 0));
+        .add(new AttemptedItemInfo<Long>(trackID, trackID, 0L, blocks, 0));
     Block[] blksMovementReport = new Block[1];
     blksMovementReport[0] = new Block(item);
     bsmAttemptedItems.notifyMovementTriedBlocks(blksMovementReport);
@@ -188,7 +187,7 @@ public class TestBlockStorageMovementAttemptedItems {
     List<Block> blocks = new ArrayList<>();
     blocks.add(new Block(item));
     bsmAttemptedItems
-        .add(new AttemptedItemInfo(trackID, trackID, 0L, blocks, 0));
+        .add(new AttemptedItemInfo<Long>(trackID, trackID, 0L, blocks, 0));
     Block[] blksMovementReport = new Block[1];
     blksMovementReport[0] = new Block(item);
     bsmAttemptedItems.notifyMovementTriedBlocks(blksMovementReport);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8467ec24/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/sps/TestStoragePolicySatisfier.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/sps/TestStoragePolicySatisfier.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/sps/TestStoragePolicySatisfier.java
index 6f7fe89..2a3d0c8 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/sps/TestStoragePolicySatisfier.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/sps/TestStoragePolicySatisfier.java
@@ -105,7 +105,7 @@ public class TestStoragePolicySatisfier {
   public static final int NUM_OF_DATANODES = 3;
   public static final int STORAGES_PER_DATANODE = 2;
   public static final long CAPACITY = 2 * 256 * 1024 * 1024;
-  public static final String FILE = "/testMoveWhenStoragePolicyNotSatisfying";
+  public static final String FILE = "/testMoveToSatisfyStoragePolicy";
   public static final int DEFAULT_BLOCK_SIZE = 1024;
 
   /**
@@ -1269,8 +1269,9 @@ public class TestStoragePolicySatisfier {
 
     //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 = new StoragePolicySatisfier(config);
-    Context ctxt = new IntraSPSNameNodeContext(hdfsCluster.getNamesystem(),
+    StoragePolicySatisfier<Long> sps = new StoragePolicySatisfier<Long>(config);
+    Context<Long> ctxt = new IntraSPSNameNodeContext(
+        hdfsCluster.getNamesystem(),
         hdfsCluster.getNamesystem().getBlockManager(), sps) {
       @Override
       public boolean isInSafeMode() {
@@ -1283,7 +1284,7 @@ public class TestStoragePolicySatisfier {
       }
     };
 
-    FileIdCollector fileIDCollector = createFileIdCollector(sps, ctxt);
+    FileCollector<Long> fileIDCollector = createFileIdCollector(sps, ctxt);
     sps.init(ctxt, fileIDCollector, null, null);
     sps.getStorageMovementQueue().activate();
 
@@ -1300,9 +1301,9 @@ public class TestStoragePolicySatisfier {
     dfs.delete(new Path("/root"), true);
   }
 
-  public FileIdCollector createFileIdCollector(StoragePolicySatisfier sps,
-      Context ctxt) {
-    FileIdCollector fileIDCollector = new IntraSPSNameNodeFileIdCollector(
+  public FileCollector<Long> createFileIdCollector(
+      StoragePolicySatisfier<Long> sps, Context<Long> ctxt) {
+    FileCollector<Long> fileIDCollector = new IntraSPSNameNodeFileIdCollector(
         hdfsCluster.getNamesystem().getFSDirectory(), sps);
     return fileIDCollector;
   }
@@ -1337,8 +1338,9 @@ public class TestStoragePolicySatisfier {
 
     // 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 = new StoragePolicySatisfier(config);
-    Context ctxt = new IntraSPSNameNodeContext(hdfsCluster.getNamesystem(),
+    StoragePolicySatisfier<Long> sps = new StoragePolicySatisfier<Long>(config);
+    Context<Long> ctxt = new IntraSPSNameNodeContext(
+        hdfsCluster.getNamesystem(),
         hdfsCluster.getNamesystem().getBlockManager(), sps) {
       @Override
       public boolean isInSafeMode() {
@@ -1350,7 +1352,7 @@ public class TestStoragePolicySatisfier {
         return true;
       }
     };
-    FileIdCollector fileIDCollector = createFileIdCollector(sps, ctxt);
+    FileCollector<Long> fileIDCollector = createFileIdCollector(sps, ctxt);
     sps.init(ctxt, fileIDCollector, null, null);
     sps.getStorageMovementQueue().activate();
 
@@ -1368,16 +1370,16 @@ public class TestStoragePolicySatisfier {
   }
 
   private void assertTraversal(List<String> expectedTraverseOrder,
-      FSDirectory fsDir, StoragePolicySatisfier sps)
+      FSDirectory fsDir, StoragePolicySatisfier<Long> sps)
           throws InterruptedException {
     // Remove 10 element and make queue free, So other traversing will start.
     for (int i = 0; i < 10; i++) {
       String path = expectedTraverseOrder.remove(0);
-      ItemInfo itemInfo = sps.getStorageMovementQueue().get();
+      ItemInfo<Long> itemInfo = sps.getStorageMovementQueue().get();
       if (itemInfo == null) {
         continue;
       }
-      long trackId = itemInfo.getFileId();
+      Long trackId = itemInfo.getFile();
       INode inode = fsDir.getInode(trackId);
       assertTrue("Failed to traverse tree, expected " + path + " but got "
           + inode.getFullPathName(), path.equals(inode.getFullPathName()));
@@ -1388,11 +1390,11 @@ public class TestStoragePolicySatisfier {
     // 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) {
-      ItemInfo itemInfo = sps.getStorageMovementQueue().get();
+      ItemInfo<Long> itemInfo = sps.getStorageMovementQueue().get();
       if (itemInfo == null) {
         continue;
       }
-      long trackId = itemInfo.getFileId();
+      Long trackId = itemInfo.getFile();
       INode inode = fsDir.getInode(trackId);
       assertTrue("Failed to traverse tree, expected " + path + " but got "
           + inode.getFullPathName(), path.equals(inode.getFullPathName()));
@@ -1696,39 +1698,41 @@ public class TestStoragePolicySatisfier {
     return file1;
   }
 
-  private void waitForAttemptedItems(long expectedBlkMovAttemptedCount,
+  public void waitForAttemptedItems(long expectedBlkMovAttemptedCount,
       int timeout) throws TimeoutException, InterruptedException {
     BlockManager blockManager = hdfsCluster.getNamesystem().getBlockManager();
-    final StoragePolicySatisfier sps = (StoragePolicySatisfier) blockManager
-        .getSPSManager().getInternalSPSService();
+    final StoragePolicySatisfier<Long> sps =
+        (StoragePolicySatisfier<Long>) blockManager.getSPSManager()
+        .getInternalSPSService();
     GenericTestUtils.waitFor(new Supplier<Boolean>() {
       @Override
       public Boolean get() {
         LOG.info("expectedAttemptedItemsCount={} actualAttemptedItemsCount={}",
             expectedBlkMovAttemptedCount,
-            ((BlockStorageMovementAttemptedItems) (sps
+            ((BlockStorageMovementAttemptedItems<Long>) (sps
                 .getAttemptedItemsMonitor())).getAttemptedItemsCount());
-        return ((BlockStorageMovementAttemptedItems) (sps
+        return ((BlockStorageMovementAttemptedItems<Long>) (sps
             .getAttemptedItemsMonitor()))
             .getAttemptedItemsCount() == expectedBlkMovAttemptedCount;
       }
     }, 100, timeout);
   }
 
-  private void waitForBlocksMovementAttemptReport(
+  public void waitForBlocksMovementAttemptReport(
       long expectedMovementFinishedBlocksCount, int timeout)
           throws TimeoutException, InterruptedException {
     BlockManager blockManager = hdfsCluster.getNamesystem().getBlockManager();
-    final StoragePolicySatisfier sps = (StoragePolicySatisfier) blockManager
+    final StoragePolicySatisfier<Long> sps =
+        (StoragePolicySatisfier<Long>) blockManager
         .getSPSManager().getInternalSPSService();
     GenericTestUtils.waitFor(new Supplier<Boolean>() {
       @Override
       public Boolean get() {
         LOG.info("MovementFinishedBlocks: expectedCount={} actualCount={}",
             expectedMovementFinishedBlocksCount,
-            ((BlockStorageMovementAttemptedItems) (sps
+            ((BlockStorageMovementAttemptedItems<Long>) (sps
                 .getAttemptedItemsMonitor())).getMovementFinishedBlocksCount());
-        return ((BlockStorageMovementAttemptedItems) (sps
+        return ((BlockStorageMovementAttemptedItems<Long>) (sps
             .getAttemptedItemsMonitor()))
                 .getMovementFinishedBlocksCount()
             >= expectedMovementFinishedBlocksCount;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8467ec24/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/sps/TestStoragePolicySatisfierWithStripedFile.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/sps/TestStoragePolicySatisfierWithStripedFile.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/sps/TestStoragePolicySatisfierWithStripedFile.java
index ef12300..a39fb92 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/sps/TestStoragePolicySatisfierWithStripedFile.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/sps/TestStoragePolicySatisfierWithStripedFile.java
@@ -495,16 +495,17 @@ public class TestStoragePolicySatisfierWithStripedFile {
       long expectedBlkMovAttemptedCount, int timeout)
           throws TimeoutException, InterruptedException {
     BlockManager blockManager = cluster.getNamesystem().getBlockManager();
-    final StoragePolicySatisfier sps = (StoragePolicySatisfier) blockManager
+    final StoragePolicySatisfier<Long> sps =
+        (StoragePolicySatisfier<Long>) blockManager
         .getSPSManager().getInternalSPSService();
     GenericTestUtils.waitFor(new Supplier<Boolean>() {
       @Override
       public Boolean get() {
         LOG.info("expectedAttemptedItemsCount={} actualAttemptedItemsCount={}",
             expectedBlkMovAttemptedCount,
-            ((BlockStorageMovementAttemptedItems) sps
+            ((BlockStorageMovementAttemptedItems<Long>) sps
                 .getAttemptedItemsMonitor()).getAttemptedItemsCount());
-        return ((BlockStorageMovementAttemptedItems) sps
+        return ((BlockStorageMovementAttemptedItems<Long>) sps
             .getAttemptedItemsMonitor())
                 .getAttemptedItemsCount() == expectedBlkMovAttemptedCount;
       }
@@ -567,7 +568,8 @@ public class TestStoragePolicySatisfierWithStripedFile {
       long expectedMoveFinishedBlks, int timeout)
           throws TimeoutException, InterruptedException {
     BlockManager blockManager = cluster.getNamesystem().getBlockManager();
-    final StoragePolicySatisfier sps = (StoragePolicySatisfier) blockManager
+    final StoragePolicySatisfier<Long> sps =
+        (StoragePolicySatisfier<Long>) blockManager
         .getSPSManager().getInternalSPSService();
     Assert.assertNotNull("Failed to get SPS object reference!", sps);
 
@@ -575,9 +577,10 @@ public class TestStoragePolicySatisfierWithStripedFile {
       @Override
       public Boolean get() {
         LOG.info("MovementFinishedBlocks: expectedCount={} actualCount={}",
-            expectedMoveFinishedBlks, ((BlockStorageMovementAttemptedItems) sps
+            expectedMoveFinishedBlks,
+            ((BlockStorageMovementAttemptedItems<Long>) sps
                 .getAttemptedItemsMonitor()).getMovementFinishedBlocksCount());
-        return ((BlockStorageMovementAttemptedItems) sps
+        return ((BlockStorageMovementAttemptedItems<Long>) sps
             .getAttemptedItemsMonitor())
                 .getMovementFinishedBlocksCount() >= expectedMoveFinishedBlks;
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8467ec24/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/sps/TestExternalStoragePolicySatisfier.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/sps/TestExternalStoragePolicySatisfier.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/sps/TestExternalStoragePolicySatisfier.java
index 0546f39..28e172a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/sps/TestExternalStoragePolicySatisfier.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/sps/TestExternalStoragePolicySatisfier.java
@@ -43,23 +43,23 @@ import java.util.ArrayList;
 import java.util.Collection;
 import java.util.List;
 import java.util.Properties;
+import java.util.concurrent.TimeoutException;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileUtil;
 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.DFSUtil;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.StoragePolicySatisfierMode;
 import org.apache.hadoop.hdfs.server.balancer.NameNodeConnector;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.namenode.sps.BlockMovementListener;
-import org.apache.hadoop.hdfs.server.namenode.sps.Context;
-import org.apache.hadoop.hdfs.server.namenode.sps.FileIdCollector;
-import org.apache.hadoop.hdfs.server.namenode.sps.SPSService;
+import org.apache.hadoop.hdfs.server.namenode.sps.BlockStorageMovementAttemptedItems;
 import org.apache.hadoop.hdfs.server.namenode.sps.StoragePolicySatisfier;
 import org.apache.hadoop.hdfs.server.namenode.sps.TestStoragePolicySatisfier;
 import org.apache.hadoop.http.HttpConfig;
@@ -74,6 +74,8 @@ import org.junit.Assert;
 import org.junit.Ignore;
 import org.junit.Test;
 
+import com.google.common.base.Supplier;
+
 /**
  * Tests the external sps service plugins.
  */
@@ -88,6 +90,8 @@ public class TestExternalStoragePolicySatisfier
   private String principal;
   private MiniKdc kdc;
   private File baseDir;
+  private StoragePolicySatisfier<String> externalSps;
+  private ExternalSPSContext externalCtxt;
 
   @After
   public void destroy() throws Exception {
@@ -98,6 +102,14 @@ public class TestExternalStoragePolicySatisfier
   }
 
   @Override
+  public void shutdownCluster() {
+    if (externalSps != null) {
+      externalSps.stopGracefully();
+    }
+    super.shutdownCluster();
+  }
+
+  @Override
   public void setUp() {
     super.setUp();
 
@@ -131,60 +143,44 @@ public class TestExternalStoragePolicySatisfier
 
     nnc = getNameNodeConnector(getConf());
 
-    BlockManager blkMgr = cluster.getNameNode().getNamesystem()
-        .getBlockManager();
-    SPSService spsService = blkMgr.getSPSManager().getInternalSPSService();
-    spsService.stopGracefully();
-
-    ExternalSPSContext context = new ExternalSPSContext(spsService,
+    externalSps = new StoragePolicySatisfier<String>(getConf());
+    externalCtxt = new ExternalSPSContext(externalSps,
         getNameNodeConnector(conf));
 
     ExternalBlockMovementListener blkMoveListener =
         new ExternalBlockMovementListener();
     ExternalSPSBlockMoveTaskHandler externalHandler =
         new ExternalSPSBlockMoveTaskHandler(conf, nnc,
-            blkMgr.getSPSManager().getInternalSPSService());
+            externalSps);
     externalHandler.init();
-    spsService.init(context,
-        new ExternalSPSFileIDCollector(context,
-            blkMgr.getSPSManager().getInternalSPSService()),
-        externalHandler, blkMoveListener);
-    spsService.start(true, StoragePolicySatisfierMode.EXTERNAL);
+    externalSps.init(externalCtxt,
+        new ExternalSPSFilePathCollector(externalSps), externalHandler,
+        blkMoveListener);
+    externalSps.start(true, StoragePolicySatisfierMode.EXTERNAL);
     return cluster;
   }
 
   public void restartNamenode() throws IOException{
-    BlockManager blkMgr = getCluster().getNameNode().getNamesystem()
-        .getBlockManager();
-    SPSService spsService = blkMgr.getSPSManager().getInternalSPSService();
-    spsService.stopGracefully();
+    if (externalSps != null) {
+      externalSps.stopGracefully();
+    }
 
     getCluster().restartNameNodes();
     getCluster().waitActive();
-    blkMgr = getCluster().getNameNode().getNamesystem()
-        .getBlockManager();
-    spsService = blkMgr.getSPSManager().getInternalSPSService();
-    spsService.stopGracefully();
+    externalSps = new StoragePolicySatisfier<>(getConf());
 
-    ExternalSPSContext context = new ExternalSPSContext(spsService,
+    externalCtxt = new ExternalSPSContext(externalSps,
         getNameNodeConnector(getConf()));
     ExternalBlockMovementListener blkMoveListener =
         new ExternalBlockMovementListener();
     ExternalSPSBlockMoveTaskHandler externalHandler =
         new ExternalSPSBlockMoveTaskHandler(getConf(), nnc,
-            blkMgr.getSPSManager().getInternalSPSService());
+            externalSps);
     externalHandler.init();
-    spsService.init(context,
-        new ExternalSPSFileIDCollector(context,
-            blkMgr.getSPSManager().getInternalSPSService()),
-        externalHandler, blkMoveListener);
-    spsService.start(true, StoragePolicySatisfierMode.EXTERNAL);
-  }
-
-  @Override
-  public FileIdCollector createFileIdCollector(StoragePolicySatisfier sps,
-      Context ctxt) {
-    return new ExternalSPSFileIDCollector(ctxt, sps);
+    externalSps.init(externalCtxt,
+        new ExternalSPSFilePathCollector(externalSps), externalHandler,
+        blkMoveListener);
+    externalSps.start(true, StoragePolicySatisfierMode.EXTERNAL);
   }
 
   private class ExternalBlockMovementListener implements BlockMovementListener {
@@ -204,7 +200,7 @@ public class TestExternalStoragePolicySatisfier
       throws IOException {
     final Collection<URI> namenodes = DFSUtil.getInternalNsRpcUris(conf);
     Assert.assertEquals(1, namenodes.size());
-    final Path externalSPSPathId = new Path("/system/tmp.id");
+    final Path externalSPSPathId = HdfsServerConstants.MOVER_ID_PATH;
     NameNodeConnector.checkOtherInstanceRunning(false);
     while (true) {
       try {
@@ -222,6 +218,40 @@ public class TestExternalStoragePolicySatisfier
     }
   }
 
+  public void waitForAttemptedItems(long expectedBlkMovAttemptedCount,
+      int timeout) throws TimeoutException, InterruptedException {
+    GenericTestUtils.waitFor(new Supplier<Boolean>() {
+      @Override
+      public Boolean get() {
+        LOG.info("expectedAttemptedItemsCount={} actualAttemptedItemsCount={}",
+            expectedBlkMovAttemptedCount,
+            ((BlockStorageMovementAttemptedItems<String>) (externalSps
+                .getAttemptedItemsMonitor())).getAttemptedItemsCount());
+        return ((BlockStorageMovementAttemptedItems<String>) (externalSps
+            .getAttemptedItemsMonitor()))
+            .getAttemptedItemsCount() == expectedBlkMovAttemptedCount;
+      }
+    }, 100, timeout);
+  }
+
+  public void waitForBlocksMovementAttemptReport(
+      long expectedMovementFinishedBlocksCount, int timeout)
+          throws TimeoutException, InterruptedException {
+    GenericTestUtils.waitFor(new Supplier<Boolean>() {
+      @Override
+      public Boolean get() {
+        LOG.info("MovementFinishedBlocks: expectedCount={} actualCount={}",
+            expectedMovementFinishedBlocksCount,
+            ((BlockStorageMovementAttemptedItems<String>) (externalSps
+                .getAttemptedItemsMonitor())).getMovementFinishedBlocksCount());
+        return ((BlockStorageMovementAttemptedItems<String>) (externalSps
+            .getAttemptedItemsMonitor()))
+                .getMovementFinishedBlocksCount()
+            >= expectedMovementFinishedBlocksCount;
+      }
+    }, 100, timeout);
+  }
+
   private void initSecureConf(Configuration conf) throws Exception {
     String username = "externalSPS";
     baseDir = GenericTestUtils
@@ -321,10 +351,6 @@ public class TestExternalStoragePolicySatisfier
       List<String> files = new ArrayList<>();
       files.add(FILE);
       DistributedFileSystem fs = getFS();
-      BlockManager blkMgr = getCluster().getNameNode().getNamesystem()
-          .getBlockManager();
-      SPSService spsService = blkMgr.getSPSManager().getInternalSPSService();
-      spsService.stopGracefully(); // stops SPS
 
       // Creates 4 more files. Send all of them for satisfying the storage
       // policy together.
@@ -367,6 +393,28 @@ public class TestExternalStoragePolicySatisfier
   }
 
   /**
+   * 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 the case by creating MOVER_ID file
+      DFSTestUtil.createFile(getCluster().getFileSystem(),
+          HdfsServerConstants.MOVER_ID_PATH, 0, (short) 1, 0);
+      restartNamenode();
+      boolean running = externalCtxt.isRunning();
+      Assert.assertTrue("SPS should be running as "
+          + "no Mover really running", running);
+    } finally {
+      shutdownCluster();
+    }
+  }
+
+  /**
    * This test need not run as external scan is not a batch based scanning right
    * now.
    */
@@ -389,4 +437,20 @@ public class TestExternalStoragePolicySatisfier
   @Ignore("Status is not supported for external SPS. So, ignoring it.")
   public void testMaxRetryForFailedBlock() throws Exception {
   }
+
+  /**
+   * This test is specific to internal SPS. So, ignoring it.
+   */
+  @Ignore("This test is specific to internal SPS. So, ignoring it.")
+  @Override
+  public void testTraverseWhenParentDeleted() throws Exception {
+  }
+
+  /**
+   * This test is specific to internal SPS. So, ignoring it.
+   */
+  @Ignore("This test is specific to internal SPS. So, ignoring it.")
+  @Override
+  public void testTraverseWhenRootParentDeleted() throws Exception {
+  }
 }


---------------------------------------------------------------------
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-11874. [SPS]: Document the SPS feature. Contributed by Uma Maheswara Rao G

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

Branch: refs/heads/HDFS-10285
Commit: 68af4e199a754ca6c727b844a22ecabe9dc7cc68
Parents: 5eb24ef
Author: Rakesh Radhakrishnan <ra...@apache.org>
Authored: Fri Jul 14 22:36:09 2017 +0530
Committer: Uma Maheswara Rao Gangumalla <um...@apache.org>
Committed: Sun Aug 12 03:06:00 2018 -0700

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


http://git-wip-us.apache.org/repos/asf/hadoop/blob/68af4e19/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 a56cf8b..9098616 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/ArchivalStorage.md
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/ArchivalStorage.md
@@ -97,8 +97,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.
 
@@ -115,6 +151,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
 -----------------------
 
@@ -173,7 +213,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:
 
@@ -193,4 +234,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


[12/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 um...@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/bfd3f8bd
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/bfd3f8bd
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/bfd3f8bd

Branch: refs/heads/HDFS-10285
Commit: bfd3f8bd8a9ae2186ec3e4addc71f912ec7b8923
Parents: 7ea24fc
Author: Uma Maheswara Rao G <um...@intel.com>
Authored: Sat Sep 30 06:31:52 2017 -0700
Committer: Uma Maheswara Rao Gangumalla <um...@apache.org>
Committed: Sun Aug 12 03:06:01 2018 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/hdfs/DFSConfigKeys.java   |   8 +
 .../java/org/apache/hadoop/hdfs/DFSUtil.java    |  22 +-
 .../BlockStorageMovementAttemptedItems.java     |   8 +-
 .../namenode/BlockStorageMovementNeeded.java    | 277 +++++++++++---
 .../server/namenode/ReencryptionHandler.java    |   1 +
 .../server/namenode/StoragePolicySatisfier.java |  43 ++-
 .../src/main/resources/hdfs-default.xml         |  23 ++
 .../src/site/markdown/ArchivalStorage.md        |   3 +-
 .../TestBlockStorageMovementAttemptedItems.java |   2 +-
 .../TestPersistentStoragePolicySatisfier.java   |   8 +-
 .../namenode/TestStoragePolicySatisfier.java    | 377 ++++++++++++++++++-
 11 files changed, 689 insertions(+), 83 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/bfd3f8bd/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 e66806f..c90ca33 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
@@ -618,6 +618,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/bfd3f8bd/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 f5ceeaf..c26599c 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
@@ -1457,7 +1457,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/bfd3f8bd/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/bfd3f8bd/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/bfd3f8bd/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 b92fe9f..feacd74 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
@@ -20,6 +20,7 @@ package org.apache.hadoop.hdfs.server.namenode;
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
 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;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bfd3f8bd/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/bfd3f8bd/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 17f7795..41a74a7 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
@@ -4510,6 +4510,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/bfd3f8bd/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 f6bbd10..c8a9466 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/ArchivalStorage.md
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/ArchivalStorage.md
@@ -112,7 +112,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`
@@ -214,7 +214,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/bfd3f8bd/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/bfd3f8bd/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/bfd3f8bd/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


[18/50] [abbrv] hadoop git commit: HDFS-12955: [SPS]: Move SPS classes to a separate package. Contributed by Rakesh R.

Posted by um...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/78420719/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/sps/TestStoragePolicySatisfier.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/sps/TestStoragePolicySatisfier.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/sps/TestStoragePolicySatisfier.java
new file mode 100644
index 0000000..8dc52dc
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/sps/TestStoragePolicySatisfier.java
@@ -0,0 +1,1779 @@
+/**
+ * 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.sps;
+
+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 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;
+import java.net.InetSocketAddress;
+import java.util.ArrayList;
+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;
+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.MiniDFSCluster.DataNodeProperties;
+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;
+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;
+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.hdfs.server.namenode.FSDirectory;
+import org.apache.hadoop.hdfs.server.namenode.FSEditLog;
+import org.apache.hadoop.hdfs.server.namenode.FSTreeTraverser;
+import org.apache.hadoop.hdfs.server.namenode.INode;
+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;
+
+/**
+ * Tests that StoragePolicySatisfier daemon is able to check the blocks to be
+ * 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 =
+      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 dfs = null;
+  private static final int DEFAULT_BLOCK_SIZE = 1024;
+
+  private void shutdownCluster() {
+    if (hdfsCluster != null) {
+      hdfsCluster.shutdown();
+    }
+  }
+
+  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();
+    writeContent(file);
+  }
+
+  @Test(timeout = 300000)
+  public void testWhenStoragePolicySetToCOLD()
+      throws Exception {
+
+    try {
+      createCluster();
+      doTestWhenStoragePolicySetToCOLD();
+    } finally {
+      shutdownCluster();
+    }
+  }
+
+  private void doTestWhenStoragePolicySetToCOLD() throws Exception {
+    // 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);
+
+    dfs.satisfyStoragePolicy(new Path(file));
+
+    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 {
+    try {
+      createCluster();
+      // Change policy to ALL_SSD
+      dfs.setStoragePolicy(new Path(file), "ALL_SSD");
+
+      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);
+      dfs.satisfyStoragePolicy(new Path(file));
+      hdfsCluster.triggerHeartbeats();
+      // Wait till StorgePolicySatisfier Identified that block to move to SSD
+      // areas
+      DFSTestUtil.waitExpectedStorageType(
+          file, StorageType.SSD, 3, 30000, dfs);
+    } finally {
+      shutdownCluster();
+    }
+  }
+
+  @Test(timeout = 300000)
+  public void testWhenStoragePolicySetToONESSD()
+      throws Exception {
+    try {
+      createCluster();
+      // Change policy to ONE_SSD
+      dfs.setStoragePolicy(new Path(file), ONE_SSD);
+
+      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);
+      dfs.satisfyStoragePolicy(new Path(file));
+      hdfsCluster.triggerHeartbeats();
+      // Wait till StorgePolicySatisfier Identified that block to move to SSD
+      // areas
+      DFSTestUtil.waitExpectedStorageType(
+          file, StorageType.SSD, 1, 30000, dfs);
+      DFSTestUtil.waitExpectedStorageType(
+          file, StorageType.DISK, 2, 30000, dfs);
+    } finally {
+      shutdownCluster();
+    }
+  }
+
+  /**
+   * Tests to verify that the block storage movement report will be propagated
+   * to Namenode via datanode heartbeat.
+   */
+  @Test(timeout = 300000)
+  public void testBlksStorageMovementAttemptFinishedReport() throws Exception {
+    try {
+      createCluster();
+      // Change policy to ONE_SSD
+      dfs.setStoragePolicy(new Path(file), ONE_SSD);
+
+      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);
+      dfs.satisfyStoragePolicy(new Path(file));
+      hdfsCluster.triggerHeartbeats();
+
+      // Wait till the block is moved to SSD areas
+      DFSTestUtil.waitExpectedStorageType(
+          file, StorageType.SSD, 1, 30000, dfs);
+      DFSTestUtil.waitExpectedStorageType(
+          file, StorageType.DISK, 2, 30000, dfs);
+
+      waitForBlocksMovementAttemptReport(1, 30000);
+    } finally {
+      shutdownCluster();
+    }
+  }
+
+  /**
+   * 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 {
+    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);
+      }
+      // Change policy to ONE_SSD
+      for (String fileName : files) {
+        dfs.setStoragePolicy(new Path(fileName), ONE_SSD);
+        dfs.satisfyStoragePolicy(new Path(fileName));
+      }
+
+      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);
+      hdfsCluster.triggerHeartbeats();
+
+      for (String fileName : files) {
+        // Wait till the block is moved to SSD areas
+        DFSTestUtil.waitExpectedStorageType(
+            fileName, StorageType.SSD, 1, 30000, dfs);
+        DFSTestUtil.waitExpectedStorageType(
+            fileName, StorageType.DISK, 2, 30000, dfs);
+      }
+
+      waitForBlocksMovementAttemptReport(files.size(), 30000);
+    } finally {
+      shutdownCluster();
+    }
+  }
+
+  /**
+   * Tests to verify hdfsAdmin.satisfyStoragePolicy works well for file.
+   * @throws Exception
+   */
+  @Test(timeout = 300000)
+  public void testSatisfyFileWithHdfsAdmin() throws Exception {
+    try {
+      createCluster();
+      HdfsAdmin hdfsAdmin =
+          new HdfsAdmin(FileSystem.getDefaultUri(config), config);
+      // Change policy to COLD
+      dfs.setStoragePolicy(new Path(file), COLD);
+
+      StorageType[][] newtypes =
+          new StorageType[][]{{StorageType.DISK, StorageType.ARCHIVE},
+              {StorageType.DISK, StorageType.ARCHIVE},
+              {StorageType.DISK, 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
+      DFSTestUtil.waitExpectedStorageType(
+          file, StorageType.ARCHIVE, 3, 30000, dfs);
+    } finally {
+      shutdownCluster();
+    }
+  }
+
+  /**
+   * Tests to verify hdfsAdmin.satisfyStoragePolicy works well for dir.
+   * @throws Exception
+   */
+  @Test(timeout = 300000)
+  public void testSatisfyDirWithHdfsAdmin() throws Exception {
+    try {
+      createCluster();
+      HdfsAdmin hdfsAdmin =
+          new HdfsAdmin(FileSystem.getDefaultUri(config), config);
+      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.
+      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.
+      DFSTestUtil.waitExpectedStorageType(
+          subFile2, StorageType.SSD, 1, 30000, dfs);
+      DFSTestUtil.waitExpectedStorageType(
+          subFile2, StorageType.DISK, 2, 30000, dfs);
+    } finally {
+      shutdownCluster();
+    }
+  }
+
+  /**
+   * Tests to verify hdfsAdmin.satisfyStoragePolicy exceptions.
+   * @throws Exception
+   */
+  @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);
+      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) {
+
+      }
+
+      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();
+    }
+  }
+
+  /**
+   * 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 {
+      createCluster();
+      // Change policy to COLD
+      dfs.setStoragePolicy(new Path(file), COLD);
+
+      StorageType[][] newtypes =
+          new StorageType[][]{{StorageType.ARCHIVE, StorageType.ARCHIVE}};
+
+      // Adding ARCHIVE based datanodes.
+      startAdditionalDNs(config, 1, numOfDatanodes, newtypes,
+          storagesPerDatanode, capacity, hdfsCluster);
+
+      dfs.satisfyStoragePolicy(new Path(file));
+      hdfsCluster.triggerHeartbeats();
+      // Wait till StorgePolicySatisfier identified that block to move to
+      // ARCHIVE area.
+      DFSTestUtil.waitExpectedStorageType(
+          file, StorageType.ARCHIVE, 1, 30000, dfs);
+      DFSTestUtil.waitExpectedStorageType(
+          file, StorageType.DISK, 2, 30000, dfs);
+
+      waitForBlocksMovementAttemptReport(1, 30000);
+    } finally {
+      shutdownCluster();
+    }
+  }
+
+  /**
+   * 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 {
+      createCluster();
+      // Change policy to COLD
+      dfs.setStoragePolicy(new Path(file), COLD);
+
+      StorageType[][] newtypes =
+          new StorageType[][]{{StorageType.DISK, StorageType.DISK}};
+      // Adding DISK based datanodes
+      startAdditionalDNs(config, 1, numOfDatanodes, newtypes,
+          storagesPerDatanode, capacity, hdfsCluster);
+
+      dfs.satisfyStoragePolicy(new Path(file));
+      hdfsCluster.triggerHeartbeats();
+
+      // 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);
+      // Since there is no target node the item will get timed out and then
+      // re-attempted.
+      waitForAttemptedItems(1, 30000);
+    } finally {
+      shutdownCluster();
+    }
+  }
+
+  /**
+   * Tests to verify that SPS should not start when a Mover instance
+   * is running.
+   */
+  @Test(timeout = 300000)
+  public void testWhenMoverIsAlreadyRunningBeforeStoragePolicySatisfier()
+      throws Exception {
+    boolean running;
+    FSDataOutputStream out = null;
+    try {
+      createCluster();
+      // Stop SPS
+      hdfsCluster.getNameNode().reconfigureProperty(
+          DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_ENABLED_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().reconfigureProperty(
+          DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_ENABLED_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().reconfigureProperty(
+          DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_ENABLED_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_ENABLED_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 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.assertTrue("SPS should be running as "
+          + "no Mover really running", running);
+    } finally {
+      if (hdfsCluster != null) {
+        hdfsCluster.shutdown();
+      }
+    }
+  }
+
+  /**
+   * 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 {
+    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();
+
+      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);
+
+      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);
+
+      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);
+      waitForBlocksMovementAttemptReport(1, 30000);
+      DFSTestUtil.waitExpectedStorageType(
+          file1, StorageType.ARCHIVE, 1, 30000, dfs);
+      DFSTestUtil.waitExpectedStorageType(
+          file1, StorageType.DISK, 2, 30000, dfs);
+    } finally {
+      if (hdfsCluster != null) {
+        hdfsCluster.shutdown();
+      }
+    }
+  }
+
+  /**
+   * 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);
+      config.setBoolean(DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_ENABLED_KEY,
+          true);
+      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);
+
+      dfs.satisfyStoragePolicy(new Path(file));
+      hdfsCluster.triggerHeartbeats();
+      // Wait till StorgePolicySatisfier identified that block to move to
+      // ARCHIVE area.
+      DFSTestUtil.waitExpectedStorageType(
+          file, StorageType.ARCHIVE, 2, 30000, dfs);
+      DFSTestUtil.waitExpectedStorageType(
+          file, StorageType.DISK, 3, 30000, dfs);
+
+      waitForBlocksMovementAttemptReport(1, 30000);
+    } finally {
+      shutdownCluster();
+    }
+  }
+
+  /**
+   * 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);
+    config.setBoolean(DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_ENABLED_KEY,
+        true);
+    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);
+
+      dfs.satisfyStoragePolicy(new Path(file));
+      hdfsCluster.triggerHeartbeats();
+      DFSTestUtil.waitExpectedStorageType(
+          file, StorageType.SSD, 1, 30000, dfs);
+      DFSTestUtil.waitExpectedStorageType(
+          file, StorageType.DISK, 2, 30000, dfs);
+
+    } 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);
+    config.setBoolean(DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_ENABLED_KEY,
+        true);
+    try {
+      hdfsCluster = startCluster(config, diskTypes, diskTypes.length,
+          storagesPerDatanode, capacity);
+      dfs = hdfsCluster.getFileSystem();
+      writeContent(file);
+
+      // Change policy to WARM
+      dfs.setStoragePolicy(new Path(file), "WARM");
+      dfs.satisfyStoragePolicy(new Path(file));
+      hdfsCluster.triggerHeartbeats();
+
+      DFSTestUtil.waitExpectedStorageType(
+          file, StorageType.DISK, 1, 30000, dfs);
+      DFSTestUtil.waitExpectedStorageType(
+          file, StorageType.ARCHIVE, 2, 30000, dfs);
+    } finally {
+      shutdownCluster();
+    }
+  }
+
+  /**
+   * 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 {
+      config.setBoolean(DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_ENABLED_KEY,
+          true);
+      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.
+   */
+  @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);
+    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,
+          storagesPerDatanode, dnCapacity);
+      dfs = hdfsCluster.getFileSystem();
+      writeContent(file);
+
+      // Change policy to ONE_SSD
+      dfs.setStoragePolicy(new Path(file), ONE_SSD);
+      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);
+      }
+      dfs.satisfyStoragePolicy(new Path(file));
+
+      // 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(1));
+      }
+
+      // 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();
+    }
+  }
+
+  /**
+   * 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 =
+        StripedFileTestUtil.getDefaultECPolicy().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);
+    config.setBoolean(DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_ENABLED_KEY,
+        true);
+    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,
+          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,
+          StripedFileTestUtil.getDefaultECPolicy().getName());
+
+      // write file to fooDir
+      final String testFile = "/foo/bar";
+      long fileLen = 20 * defaultStripedBlockSize;
+      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();
+    }
+  }
+
+  /**
+   * 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 {
+      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");
+      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();
+      }
+    }
+  }
+
+  /**
+   * 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.setBoolean(DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_ENABLED_KEY,
+          true);
+      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), false);
+      cluster.restartDataNode(list.get(1), false);
+      cluster.waitActive();
+      fs.satisfyStoragePolicy(filePath);
+      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 {
+      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.setBoolean(DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_ENABLED_KEY,
+          true);
+      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();
+      }
+    }
+  }
+
+  /**
+   * 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 = 3000000)
+  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);
+      }
+      waitForBlocksMovementAttemptReport(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(timeout = 300000)
+  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(hdfsCluster.getNamesystem(), 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(timeout = 300000)
+  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(hdfsCluster.getNamesystem(), 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);
+  }
+
+  /**
+   * 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_LOW_MAX_STREAMS_PREFERENCE_KEY,
+          false);
+
+      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();
+    }
+  }
+
+  @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_LOW_MAX_STREAMS_PREFERENCE_KEY,
+          false);
+
+      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();
+    }
+  }
+
+  @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
+    /*
+     *                           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();
+    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());
+      InternalDataNodeTestUtils.mockDatanodeBlkPinning(dn, true);
+      favoredNodesCount--;
+      if (favoredNodesCount <= 0) {
+        break; // marked favoredNodesCount number of pinned block location
+      }
+    }
+    return file1;
+  }
+
+  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 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("MovementFinishedBlocks: expectedCount={} actualCount={}",
+            expectedMovementFinishedBlocksCount,
+            sps.getAttemptedItemsMonitor().getMovementFinishedBlocksCount());
+        return sps.getAttemptedItemsMonitor().getMovementFinishedBlocksCount()
+            >= expectedMovementFinishedBlocksCount;
+      }
+    }, 100, timeout);
+  }
+
+  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),
+        replicatonFactor);
+    for (int i = 0; i < 1024; i++) {
+      out.write(i);
+    }
+    out.close();
+  }
+
+  private void startAdditionalDNs(final Configuration conf,
+      int newNodesRequired, int existingNodesNum, StorageType[][] newTypes,
+      int storagesPerDn, long nodeCapacity, final MiniDFSCluster cluster)
+          throws IOException {
+    long[][] capacities;
+    existingNodesNum += newNodesRequired;
+    capacities = new long[newNodesRequired][storagesPerDn];
+    for (int i = 0; i < newNodesRequired; i++) {
+      for (int j = 0; j < storagesPerDn; j++) {
+        capacities[i][j] = nodeCapacity;
+      }
+    }
+
+    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;
+  }
+}


---------------------------------------------------------------------
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-12146. [SPS]: Fix TestStoragePolicySatisfierWithStripedFile#testSPSWhenFileHasLowRedundancyBlocks. Contributed by Surendra Singh Lilhore.

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

Branch: refs/heads/HDFS-10285
Commit: 9e82e5a86ea66b9d24d38b922ee5fa97b3391475
Parents: 68af4e1
Author: Rakesh Radhakrishnan <ra...@apache.org>
Authored: Mon Jul 17 22:40:03 2017 +0530
Committer: Uma Maheswara Rao Gangumalla <um...@apache.org>
Committed: Sun Aug 12 03:06:00 2018 -0700

----------------------------------------------------------------------
 .../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/9e82e5a8/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/9e82e5a8/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


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

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

Branch: refs/heads/HDFS-10285
Commit: 4bcf61c696909342f1a238f614d4471c4b6fbad0
Parents: 9e82e5a
Author: Uma Maheswara Rao G <um...@intel.com>
Authored: Mon Jul 17 10:24:06 2017 -0700
Committer: Uma Maheswara Rao Gangumalla <um...@apache.org>
Committed: Sun Aug 12 03:06:00 2018 -0700

----------------------------------------------------------------------
 .../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/4bcf61c6/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 988067c..8b7abaa 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
@@ -427,7 +427,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/4bcf61c6/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/4bcf61c6/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/4bcf61c6/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/4bcf61c6/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 c396387..f0f264c 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
@@ -126,9 +126,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/4bcf61c6/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


[33/50] [abbrv] hadoop git commit: HDFS-13097: [SPS]: Fix the branch review comments(Part1). Contributed by Surendra Singh.

Posted by um...@apache.org.
HDFS-13097: [SPS]: Fix the branch review comments(Part1). Contributed by Surendra Singh.


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

Branch: refs/heads/HDFS-10285
Commit: 4402f3f8557527d5c6cdad6f5bdcbd707b8cbf52
Parents: d3de4fb
Author: Uma Maheswara Rao G <um...@intel.com>
Authored: Wed Feb 7 02:28:23 2018 -0800
Committer: Uma Maheswara Rao Gangumalla <um...@apache.org>
Committed: Sun Aug 12 03:06:04 2018 -0700

----------------------------------------------------------------------
 .../java/org/apache/hadoop/hdfs/DFSClient.java  |   4 +-
 .../hadoop/hdfs/protocol/ClientProtocol.java    |   6 +-
 .../ClientNamenodeProtocolTranslatorPB.java     |  14 +-
 .../src/main/proto/ClientNamenodeProtocol.proto |   8 +-
 .../federation/router/RouterRpcServer.java      |   2 +-
 .../java/org/apache/hadoop/hdfs/DFSUtil.java    |  61 ---
 ...tNamenodeProtocolServerSideTranslatorPB.java |  16 +-
 .../server/blockmanagement/BlockManager.java    | 255 +-----------
 .../blockmanagement/DatanodeDescriptor.java     |  33 +-
 .../hdfs/server/common/HdfsServerConstants.java |   2 +-
 .../datanode/StoragePolicySatisfyWorker.java    |  15 +-
 .../apache/hadoop/hdfs/server/mover/Mover.java  |   2 +-
 .../namenode/FSDirSatisfyStoragePolicyOp.java   |  26 +-
 .../server/namenode/FSDirStatAndListingOp.java  |   1 -
 .../hdfs/server/namenode/FSDirXAttrOp.java      |   2 +-
 .../hdfs/server/namenode/FSDirectory.java       |   2 +-
 .../hdfs/server/namenode/FSNamesystem.java      |  46 +--
 .../hadoop/hdfs/server/namenode/NameNode.java   |  30 +-
 .../hdfs/server/namenode/NameNodeRpcServer.java |  21 +-
 .../sps/BlockStorageMovementNeeded.java         |   4 +-
 .../namenode/sps/IntraSPSNameNodeContext.java   |   6 +-
 .../hdfs/server/namenode/sps/SPSPathIds.java    |  70 ----
 .../hdfs/server/namenode/sps/SPSService.java    |  10 +-
 .../namenode/sps/StoragePolicySatisfier.java    | 137 ++++---
 .../sps/StoragePolicySatisfyManager.java        | 399 +++++++++++++++++++
 .../sps/ExternalStoragePolicySatisfier.java     |   2 +-
 .../hadoop/hdfs/tools/StoragePolicyAdmin.java   |   2 +-
 .../namenode/TestNameNodeReconfigure.java       |  19 +-
 .../TestPersistentStoragePolicySatisfier.java   |   3 +-
 .../TestStoragePolicySatisfierWithHA.java       |   6 +-
 .../sps/TestStoragePolicySatisfier.java         |  35 +-
 ...stStoragePolicySatisfierWithStripedFile.java |   6 +-
 .../sps/TestExternalStoragePolicySatisfier.java |  24 +-
 33 files changed, 665 insertions(+), 604 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/4402f3f8/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 471ab2c..b6f9bdd 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
@@ -3110,8 +3110,8 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
     }
   }
 
-  public boolean isStoragePolicySatisfierRunning() throws IOException {
-    return namenode.isStoragePolicySatisfierRunning();
+  public boolean isInternalSatisfierRunning() throws IOException {
+    return namenode.isInternalSatisfierRunning();
   }
 
   Tracer getTracer() {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4402f3f8/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 360fd63..5c51c22 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
@@ -1759,12 +1759,12 @@ public interface ClientProtocol {
   void satisfyStoragePolicy(String path) throws IOException;
 
   /**
-   * Check if StoragePolicySatisfier is running.
-   * @return true if StoragePolicySatisfier is running
+   * Check if internal StoragePolicySatisfier is running.
+   * @return true if internal StoragePolicySatisfier is running
    * @throws IOException
    */
   @Idempotent
-  boolean isStoragePolicySatisfierRunning() throws IOException;
+  boolean isInternalSatisfierRunning() throws IOException;
 
   /**
    * Check the storage policy satisfy status of the path for which

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4402f3f8/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 cdc8eac..683ccca 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
@@ -150,8 +150,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.IsInternalSatisfierRunningRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.IsInternalSatisfierRunningResponseProto;
 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;
@@ -301,8 +301,8 @@ 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
+  private final static IsInternalSatisfierRunningRequestProto
+      VOID_IS_SPS_RUNNING_REQUEST = IsInternalSatisfierRunningRequestProto
       .newBuilder().build();
 
 
@@ -1912,10 +1912,10 @@ public class ClientNamenodeProtocolTranslatorPB implements
   }
 
   @Override
-  public boolean isStoragePolicySatisfierRunning() throws IOException {
+  public boolean isInternalSatisfierRunning() throws IOException {
     try {
-      IsStoragePolicySatisfierRunningResponseProto rep =
-          rpcProxy.isStoragePolicySatisfierRunning(null,
+      IsInternalSatisfierRunningResponseProto rep =
+          rpcProxy.isInternalSatisfierRunning(null,
               VOID_IS_SPS_RUNNING_REQUEST);
       return rep.getRunning();
     } catch (ServiceException e) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4402f3f8/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 933a19a..e8e3a58 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
@@ -838,10 +838,10 @@ message SatisfyStoragePolicyResponseProto {
 
 }
 
-message IsStoragePolicySatisfierRunningRequestProto {  // no parameters
+message IsInternalSatisfierRunningRequestProto {  // no parameters
 }
 
-message IsStoragePolicySatisfierRunningResponseProto {
+message IsInternalSatisfierRunningResponseProto {
   required bool running = 1;
 }
 
@@ -1048,8 +1048,8 @@ service ClientNamenodeProtocol {
       returns(ListOpenFilesResponseProto);
   rpc satisfyStoragePolicy(SatisfyStoragePolicyRequestProto)
       returns(SatisfyStoragePolicyResponseProto);
-  rpc isStoragePolicySatisfierRunning(IsStoragePolicySatisfierRunningRequestProto)
-      returns(IsStoragePolicySatisfierRunningResponseProto);
+  rpc isInternalSatisfierRunning(IsInternalSatisfierRunningRequestProto)
+      returns(IsInternalSatisfierRunningResponseProto);
   rpc checkStoragePolicySatisfyPathStatus(CheckStoragePolicySatisfyPathStatusRequestProto)
       returns(CheckStoragePolicySatisfyPathStatusResponseProto);
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4402f3f8/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcServer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcServer.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcServer.java
index c5458f0..d93f99d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcServer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcServer.java
@@ -2498,7 +2498,7 @@ public class RouterRpcServer extends AbstractService
   }
 
   @Override
-  public boolean isStoragePolicySatisfierRunning() throws IOException {
+  public boolean isInternalSatisfierRunning() throws IOException {
     checkOperation(OperationCategory.READ, false);
     return false;
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4402f3f8/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 c26599c..23f0478 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,7 +53,6 @@ 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;
@@ -74,7 +73,6 @@ 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;
@@ -1459,26 +1457,6 @@ public class DFSUtil {
   }
 
   /**
-   * 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
    * 
    * @param conf Configuration
@@ -1738,43 +1716,4 @@ 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/4402f3f8/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 09f7ce2..b0816cb 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
@@ -162,8 +162,8 @@ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.IsFile
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.IsFileClosedResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.UpgradeStatusRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.UpgradeStatusResponseProto;
-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.IsInternalSatisfierRunningRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.IsInternalSatisfierRunningResponseProto;
 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;
@@ -1865,14 +1865,14 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
   }
 
   @Override
-  public IsStoragePolicySatisfierRunningResponseProto
-      isStoragePolicySatisfierRunning(RpcController controller,
-      IsStoragePolicySatisfierRunningRequestProto req)
+  public IsInternalSatisfierRunningResponseProto
+      isInternalSatisfierRunning(RpcController controller,
+      IsInternalSatisfierRunningRequestProto req)
       throws ServiceException {
     try {
-      boolean ret = server.isStoragePolicySatisfierRunning();
-      IsStoragePolicySatisfierRunningResponseProto.Builder builder =
-          IsStoragePolicySatisfierRunningResponseProto.newBuilder();
+      boolean ret = server.isInternalSatisfierRunning();
+      IsInternalSatisfierRunningResponseProto.Builder builder =
+          IsInternalSatisfierRunningResponseProto.newBuilder();
       builder.setRunning(ret);
       return builder.build();
     } catch (IOException e) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4402f3f8/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 9205910..e7979b4 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
@@ -69,8 +69,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.protocol.HdfsConstants.DatanodeReportType;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants.StoragePolicySatisfierMode;
-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;
@@ -94,12 +92,7 @@ import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.namenode.Namesystem;
 import org.apache.hadoop.hdfs.server.namenode.ha.HAContext;
 import org.apache.hadoop.hdfs.server.namenode.metrics.NameNodeMetrics;
-import org.apache.hadoop.hdfs.server.namenode.sps.IntraSPSNameNodeBlockMoveTaskHandler;
-import org.apache.hadoop.hdfs.server.namenode.sps.IntraSPSNameNodeContext;
-import org.apache.hadoop.hdfs.server.namenode.sps.IntraSPSNameNodeFileIdCollector;
-import org.apache.hadoop.hdfs.server.namenode.sps.SPSPathIds;
-import org.apache.hadoop.hdfs.server.namenode.sps.SPSService;
-import org.apache.hadoop.hdfs.server.namenode.sps.StoragePolicySatisfier;
+import org.apache.hadoop.hdfs.server.namenode.sps.StoragePolicySatisfyManager;
 import org.apache.hadoop.hdfs.server.protocol.BlockCommand;
 import org.apache.hadoop.hdfs.server.protocol.BlockReportContext;
 import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations;
@@ -435,11 +428,7 @@ public class BlockManager implements BlockStatsMXBean {
   private final BlockIdManager blockIdManager;
 
   /** For satisfying block storage policies. */
-  private final StoragePolicySatisfier sps;
-  private final boolean storagePolicyEnabled;
-  private StoragePolicySatisfierMode spsMode;
-  private SPSPathIds spsPaths;
-  private final int spsOutstandingPathsLimit;
+  private final StoragePolicySatisfyManager spsManager;
 
   /** Minimum live replicas needed for the datanode to be transitioned
    * from ENTERING_MAINTENANCE to IN_MAINTENANCE.
@@ -479,19 +468,10 @@ public class BlockManager implements BlockStatsMXBean {
         DFSConfigKeys.DFS_NAMENODE_RECONSTRUCTION_PENDING_TIMEOUT_SEC_KEY,
         DFSConfigKeys.DFS_NAMENODE_RECONSTRUCTION_PENDING_TIMEOUT_SEC_DEFAULT)
         * 1000L);
-    // StoragePolicySatisfier(SPS) configs
-    storagePolicyEnabled =
-        conf.getBoolean(DFSConfigKeys.DFS_STORAGE_POLICY_ENABLED_KEY,
-            DFSConfigKeys.DFS_STORAGE_POLICY_ENABLED_DEFAULT);
-    String spsModeVal = conf.get(
-        DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_MODE_KEY,
-        DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_MODE_DEFAULT);
-    spsOutstandingPathsLimit = conf.getInt(
-        DFSConfigKeys.DFS_SPS_MAX_OUTSTANDING_PATHS_KEY,
-        DFSConfigKeys.DFS_SPS_MAX_OUTSTANDING_PATHS_DEFAULT);
-    spsMode = StoragePolicySatisfierMode.fromString(spsModeVal);
-    spsPaths = new SPSPathIds();
-    sps = new StoragePolicySatisfier(conf);
+
+    // sps manager manages the user invoked sps paths and does the movement.
+    spsManager = new StoragePolicySatisfyManager(conf, namesystem, this);
+
     blockTokenSecretManager = createBlockTokenSecretManager(conf);
 
     providedStorageMap = new ProvidedStorageMap(namesystem, this, conf);
@@ -719,7 +699,7 @@ public class BlockManager implements BlockStatsMXBean {
   }
 
   public void close() {
-    stopSPS(false);
+    getSPSManager().stop();
     bmSafeMode.close();
     try {
       redundancyThread.interrupt();
@@ -733,7 +713,7 @@ public class BlockManager implements BlockStatsMXBean {
     datanodeManager.close();
     pendingReconstruction.stop();
     blocksMap.close();
-    stopSPSGracefully();
+    getSPSManager().stopGracefully();
   }
 
   /** @return the datanodeManager */
@@ -5046,222 +5026,9 @@ public class BlockManager implements BlockStatsMXBean {
   }
 
   /**
-   * Gets the storage policy satisfier instance.
-   *
-   * @return sps
-   */
-  public StoragePolicySatisfier getStoragePolicySatisfier() {
-    return sps;
-  }
-
-  /**
-   * Start storage policy satisfier service.
-   */
-  public void startSPS() {
-    if (!(storagePolicyEnabled && spsMode != StoragePolicySatisfierMode.NONE)) {
-      LOG.info(
-          "Failed to start StoragePolicySatisfier "
-              + " as {} set to {} and {} set to {}.",
-          DFSConfigKeys.DFS_STORAGE_POLICY_ENABLED_KEY, storagePolicyEnabled,
-          DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_MODE_KEY, spsMode);
-      return;
-    } else if (sps.isRunning()) {
-      LOG.info("Storage policy satisfier is already running"
-          + " as internal service.");
-      return;
-    }
-    // starting internal SPS service
-    if (spsMode == StoragePolicySatisfierMode.INTERNAL) {
-      sps.start(false, spsMode);
-    }
-  }
-
-  /**
-   * Stop storage policy satisfier service.
-   *
-   * @param forceStop
-   *          true represents that it should stop SPS service by clearing all
-   *          pending SPS work
-   */
-  public void stopSPS(boolean forceStop) {
-    if (!(storagePolicyEnabled
-        && (spsMode != StoragePolicySatisfierMode.NONE))) {
-      LOG.info("Storage policy satisfier is not enabled.");
-      return;
-    } else if (!sps.isRunning()) {
-      removeAllSPSPathIds();
-      LOG.info("Storage policy satisfier is not running.");
-      return;
-    }
-
-    sps.disable(forceStop);
-  }
-
-  /**
-   * Enable storage policy satisfier by starting its service.
-   */
-  public void enableInternalSPS() {
-    if (!storagePolicyEnabled){
-      LOG.info("Failed to start StoragePolicySatisfier as {} set to {}.",
-          DFSConfigKeys.DFS_STORAGE_POLICY_ENABLED_KEY, storagePolicyEnabled);
-      return;
-    }
-    if (sps.isRunning()) {
-      LOG.info("Storage policy satisfier is already running as SPS mode:{}.",
-          spsMode);
-      return;
-    }
-    updateSPSMode(StoragePolicySatisfierMode.INTERNAL);
-    sps.init(new IntraSPSNameNodeContext(this.namesystem, this, sps),
-        new IntraSPSNameNodeFileIdCollector(this.namesystem.getFSDirectory(),
-            sps),
-        new IntraSPSNameNodeBlockMoveTaskHandler(this, this.namesystem), null);
-    sps.start(true, spsMode);
-  }
-
-
-
-  /**
-   * Enable storage policy satisfier by starting its service.
-   */
-  public void enableExternalSPS() {
-    if (!storagePolicyEnabled){
-      LOG.info("Failed to start StoragePolicySatisfier as {} set to {}.",
-          DFSConfigKeys.DFS_STORAGE_POLICY_ENABLED_KEY, storagePolicyEnabled);
-      return;
-    }
-    if (spsMode == StoragePolicySatisfierMode.EXTERNAL) {
-      LOG.info("Storage policy satisfier is already enabled as SPS mode:{}.",
-          spsMode);
-      return;
-    }
-    updateSPSMode(StoragePolicySatisfierMode.EXTERNAL);
-    sps.stopGracefully();
-  }
-
-  private void updateSPSMode(StoragePolicySatisfierMode newSpsMode) {
-    LOG.debug("Updating SPS service status, current mode:{}, new mode:{}",
-        spsMode, newSpsMode);
-    spsMode = newSpsMode;
-  }
-
-  /**
-   * Disable the storage policy satisfier by stopping its services.
-   */
-  public void disableSPS() {
-    switch (spsMode) {
-    case NONE:
-      break;
-    case INTERNAL:
-    case EXTERNAL:
-      if (!sps.isRunning()) {
-        LOG.info("Storage policy satisfier is already stopped.");
-      } else {
-        LOG.info("Stopping StoragePolicySatisfier mode {}, as admin "
-            + "requested to stop it.", spsMode);
-        sps.disable(true);
-      }
-      removeAllSPSPathIds();
-      break;
-    default:
-      // nothing
-      break;
-    }
-    updateSPSMode(StoragePolicySatisfierMode.NONE);
-  }
-
-  /**
-   * Timed wait to stop storage policy satisfier daemon threads.
-   */
-  public void stopSPSGracefully() {
-    removeAllSPSPathIds();
-    sps.stopGracefully();
-  }
-  /**
-   * @return True if storage policy satisfier running.
-   */
-  public boolean isStoragePolicySatisfierRunning() {
-    return sps.isRunning();
-  }
-
-  /**
-   * @return status
-   *                Storage policy satisfy status of the path.
-   * @throws IOException
-   */
-  public StoragePolicySatisfyPathStatus checkStoragePolicySatisfyPathStatus(
-      String path) throws IOException {
-    if (spsMode != StoragePolicySatisfierMode.INTERNAL) {
-      LOG.debug("Satisfier is not running inside namenode, so status "
-          + "can't be returned.");
-      throw new IOException("Satisfier is not running inside namenode, "
-          + "so status can't be returned.");
-    }
-    return sps.checkStoragePolicySatisfyPathStatus(path);
-  }
-
-  /**
-   * @return SPS service instance.
-   */
-  public SPSService getSPSService() {
-    return this.sps;
-  }
-
-  /**
-   * @return the next SPS path id, on which path users has invoked to satisfy
-   *         storages.
-   */
-  public Long getNextSPSPathId() {
-    return spsPaths.pollNext();
-  }
-
-  /**
-   * Verify that satisfier queue limit exceeds allowed outstanding limit.
-   */
-  public void verifyOutstandingSPSPathQLimit() throws IOException {
-    long size = spsPaths.size();
-    // Checking that the SPS call Q exceeds the allowed limit.
-    if (spsOutstandingPathsLimit - size <= 0) {
-      LOG.debug("Satisifer Q - outstanding limit:{}, current size:{}",
-          spsOutstandingPathsLimit, size);
-      throw new IOException("Outstanding satisfier queue limit: "
-          + spsOutstandingPathsLimit + " exceeded, try later!");
-    }
-  }
-
-  /**
-   * Removes the SPS path id from the list of sps paths.
-   */
-  public void removeSPSPathId(long trackId) {
-    spsPaths.remove(trackId);
-  }
-
-  /**
-   * Clean up all sps path ids.
-   */
-  public void removeAllSPSPathIds() {
-    spsPaths.clear();
-  }
-
-  /**
-   * Adds the sps path to SPSPathIds list.
-   */
-  public void addSPSPathId(long id) {
-    spsPaths.add(id);
-  }
-
-  /**
-   * @return true if sps is running as an internal service or external service.
-   */
-  public boolean isSPSEnabled() {
-    return spsMode == StoragePolicySatisfierMode.INTERNAL
-        || spsMode == StoragePolicySatisfierMode.EXTERNAL;
-  }
-
-  /**
-   * @return sps service mode.
+   * @return sps manager.
    */
-  public StoragePolicySatisfierMode getSPSMode() {
-    return spsMode;
+  public StoragePolicySatisfyManager getSPSManager() {
+    return spsManager;
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4402f3f8/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 b09d908..24b948c 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
@@ -211,8 +211,8 @@ public class DatanodeDescriptor extends DatanodeInfo {
    * A queue of blocks corresponding to trackID for moving its storage
    * placements by this datanode.
    */
-  private final Queue<BlockMovingInfo> storageMovementBlocks =
-      new LinkedList<>();
+  private final BlockQueue<BlockMovingInfo> storageMovementBlocks =
+      new BlockQueue<>();
   private volatile boolean dropSPSWork = false;
 
   /* Variables for maintaining number of blocks scheduled to be written to
@@ -369,6 +369,7 @@ public class DatanodeDescriptor extends DatanodeInfo {
     this.pendingCached.clear();
     this.cached.clear();
     this.pendingUncached.clear();
+    this.storageMovementBlocks.clear();
   }
 
   public int numBlocks() {
@@ -1082,9 +1083,10 @@ public class DatanodeDescriptor extends DatanodeInfo {
    *          - storage mismatched block info
    */
   public void addBlocksToMoveStorage(BlockMovingInfo blkMovingInfo) {
-    synchronized (storageMovementBlocks) {
-      storageMovementBlocks.offer(blkMovingInfo);
-    }
+    storageMovementBlocks.offer(blkMovingInfo);
+    BlockManager.LOG
+        .debug("Adding block move task " + blkMovingInfo + " to " + getName()
+            + ", current queue size is " + storageMovementBlocks.size());
   }
 
   /**
@@ -1101,23 +1103,18 @@ public class DatanodeDescriptor extends DatanodeInfo {
    *          total number of blocks which will be send to this datanode for
    *          block movement.
    *
-   * @return block infos which needs to move its storage locations.
+   * @return block infos which needs to move its storage locations or null if
+   *         there is no block infos to move.
    */
   public BlockMovingInfo[] getBlocksToMoveStorages(int numBlocksToMoveTasks) {
-    synchronized (storageMovementBlocks) {
-      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;
-      }
+    List<BlockMovingInfo> blockMovingInfos = storageMovementBlocks
+        .poll(numBlocksToMoveTasks);
+    if (blockMovingInfos == null || blockMovingInfos.size() <= 0) {
       return null;
     }
+    BlockMovingInfo[] blkMoveArray = new BlockMovingInfo[blockMovingInfos
+        .size()];
+    return blockMovingInfos.toArray(blkMoveArray);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4402f3f8/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 1378de2..c6e2263 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,7 +365,7 @@ public interface HdfsServerConstants {
   String XATTR_ERASURECODING_POLICY =
       "system.hdfs.erasurecoding.policy";
 
-  String XATTR_SATISFY_STORAGE_POLICY = "user.hdfs.sps.xattr";
+  String XATTR_SATISFY_STORAGE_POLICY = "user.hdfs.sps";
 
   Path MOVER_ID_PATH = new Path("/system/mover.id");
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4402f3f8/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 42f2e93..af6137c 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
@@ -75,9 +75,8 @@ public class StoragePolicySatisfyWorker {
 
   public StoragePolicySatisfyWorker(Configuration conf, DataNode datanode) {
     this.datanode = datanode;
-
-    moverThreads = conf.getInt(DFSConfigKeys.DFS_MOVER_MOVERTHREADS_KEY,
-        DFSConfigKeys.DFS_MOVER_MOVERTHREADS_DEFAULT);
+    // Defaulting to 10. This is to minimise the number of move ops.
+    moverThreads = conf.getInt(DFSConfigKeys.DFS_MOVER_MOVERTHREADS_KEY, 10);
     moveExecutor = initializeBlockMoverThreadPool(moverThreads);
     moverCompletionService = new ExecutorCompletionService<>(moveExecutor);
     handler = new BlocksMovementsStatusHandler();
@@ -127,21 +126,13 @@ 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);
             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);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4402f3f8/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 2cc0e27..af5ab2d 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
@@ -661,7 +661,7 @@ public class Mover {
           boolean spsRunning;
           try {
             spsRunning = nnc.getDistributedFileSystem().getClient()
-                .isStoragePolicySatisfierRunning();
+                .isInternalSatisfierRunning();
           } catch (RemoteException e) {
             IOException cause = e.unwrapRemoteException();
             if (cause instanceof StandbyException) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4402f3f8/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 5ffd6e8..45d6218 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,6 +20,7 @@ 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.Arrays;
 import java.util.EnumSet;
 import java.util.List;
 
@@ -75,24 +76,33 @@ final class FSDirSatisfyStoragePolicyOp {
         fsd.checkPathAccess(pc, iip, FsAction.WRITE);
       }
       INode inode = FSDirectory.resolveLastINode(iip);
-      if (inodeHasSatisfyXAttr(inode)) {
-        throw new IOException(
-            "Cannot request to call satisfy storage policy on path "
+      if (inode.isFile() && inode.asFile().numBlocks() == 0) {
+        if (NameNode.LOG.isInfoEnabled()) {
+          NameNode.LOG.info(
+              "Skipping satisfy storage policy on path:{} as "
+                  + "this file doesn't have any blocks!",
+              inode.getFullPathName());
+        }
+      } else if (inodeHasSatisfyXAttr(inode)) {
+        NameNode.LOG
+            .warn("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)) {
+      } else {
         XAttr satisfyXAttr = XAttrHelper
             .buildXAttr(XATTR_SATISFY_STORAGE_POLICY);
-        List<XAttr> xAttrs = Lists.newArrayListWithCapacity(1);
-        xAttrs.add(satisfyXAttr);
+        List<XAttr> xAttrs = Arrays.asList(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);
+
+        // Adding directory in the pending queue, so FileInodeIdCollector
+        // process directory child in batch and recursively
+        fsd.getBlockManager().getSPSManager().addPathId(inode.getId());
       }
     } finally {
       fsd.writeUnlock();
@@ -106,7 +116,7 @@ final class FSDirSatisfyStoragePolicyOp {
     } else {
       // Adding directory in the pending queue, so FileInodeIdCollector process
       // directory child in batch and recursively
-      fsd.getBlockManager().addSPSPathId(inode.getId());
+      fsd.getBlockManager().getSPSManager().addPathId(inode.getId());
       return true;
     }
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4402f3f8/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirStatAndListingOp.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirStatAndListingOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirStatAndListingOp.java
index 709e270..7e22ae1 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirStatAndListingOp.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirStatAndListingOp.java
@@ -90,7 +90,6 @@ class FSDirStatAndListingOp {
    * @param srcArg The string representation of the path to the file
    * @param resolveLink whether to throw UnresolvedLinkException
    *        if src refers to a symlink
-   * @param needLocation if blockLocations need to be returned
    *
    * @param needLocation Include {@link LocatedBlocks} in result.
    * @param needBlockToken Include block tokens in {@link LocatedBlocks}.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4402f3f8/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 459e697..1150a72 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
@@ -209,7 +209,7 @@ class FSDirXAttrOp {
       for (XAttr xattr : toRemove) {
         if (XATTR_SATISFY_STORAGE_POLICY
             .equals(XAttrHelper.getPrefixedName(xattr))) {
-          fsd.getBlockManager().getStoragePolicySatisfier()
+          fsd.getBlockManager().getSPSManager().getInternalSPSService()
               .clearQueue(inode.getId());
           break;
         }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4402f3f8/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 2c9d627..6539b51 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
@@ -1401,7 +1401,7 @@ public class FSDirectory implements Closeable {
       if (!inode.isSymlink()) {
         final XAttrFeature xaf = inode.getXAttrFeature();
         addEncryptionZone((INodeWithAdditionalFields) inode, xaf);
-        if (namesystem.getBlockManager().isSPSEnabled()) {
+        if (namesystem.getBlockManager().getSPSManager().isEnabled()) {
           addStoragePolicySatisfier((INodeWithAdditionalFields) inode, xaf);
         }
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4402f3f8/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 5dfec25..8c5a410 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
@@ -259,10 +259,7 @@ import org.apache.hadoop.hdfs.server.namenode.metrics.NameNodeMetrics;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.DirectorySnapshottableFeature;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotManager;
-import org.apache.hadoop.hdfs.server.namenode.sps.IntraSPSNameNodeBlockMoveTaskHandler;
-import org.apache.hadoop.hdfs.server.namenode.sps.IntraSPSNameNodeContext;
-import org.apache.hadoop.hdfs.server.namenode.sps.IntraSPSNameNodeFileIdCollector;
-import org.apache.hadoop.hdfs.server.namenode.sps.StoragePolicySatisfier;
+import org.apache.hadoop.hdfs.server.namenode.sps.SPSService;
 import org.apache.hadoop.hdfs.server.namenode.startupprogress.Phase;
 import org.apache.hadoop.hdfs.server.namenode.startupprogress.StartupProgress;
 import org.apache.hadoop.hdfs.server.namenode.startupprogress.Step;
@@ -1295,13 +1292,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
         FSDirEncryptionZoneOp.warmUpEdekCache(edekCacheLoader, dir,
             edekCacheLoaderDelay, edekCacheLoaderInterval);
       }
-      blockManager.getSPSService().init(
-          new IntraSPSNameNodeContext(this, blockManager,
-              blockManager.getSPSService()),
-          new IntraSPSNameNodeFileIdCollector(getFSDirectory(),
-              blockManager.getSPSService()),
-          new IntraSPSNameNodeBlockMoveTaskHandler(blockManager, this), null);
-      blockManager.startSPS();
+      blockManager.getSPSManager().start();
     } finally {
       startingActiveService = false;
       blockManager.checkSafeMode();
@@ -1332,7 +1323,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     writeLock();
     try {
       if (blockManager != null) {
-        blockManager.stopSPS(false);
+        blockManager.getSPSManager().stop();
       }
       stopSecretManager();
       leaseManager.stopMonitor();
@@ -1372,7 +1363,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();
+        blockManager.getSPSManager().stopGracefully();
       }
     } finally {
       writeUnlock("stopActiveServices");
@@ -2281,17 +2272,18 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
           DFS_STORAGE_POLICY_ENABLED_KEY));
     }
     // checks sps status
-    if (!blockManager.isSPSEnabled()
-        || (blockManager.getSPSMode() == StoragePolicySatisfierMode.INTERNAL
-            && !blockManager.getStoragePolicySatisfier().isRunning())) {
+    if (!blockManager.getSPSManager().isEnabled() || (blockManager
+        .getSPSManager().getMode() == StoragePolicySatisfierMode.INTERNAL
+        && !blockManager.getSPSManager().isInternalSatisfierRunning())) {
       throw new UnsupportedActionException(
           "Cannot request to satisfy storage policy "
               + "when storage policy satisfier feature has been disabled"
               + " by admin. Seek for an admin help to enable it "
               + "or use Mover tool.");
     }
-    // checks SPS Q has many outstanding requests.
-    blockManager.verifyOutstandingSPSPathQLimit();
+    // checks SPS Q has many outstanding requests. It will throw IOException if
+    // the limit exceeds.
+    blockManager.getSPSManager().verifyOutstandingPathQLimit();
   }
 
   /**
@@ -3996,17 +3988,15 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       }
 
       // 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 storage"
-                    + "  movement attempt finished block info sent by DN");
-          }
-        } else {
-          sps.notifyStorageMovementAttemptFinishedBlks(blksMovementsFinished);
+      SPSService sps = blockManager.getSPSManager().getInternalSPSService();
+      if (!sps.isRunning()) {
+        if (LOG.isDebugEnabled()) {
+          LOG.debug(
+              "Storage policy satisfier is not running. So, ignoring storage"
+                  + "  movement attempt finished block info sent by DN");
         }
+      } else {
+        sps.notifyStorageMovementAttemptFinishedBlks(blksMovementsFinished);
       }
 
       //create ha status

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4402f3f8/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 a7d8298..4e3a3ba 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
@@ -2043,7 +2043,7 @@ public class NameNode extends ReconfigurableBase implements
     } else if (property.equals(ipcClientRPCBackoffEnable)) {
       return reconfigureIPCBackoffEnabled(newVal);
     } else if (property.equals(DFS_STORAGE_POLICY_SATISFIER_MODE_KEY)) {
-      return reconfigureSPSEnabled(newVal, property);
+      return reconfigureSPSModeEvent(newVal, property);
     } else {
       throw new ReconfigurationException(property, newVal, getConf().get(
           property));
@@ -2127,39 +2127,27 @@ public class NameNode extends ReconfigurableBase implements
     return Boolean.toString(clientBackoffEnabled);
   }
 
-  String reconfigureSPSEnabled(String newVal, String property)
+  String reconfigureSPSModeEvent(String newVal, String property)
       throws ReconfigurationException {
     if (newVal == null
         || StoragePolicySatisfierMode.fromString(newVal) == null) {
       throw new ReconfigurationException(property, newVal,
           getConf().get(property),
           new HadoopIllegalArgumentException(
-              "For enabling or disabling storage policy satisfier, we must "
-                  + "pass either none/internal/external string value only"));
+              "For enabling or disabling storage policy satisfier, must "
+                  + "pass either internal/external/none string value only"));
     }
 
     if (!isActiveState()) {
       throw new ReconfigurationException(property, newVal,
-          getConf().get(property), new HadoopIllegalArgumentException(
-          "Enabling or disabling storage policy satisfier service on "
-              + state + " NameNode is not allowed"));
+          getConf().get(property),
+          new HadoopIllegalArgumentException(
+              "Enabling or disabling storage policy satisfier service on "
+                  + state + " NameNode is not allowed"));
     }
     StoragePolicySatisfierMode mode = StoragePolicySatisfierMode
         .fromString(newVal);
-    switch(mode){
-    case NONE:
-      namesystem.getBlockManager().disableSPS();
-      break;
-    case INTERNAL:
-      namesystem.getBlockManager().enableInternalSPS();
-      break;
-    case EXTERNAL:
-      namesystem.getBlockManager().enableExternalSPS();
-      break;
-    default:
-      // nothing
-      break;
-    }
+    namesystem.getBlockManager().getSPSManager().changeModeEvent(mode);
     return newVal;
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4402f3f8/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 d74dc9e..97f38c7 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
@@ -2536,15 +2536,15 @@ public class NameNodeRpcServer implements NamenodeProtocols {
   }
 
   @Override
-  public boolean isStoragePolicySatisfierRunning() throws IOException {
+  public boolean isInternalSatisfierRunning() throws IOException {
     checkNNStartup();
-    String operationName = "isStoragePolicySatisfierRunning";
+    String operationName = "isInternalSatisfierRunning";
     namesystem.checkSuperuserPrivilege(operationName);
     if (nn.isStandbyState()) {
       throw new StandbyException("Not supported by Standby Namenode.");
     }
-    boolean isSPSRunning =
-        namesystem.getBlockManager().isStoragePolicySatisfierRunning();
+    boolean isSPSRunning = namesystem.getBlockManager().getSPSManager()
+        .isInternalSatisfierRunning();
     namesystem.logAuditEvent(true, operationName, null);
     return isSPSRunning;
   }
@@ -2556,8 +2556,8 @@ public class NameNodeRpcServer implements NamenodeProtocols {
     if (nn.isStandbyState()) {
       throw new StandbyException("Not supported by Standby Namenode.");
     }
-    return namesystem.getBlockManager().checkStoragePolicySatisfyPathStatus(
-        path);
+    return namesystem.getBlockManager().getSPSManager()
+        .checkStoragePolicySatisfyPathStatus(path);
   }
 
   @Override
@@ -2579,17 +2579,16 @@ public class NameNodeRpcServer implements NamenodeProtocols {
     if (nn.isStandbyState()) {
       throw new StandbyException("Not supported by Standby Namenode.");
     }
-    // Check that internal SPS service is running
-    if (namesystem.getBlockManager()
-        .getSPSMode() == StoragePolicySatisfierMode.INTERNAL
-        && namesystem.getBlockManager().getSPSService().isRunning()) {
+    // Check that SPS daemon service is running inside namenode
+    if (namesystem.getBlockManager().getSPSManager()
+        .getMode() == StoragePolicySatisfierMode.INTERNAL) {
       LOG.debug("SPS service is internally enabled and running inside "
           + "namenode, so external SPS is not allowed to fetch the path Ids");
       throw new IOException("SPS service is internally enabled and running"
           + " inside namenode, so external SPS is not allowed to fetch"
           + " the path Ids");
     }
-    return namesystem.getBlockManager().getNextSPSPathId();
+    return namesystem.getBlockManager().getSPSManager().getNextPathId();
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4402f3f8/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/BlockStorageMovementNeeded.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/BlockStorageMovementNeeded.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/BlockStorageMovementNeeded.java
index 8a10183..c683a63 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/BlockStorageMovementNeeded.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/BlockStorageMovementNeeded.java
@@ -311,7 +311,7 @@ public class BlockStorageMovementNeeded {
             if (Time.monotonicNow()
                 - lastStatusCleanTime > statusClearanceElapsedTimeMs) {
               lastStatusCleanTime = Time.monotonicNow();
-              cleanSpsStatus();
+              cleanSPSStatus();
             }
             startINodeId = null; // Current inode id successfully scanned.
           }
@@ -333,7 +333,7 @@ public class BlockStorageMovementNeeded {
       }
     }
 
-    private synchronized void cleanSpsStatus() {
+    private synchronized void cleanSPSStatus() {
       for (Iterator<Entry<Long, StoragePolicySatisfyPathStatusInfo>> it =
           spsStatus.entrySet().iterator(); it.hasNext();) {
         Entry<Long, StoragePolicySatisfyPathStatusInfo> entry = it.next();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4402f3f8/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/IntraSPSNameNodeContext.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/IntraSPSNameNodeContext.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/IntraSPSNameNodeContext.java
index ff6cc21..495d1c4 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/IntraSPSNameNodeContext.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/IntraSPSNameNodeContext.java
@@ -178,17 +178,17 @@ public class IntraSPSNameNodeContext implements Context {
 
   @Override
   public Long getNextSPSPathId() {
-    return blockManager.getNextSPSPathId();
+    return blockManager.getSPSManager().getNextPathId();
   }
 
   @Override
   public void removeSPSPathId(long trackId) {
-    blockManager.removeSPSPathId(trackId);
+    blockManager.getSPSManager().removePathId(trackId);
   }
 
   @Override
   public void removeAllSPSPathIds() {
-    blockManager.removeAllSPSPathIds();
+    blockManager.getSPSManager().removeAllPathIds();
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4402f3f8/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/SPSPathIds.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/SPSPathIds.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/SPSPathIds.java
deleted file mode 100644
index 6c0f8b2..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/SPSPathIds.java
+++ /dev/null
@@ -1,70 +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.sps;
-
-import java.util.LinkedList;
-import java.util.Queue;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-
-/**
- * A class which holds the SPS invoked path ids.
- */
-@InterfaceAudience.Private
-@InterfaceStability.Evolving
-public class SPSPathIds {
-
-  // List of pending dir to satisfy the policy
-  private final Queue<Long> spsDirsToBeTraveresed = new LinkedList<Long>();
-
-  /**
-   * Add the path id to queue.
-   */
-  public synchronized void add(long pathId) {
-    spsDirsToBeTraveresed.add(pathId);
-  }
-
-  /**
-   * Removes the path id.
-   */
-  public synchronized void remove(long pathId) {
-    spsDirsToBeTraveresed.remove(pathId);
-  }
-
-  /**
-   * Clears all path ids.
-   */
-  public synchronized void clear() {
-    spsDirsToBeTraveresed.clear();
-  }
-
-  /**
-   * @return next path id available in queue.
-   */
-  public synchronized Long pollNext() {
-    return spsDirsToBeTraveresed.poll();
-  }
-
-  /**
-   * @return the size of the queue.
-   */
-  public synchronized long size() {
-    return spsDirsToBeTraveresed.size();
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4402f3f8/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/SPSService.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/SPSService.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/SPSService.java
index ceec3f3..da6e365 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/SPSService.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/SPSService.java
@@ -67,11 +67,12 @@ public interface SPSService {
   void stopGracefully();
 
   /**
-   * Disable the SPS service.
+   * Stops the SPS service.
    *
    * @param forceStop
+   *          true represents to clear all the sps path's hint, false otherwise.
    */
-  void disable(boolean forceStop);
+  void stop(boolean forceStop);
 
   /**
    * Check whether StoragePolicySatisfier is running.
@@ -106,6 +107,11 @@ public interface SPSService {
   int processingQueueSize();
 
   /**
+   * Clear inodeId present in the processing queue.
+   */
+  void clearQueue(long inodeId);
+
+  /**
    * @return the configuration.
    */
   Configuration getConf();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4402f3f8/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/StoragePolicySatisfier.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/StoragePolicySatisfier.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/StoragePolicySatisfier.java
index 87faced..6b449aa 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/StoragePolicySatisfier.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/StoragePolicySatisfier.java
@@ -32,7 +32,6 @@ 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;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
@@ -60,6 +59,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
 
 /**
  * Setting storagePolicy on a file after the file write will only update the new
@@ -145,7 +145,7 @@ public class StoragePolicySatisfier implements SPSService, Runnable {
         new BlockStorageMovementAttemptedItems(this,
         storageMovementNeeded, blockMovementListener);
     this.blockMoveTaskHandler = blockMovementTaskHandler;
-    this.spsWorkMultiplier = DFSUtil.getSPSWorkMultiplier(getConf());
+    this.spsWorkMultiplier = getSPSWorkMultiplier(getConf());
     this.blockMovementMaxRetry = getConf().getInt(
         DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_MAX_RETRY_ATTEMPTS_KEY,
         DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_MAX_RETRY_ATTEMPTS_DEFAULT);
@@ -163,8 +163,6 @@ public class StoragePolicySatisfier implements SPSService, Runnable {
           serviceMode);
       return;
     }
-    isRunning = true;
-    this.spsMode = serviceMode;
     if (spsMode == StoragePolicySatisfierMode.INTERNAL
         && ctxt.isMoverRunning()) {
       isRunning = false;
@@ -182,6 +180,8 @@ public class StoragePolicySatisfier implements SPSService, Runnable {
           StringUtils.toLowerCase(spsMode.toString()));
     }
 
+    isRunning = true;
+    this.spsMode = serviceMode;
     // Ensure that all the previously submitted block movements(if any) have to
     // be stopped in all datanodes.
     addDropSPSWorkCommandsToAllDNs();
@@ -193,7 +193,7 @@ public class StoragePolicySatisfier implements SPSService, Runnable {
   }
 
   @Override
-  public synchronized void disable(boolean forceStop) {
+  public synchronized void stop(boolean forceStop) {
     isRunning = false;
     if (storagePolicySatisfierThread == null) {
       return;
@@ -214,19 +214,22 @@ public class StoragePolicySatisfier implements SPSService, Runnable {
   @Override
   public synchronized void stopGracefully() {
     if (isRunning) {
-      disable(true);
+      stop(false);
     }
 
     if (this.storageMovementsMonitor != null) {
       this.storageMovementsMonitor.stopGracefully();
     }
 
-    if (storagePolicySatisfierThread == null) {
-      return;
-    }
-    try {
-      storagePolicySatisfierThread.join(3000);
-    } catch (InterruptedException ie) {
+    if (storagePolicySatisfierThread != null) {
+      try {
+        storagePolicySatisfierThread.join(3000);
+      } catch (InterruptedException ie) {
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("Interrupted Exception while waiting to join sps thread,"
+              + " ignoring it", ie);
+        }
+      }
     }
   }
 
@@ -351,32 +354,26 @@ public class StoragePolicySatisfier implements SPSService, Runnable {
           Thread.sleep(3000);
           blockCount = 0L;
         }
+      } catch (IOException e) {
+        LOG.error("Exception during StoragePolicySatisfier execution - "
+            + "will continue next cycle", e);
       } catch (Throwable t) {
-        handleException(t);
-      }
-    }
-  }
-
-  private void handleException(Throwable t) {
-    // double check to avoid entering into synchronized block.
-    if (isRunning) {
-      synchronized (this) {
-        if (isRunning) {
-          if (t instanceof InterruptedException) {
+        synchronized (this) {
+          if (isRunning) {
             isRunning = false;
-            LOG.info("Stopping StoragePolicySatisfier.");
+            if (t instanceof InterruptedException) {
+              LOG.info("Stopping StoragePolicySatisfier.", t);
+            } else {
+              LOG.error("StoragePolicySatisfier thread received "
+                  + "runtime exception.", t);
+            }
             // Stopping monitor thread and clearing queues as well
             this.clearQueues();
             this.storageMovementsMonitor.stopGracefully();
-          } else {
-            LOG.error(
-                "StoragePolicySatisfier thread received runtime exception, "
-                    + "ignoring", t);
           }
         }
       }
     }
-    return;
   }
 
   private BlocksMovingAnalysis analyseBlocksStorageMovementsAndAssignToDN(
@@ -434,7 +431,7 @@ public class StoragePolicySatisfier implements SPSService, Runnable {
 
       List<StorageType> existing = new LinkedList<StorageType>(
           Arrays.asList(blockInfo.getStorageTypes()));
-      if (!DFSUtil.removeOverlapBetweenStorageTypes(expectedStorageTypes,
+      if (!removeOverlapBetweenStorageTypes(expectedStorageTypes,
           existing, true)) {
         boolean blocksPaired = computeBlockMovingInfos(blockMovingInfos,
             blockInfo, expectedStorageTypes, existing, blockInfo.getLocations(),
@@ -499,7 +496,7 @@ public class StoragePolicySatisfier implements SPSService, Runnable {
       DatanodeInfo[] storages, DatanodeStorageReport[] liveDns,
       ErasureCodingPolicy ecPolicy) {
     boolean foundMatchingTargetNodesForBlock = true;
-    if (!DFSUtil.removeOverlapBetweenStorageTypes(expectedStorageTypes,
+    if (!removeOverlapBetweenStorageTypes(expectedStorageTypes,
         existing, true)) {
       List<StorageTypeNodePair> sourceWithStorageMap =
           new ArrayList<StorageTypeNodePair>();
@@ -881,21 +878,6 @@ public class StoragePolicySatisfier implements SPSService, Runnable {
   }
 
   /**
-   * Set file inode in queue for which storage movement needed for its blocks.
-   *
-   * @param inodeId
-   *          - file inode/blockcollection id.
-   */
-  public void satisfyStoragePolicy(Long inodeId) {
-    //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 "
-          + "storageMovementNeeded queue", inodeId);
-    }
-  }
-
-  /**
    * Clear queues for given track id.
    */
   public void clearQueue(long trackId) {
@@ -958,6 +940,10 @@ public class StoragePolicySatisfier implements SPSService, Runnable {
   @Override
   public void addFileIdToProcess(ItemInfo trackInfo, boolean scanCompleted) {
     storageMovementNeeded.add(trackInfo, scanCompleted);
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Added track info for inode {} to block "
+          + "storageMovementNeeded queue", trackInfo.getFileId());
+    }
   }
 
   @Override
@@ -993,4 +979,63 @@ public class StoragePolicySatisfier implements SPSService, Runnable {
     //TODO Add join here on SPS rpc server also
     storagePolicySatisfierThread.join();
   }
+
+  /**
+   * 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.
+   */
+  private 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();
+      }
+    }
+  }
+
+  /**
+   * Get DFS_SPS_WORK_MULTIPLIER_PER_ITERATION from
+   * configuration.
+   *
+   * @param conf Configuration
+   * @return Value of DFS_SPS_WORK_MULTIPLIER_PER_ITERATION
+   */
+  private 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;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4402f3f8/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/StoragePolicySatisfyManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/StoragePolicySatisfyManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/StoragePolicySatisfyManager.java
new file mode 100644
index 0000000..5bdf6ae
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/StoragePolicySatisfyManager.java
@@ -0,0 +1,399 @@
+/**
+ * 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.sps;
+
+import java.io.IOException;
+import java.util.LinkedList;
+import java.util.Queue;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants.StoragePolicySatisfierMode;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants.StoragePolicySatisfyPathStatus;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
+import org.apache.hadoop.hdfs.server.namenode.Namesystem;
+import org.apache.hadoop.hdfs.server.sps.ExternalStoragePolicySatisfier;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * This manages satisfy storage policy invoked path ids and expose methods to
+ * process these path ids. It maintains sps mode(INTERNAL/EXTERNAL/NONE)
+ * configured by the administrator.
+ *
+ * <p>
+ * If the configured mode is {@link StoragePolicySatisfierMode.INTERNAL}, then
+ * it will start internal sps daemon service inside namenode and process sps
+ * invoked path ids to satisfy the storage policy.
+ *
+ * <p>
+ * If the configured mode is {@link StoragePolicySatisfierMode.EXTERNAL}, then
+ * it won't do anything, just maintains the sps invoked path ids. Administrator
+ * requires to start external sps service explicitly, to fetch the sps invoked
+ * path ids from namenode, then do necessary computations and block movement in
+ * order to satisfy the storage policy. Please refer
+ * {@link ExternalStoragePolicySatisfier} class to understand more about the
+ * external sps service functionality.
+ *
+ * <p>
+ * If the configured mode is {@link StoragePolicySatisfierMode.NONE}, then it
+ * will disable the sps feature completely by clearing all queued up sps path's
+ * hint.
+ *
+ * This class is instantiated by the BlockManager.
+ */
+public class StoragePolicySatisfyManager {
+  private static final Logger LOG = LoggerFactory
+      .getLogger(StoragePolicySatisfyManager.class);
+  private final StoragePolicySatisfier spsService;
+  private final boolean storagePolicyEnabled;
+  private volatile StoragePolicySatisfierMode mode;
+  private final Queue<Long> pathsToBeTraveresed;
+  private final int outstandingPathsLimit;
+  private final Namesystem namesystem;
+  private final BlockManager blkMgr;
+
+  public StoragePolicySatisfyManager(Configuration conf, Namesystem namesystem,
+      BlockManager blkMgr) {
+    // StoragePolicySatisfier(SPS) configs
+    storagePolicyEnabled = conf.getBoolean(
+        DFSConfigKeys.DFS_STORAGE_POLICY_ENABLED_KEY,
+        DFSConfigKeys.DFS_STORAGE_POLICY_ENABLED_DEFAULT);
+    String modeVal = conf.get(
+        DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_MODE_KEY,
+        DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_MODE_DEFAULT);
+    outstandingPathsLimit = conf.getInt(
+        DFSConfigKeys.DFS_SPS_MAX_OUTSTANDING_PATHS_KEY,
+        DFSConfigKeys.DFS_SPS_MAX_OUTSTANDING_PATHS_DEFAULT);
+    mode = StoragePolicySatisfierMode.fromString(modeVal);
+    pathsToBeTraveresed = new LinkedList<Long>();
+    // instantiate SPS service by just keeps config reference and not starting
+    // any supporting threads.
+    spsService = new StoragePolicySatisfier(conf);
+    this.namesystem = namesystem;
+    this.blkMgr = blkMgr;
+  }
+
+  /**
+   * This function will do following logic based on the configured sps mode:
+   *
+   * <p>
+   * If the configured mode is {@link StoragePolicySatisfierMode.INTERNAL}, then
+   * starts internal daemon service inside namenode.
+   *
+   * <p>
+   * If the configured mode is {@link StoragePolicySatisfierMode.EXTERNAL}, then
+   * it won't do anything. Administrator requires to start external sps service
+   * explicitly.
+   *
+   * <p>
+   * If the configured mode is {@link StoragePolicySatisfierMode.NONE}, then the
+   * service is disabled and won't do any action.
+   */
+  public void start() {
+    if (!storagePolicyEnabled) {
+      LOG.info("Disabling StoragePolicySatisfier service as {} set to {}.",
+          DFSConfigKeys.DFS_STORAGE_POLICY_ENABLED_KEY, storagePolicyEnabled);
+      return;
+    }
+
+    switch (mode) {
+    case INTERNAL:
+      if (spsService.isRunning()) {
+        LOG.info("Storage policy satisfier is already running"
+            + " as internal daemon service inside namenode.");
+        return;
+      }
+      // starts internal daemon service inside namenode
+      spsService.init(
+          new IntraSPSNameNodeContext(namesystem, blkMgr, spsService),
+          new IntraSPSNameNodeFileIdCollector(namesystem.getFSDirectory(),
+              spsService),
+          new IntraSPSNameNodeBlockMoveTaskHandler(blkMgr, namesystem), null);
+      spsService.start(false, mode);
+      break;
+    case EXTERNAL:
+      LOG.info("Storage policy satisfier is configured as external, "
+          + "please start external sps service explicitly to satisfy policy");
+      break;
+    case NONE:
+      LOG.info("Storage policy satisfier is disabled");
+      break;
+    default:
+      LOG.info("Given mode: {} is invalid", mode);
+      break;
+    }
+  }
+
+  /**
+   * This function will do following logic based on the configured sps mode:
+   *
+   * <p>
+   * If the configured mode is {@link StoragePolicySatisfierMode.INTERNAL}, then
+   * stops internal daemon service inside namenode.
+   *
+   * <p>
+   * If the configured mode is {@link StoragePolicySatisfierMode.EXTERNAL}, then
+   * it won't do anything. Administrator requires to stop external sps service
+   * explicitly, if needed.
+   *
+   * <p>
+   * If the configured mode is {@link StoragePolicySatisfierMode.NONE}, then the
+   * service is disabled and won't do any action.
+   */
+  public void stop() {
+    if (!storagePolicyEnabled) {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Storage policy is not enabled, ignoring");
+      }
+      return;
+    }
+
+    switch (mode) {
+    case INTERNAL:
+      removeAllPathIds();
+      if (!spsService.isRunning()) {
+        LOG.info("Internal storage policy satisfier daemon service"
+            + " is not running");
+        return;
+      }
+      // stops internal daemon service running inside namenode
+      spsService.stop(false);
+      break;
+    case EXTERNAL:
+      removeAllPathIds();
+      if (LOG.isDebugEnabled()) {
+        LOG.debug(
+            "Storage policy satisfier service is running outside namenode"
+            + ", ignoring");
+      }
+      break;
+    case NONE:
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Storage policy satisfier is not enabled, ignoring");
+      }
+      break;
+    default:
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Invalid mode:{}, ignoring", mode);
+      }
+      break;
+    }
+  }
+
+  /**
+   * Sets new sps mode. If the new mode is internal, then it will start internal
+   * sps service inside namenode. If the new mode is external, then stops
+   * internal sps service running(if any) inside namenode. If the new mode is
+   * none, then it will disable the sps feature completely by clearing all
+   * queued up sps path's hint.
+   */
+  public void changeModeEvent(StoragePolicySatisfierMode newMode) {
+    if (!storagePolicyEnabled) {
+      LOG.info("Failed to change storage policy satisfier as {} set to {}.",
+          DFSConfigKeys.DFS_STORAGE_POLICY_ENABLED_KEY, storagePolicyEnabled);
+      return;
+    }
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Updating SPS service status, current mode:{}, new mode:{}",
+          mode, newMode);
+    }
+
+    switch (newMode) {
+    case INTERNAL:
+      if (spsService.isRunning()) {
+        LOG.info("Storage policy satisfier is already running as {} mode.",
+            mode);
+        return;
+      }
+      spsService.init(
+          new IntraSPSNameNodeContext(this.namesystem, this.blkMgr, spsService),
+          new IntraSPSNameNodeFileIdCollector(this.namesystem.getFSDirectory(),
+              spsService),
+          new IntraSPSNameNodeBlockMoveTaskHandler(this.blkMgr,
+              this.namesystem),
+          null);
+      spsService.start(true, newMode);
+      break;
+    case EXTERNAL:
+      if (mode == newMode) {
+        LOG.info("Storage policy satisfier is already in mode:{},"
+            + " so ignoring change mode event.", newMode);
+        return;
+      }
+      spsService.stopGracefully();
+      break;
+    case NONE:
+      if (mode == newMode) {
+        LOG.info("Storage policy satisfier is already disabled, mode:{}"
+            + " so ignoring change mode event.", newMode);
+        return;
+      }
+      LOG.info("Disabling StoragePolicySatisfier, mode:{}", newMode);
+      spsService.stop(true);
+      removeAllPathIds();
+      break;
+    default:
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Given mode: {} is invalid", newMode);
+      }
+      break;
+    }
+
+    // update sps mode
+    mode = newMode;
+  }
+
+  /**
+   * This function will do following logic based on the configured sps mode:
+   *
+   * <p>
+   * If the configured mode is {@link StoragePolicySatisfierMode.INTERNAL}, then
+   * timed wait to stop internal storage policy satisfier daemon threads.
+   *
+   * <p>
+   * If the configured mode is {@link StoragePolicySatisfierMode.EXTERNAL}, then
+   * it won't do anything, just ignore it.
+   *
+   * <p>
+   * If the configured mode is {@link StoragePolicySatisfierMode.NONE}, then the
+   * service is disabled. It won't do any action, just ignore it.
+   */
+  public void stopGracefully() {
+    switch (mode) {
+    case INTERNAL:
+      spsService.stopGracefully();
+      break;
+    case EXTERNAL:
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Ignoring, StoragePolicySatisfier feature is running"
+            + " outside namenode");
+      }
+      break;
+    case NONE:
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Ignoring, StoragePolicySatisfier feature is disabled");
+      }
+      break;
+    default:
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Invalid mode:{}", mode);
+      }
+      break;
+    }
+  }
+
+  /**
+   * @return true if the internal storage policy satisfier daemon is running,
+   *         false otherwise.
+   */
+  public boolean isInternalSatisfierRunning() {
+    return spsService.isRunning();
+  }
+
+  /**
+   * @return internal SPS service instance.
+   */
+  public SPSService getInternalSPSService() {
+    return this.spsService;
+  }
+
+  /**
+   * @return status Storage policy satisfy status of the path. It is supported
+   *         only for the internal sps daemon service.
+   * @throws IOException
+   *           if the Satisfier is not running inside namenode.
+   */
+  public StoragePolicySatisfyPathStatus checkStoragePolicySatisfyPathStatus(
+      String path) throws IOException {
+    if (mode != StoragePolicySatisfierMode.INTERNAL) {
+      LOG.debug("Satisfier is not running inside namenode, so status "
+          + "can't be returned.");
+      throw new IOException("Satisfier is not running inside namenode, "
+          + "so status can't be returned.");
+    }
+    return spsService.checkStoragePolicySatisfyPathStatus(path);
+  }
+
+  /**
+   * @return the next SPS path id, on which path users has invoked to satisfy
+   *         storages.
+   */
+  public Long getNextPathId() {
+    synchronized (pathsToBeTraveresed) {
+      return pathsToBeTraveresed.poll();
+    }
+  }
+
+  /**
+   * Verify that satisfier queue limit exceeds allowed outstanding limit.
+   */
+  public void verifyOutstandingPathQLimit() throws IOException {
+    long size = pathsToBeTraveresed.size();
+    // Checking that the SPS call Q exceeds the allowed limit.
+    if (outstandingPathsLimit - size <= 0) {
+      LOG.debug("Satisifer Q - outstanding limit:{}, current size:{}",
+          outstandingPathsLimit, size);
+      throw new IOException("Outstanding satisfier queue limit: "
+          + outstandingPathsLimit + " exceeded, try later!");
+    }
+  }
+
+  /**
+   * Removes the SPS path id from the list of sps paths.
+   */
+  public void removePathId(long trackId) {
+    synchronized (pathsToBeTraveresed) {
+      pathsToBeTraveresed.remove(trackId);
+    }
+  }
+
+  /**
+   * Clean up all sps path ids.
+   */
+  public void removeAllPathIds() {
+    synchronized (pathsToBeTraveresed) {
+      pathsToBeTraveresed.clear();
+    }
+  }
+
+  /**
+   * Adds the sps path to SPSPathIds list.
+   */
+  public void addPathId(long id) {
+    synchronized (pathsToBeTraveresed) {
+      pathsToBeTraveresed.add(id);
+    }
+  }
+
+  /**
+   * @return true if sps is configured as an internal service or external
+   *         service, false otherwise.
+   */
+  public boolean isEnabled() {
+    return mode == StoragePolicySatisfierMode.INTERNAL
+        || mode == StoragePolicySatisfierMode.EXTERNAL;
+  }
+
+  /**
+   * @return sps service mode.
+   */
+  public StoragePolicySatisfierMode getMode() {
+    return mode;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4402f3f8/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/sps/ExternalStoragePolicySatisfier.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/sps/ExternalStoragePolicySatisfier.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/sps/ExternalStoragePolicySatisfier.java
index 59935b6..33448db 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/sps/ExternalStoragePolicySatisfier.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/sps/ExternalStoragePolicySatisfier.java
@@ -73,7 +73,7 @@ public final class ExternalStoragePolicySatisfier {
 
       boolean spsRunning;
       spsRunning = nnc.getDistributedFileSystem().getClient()
-          .isStoragePolicySatisfierRunning();
+          .isInternalSatisfierRunning();
       if (spsRunning) {
         throw new RuntimeException(
             "Startup failed due to StoragePolicySatisfier"

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4402f3f8/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 3a2ad48..d8392fa 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
@@ -374,7 +374,7 @@ public class StoragePolicyAdmin extends Configured implements Tool {
       }
       final DistributedFileSystem dfs = AdminHelper.getDFS(conf);
       try {
-        if(dfs.getClient().isStoragePolicySatisfierRunning()){
+        if(dfs.getClient().isInternalSatisfierRunning()){
           System.out.println("yes");
         }else{
           System.out.println("no");


---------------------------------------------------------------------
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-12911. [SPS]: Modularize the SPS code and expose necessary interfaces for external/internal implementations. Contributed by Uma Maheswara Rao G

Posted by um...@apache.org.
HDFS-12911. [SPS]: Modularize the SPS code and expose necessary interfaces for external/internal implementations. 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/8d4f74e7
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/8d4f74e7
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/8d4f74e7

Branch: refs/heads/HDFS-10285
Commit: 8d4f74e7339abc77dc0daa162d7bd2814bd79b3d
Parents: 05d4daf
Author: Rakesh Radhakrishnan <ra...@apache.org>
Authored: Fri Jan 19 08:51:49 2018 +0530
Committer: Uma Maheswara Rao Gangumalla <um...@apache.org>
Committed: Sun Aug 12 03:06:03 2018 -0700

----------------------------------------------------------------------
 .../server/blockmanagement/BlockManager.java    |  61 +++++-
 .../namenode/FSDirSatisfyStoragePolicyOp.java   |  16 +-
 .../hdfs/server/namenode/FSDirectory.java       |   6 +-
 .../hdfs/server/namenode/FSNamesystem.java      |  10 +-
 .../namenode/sps/BlockMoveTaskHandler.java      |  44 ++++
 .../namenode/sps/BlockMovementListener.java     |  40 ++++
 .../sps/BlockStorageMovementAttemptedItems.java |  28 +--
 .../sps/BlockStorageMovementNeeded.java         | 207 ++++---------------
 .../hdfs/server/namenode/sps/Context.java       |  43 ++--
 .../server/namenode/sps/FileIdCollector.java    |  43 ++++
 .../IntraSPSNameNodeBlockMoveTaskHandler.java   |  62 ++++++
 .../namenode/sps/IntraSPSNameNodeContext.java   |  62 ++----
 .../sps/IntraSPSNameNodeFileIdCollector.java    | 178 ++++++++++++++++
 .../hdfs/server/namenode/sps/ItemInfo.java      |  81 ++++++++
 .../hdfs/server/namenode/sps/SPSPathIds.java    |  63 ++++++
 .../hdfs/server/namenode/sps/SPSService.java    | 107 ++++++++++
 .../namenode/sps/StoragePolicySatisfier.java    | 175 +++++++---------
 .../TestBlockStorageMovementAttemptedItems.java |  19 +-
 .../sps/TestStoragePolicySatisfier.java         | 111 ++++++----
 ...stStoragePolicySatisfierWithStripedFile.java |  19 +-
 20 files changed, 938 insertions(+), 437 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/8d4f74e7/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 5ee4026..d12cb01 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
@@ -93,8 +93,8 @@ import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.namenode.Namesystem;
 import org.apache.hadoop.hdfs.server.namenode.ha.HAContext;
 import org.apache.hadoop.hdfs.server.namenode.metrics.NameNodeMetrics;
-import org.apache.hadoop.hdfs.server.namenode.sps.Context;
-import org.apache.hadoop.hdfs.server.namenode.sps.IntraSPSNameNodeContext;
+import org.apache.hadoop.hdfs.server.namenode.sps.SPSPathIds;
+import org.apache.hadoop.hdfs.server.namenode.sps.SPSService;
 import org.apache.hadoop.hdfs.server.namenode.sps.StoragePolicySatisfier;
 import org.apache.hadoop.hdfs.server.protocol.BlockCommand;
 import org.apache.hadoop.hdfs.server.protocol.BlockReportContext;
@@ -434,7 +434,8 @@ public class BlockManager implements BlockStatsMXBean {
   private final StoragePolicySatisfier sps;
   private final boolean storagePolicyEnabled;
   private boolean spsEnabled;
-  private Context spsctxt = null;
+  private final SPSPathIds spsPaths;
+
   /** Minimum live replicas needed for the datanode to be transitioned
    * from ENTERING_MAINTENANCE to IN_MAINTENANCE.
    */
@@ -481,8 +482,8 @@ public class BlockManager implements BlockStatsMXBean {
         conf.getBoolean(
             DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_ENABLED_KEY,
             DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_ENABLED_DEFAULT);
-    spsctxt = new IntraSPSNameNodeContext(namesystem, this, conf);
-    sps = new StoragePolicySatisfier(spsctxt);
+    sps = new StoragePolicySatisfier(conf);
+    spsPaths = new SPSPathIds();
     blockTokenSecretManager = createBlockTokenSecretManager(conf);
 
     providedStorageMap = new ProvidedStorageMap(namesystem, this, conf);
@@ -5041,8 +5042,7 @@ public class BlockManager implements BlockStatsMXBean {
       LOG.info("Storage policy satisfier is already running.");
       return;
     }
-    // TODO: FSDirectory will get removed via HDFS-12911 modularization work
-    sps.start(false, namesystem.getFSDirectory());
+    sps.start(false);
   }
 
   /**
@@ -5078,8 +5078,7 @@ public class BlockManager implements BlockStatsMXBean {
       LOG.info("Storage policy satisfier is already running.");
       return;
     }
-    // TODO: FSDirectory will get removed via HDFS-12911 modularization work
-    sps.start(true, namesystem.getFSDirectory());
+    sps.start(true);
   }
 
   /**
@@ -5119,4 +5118,48 @@ public class BlockManager implements BlockStatsMXBean {
       String path) throws IOException {
     return sps.checkStoragePolicySatisfyPathStatus(path);
   }
+
+  /**
+   * @return SPS service instance.
+   */
+  public SPSService getSPSService() {
+    return this.sps;
+  }
+
+  /**
+   * @return the next SPS path id, on which path users has invoked to satisfy
+   *         storages.
+   */
+  public Long getNextSPSPathId() {
+    return spsPaths.pollNext();
+  }
+
+  /**
+   * Removes the SPS path id from the list of sps paths.
+   */
+  public void removeSPSPathId(long trackId) {
+    spsPaths.remove(trackId);
+  }
+
+  /**
+   * Clean up all sps path ids.
+   */
+  public void removeAllSPSPathIds() {
+    spsPaths.clear();
+  }
+
+  /**
+   * Adds the sps path to SPSPathIds list.
+   */
+  public void addSPSPathId(long id) {
+    spsPaths.add(id);
+  }
+
+  /**
+   * @return true if sps enabled.
+   */
+  public boolean isSPSEnabled() {
+    return spsEnabled;
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8d4f74e7/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 fb6eec9..eed6e52 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
@@ -30,7 +30,6 @@ 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;
 
@@ -87,21 +86,14 @@ final class FSDirSatisfyStoragePolicyOp {
   }
 
   static boolean unprotectedSatisfyStoragePolicy(INode inode, FSDirectory fsd) {
-    if (inode.isFile() && inode.asFile().numBlocks() != 0) {
-      // 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) {
+    if (inode.isFile() && inode.asFile().numBlocks() == 0) {
+      return false;
+    } else {
       // Adding directory in the pending queue, so FileInodeIdCollector process
       // directory child in batch and recursively
-      fsd.getBlockManager().getStoragePolicySatisfier()
-          .addInodeToPendingDirQueue(inode.getId());
+      fsd.getBlockManager().addSPSPathId(inode.getId());
       return true;
     }
-    return false;
   }
 
   private static boolean inodeHasSatisfyXAttr(INode inode) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8d4f74e7/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 35341d7..2c9d627 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
@@ -1401,14 +1401,16 @@ public class FSDirectory implements Closeable {
       if (!inode.isSymlink()) {
         final XAttrFeature xaf = inode.getXAttrFeature();
         addEncryptionZone((INodeWithAdditionalFields) inode, xaf);
-        addStoragePolicySatisfier((INodeWithAdditionalFields) inode, xaf);
+        if (namesystem.getBlockManager().isSPSEnabled()) {
+          addStoragePolicySatisfier((INodeWithAdditionalFields) inode, xaf);
+        }
       }
     }
   }
 
   private void addStoragePolicySatisfier(INodeWithAdditionalFields inode,
       XAttrFeature xaf) {
-    if (xaf == null || inode.isDirectory()) {
+    if (xaf == null) {
       return;
     }
     XAttr xattr = xaf.getXAttr(XATTR_SATISFY_STORAGE_POLICY);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8d4f74e7/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 e4a495b..ed1c823 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
@@ -258,6 +258,9 @@ import org.apache.hadoop.hdfs.server.namenode.metrics.NameNodeMetrics;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.DirectorySnapshottableFeature;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotManager;
+import org.apache.hadoop.hdfs.server.namenode.sps.IntraSPSNameNodeBlockMoveTaskHandler;
+import org.apache.hadoop.hdfs.server.namenode.sps.IntraSPSNameNodeContext;
+import org.apache.hadoop.hdfs.server.namenode.sps.IntraSPSNameNodeFileIdCollector;
 import org.apache.hadoop.hdfs.server.namenode.sps.StoragePolicySatisfier;
 import org.apache.hadoop.hdfs.server.namenode.startupprogress.Phase;
 import org.apache.hadoop.hdfs.server.namenode.startupprogress.StartupProgress;
@@ -1291,7 +1294,12 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
         FSDirEncryptionZoneOp.warmUpEdekCache(edekCacheLoader, dir,
             edekCacheLoaderDelay, edekCacheLoaderInterval);
       }
-
+      blockManager.getSPSService().init(
+          new IntraSPSNameNodeContext(this, blockManager,
+              blockManager.getSPSService()),
+          new IntraSPSNameNodeFileIdCollector(getFSDirectory(),
+              blockManager.getSPSService()),
+          new IntraSPSNameNodeBlockMoveTaskHandler(blockManager, this));
       blockManager.startSPS();
     } finally {
       startingActiveService = false;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8d4f74e7/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/BlockMoveTaskHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/BlockMoveTaskHandler.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/BlockMoveTaskHandler.java
new file mode 100644
index 0000000..e6f78e1
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/BlockMoveTaskHandler.java
@@ -0,0 +1,44 @@
+/**
+ * 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.sps;
+
+import java.io.IOException;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hdfs.server.protocol.BlockStorageMovementCommand.BlockMovingInfo;
+
+/**
+ * Interface for implementing different ways of block moving approaches. One can
+ * connect directly to DN and request block move, and other can talk NN to
+ * schedule via heart-beats.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public interface BlockMoveTaskHandler {
+
+  /**
+   * This is an interface method to handle the move tasks. BlockMovingInfo must
+   * contain the required info to move the block, that source location,
+   * destination location and storage types.
+   */
+  void submitMoveTask(BlockMovingInfo blkMovingInfo,
+      BlockMovementListener blockMoveCompletionListener) throws IOException;
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8d4f74e7/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/BlockMovementListener.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/BlockMovementListener.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/BlockMovementListener.java
new file mode 100644
index 0000000..36473f3
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/BlockMovementListener.java
@@ -0,0 +1,40 @@
+/**
+ * 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.sps;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hdfs.protocol.Block;
+
+/**
+ * Interface for notifying about block movement attempt completion.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public interface BlockMovementListener {
+
+  /**
+   * This method used to notify to the SPS about block movement attempt
+   * finished. Then SPS will re-check whether it needs retry or not.
+   *
+   * @param moveAttemptFinishedBlks
+   *          -list of movement attempt finished blocks
+   */
+  void notifyMovementTriedBlocks(Block[] moveAttemptFinishedBlks);
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8d4f74e7/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/BlockStorageMovementAttemptedItems.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/BlockStorageMovementAttemptedItems.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/BlockStorageMovementAttemptedItems.java
index 1cae027..3f0155d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/BlockStorageMovementAttemptedItems.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/BlockStorageMovementAttemptedItems.java
@@ -32,7 +32,6 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_
 
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.server.namenode.sps.StoragePolicySatisfier.AttemptedItemInfo;
-import org.apache.hadoop.hdfs.server.namenode.sps.StoragePolicySatisfier.ItemInfo;
 import org.apache.hadoop.util.Daemon;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -47,7 +46,8 @@ import com.google.common.annotations.VisibleForTesting;
  * finished for a longer time period, then such items will retries automatically
  * after timeout. The default timeout would be 5 minutes.
  */
-public class BlockStorageMovementAttemptedItems {
+public class BlockStorageMovementAttemptedItems
+    implements BlockMovementListener {
   private static final Logger LOG =
       LoggerFactory.getLogger(BlockStorageMovementAttemptedItems.class);
 
@@ -71,19 +71,19 @@ public class BlockStorageMovementAttemptedItems {
   //
   private long minCheckTimeout = 1 * 60 * 1000; // minimum value
   private BlockStorageMovementNeeded blockStorageMovementNeeded;
-  private final Context ctxt;
+  private final SPSService service;
 
-  public BlockStorageMovementAttemptedItems(Context context,
+  public BlockStorageMovementAttemptedItems(SPSService service,
       BlockStorageMovementNeeded unsatisfiedStorageMovementFiles) {
-    this.ctxt = context;
-    long recheckTimeout = ctxt.getConf().getLong(
+    this.service = service;
+    long recheckTimeout = this.service.getConf().getLong(
         DFS_STORAGE_POLICY_SATISFIER_RECHECK_TIMEOUT_MILLIS_KEY,
         DFS_STORAGE_POLICY_SATISFIER_RECHECK_TIMEOUT_MILLIS_DEFAULT);
     if (recheckTimeout > 0) {
       this.minCheckTimeout = Math.min(minCheckTimeout, recheckTimeout);
     }
 
-    this.selfRetryTimeout = ctxt.getConf().getLong(
+    this.selfRetryTimeout = this.service.getConf().getLong(
         DFS_STORAGE_POLICY_SATISFIER_SELF_RETRY_TIMEOUT_MILLIS_KEY,
         DFS_STORAGE_POLICY_SATISFIER_SELF_RETRY_TIMEOUT_MILLIS_DEFAULT);
     this.blockStorageMovementNeeded = unsatisfiedStorageMovementFiles;
@@ -111,7 +111,7 @@ public class BlockStorageMovementAttemptedItems {
    * @param moveAttemptFinishedBlks
    *          storage movement attempt finished blocks
    */
-  public void addReportedMovedBlocks(Block[] moveAttemptFinishedBlks) {
+  public void notifyMovementTriedBlocks(Block[] moveAttemptFinishedBlks) {
     if (moveAttemptFinishedBlks.length == 0) {
       return;
     }
@@ -191,7 +191,7 @@ public class BlockStorageMovementAttemptedItems {
         AttemptedItemInfo itemInfo = iter.next();
         if (now > itemInfo.getLastAttemptedOrReportedTime()
             + selfRetryTimeout) {
-          Long blockCollectionID = itemInfo.getTrackId();
+          Long blockCollectionID = itemInfo.getFileId();
           synchronized (movementFinishedBlocks) {
             ItemInfo candidate = new ItemInfo(itemInfo.getStartId(),
                 blockCollectionID, itemInfo.getRetryCount() + 1);
@@ -223,7 +223,7 @@ public class BlockStorageMovementAttemptedItems {
               // gets the chance first and can be cleaned from queue quickly as
               // all movements already done.
               blockStorageMovementNeeded.add(new ItemInfo(attemptedItemInfo
-                  .getStartId(), attemptedItemInfo.getTrackId(),
+                  .getStartId(), attemptedItemInfo.getFileId(),
                   attemptedItemInfo.getRetryCount() + 1));
               iterator.remove();
             }
@@ -246,7 +246,11 @@ public class BlockStorageMovementAttemptedItems {
   }
 
   public void clearQueues() {
-    movementFinishedBlocks.clear();
-    storageMovementAttemptedItems.clear();
+    synchronized (movementFinishedBlocks) {
+      movementFinishedBlocks.clear();
+    }
+    synchronized (storageMovementAttemptedItems) {
+      storageMovementAttemptedItems.clear();
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8d4f74e7/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/BlockStorageMovementNeeded.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/BlockStorageMovementNeeded.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/BlockStorageMovementNeeded.java
index 80f1893..39a0051 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/BlockStorageMovementNeeded.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/BlockStorageMovementNeeded.java
@@ -17,11 +17,7 @@
  */
 package org.apache.hadoop.hdfs.server.namenode.sps;
 
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_QUEUE_LIMIT_DEFAULT;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_QUEUE_LIMIT_KEY;
-
 import java.io.IOException;
-import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.Iterator;
 import java.util.LinkedList;
@@ -33,12 +29,6 @@ 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.FSDirectory;
-import org.apache.hadoop.hdfs.server.namenode.FSTreeTraverser;
-import org.apache.hadoop.hdfs.server.namenode.FSTreeTraverser.TraverseInfo;
-import org.apache.hadoop.hdfs.server.namenode.INode;
-import org.apache.hadoop.hdfs.server.namenode.sps.StoragePolicySatisfier.ItemInfo;
 import org.apache.hadoop.util.Daemon;
 import org.apache.hadoop.util.Time;
 import org.slf4j.Logger;
@@ -75,22 +65,21 @@ public class BlockStorageMovementNeeded {
 
   private final Context ctxt;
 
-  // List of pending dir to satisfy the policy
-  private final Queue<Long> spsDirsToBeTraveresed = new LinkedList<Long>();
+  private Daemon pathIdCollector;
 
-  private Daemon inodeIdCollector;
+  private FileIdCollector fileIDCollector;
 
-  private final int maxQueuedItem;
+  private SPSPathIdProcessor pathIDProcessor;
 
   // Amount of time to cache the SUCCESS status of path before turning it to
   // NOT_AVAILABLE.
   private static long statusClearanceElapsedTimeMs = 300000;
 
-  public BlockStorageMovementNeeded(Context context) {
+  public BlockStorageMovementNeeded(Context context,
+      FileIdCollector fileIDCollector) {
     this.ctxt = context;
-    this.maxQueuedItem = ctxt.getConf().getInt(
-                  DFS_STORAGE_POLICY_SATISFIER_QUEUE_LIMIT_KEY,
-                  DFS_STORAGE_POLICY_SATISFIER_QUEUE_LIMIT_DEFAULT);
+    this.fileIDCollector = fileIDCollector;
+    pathIDProcessor = new SPSPathIdProcessor();
   }
 
   /**
@@ -140,29 +129,6 @@ public class BlockStorageMovementNeeded {
     return storageMovementNeeded.poll();
   }
 
-  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.
-    synchronized (spsDirsToBeTraveresed) {
-      spsDirsToBeTraveresed.notify();
-    }
-  }
-
-  /**
-   * Returns queue remaining capacity.
-   */
-  public synchronized int remainingCapacity() {
-    int size = storageMovementNeeded.size();
-    if (size >= maxQueuedItem) {
-      return 0;
-    } else {
-      return (maxQueuedItem - size);
-    }
-  }
-
   /**
    * Returns queue size.
    */
@@ -171,7 +137,7 @@ public class BlockStorageMovementNeeded {
   }
 
   public synchronized void clearAll() {
-    spsDirsToBeTraveresed.clear();
+    ctxt.removeAllSPSPathIds();
     storageMovementNeeded.clear();
     pendingWorkForDirectory.clear();
   }
@@ -206,13 +172,13 @@ public class BlockStorageMovementNeeded {
     } else {
       // Remove xAttr if trackID doesn't exist in
       // storageMovementAttemptedItems or file policy satisfied.
-      ctxt.removeSPSHint(trackInfo.getTrackId());
+      ctxt.removeSPSHint(trackInfo.getFileId());
       updateStatus(trackInfo.getStartId(), isSuccess);
     }
   }
 
   public synchronized void clearQueue(long trackId) {
-    spsDirsToBeTraveresed.remove(trackId);
+    ctxt.removeSPSPathId(trackId);
     Iterator<ItemInfo> iterator = storageMovementNeeded.iterator();
     while (iterator.hasNext()) {
       ItemInfo next = iterator.next();
@@ -249,7 +215,7 @@ public class BlockStorageMovementNeeded {
   public synchronized void clearQueuesWithNotification() {
     // Remove xAttr from directories
     Long trackId;
-    while ((trackId = spsDirsToBeTraveresed.poll()) != null) {
+    while ((trackId = ctxt.getNextSPSPathId()) != null) {
       try {
         // Remove xAttr for file
         ctxt.removeSPSHint(trackId);
@@ -265,12 +231,12 @@ public class BlockStorageMovementNeeded {
       try {
         // Remove xAttr for file
         if (!itemInfo.isDir()) {
-          ctxt.removeSPSHint(itemInfo.getTrackId());
+          ctxt.removeSPSHint(itemInfo.getFileId());
         }
       } catch (IOException ie) {
         LOG.warn(
             "Failed to remove SPS xattr for track id "
-                + itemInfo.getTrackId(), ie);
+                + itemInfo.getFileId(), ie);
       }
     }
     this.clearAll();
@@ -280,57 +246,33 @@ public class BlockStorageMovementNeeded {
    * Take dir tack ID from the spsDirsToBeTraveresed queue and collect child
    * ID's to process for satisfy the policy.
    */
-  private class StorageMovementPendingInodeIdCollector extends FSTreeTraverser
-      implements Runnable {
-
-    private int remainingCapacity = 0;
-
-    private List<ItemInfo> currentBatch = new ArrayList<>(maxQueuedItem);
-
-    StorageMovementPendingInodeIdCollector(FSDirectory dir) {
-      super(dir);
-    }
+  private class SPSPathIdProcessor implements Runnable {
 
     @Override
     public void run() {
       LOG.info("Starting FileInodeIdCollector!.");
       long lastStatusCleanTime = 0;
       while (ctxt.isRunning()) {
+        LOG.info("Running FileInodeIdCollector!.");
         try {
           if (!ctxt.isInSafeMode()) {
-            Long startINodeId = spsDirsToBeTraveresed.poll();
+            Long startINodeId = ctxt.getNextSPSPathId();
             if (startINodeId == null) {
               // Waiting for SPS path
-              synchronized (spsDirsToBeTraveresed) {
-                spsDirsToBeTraveresed.wait(5000);
-              }
+              Thread.sleep(3000);
             } else {
-              INode startInode = getFSDirectory().getInode(startINodeId);
-              if (startInode != null) {
-                try {
-                  remainingCapacity = remainingCapacity();
-                  spsStatus.put(startINodeId,
-                      new StoragePolicySatisfyPathStatusInfo(
-                          StoragePolicySatisfyPathStatus.IN_PROGRESS));
-                  readLock();
-                  traverseDir(startInode.asDirectory(), startINodeId,
-                      HdfsFileStatus.EMPTY_NAME,
-                      new SPSTraverseInfo(startINodeId));
-                } finally {
-                  readUnlock();
-                }
-                // 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()) {
-                  ctxt.removeSPSHint(startInode.getId());
-                  pendingWorkForDirectory.remove(startInode.getId());
-                  updateStatus(startInode.getId(), true);
-                }
+              spsStatus.put(startINodeId,
+                  new StoragePolicySatisfyPathStatusInfo(
+                      StoragePolicySatisfyPathStatus.IN_PROGRESS));
+              fileIDCollector.scanAndCollectFileIds(startINodeId);
+              // check if directory was empty and no child added to queue
+              DirPendingWorkInfo dirPendingWorkInfo =
+                  pendingWorkForDirectory.get(startINodeId);
+              if (dirPendingWorkInfo != null
+                  && dirPendingWorkInfo.isDirWorkDone()) {
+                ctxt.removeSPSHint(startINodeId);
+                pendingWorkForDirectory.remove(startINodeId);
+                updateStatus(startINodeId, true);
               }
             }
             //Clear the SPS status if status is in SUCCESS more than 5 min.
@@ -355,71 +297,6 @@ public class BlockStorageMovementNeeded {
         }
       }
     }
-
-    @Override
-    protected void checkPauseForTesting() throws InterruptedException {
-      // TODO implement if needed
-    }
-
-    @Override
-    protected boolean processFileInode(INode inode, TraverseInfo traverseInfo)
-        throws IOException, InterruptedException {
-      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 {
-      // SPS work won't be scheduled if NN is in standby. So, skipping NN
-      // standby check.
-      return;
-    }
-
-    @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 {
-      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;
-    }
   }
 
   /**
@@ -476,29 +353,15 @@ public class BlockStorageMovementNeeded {
     }
   }
 
-  // TODO: FSDirectory will get removed via HDFS-12911 modularization work
-  public void init(FSDirectory fsd) {
-    inodeIdCollector = new Daemon(new StorageMovementPendingInodeIdCollector(
-        fsd));
-    inodeIdCollector.setName("FileInodeIdCollector");
-    inodeIdCollector.start();
+  public void activate() {
+    pathIdCollector = new Daemon(pathIDProcessor);
+    pathIdCollector.setName("SPSPathIdProcessor");
+    pathIdCollector.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;
+    if (pathIdCollector != null) {
+      pathIdCollector.interrupt();
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8d4f74e7/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/Context.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/Context.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/Context.java
index d11e26f..b7053b9 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/Context.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/Context.java
@@ -19,11 +19,9 @@
 package org.apache.hadoop.hdfs.server.namenode.sps;
 
 import java.io.IOException;
-import java.util.function.Supplier;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.ParentNotDirectoryException;
 import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.fs.UnresolvedLinkException;
@@ -31,7 +29,6 @@ import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeStorageReport;
-import org.apache.hadoop.hdfs.server.protocol.BlockStorageMovementCommand.BlockMovingInfo;
 import org.apache.hadoop.net.NetworkTopology;
 import org.apache.hadoop.security.AccessControlException;
 
@@ -43,24 +40,11 @@ import org.apache.hadoop.security.AccessControlException;
 public interface Context {
 
   /**
-   * Returns configuration object.
-   */
-  Configuration getConf();
-
-  /**
    * Returns true if the SPS is running, false otherwise.
    */
   boolean isRunning();
 
   /**
-   * Update the SPS running status.
-   *
-   * @param isSpsRunning
-   *          true represents running, false otherwise
-   */
-  void setSPSRunning(Supplier<Boolean> isSpsRunning);
-
-  /**
    * Returns true if the Namenode in safe mode, false otherwise.
    */
   boolean isInSafeMode();
@@ -153,17 +137,6 @@ public interface Context {
   boolean hasLowRedundancyBlocks(long inodeID);
 
   /**
-   * Assign the given block movement task to the target node present in
-   * {@link BlockMovingInfo}.
-   *
-   * @param blkMovingInfo
-   *          block to storage info
-   * @throws IOException
-   */
-  void assignBlockMoveTaskToTargetNode(BlockMovingInfo blkMovingInfo)
-      throws IOException;
-
-  /**
    * Checks whether the given datanode has sufficient space to occupy the given
    * blockSize data.
    *
@@ -178,4 +151,20 @@ public interface Context {
    */
   boolean verifyTargetDatanodeHasSpaceForScheduling(DatanodeInfo dn,
       StorageType type, long blockSize);
+
+  /**
+   * @return next SPS path id to process.
+   */
+  Long getNextSPSPathId();
+
+  /**
+   * Removes the SPS path id.
+   */
+  void removeSPSPathId(long pathId);
+
+  /**
+   * Removes all SPS path ids.
+   */
+  void removeAllSPSPathIds();
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8d4f74e7/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/FileIdCollector.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/FileIdCollector.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/FileIdCollector.java
new file mode 100644
index 0000000..7cf77f0
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/FileIdCollector.java
@@ -0,0 +1,43 @@
+/**
+ * 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.sps;
+
+import java.io.IOException;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * An interface for scanning the directory recursively and collect file ids
+ * under the given directory.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public interface FileIdCollector {
+
+  /**
+   * Scans the given inode directory and collects the file ids under that
+   * directory and adds to the given BlockStorageMovementNeeded.
+   *
+   * @param inodeID
+   *          - The directory ID
+   */
+  void scanAndCollectFileIds(Long inodeId)
+      throws IOException, InterruptedException;
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8d4f74e7/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/IntraSPSNameNodeBlockMoveTaskHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/IntraSPSNameNodeBlockMoveTaskHandler.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/IntraSPSNameNodeBlockMoveTaskHandler.java
new file mode 100644
index 0000000..1da4af9
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/IntraSPSNameNodeBlockMoveTaskHandler.java
@@ -0,0 +1,62 @@
+/**
+ * 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.sps;
+
+import java.io.IOException;
+
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
+import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
+import org.apache.hadoop.hdfs.server.namenode.Namesystem;
+import org.apache.hadoop.hdfs.server.protocol.BlockStorageMovementCommand.BlockMovingInfo;
+
+/**
+ * This class handles the internal SPS block movements. This will assign block
+ * movement tasks to target datanode descriptors.
+ */
+public class IntraSPSNameNodeBlockMoveTaskHandler
+    implements BlockMoveTaskHandler {
+
+  private BlockManager blockManager;
+  private Namesystem namesystem;
+
+  public IntraSPSNameNodeBlockMoveTaskHandler(BlockManager blockManager,
+      Namesystem namesytem) {
+    this.blockManager = blockManager;
+    this.namesystem = namesytem;
+  }
+
+  @Override
+  public void submitMoveTask(BlockMovingInfo blkMovingInfo,
+      BlockMovementListener blockMoveCompletionListener) throws IOException {
+    namesystem.readLock();
+    try {
+      DatanodeDescriptor dn = blockManager.getDatanodeManager()
+          .getDatanode(blkMovingInfo.getTarget().getDatanodeUuid());
+      if (dn == null) {
+        throw new IOException("Failed to schedule block movement task:"
+            + blkMovingInfo + " as target datanode: "
+            + blkMovingInfo.getTarget() + " doesn't exists");
+      }
+      dn.incrementBlocksScheduled(blkMovingInfo.getTargetStorageType());
+      dn.addBlocksToMoveStorage(blkMovingInfo);
+    } finally {
+      namesystem.readUnlock();
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8d4f74e7/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/IntraSPSNameNodeContext.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/IntraSPSNameNodeContext.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/IntraSPSNameNodeContext.java
index 6654212..cef26ed 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/IntraSPSNameNodeContext.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/IntraSPSNameNodeContext.java
@@ -20,10 +20,8 @@ package org.apache.hadoop.hdfs.server.namenode.sps;
 import static org.apache.hadoop.hdfs.server.common.HdfsServerConstants.XATTR_SATISFY_STORAGE_POLICY;
 
 import java.io.IOException;
-import java.util.function.Supplier;
 
 import org.apache.commons.lang.StringUtils;
-import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.ParentNotDirectoryException;
 import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.fs.UnresolvedLinkException;
@@ -38,7 +36,6 @@ import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.namenode.INode;
 import org.apache.hadoop.hdfs.server.namenode.Namesystem;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeStorageReport;
-import org.apache.hadoop.hdfs.server.protocol.BlockStorageMovementCommand.BlockMovingInfo;
 import org.apache.hadoop.net.NetworkTopology;
 import org.apache.hadoop.security.AccessControlException;
 import org.slf4j.Logger;
@@ -55,15 +52,14 @@ public class IntraSPSNameNodeContext implements Context {
 
   private final Namesystem namesystem;
   private final BlockManager blockManager;
-  private final Configuration conf;
-  private Supplier<Boolean> isSpsRunning;
+
+  private SPSService service;
 
   public IntraSPSNameNodeContext(Namesystem namesystem,
-      BlockManager blockManager, Configuration conf) {
+      BlockManager blockManager, SPSService service) {
     this.namesystem = namesystem;
     this.blockManager = blockManager;
-    this.conf = conf;
-    isSpsRunning = () -> false;
+    this.service = service;
   }
 
   @Override
@@ -111,11 +107,6 @@ public class IntraSPSNameNodeContext implements Context {
   }
 
   @Override
-  public Configuration getConf() {
-    return conf;
-  }
-
-  @Override
   public boolean isFileExist(long inodeId) {
     return namesystem.getFSDirectory().getInode(inodeId) != null;
   }
@@ -127,16 +118,7 @@ public class IntraSPSNameNodeContext implements Context {
 
   @Override
   public boolean isRunning() {
-    // TODO : 'isSpsRunning' flag has been added to avoid the NN lock inside
-    // SPS. Context interface will be further refined as part of HDFS-12911
-    // modularization task. One idea is to introduce a cleaner interface similar
-    // to Namesystem for better abstraction.
-    return namesystem.isRunning() && isSpsRunning.get();
-  }
-
-  @Override
-  public void setSPSRunning(Supplier<Boolean> spsRunningFlag) {
-    this.isSpsRunning = spsRunningFlag;
+    return namesystem.isRunning() && service.isRunning();
   }
 
   @Override
@@ -183,25 +165,6 @@ public class IntraSPSNameNodeContext implements Context {
   }
 
   @Override
-  public void assignBlockMoveTaskToTargetNode(BlockMovingInfo blkMovingInfo)
-      throws IOException {
-    namesystem.readLock();
-    try {
-      DatanodeDescriptor dn = blockManager.getDatanodeManager()
-          .getDatanode(blkMovingInfo.getTarget().getDatanodeUuid());
-      if (dn == null) {
-        throw new IOException("Failed to schedule block movement task:"
-            + blkMovingInfo + " as target datanode: "
-            + blkMovingInfo.getTarget() + " doesn't exists");
-      }
-      dn.addBlocksToMoveStorage(blkMovingInfo);
-      dn.incrementBlocksScheduled(blkMovingInfo.getTargetStorageType());
-    } finally {
-      namesystem.readUnlock();
-    }
-  }
-
-  @Override
   public boolean verifyTargetDatanodeHasSpaceForScheduling(DatanodeInfo dn,
       StorageType type, long blockSize) {
     namesystem.readLock();
@@ -217,4 +180,19 @@ public class IntraSPSNameNodeContext implements Context {
       namesystem.readUnlock();
     }
   }
+
+  @Override
+  public Long getNextSPSPathId() {
+    return blockManager.getNextSPSPathId();
+  }
+
+  @Override
+  public void removeSPSPathId(long trackId) {
+    blockManager.removeSPSPathId(trackId);
+  }
+
+  @Override
+  public void removeAllSPSPathIds() {
+    blockManager.removeAllSPSPathIds();
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8d4f74e7/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/IntraSPSNameNodeFileIdCollector.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/IntraSPSNameNodeFileIdCollector.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/IntraSPSNameNodeFileIdCollector.java
new file mode 100644
index 0000000..c6834c1
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/IntraSPSNameNodeFileIdCollector.java
@@ -0,0 +1,178 @@
+/**
+ * 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.sps;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
+import org.apache.hadoop.hdfs.server.namenode.FSDirectory;
+import org.apache.hadoop.hdfs.server.namenode.FSTreeTraverser;
+import org.apache.hadoop.hdfs.server.namenode.INode;
+
+/**
+ * A specific implementation for scanning the directory with Namenode internal
+ * Inode structure and collects the file ids under the given directory ID.
+ */
+public class IntraSPSNameNodeFileIdCollector extends FSTreeTraverser
+    implements FileIdCollector {
+  private int maxQueueLimitToScan;
+  private final SPSService service;
+
+  private int remainingCapacity = 0;
+
+  private List<ItemInfo> currentBatch;
+
+  public IntraSPSNameNodeFileIdCollector(FSDirectory dir, SPSService service) {
+    super(dir);
+    this.service = service;
+    this.maxQueueLimitToScan = service.getConf().getInt(
+        DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_QUEUE_LIMIT_KEY,
+        DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_QUEUE_LIMIT_DEFAULT);
+    currentBatch = new ArrayList<>(maxQueueLimitToScan);
+  }
+
+  @Override
+  protected boolean processFileInode(INode inode, TraverseInfo traverseInfo)
+      throws IOException, InterruptedException {
+    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 {
+    // SPS work won't be scheduled if NN is in standby. So, skipping NN
+    // standby check.
+    return;
+  }
+
+  @Override
+  protected void submitCurrentBatch(long startId)
+      throws IOException, InterruptedException {
+    // Add current child's to queue
+    service.addAllFileIdsToProcess(startId,
+        currentBatch, false);
+    currentBatch.clear();
+  }
+
+  @Override
+  protected void throttle() throws InterruptedException {
+    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;
+  }
+
+  @Override
+  protected void checkPauseForTesting() throws InterruptedException {
+    // Nothing to do
+  }
+
+  @Override
+  public void scanAndCollectFileIds(final Long startINodeId)
+      throws IOException, InterruptedException {
+    FSDirectory fsd = getFSDirectory();
+    INode startInode = fsd.getInode(startINodeId);
+    if (startInode != null) {
+      remainingCapacity = remainingCapacity();
+      if (remainingCapacity == 0) {
+        throttle();
+      }
+      if (startInode.isFile()) {
+        currentBatch.add(new ItemInfo(startInode.getId(), startInode.getId()));
+      } else {
+
+        readLock();
+        // NOTE: this lock will not be held until full directory scanning. It is
+        // basically a sliced locking. Once it collects a batch size( at max the
+        // size of maxQueueLimitToScan (default 1000)) file ids, then it will
+        // unlock and submits the current batch to SPSService. Once
+        // service.processingQueueSize() shows empty slots, then lock will be
+        // resumed and scan also will be resumed. This logic was re-used from
+        // EDEK feature.
+        try {
+          traverseDir(startInode.asDirectory(), startINodeId,
+              HdfsFileStatus.EMPTY_NAME, new SPSTraverseInfo(startINodeId));
+        } finally {
+          readUnlock();
+        }
+      }
+      // Mark startInode traverse is done, this is last-batch
+      service.addAllFileIdsToProcess(startInode.getId(), currentBatch, true);
+      currentBatch.clear();
+    }
+  }
+
+  /**
+   * Returns queue remaining capacity.
+   */
+  public synchronized int remainingCapacity() {
+    int size = service.processingQueueSize();
+    if (size >= maxQueueLimitToScan) {
+      return 0;
+    } else {
+      return (maxQueueLimitToScan - size);
+    }
+  }
+
+  class SPSTraverseInfo extends TraverseInfo {
+    private long startId;
+
+    SPSTraverseInfo(long startId) {
+      this.startId = startId;
+    }
+
+    public long getStartId() {
+      return startId;
+    }
+  }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8d4f74e7/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/ItemInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/ItemInfo.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/ItemInfo.java
new file mode 100644
index 0000000..47c64cc
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/ItemInfo.java
@@ -0,0 +1,81 @@
+/**
+ * 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.sps;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * ItemInfo is a file info object for which need to satisfy the policy.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class ItemInfo {
+  private long startId;
+  private long fileId;
+  private int retryCount;
+
+  public ItemInfo(long startId, long fileId) {
+    this.startId = startId;
+    this.fileId = fileId;
+    // set 0 when item is getting added first time in queue.
+    this.retryCount = 0;
+  }
+
+  public ItemInfo(final long startId, final long fileId, final int retryCount) {
+    this.startId = startId;
+    this.fileId = fileId;
+    this.retryCount = retryCount;
+  }
+
+  /**
+   * Return the start inode id of the current track Id. This indicates that SPS
+   * was invoked on this inode id.
+   */
+  public long getStartId() {
+    return startId;
+  }
+
+  /**
+   * Return the File inode Id for which needs to satisfy the policy.
+   */
+  public long getFileId() {
+    return fileId;
+  }
+
+  /**
+   * Returns true if the tracking path is a directory, false otherwise.
+   */
+  public boolean isDir() {
+    return (startId != fileId);
+  }
+
+  /**
+   * Get the attempted retry count of the block for satisfy the policy.
+   */
+  public int getRetryCount() {
+    return retryCount;
+  }
+
+  /**
+   * Increments the retry count.
+   */
+  public void increRetryCount() {
+    this.retryCount++;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8d4f74e7/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/SPSPathIds.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/SPSPathIds.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/SPSPathIds.java
new file mode 100644
index 0000000..cd6ad22
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/SPSPathIds.java
@@ -0,0 +1,63 @@
+/**
+ * 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.sps;
+
+import java.util.LinkedList;
+import java.util.Queue;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * A class which holds the SPS invoked path ids.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class SPSPathIds {
+
+  // List of pending dir to satisfy the policy
+  private final Queue<Long> spsDirsToBeTraveresed = new LinkedList<Long>();
+
+  /**
+   * Add the path id to queue.
+   */
+  public synchronized void add(long pathId) {
+    spsDirsToBeTraveresed.add(pathId);
+  }
+
+  /**
+   * Removes the path id.
+   */
+  public synchronized void remove(long pathId) {
+    spsDirsToBeTraveresed.remove(pathId);
+  }
+
+  /**
+   * Clears all path ids.
+   */
+  public synchronized void clear() {
+    spsDirsToBeTraveresed.clear();
+  }
+
+  /**
+   * @return next path id available in queue.
+   */
+  public synchronized Long pollNext() {
+    return spsDirsToBeTraveresed.poll();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8d4f74e7/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/SPSService.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/SPSService.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/SPSService.java
new file mode 100644
index 0000000..6d85ea6
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/SPSService.java
@@ -0,0 +1,107 @@
+/**
+ * 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.sps;
+
+import java.util.List;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+
+/**
+ * An interface for SPSService, which exposes life cycle and processing APIs.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public interface SPSService {
+
+  /**
+   * Initializes the helper services.
+   *
+   * @param ctxt
+   *          - context is an helper service to provide communication channel
+   *          between NN and SPS
+   * @param fileIDCollector
+   *          - a helper service for scanning the files under a given directory
+   *          id
+   * @param handler
+   *          - a helper service for moving the blocks
+   */
+  void init(Context ctxt, FileIdCollector fileIDCollector,
+      BlockMoveTaskHandler handler);
+
+  /**
+   * Starts the SPS service. Make sure to initialize the helper services before
+   * invoking this method.
+   *
+   * @param reconfigStart
+   *          - to indicate whether the SPS startup requested from
+   *          reconfiguration service
+   */
+  void start(boolean reconfigStart);
+
+  /**
+   * Stops the SPS service gracefully. Timed wait to stop storage policy
+   * satisfier daemon threads.
+   */
+  void stopGracefully();
+
+  /**
+   * Disable the SPS service.
+   *
+   * @param forceStop
+   */
+  void disable(boolean forceStop);
+
+  /**
+   * Check whether StoragePolicySatisfier is running.
+   *
+   * @return true if running
+   */
+  boolean isRunning();
+
+  /**
+   * Adds the Item information(file id etc) to processing queue.
+   *
+   * @param itemInfo
+   */
+  void addFileIdToProcess(ItemInfo itemInfo);
+
+  /**
+   * Adds all the Item information(file id etc) to processing queue.
+   *
+   * @param startId
+   *          - directory/file id, on which SPS was called.
+   * @param itemInfoList
+   *          - list of item infos
+   * @param scanCompleted
+   *          - whether the scanning of directory fully done with itemInfoList
+   */
+  void addAllFileIdsToProcess(long startId, List<ItemInfo> itemInfoList,
+      boolean scanCompleted);
+
+  /**
+   * @return current processing queue size.
+   */
+  int processingQueueSize();
+
+  /**
+   * @return the configuration.
+   */
+  Configuration getConf();
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8d4f74e7/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/StoragePolicySatisfier.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/StoragePolicySatisfier.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/StoragePolicySatisfier.java
index b3e6b78..28c1372 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/StoragePolicySatisfier.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/StoragePolicySatisfier.java
@@ -29,6 +29,7 @@ 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;
@@ -47,7 +48,6 @@ import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock;
 import org.apache.hadoop.hdfs.server.balancer.Matcher;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.namenode.ErasureCodingPolicyManager;
-import org.apache.hadoop.hdfs.server.namenode.FSDirectory;
 import org.apache.hadoop.hdfs.server.protocol.BlockStorageMovementCommand.BlockMovingInfo;
 import org.apache.hadoop.hdfs.server.protocol.BlocksStorageMoveAttemptFinished;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeStorageReport;
@@ -64,28 +64,34 @@ import com.google.common.annotations.VisibleForTesting;
  * 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.
+ * where users may want to physically move the blocks by a dedidated daemon (can
+ * run inside Namenode or stand alone) 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 SPS 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 requests to send to Datanode for processing the
+ * physical block movements.
  */
 @InterfaceAudience.Private
-public class StoragePolicySatisfier implements Runnable {
+public class StoragePolicySatisfier implements SPSService, Runnable {
   public static final Logger LOG =
       LoggerFactory.getLogger(StoragePolicySatisfier.class);
   private Daemon storagePolicySatisfierThread;
-  private final BlockStorageMovementNeeded storageMovementNeeded;
-  private final BlockStorageMovementAttemptedItems storageMovementsMonitor;
+  private BlockStorageMovementNeeded storageMovementNeeded;
+  private BlockStorageMovementAttemptedItems storageMovementsMonitor;
   private volatile boolean isRunning = false;
   private int spsWorkMultiplier;
   private long blockCount = 0L;
   private int blockMovementMaxRetry;
-  private final Context ctxt;
+  private Context ctxt;
+  private BlockMoveTaskHandler blockMoveTaskHandler;
+  private Configuration conf;
 
+  public StoragePolicySatisfier(Configuration conf) {
+    this.conf = conf;
+  }
   /**
    * Represents the collective analysis status for all blocks.
    */
@@ -125,13 +131,17 @@ public class StoragePolicySatisfier implements Runnable {
     }
   }
 
-  public StoragePolicySatisfier(Context ctxt) {
-    this.ctxt = ctxt;
-    this.storageMovementNeeded = new BlockStorageMovementNeeded(ctxt);
-    this.storageMovementsMonitor = new BlockStorageMovementAttemptedItems(ctxt,
+  public void init(final Context context, final FileIdCollector fileIDCollector,
+      final BlockMoveTaskHandler blockMovementTaskHandler) {
+    this.ctxt = context;
+    this.storageMovementNeeded =
+        new BlockStorageMovementNeeded(context, fileIDCollector);
+    this.storageMovementsMonitor =
+        new BlockStorageMovementAttemptedItems(this,
         storageMovementNeeded);
-    this.spsWorkMultiplier = DFSUtil.getSPSWorkMultiplier(ctxt.getConf());
-    this.blockMovementMaxRetry = ctxt.getConf().getInt(
+    this.blockMoveTaskHandler = blockMovementTaskHandler;
+    this.spsWorkMultiplier = DFSUtil.getSPSWorkMultiplier(getConf());
+    this.blockMovementMaxRetry = getConf().getInt(
         DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_MAX_RETRY_ATTEMPTS_KEY,
         DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_MAX_RETRY_ATTEMPTS_DEFAULT);
   }
@@ -139,12 +149,10 @@ public class StoragePolicySatisfier implements Runnable {
   /**
    * Start storage policy satisfier demon thread. Also start block storage
    * movements monitor for retry the attempts if needed.
-   *
-   * // TODO: FSDirectory will get removed via HDFS-12911 modularization work.
    */
-  public synchronized void start(boolean reconfigStart, FSDirectory fsd) {
+  @Override
+  public synchronized void start(boolean reconfigStart) {
     isRunning = true;
-    ctxt.setSPSRunning(this::isRunning);
     if (ctxt.isMoverRunning()) {
       isRunning = false;
       LOG.error(
@@ -163,20 +171,14 @@ public class StoragePolicySatisfier implements Runnable {
     // Ensure that all the previously submitted block movements(if any) have to
     // be stopped in all datanodes.
     addDropSPSWorkCommandsToAllDNs();
-    storageMovementNeeded.init(fsd);
     storagePolicySatisfierThread = new Daemon(this);
     storagePolicySatisfierThread.setName("StoragePolicySatisfier");
     storagePolicySatisfierThread.start();
     this.storageMovementsMonitor.start();
+    this.storageMovementNeeded.activate();
   }
 
-  /**
-   * Disables storage policy satisfier by stopping its services.
-   *
-   * @param forceStop
-   *          true represents that it should stop SPS service by clearing all
-   *          pending SPS work
-   */
+  @Override
   public synchronized void disable(boolean forceStop) {
     isRunning = false;
     if (storagePolicySatisfierThread == null) {
@@ -195,14 +197,15 @@ public class StoragePolicySatisfier implements Runnable {
     }
   }
 
-  /**
-   * Timed wait to stop storage policy satisfier daemon threads.
-   */
+  @Override
   public synchronized void stopGracefully() {
     if (isRunning) {
       disable(true);
     }
-    this.storageMovementsMonitor.stopGracefully();
+
+    if (this.storageMovementsMonitor != null) {
+      this.storageMovementsMonitor.stopGracefully();
+    }
 
     if (storagePolicySatisfierThread == null) {
       return;
@@ -213,10 +216,7 @@ public class StoragePolicySatisfier implements Runnable {
     }
   }
 
-  /**
-   * Check whether StoragePolicySatisfier is running.
-   * @return true if running
-   */
+  @Override
   public boolean isRunning() {
     return isRunning;
   }
@@ -239,11 +239,11 @@ public class StoragePolicySatisfier implements Runnable {
             if(itemInfo.getRetryCount() >= blockMovementMaxRetry){
               LOG.info("Failed to satisfy the policy after "
                   + blockMovementMaxRetry + " retries. Removing inode "
-                  + itemInfo.getTrackId() + " from the queue");
+                  + itemInfo.getFileId() + " from the queue");
               storageMovementNeeded.removeItemTrackInfo(itemInfo, false);
               continue;
             }
-            long trackId = itemInfo.getTrackId();
+            long trackId = itemInfo.getFileId();
             BlocksMovingAnalysis status = null;
             DatanodeStorageReport[] liveDnReports;
             BlockStoragePolicy existingStoragePolicy;
@@ -273,7 +273,7 @@ public class StoragePolicySatisfier implements Runnable {
                 // be removed on storage movement attempt finished report.
               case BLOCKS_TARGETS_PAIRED:
                 this.storageMovementsMonitor.add(new AttemptedItemInfo(itemInfo
-                    .getStartId(), itemInfo.getTrackId(), monotonicNow(),
+                    .getStartId(), itemInfo.getFileId(), monotonicNow(),
                     status.assignedBlocks, itemInfo.getRetryCount()));
                 break;
               case NO_BLOCKS_TARGETS_PAIRED:
@@ -282,7 +282,7 @@ public class StoragePolicySatisfier implements Runnable {
                       + " back to retry queue as none of the blocks"
                       + " found its eligible targets.");
                 }
-                itemInfo.retryCount++;
+                itemInfo.increRetryCount();
                 this.storageMovementNeeded.add(itemInfo);
                 break;
               case FEW_LOW_REDUNDANCY_BLOCKS:
@@ -426,7 +426,8 @@ public class StoragePolicySatisfier implements Runnable {
     for (BlockMovingInfo blkMovingInfo : blockMovingInfos) {
       // Check for at least one block storage movement has been chosen
       try {
-        ctxt.assignBlockMoveTaskToTargetNode(blkMovingInfo);
+        blockMoveTaskHandler.submitMoveTask(blkMovingInfo,
+            storageMovementsMonitor);
         LOG.debug("BlockMovingInfo: {}", blkMovingInfo);
         assignedBlockIds.add(blkMovingInfo.getBlock());
         blockCount++;
@@ -611,7 +612,6 @@ public class StoragePolicySatisfier implements Runnable {
 
         expected.remove(chosenTarget.storageType);
         excludeNodes.add(chosenTarget.dn);
-        // TODO: We can increment scheduled block count for this node?
       } else {
         LOG.warn(
             "Failed to choose target datanode for the required"
@@ -830,11 +830,11 @@ public class StoragePolicySatisfier implements Runnable {
       return;
     }
     storageMovementsMonitor
-        .addReportedMovedBlocks(moveAttemptFinishedBlks.getBlocks());
+        .notifyMovementTriedBlocks(moveAttemptFinishedBlks.getBlocks());
   }
 
   @VisibleForTesting
-  BlockStorageMovementAttemptedItems getAttemptedItemsMonitor() {
+  BlockMovementListener getAttemptedItemsMonitor() {
     return storageMovementsMonitor;
   }
 
@@ -863,10 +863,6 @@ public class StoragePolicySatisfier implements Runnable {
     }
   }
 
-  public void addInodeToPendingDirQueue(long id) {
-    storageMovementNeeded.addToPendingDirQueue(id);
-  }
-
   /**
    * Clear queues for given track id.
    */
@@ -875,57 +871,6 @@ public class StoragePolicySatisfier implements Runnable {
   }
 
   /**
-   * ItemInfo is a file info object for which need to satisfy the
-   * policy.
-   */
-  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;
-    }
-
-    /**
-     * Return the start inode id of the current track Id.
-     */
-    public long getStartId() {
-      return startId;
-    }
-
-    /**
-     * 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 (startId != trackId);
-    }
-
-    /**
-     * Get the attempted retry count of the block for satisfy the policy.
-     */
-    public int getRetryCount() {
-      return retryCount;
-    }
-  }
-
-  /**
    * This class contains information of an attempted blocks and its last
    * attempted or reported time stamp. This is used by
    * {@link BlockStorageMovementAttemptedItems#storageMovementAttemptedItems}.
@@ -977,4 +922,30 @@ public class StoragePolicySatisfier implements Runnable {
       String path) throws IOException {
     return storageMovementNeeded.getStatus(ctxt.getFileID(path));
   }
+
+  @Override
+  public void addFileIdToProcess(ItemInfo trackInfo) {
+    storageMovementNeeded.add(trackInfo);
+  }
+
+  @Override
+  public void addAllFileIdsToProcess(long startId, List<ItemInfo> itemInfoList,
+      boolean scanCompleted) {
+    getStorageMovementQueue().addAll(startId, itemInfoList, scanCompleted);
+  }
+
+  @Override
+  public int processingQueueSize() {
+    return storageMovementNeeded.size();
+  }
+
+  @Override
+  public Configuration getConf() {
+    return conf;
+  }
+
+  @VisibleForTesting
+  public BlockStorageMovementNeeded getStorageMovementQueue() {
+    return storageMovementNeeded;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8d4f74e7/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/sps/TestBlockStorageMovementAttemptedItems.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/sps/TestBlockStorageMovementAttemptedItems.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/sps/TestBlockStorageMovementAttemptedItems.java
index f9762a8..3e2c324 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/sps/TestBlockStorageMovementAttemptedItems.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/sps/TestBlockStorageMovementAttemptedItems.java
@@ -29,7 +29,6 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.server.namenode.sps.StoragePolicySatisfier.AttemptedItemInfo;
-import org.apache.hadoop.hdfs.server.namenode.sps.StoragePolicySatisfier.ItemInfo;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
@@ -49,12 +48,14 @@ public class TestBlockStorageMovementAttemptedItems {
   public void setup() throws Exception {
     Configuration config = new HdfsConfiguration();
     Context ctxt = Mockito.mock(Context.class);
-    Mockito.when(ctxt.getConf()).thenReturn(config);
+    SPSService sps = Mockito.mock(StoragePolicySatisfier.class);
+    Mockito.when(sps.getConf()).thenReturn(config);
     Mockito.when(ctxt.isRunning()).thenReturn(true);
     Mockito.when(ctxt.isInSafeMode()).thenReturn(false);
     Mockito.when(ctxt.isFileExist(Mockito.anyLong())).thenReturn(true);
-    unsatisfiedStorageMovementFiles = new BlockStorageMovementNeeded(ctxt);
-    bsmAttemptedItems = new BlockStorageMovementAttemptedItems(ctxt,
+    unsatisfiedStorageMovementFiles =
+        new BlockStorageMovementNeeded(ctxt, null);
+    bsmAttemptedItems = new BlockStorageMovementAttemptedItems(sps,
         unsatisfiedStorageMovementFiles);
   }
 
@@ -73,7 +74,7 @@ public class TestBlockStorageMovementAttemptedItems {
     while (monotonicNow() < (stopTime)) {
       ItemInfo ele = null;
       while ((ele = unsatisfiedStorageMovementFiles.get()) != null) {
-        if (item == ele.getTrackId()) {
+        if (item == ele.getFileId()) {
           isItemFound = true;
           break;
         }
@@ -99,7 +100,7 @@ public class TestBlockStorageMovementAttemptedItems {
     bsmAttemptedItems.add(new AttemptedItemInfo(0L, 0L, 0L, blocks, 0));
     Block[] blockArray = new Block[blocks.size()];
     blocks.toArray(blockArray);
-    bsmAttemptedItems.addReportedMovedBlocks(blockArray);
+    bsmAttemptedItems.notifyMovementTriedBlocks(blockArray);
     assertEquals("Failed to receive result!", 1,
         bsmAttemptedItems.getMovementFinishedBlocksCount());
   }
@@ -137,7 +138,7 @@ public class TestBlockStorageMovementAttemptedItems {
         .add(new AttemptedItemInfo(trackID, trackID, 0L, blocks, 0));
     Block[] blksMovementReport = new Block[1];
     blksMovementReport[0] = new Block(item);
-    bsmAttemptedItems.addReportedMovedBlocks(blksMovementReport);
+    bsmAttemptedItems.notifyMovementTriedBlocks(blksMovementReport);
 
     // start block movement report monitor thread
     bsmAttemptedItems.start();
@@ -162,7 +163,7 @@ public class TestBlockStorageMovementAttemptedItems {
         .add(new AttemptedItemInfo(trackID, trackID, 0L, blocks, 0));
     Block[] blksMovementReport = new Block[1];
     blksMovementReport[0] = new Block(item);
-    bsmAttemptedItems.addReportedMovedBlocks(blksMovementReport);
+    bsmAttemptedItems.notifyMovementTriedBlocks(blksMovementReport);
 
     Thread.sleep(selfRetryTimeout * 2); // Waiting to get timed out
 
@@ -190,7 +191,7 @@ public class TestBlockStorageMovementAttemptedItems {
         .add(new AttemptedItemInfo(trackID, trackID, 0L, blocks, 0));
     Block[] blksMovementReport = new Block[1];
     blksMovementReport[0] = new Block(item);
-    bsmAttemptedItems.addReportedMovedBlocks(blksMovementReport);
+    bsmAttemptedItems.notifyMovementTriedBlocks(blksMovementReport);
     assertFalse(
         "Should not add in queue again if it is not there in"
             + " storageMovementAttemptedItems",


---------------------------------------------------------------------
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-12955: [SPS]: Move SPS classes to a separate package. Contributed by Rakesh R.

Posted by um...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/78420719/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/BlockStorageMovementNeeded.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/BlockStorageMovementNeeded.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/BlockStorageMovementNeeded.java
new file mode 100644
index 0000000..5635621
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/BlockStorageMovementNeeded.java
@@ -0,0 +1,572 @@
+/**
+ * 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.sps;
+
+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.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.FSDirectory;
+import org.apache.hadoop.hdfs.server.namenode.FSTreeTraverser;
+import org.apache.hadoop.hdfs.server.namenode.INode;
+import org.apache.hadoop.hdfs.server.namenode.Namesystem;
+import org.apache.hadoop.hdfs.server.namenode.sps.StoragePolicySatisfier.ItemInfo;
+import org.apache.hadoop.hdfs.server.namenode.FSTreeTraverser.TraverseInfo;
+import org.apache.hadoop.util.Daemon;
+import org.apache.hadoop.util.Time;
+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.
+ * 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 {
+
+  public static final Logger LOG =
+      LoggerFactory.getLogger(BlockStorageMovementNeeded.class);
+
+  private final Queue<ItemInfo> storageMovementNeeded =
+      new LinkedList<ItemInfo>();
+
+  /**
+   * 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, 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
+  private final Queue<Long> spsDirsToBeTraveresed = new LinkedList<Long>();
+
+  private final StoragePolicySatisfier sps;
+
+  private Daemon inodeIdCollector;
+
+  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;
+    this.sps = sps;
+    this.maxQueuedItem = queueLimit;
+  }
+
+  /**
+   * Add the candidate to tracking list for which storage movement
+   * expected if necessary.
+   *
+   * @param trackInfo
+   *          - track info for satisfy the policy
+   */
+  public synchronized void add(ItemInfo trackInfo) {
+    spsStatus.put(trackInfo.getStartId(),
+        new StoragePolicySatisfyPathStatusInfo(
+            StoragePolicySatisfyPathStatus.IN_PROGRESS));
+    storageMovementNeeded.add(trackInfo);
+  }
+
+  /**
+   * Add the itemInfo to tracking list for which storage movement
+   * expected if necessary.
+   * @param startId
+   *            - start id
+   * @param itemInfoList
+   *            - List of child in the directory
+   */
+  @VisibleForTesting
+  public synchronized void addAll(long startId,
+      List<ItemInfo> itemInfoList, boolean scanCompleted) {
+    storageMovementNeeded.addAll(itemInfoList);
+    DirPendingWorkInfo pendingWork = pendingWorkForDirectory.get(startId);
+    if (pendingWork == null) {
+      pendingWork = new DirPendingWorkInfo();
+      pendingWorkForDirectory.put(startId, pendingWork);
+    }
+    pendingWork.addPendingWorkCount(itemInfoList.size());
+    if (scanCompleted) {
+      pendingWork.markScanCompleted();
+    }
+  }
+
+  /**
+   * Gets the block collection id for which storage movements check necessary
+   * and make the movement if required.
+   *
+   * @return block collection ID
+   */
+  public synchronized ItemInfo get() {
+    return storageMovementNeeded.poll();
+  }
+
+  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.
+    synchronized (spsDirsToBeTraveresed) {
+      spsDirsToBeTraveresed.notify();
+    }
+  }
+
+  /**
+   * 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();
+    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,
+      boolean isSuccess) throws IOException {
+    if (trackInfo.isDir()) {
+      // 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(startId);
+        updateStatus(startId, isSuccess);
+      } else {
+        DirPendingWorkInfo pendingWork = pendingWorkForDirectory.get(startId);
+        if (pendingWork != null) {
+          pendingWork.decrementPendingWorkCount();
+          if (pendingWork.isDirWorkDone()) {
+            namesystem.removeXattr(startId, XATTR_SATISFY_STORAGE_POLICY);
+            pendingWorkForDirectory.remove(startId);
+            pendingWork.setFailure(!isSuccess);
+            updateStatus(startId, pendingWork.isPolicySatisfied());
+          }
+          pendingWork.setFailure(isSuccess);
+        }
+      }
+    } else {
+      // Remove xAttr if trackID doesn't exist in
+      // storageMovementAttemptedItems or file policy satisfied.
+      namesystem.removeXattr(trackInfo.getTrackId(),
+          XATTR_SATISFY_STORAGE_POLICY);
+      updateStatus(trackInfo.getStartId(), isSuccess);
+    }
+  }
+
+  public synchronized void clearQueue(long trackId) {
+    spsDirsToBeTraveresed.remove(trackId);
+    Iterator<ItemInfo> iterator = storageMovementNeeded.iterator();
+    while (iterator.hasNext()) {
+      ItemInfo next = iterator.next();
+      if (next.getStartId() == trackId) {
+        iterator.remove();
+      }
+    }
+    pendingWorkForDirectory.remove(trackId);
+  }
+
+  /**
+   * Mark inode status as SUCCESS in map.
+   */
+  private void updateStatus(long startId, boolean isSuccess){
+    StoragePolicySatisfyPathStatusInfo spsStatusInfo =
+        spsStatus.get(startId);
+    if (spsStatusInfo == null) {
+      spsStatusInfo = new StoragePolicySatisfyPathStatusInfo();
+      spsStatus.put(startId, spsStatusInfo);
+    }
+
+    if (isSuccess) {
+      spsStatusInfo.setSuccess();
+    } else {
+      spsStatusInfo.setFailure();
+    }
+  }
+
+  /**
+   * 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 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!.");
+      long lastStatusCleanTime = 0;
+      while (namesystem.isRunning() && sps.isRunning()) {
+        try {
+          if (!namesystem.isInSafeMode()) {
+            FSDirectory fsd = namesystem.getFSDirectory();
+            Long startINodeId = spsDirsToBeTraveresed.poll();
+            if (startINodeId == null) {
+              // Waiting for SPS path
+              synchronized (spsDirsToBeTraveresed) {
+                spsDirsToBeTraveresed.wait(5000);
+              }
+            } else {
+              INode startInode = fsd.getInode(startINodeId);
+              if (startInode != null) {
+                try {
+                  remainingCapacity = remainingCapacity();
+                  spsStatus.put(startINodeId,
+                      new StoragePolicySatisfyPathStatusInfo(
+                          StoragePolicySatisfyPathStatus.IN_PROGRESS));
+                  readLock();
+                  traverseDir(startInode.asDirectory(), startINodeId,
+                      HdfsFileStatus.EMPTY_NAME,
+                      new SPSTraverseInfo(startINodeId));
+                } finally {
+                  readUnlock();
+                }
+                // 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());
+                  updateStatus(startInode.getId(), true);
+                }
+              }
+            }
+            //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);
+        }
+      }
+    }
+
+    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
+    }
+
+    @Override
+    protected boolean processFileInode(INode inode, TraverseInfo traverseInfo)
+        throws IOException, InterruptedException {
+      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 {
+      // SPS work won't be scheduled if NN is in standby. So, skipping NN
+      // standby check.
+      return;
+    }
+
+    @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 {
+      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;
+    }
+  }
+
+  /**
+   * Info for directory recursive scan.
+   */
+  public static class DirPendingWorkInfo {
+
+    private int pendingWorkCount = 0;
+    private boolean fullyScanned = false;
+    private boolean success = true;
+
+    /**
+     * 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;
+    }
+
+    /**
+     * 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() {
+    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;
+    }
+  }
+
+  /**
+   * 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 void setFailure() {
+      this.status = StoragePolicySatisfyPathStatus.FAILURE;
+      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
+          || StoragePolicySatisfyPathStatus.FAILURE == 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/78420719/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/StoragePolicySatisfier.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/StoragePolicySatisfier.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/StoragePolicySatisfier.java
new file mode 100644
index 0000000..0d4bb19
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/StoragePolicySatisfier.java
@@ -0,0 +1,988 @@
+/**
+ * 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.sps;
+
+import static org.apache.hadoop.util.Time.monotonicNow;
+
+import java.io.IOException;
+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.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;
+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;
+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;
+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.namenode.Namesystem;
+import org.apache.hadoop.hdfs.server.protocol.BlockStorageMovementCommand.BlockMovingInfo;
+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;
+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
+ * 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;
+  private final BlockStorageMovementAttemptedItems storageMovementsMonitor;
+  private volatile boolean isRunning = false;
+  private int spsWorkMultiplier;
+  private long blockCount = 0L;
+  private int blockMovementMaxRetry;
+  private final Context ctxt;
+
+  /**
+   * An interface for analyzing and assigning the block storage movements to
+   * worker nodes.
+   */
+  // TODO: Now, added one API which is required for sps package. Will refine
+  // this interface via HDFS-12911.
+  public interface Context {
+    int getNumLiveDataNodes();
+  }
+
+  /**
+   * Represents the collective analysis status for all 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,
+      final BlockManager blkManager, Configuration conf, Context ctxt) {
+    this.namesystem = namesystem;
+    this.storageMovementNeeded = new BlockStorageMovementNeeded(namesystem,
+        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(
+            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);
+    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);
+    this.ctxt = ctxt;
+  }
+
+  /**
+   * Start storage policy satisfier demon thread. Also start block storage
+   * movements monitor for retry the attempts if needed.
+   */
+  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 "
+          + "start it.");
+    } else {
+      LOG.info("Starting StoragePolicySatisfier.");
+    }
+
+    // Ensure that all the previously submitted block movements(if any) have to
+    // be stopped in all datanodes.
+    addDropSPSWorkCommandsToAllDNs();
+    storageMovementNeeded.init();
+    storagePolicySatisfierThread = new Daemon(this);
+    storagePolicySatisfierThread.setName("StoragePolicySatisfier");
+    storagePolicySatisfierThread.start();
+    this.storageMovementsMonitor.start();
+  }
+
+  /**
+   * Disables storage policy satisfier by stopping its services.
+   *
+   * @param forceStop
+   *          true represents that it should stop SPS service by clearing all
+   *          pending SPS work
+   */
+  public synchronized void disable(boolean forceStop) {
+    isRunning = false;
+
+    if (storagePolicySatisfierThread == null) {
+      return;
+    }
+
+    storageMovementNeeded.close();
+
+    storagePolicySatisfierThread.interrupt();
+    this.storageMovementsMonitor.stop();
+    if (forceStop) {
+      storageMovementNeeded.clearQueuesWithNotification();
+      addDropSPSWorkCommandsToAllDNs();
+    } else {
+      LOG.info("Stopping StoragePolicySatisfier.");
+    }
+  }
+
+  /**
+   * Timed wait to stop storage policy satisfier daemon threads.
+   */
+  public synchronized void stopGracefully() {
+    if (isRunning) {
+      disable(true);
+    }
+    this.storageMovementsMonitor.stopGracefully();
+
+    if (storagePolicySatisfierThread == null) {
+      return;
+    }
+    try {
+      storagePolicySatisfierThread.join(3000);
+    } catch (InterruptedException ie) {
+    }
+  }
+
+  /**
+   * 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() {
+    String moverId = HdfsServerConstants.MOVER_ID_PATH.toString();
+    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() {
+    while (namesystem.isRunning() && isRunning) {
+      try {
+        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;
+            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, true);
+              } else {
+                status =
+                    analyseBlocksStorageMovementsAndAssignToDN(
+                        blockCollection);
+              }
+            } finally {
+              namesystem.readUnlock();
+            }
+            if (blockCollection != null) {
+              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 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, itemInfo.getRetryCount()));
+                break;
+              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.");
+                }
+                itemInfo.retryCount++;
+                this.storageMovementNeeded.add(itemInfo);
+                break;
+              case FEW_LOW_REDUNDANCY_BLOCKS:
+                if (LOG.isDebugEnabled()) {
+                  LOG.debug("Adding trackID " + trackId
+                      + " back to retry queue as some of the blocks"
+                      + " are low redundant.");
+                }
+                this.storageMovementNeeded.add(itemInfo);
+                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.");
+                storageMovementNeeded.removeItemTrackInfo(itemInfo, true);
+                break;
+              }
+            }
+          }
+        }
+        int numLiveDn = ctxt.getNumLiveDataNodes();
+        if (storageMovementNeeded.size() == 0
+            || blockCount > (numLiveDn * spsWorkMultiplier)) {
+          Thread.sleep(3000);
+          blockCount = 0L;
+        }
+      } catch (Throwable t) {
+        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);
+            }
+            return;
+          }
+        }
+      }
+    }
+    LOG.error("StoragePolicySatisfier thread received runtime exception. "
+        + "Stopping Storage policy satisfier work", t);
+    return;
+  }
+
+  private BlocksMovingAnalysis analyseBlocksStorageMovementsAndAssignToDN(
+      BlockCollection blockCollection) {
+    BlocksMovingAnalysis.Status status =
+        BlocksMovingAnalysis.Status.BLOCKS_ALREADY_SATISFIED;
+    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
+      LOG.info("BlockCollectionID: {} file is under construction. So, postpone"
+          + " this to the next retry iteration", blockCollection.getId());
+      return new BlocksMovingAnalysis(
+          BlocksMovingAnalysis.Status.ANALYSIS_SKIPPED_FOR_RETRY,
+          new ArrayList<>());
+    }
+
+    BlockInfo[] blocks = blockCollection.getBlocks();
+    if (blocks.length == 0) {
+      LOG.info("BlockCollectionID: {} file is not having any blocks."
+          + " So, skipping the analysis.", blockCollection.getId());
+      return new BlocksMovingAnalysis(
+          BlocksMovingAnalysis.Status.BLOCKS_TARGET_PAIRING_SKIPPED,
+          new ArrayList<>());
+    }
+    List<BlockMovingInfo> blockMovingInfos = new ArrayList<BlockMovingInfo>();
+
+    for (int i = 0; i < blocks.length; i++) {
+      BlockInfo blockInfo = blocks[i];
+      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 new BlocksMovingAnalysis(
+              BlocksMovingAnalysis.Status.BLOCKS_TARGET_PAIRING_SKIPPED,
+              new ArrayList<>());
+        }
+      } else {
+        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)) {
+        boolean blocksPaired = computeBlockMovingInfos(blockMovingInfos,
+            blockInfo, expectedStorageTypes, existing, storages);
+        if (blocksPaired) {
+          status = BlocksMovingAnalysis.Status.BLOCKS_TARGETS_PAIRED;
+        } else {
+          // 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 = BlocksMovingAnalysis.Status.FEW_LOW_REDUNDANCY_BLOCKS;
+        }
+      }
+    }
+
+    List<Block> assignedBlockIds = new ArrayList<Block>();
+    for (BlockMovingInfo blkMovingInfo : blockMovingInfos) {
+      // 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++;
+      }
+    }
+    return new BlocksMovingAnalysis(status, assignedBlockIds);
+  }
+
+  /**
+   * 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
+   * @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> existing,
+      DatanodeStorageInfo[] storages) {
+    boolean foundMatchingTargetNodesForBlock = true;
+    if (!DFSUtil.removeOverlapBetweenStorageTypes(expectedStorageTypes,
+        existing, true)) {
+      List<StorageTypeNodePair> sourceWithStorageMap =
+          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 = 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);
+
+      foundMatchingTargetNodesForBlock |= findSourceAndTargetToMove(
+          blockMovingInfos, blockInfo, sourceWithStorageMap,
+          expectedStorageTypes, locsForExpectedStorageTypes);
+    }
+    return foundMatchingTargetNodesForBlock;
+  }
+
+  /**
+   * 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 sourceWithStorageList
+   *          - Source Datanode with storages list
+   * @param expected
+   *          - Expecting storages to move
+   * @param locsForExpectedStorageTypes
+   *          - Available DNs for expected storage types
+   * @return false if some of the block locations failed to find target node to
+   *         satisfy the storage policy
+   */
+  private boolean findSourceAndTargetToMove(
+      List<BlockMovingInfo> blockMovingInfos, BlockInfo blockInfo,
+      List<StorageTypeNodePair> sourceWithStorageList,
+      List<StorageType> expected,
+      StorageTypeNodeMap locsForExpectedStorageTypes) {
+    boolean foundMatchingTargetNodesForBlock = true;
+    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);
+
+      // 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) {
+          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?
+        }
+      }
+      // To avoid choosing this excludeNodes as targets later
+      excludeNodes.add(existingTypeNodePair.dn);
+    }
+
+    // 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 (checkIfAlreadyChosen(blockMovingInfos, existingTypeNodePair.dn)) {
+        continue;
+      }
+      if (chosenTarget == null && blockManager.getDatanodeManager()
+          .getNetworkTopology().isNodeGroupAware()) {
+        chosenTarget = chooseTarget(blockInfo, existingTypeNodePair.dn,
+            expected, Matcher.SAME_NODE_GROUP, locsForExpectedStorageTypes,
+            excludeNodes);
+      }
+
+      // Then, match nodes on the same rack
+      if (chosenTarget == null) {
+        chosenTarget =
+            chooseTarget(blockInfo, existingTypeNodePair.dn, expected,
+                Matcher.SAME_RACK, locsForExpectedStorageTypes, excludeNodes);
+      }
+
+      if (chosenTarget == null) {
+        chosenTarget =
+            chooseTarget(blockInfo, existingTypeNodePair.dn, expected,
+                Matcher.ANY_OTHER, locsForExpectedStorageTypes, excludeNodes);
+      }
+      if (null != chosenTarget) {
+        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?
+      } else {
+        LOG.warn(
+            "Failed to choose target datanode for the required"
+                + " storage types {}, block:{}, existing storage type:{}",
+            expected, blockInfo, existingTypeNodePair.storageType);
+      }
+    }
+
+    if (expected.size() > 0) {
+      foundMatchingTargetNodesForBlock = false;
+    }
+
+    return foundMatchingTargetNodesForBlock;
+  }
+
+  private boolean checkIfAlreadyChosen(List<BlockMovingInfo> blockMovingInfos,
+      DatanodeDescriptor dn) {
+    for (BlockMovingInfo blockMovingInfo : blockMovingInfos) {
+      if (blockMovingInfo.getSource().equals(dn)) {
+        return true;
+      }
+    }
+    return false;
+  }
+
+  private void buildContinuousBlockMovingInfos(BlockInfo blockInfo,
+      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, sourceNode,
+        targetNode, sourceStorageType, targetStorageType);
+    blkMovingInfos.add(blkMovingInfo);
+  }
+
+  private void buildStripedBlockMovingInfos(BlockInfo blockInfo,
+      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
+    // 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();
+        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);
+        }
+      }
+    }
+  }
+
+  /**
+   * Choose the target storage within same datanode if possible.
+   *
+   * @param block
+   *          - block info
+   * @param source
+   *          - source datanode
+   * @param targetTypes
+   *          - list of target storage types
+   */
+  private StorageTypeNodePair chooseTargetTypeInSameNode(Block block,
+      DatanodeDescriptor source, List<StorageType> targetTypes) {
+    for (StorageType t : targetTypes) {
+      DatanodeStorageInfo chooseStorage4Block =
+          source.chooseStorage4Block(t, block.getNumBytes());
+      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> excludeNodes) {
+    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 (!excludeNodes.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 {
+    private StorageType storageType = null;
+    private DatanodeDescriptor dn = null;
+
+    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 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);
+
+    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);
+    }
+  }
+
+  /**
+   * Receives set of storage movement attempt finished blocks report.
+   *
+   * @param moveAttemptFinishedBlks
+   *          set of storage movement attempt finished blocks.
+   */
+  public void handleStorageMovementAttemptFinishedBlks(
+      BlocksStorageMoveAttemptFinished moveAttemptFinishedBlks) {
+    if (moveAttemptFinishedBlks.getBlocks().length <= 0) {
+      return;
+    }
+    storageMovementsMonitor
+        .addReportedMovedBlocks(moveAttemptFinishedBlks.getBlocks());
+  }
+
+  @VisibleForTesting
+  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();
+  }
+
+  /**
+   * Set file inode in queue for which storage movement needed for its blocks.
+   *
+   * @param inodeId
+   *          - file inode/blockcollection id.
+   */
+  public void satisfyStoragePolicy(Long inodeId) {
+    //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 "
+          + "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);
+  }
+
+  /**
+   * ItemInfo is a file info object for which need to satisfy the
+   * policy.
+   */
+  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;
+    }
+
+    /**
+     * Return the start inode id of the current track Id.
+     */
+    public long getStartId() {
+      return startId;
+    }
+
+    /**
+     * 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 (startId != trackId);
+    }
+
+    /**
+     * Get the attempted retry count of the block for satisfy the policy.
+     */
+    public int getRetryCount() {
+      return retryCount;
+    }
+  }
+
+  /**
+   * 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, int retryCount) {
+      super(rootId, trackId, retryCount);
+      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;
+    }
+
+  }
+
+  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/78420719/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/package-info.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/package-info.java
new file mode 100644
index 0000000..d1d69fb
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/package-info.java
@@ -0,0 +1,28 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+/**
+ * This package provides a mechanism for satisfying the storage policy of a
+ * path.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+package org.apache.hadoop.hdfs.server.namenode.sps;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/78420719/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
deleted file mode 100644
index d4ccb3e..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestBlockStorageMovementAttemptedItems.java
+++ /dev/null
@@ -1,196 +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 static org.apache.hadoop.util.Time.monotonicNow;
-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.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
- * processed them correctly or not.
- */
-public class TestBlockStorageMovementAttemptedItems {
-
-  private BlockStorageMovementAttemptedItems bsmAttemptedItems = null;
-  private BlockStorageMovementNeeded unsatisfiedStorageMovementFiles = null;
-  private final int selfRetryTimeout = 500;
-
-  @Before
-  public void setup() throws Exception {
-    unsatisfiedStorageMovementFiles = new BlockStorageMovementNeeded(
-        Mockito.mock(Namesystem.class),
-        Mockito.mock(StoragePolicySatisfier.class), 100);
-    bsmAttemptedItems = new BlockStorageMovementAttemptedItems(100,
-        selfRetryTimeout, unsatisfiedStorageMovementFiles);
-  }
-
-  @After
-  public void teardown() {
-    if (bsmAttemptedItems != null) {
-      bsmAttemptedItems.stop();
-      bsmAttemptedItems.stopGracefully();
-    }
-  }
-
-  private boolean checkItemMovedForRetry(Long item, long retryTimeout)
-      throws InterruptedException {
-    long stopTime = monotonicNow() + (retryTimeout * 2);
-    boolean isItemFound = false;
-    while (monotonicNow() < (stopTime)) {
-      ItemInfo ele = null;
-      while ((ele = unsatisfiedStorageMovementFiles.get()) != null) {
-        if (item == ele.getTrackId()) {
-          isItemFound = true;
-          break;
-        }
-      }
-      if (!isItemFound) {
-        Thread.sleep(100);
-      } else {
-        break;
-      }
-    }
-    return isItemFound;
-  }
-
-  /**
-   * Verify that moved blocks reporting should queued up the block info.
-   */
-  @Test(timeout = 30000)
-  public void testAddReportedMoveAttemptFinishedBlocks() throws Exception {
-    bsmAttemptedItems.start(); // start block movement result monitor thread
-    Long item = new Long(1234);
-    List<Block> blocks = new ArrayList<Block>();
-    blocks.add(new Block(item));
-    bsmAttemptedItems.add(new AttemptedItemInfo(0L, 0L, 0L, blocks, 0));
-    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 testNoBlockMovementAttemptFinishedReportAdded() throws Exception {
-    bsmAttemptedItems.start(); // start block movement report monitor thread
-    Long item = new Long(1234);
-    List<Block> blocks = new ArrayList<>();
-    blocks.add(new Block(item));
-    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,
-        bsmAttemptedItems.getAttemptedItemsCount());
-  }
-
-  /**
-   * 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);
-    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, 0));
-    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(trackID, 5000));
-    assertEquals("Failed to remove from the attempted list", 0,
-        bsmAttemptedItems.getAttemptedItemsCount());
-  }
-
-  /**
-   * Partial block movement. Here, first occurrence is
-   * #blocksStorageMovementUnReportedItemsCheck() and then
-   * #blockStorageMovementReportedItemsCheck().
-   */
-  @Test(timeout = 30000)
-  public void testPartialBlockMovementShouldBeRetried2() throws Exception {
-    Long item = new Long(1234);
-    Long trackID = 0L;
-    List<Block> blocks = new ArrayList<>();
-    blocks.add(new Block(item));
-    bsmAttemptedItems
-        .add(new AttemptedItemInfo(trackID, trackID, 0L, blocks, 0));
-    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.blockStorageMovementReportedItemsCheck();
-
-    assertTrue("Failed to add to the retry list",
-        checkItemMovedForRetry(trackID, 5000));
-    assertEquals("Failed to remove from the attempted list", 0,
-        bsmAttemptedItems.getAttemptedItemsCount());
-  }
-
-  /**
-   * 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);
-    Long trackID = 0L;
-    List<Block> blocks = new ArrayList<>();
-    blocks.add(new Block(item));
-    bsmAttemptedItems
-        .add(new AttemptedItemInfo(trackID, trackID, 0L, blocks, 0));
-    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(trackID, 5000));
-    assertEquals("Failed to remove from the attempted list", 1,
-        bsmAttemptedItems.getAttemptedItemsCount());
-  }
-}


---------------------------------------------------------------------
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-12790: [SPS]: Rebasing HDFS-10285 branch after HDFS-10467, HDFS-12599 and HDFS-11968 commits. Contributed by Rakesh R.

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

Branch: refs/heads/HDFS-10285
Commit: 9b83f94f35eb8cd20d9f3e0cbbeecbbbbb6ffb5b
Parents: 68017e3
Author: Rakesh Radhakrishnan <ra...@apache.org>
Authored: Fri Nov 10 10:06:43 2017 +0530
Committer: Uma Maheswara Rao Gangumalla <um...@apache.org>
Committed: Sun Aug 12 03:06:02 2018 -0700

----------------------------------------------------------------------
 .../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/9b83f94f/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcServer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcServer.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcServer.java
index 027db8a..c5458f0 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcServer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcServer.java
@@ -87,6 +87,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.HdfsLocatedFileStatus;
 import org.apache.hadoop.hdfs.protocol.LastBlockWithStatus;
@@ -2490,4 +2491,22 @@ public class RouterRpcServer extends AbstractService
   public FederationRPCMetrics getRPCMetrics() {
     return this.rpcMonitor.getRPCMetrics();
   }
+
+  @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/9b83f94f/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/9b83f94f/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/9b83f94f/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/9b83f94f/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


[35/50] [abbrv] hadoop git commit: HDFS-13057: [SPS]: Revisit configurations to make SPS service modes internal/external/none. Contributed by Rakesh R.

Posted by um...@apache.org.
HDFS-13057: [SPS]: Revisit configurations to make SPS service modes internal/external/none. 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/3b83110d
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/3b83110d
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/3b83110d

Branch: refs/heads/HDFS-10285
Commit: 3b83110d5ed582b9f913ecf3f62ce410535f8fca
Parents: b0cb8d9
Author: Uma Maheswara Rao G <um...@intel.com>
Authored: Fri Jan 26 08:57:29 2018 -0800
Committer: Uma Maheswara Rao Gangumalla <um...@apache.org>
Committed: Sun Aug 12 03:06:04 2018 -0700

----------------------------------------------------------------------
 .../hadoop/hdfs/protocol/HdfsConstants.java     |  39 ++++
 .../org/apache/hadoop/hdfs/DFSConfigKeys.java   |   9 +-
 .../server/blockmanagement/BlockManager.java    | 105 +++++++---
 .../hdfs/server/namenode/FSNamesystem.java      |   6 +-
 .../hdfs/server/namenode/FSTreeTraverser.java   |   2 +-
 .../hadoop/hdfs/server/namenode/NameNode.java   |  34 ++--
 .../sps/BlockStorageMovementNeeded.java         |   2 +-
 .../namenode/sps/IntraSPSNameNodeContext.java   |   3 +
 .../hdfs/server/namenode/sps/SPSService.java    |   4 +-
 .../namenode/sps/StoragePolicySatisfier.java    |  17 +-
 .../server/sps/ExternalSPSFileIDCollector.java  |  32 ++-
 .../hadoop/hdfs/tools/StoragePolicyAdmin.java   |  16 +-
 .../src/main/resources/hdfs-default.xml         |  11 +-
 .../src/site/markdown/ArchivalStorage.md        |  17 +-
 .../TestStoragePolicySatisfyWorker.java         |   5 +-
 .../hadoop/hdfs/server/mover/TestMover.java     |  45 +++--
 .../hdfs/server/mover/TestStorageMover.java     |   4 +-
 .../namenode/TestNameNodeReconfigure.java       | 105 +++++-----
 .../TestPersistentStoragePolicySatisfier.java   |   9 +-
 .../TestStoragePolicySatisfierWithHA.java       |  12 +-
 .../sps/TestStoragePolicySatisfier.java         | 202 +++++++------------
 ...stStoragePolicySatisfierWithStripedFile.java |  17 +-
 .../sps/TestExternalStoragePolicySatisfier.java | 112 +++++++---
 .../hdfs/tools/TestStoragePolicyCommands.java   |   5 +-
 .../TestStoragePolicySatisfyAdminCommands.java  |  14 +-
 25 files changed, 500 insertions(+), 327 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/3b83110d/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 aabcdd9..ab48dcd 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
@@ -129,6 +129,45 @@ public final class HdfsConstants {
   }
 
   /**
+   * Storage policy satisfier service modes.
+   */
+  public enum StoragePolicySatisfierMode {
+
+    /**
+     * This mode represents that SPS service is running inside Namenode and can
+     * accept any SPS call request.
+     */
+    INTERNAL,
+
+    /**
+     * This mode represents that SPS service is running outside Namenode as an
+     * external service and can accept any SPS call request.
+     */
+    EXTERNAL,
+
+    /**
+     * This mode represents that SPS service is disabled and cannot accept any
+     * SPS call request.
+     */
+    NONE;
+
+    private static final Map<String, StoragePolicySatisfierMode> MAP =
+        new HashMap<>();
+
+    static {
+      for (StoragePolicySatisfierMode a : values()) {
+        MAP.put(a.name(), a);
+      }
+    }
+
+    /** Convert the given String to a StoragePolicySatisfierMode. */
+    public static StoragePolicySatisfierMode fromString(String s) {
+      return MAP.get(StringUtils.toUpperCase(s));
+    }
+  }
+
+
+  /**
    * Storage policy satisfy path status.
    */
   public enum StoragePolicySatisfyPathStatus {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3b83110d/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 f1a59d3..bf29d14 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
@@ -23,6 +23,7 @@ import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
 import org.apache.hadoop.hdfs.net.DFSNetworkTopology;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants.StoragePolicySatisfierMode;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementPolicyDefault;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementPolicyRackFaultTolerant;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.RamDiskReplicaLruTracker;
@@ -614,10 +615,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_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_MODE_KEY =
+      "dfs.storage.policy.satisfier.mode";
+  public static final String DFS_STORAGE_POLICY_SATISFIER_MODE_DEFAULT =
+      StoragePolicySatisfierMode.NONE.toString();
   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 =

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3b83110d/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 d12cb01..9ef1045 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
@@ -69,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.StoragePolicySatisfierMode;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.StoragePolicySatisfyPathStatus;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
@@ -433,8 +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 SPSPathIds spsPaths;
+  private StoragePolicySatisfierMode spsMode;
+  private SPSPathIds spsPaths;
 
   /** Minimum live replicas needed for the datanode to be transitioned
    * from ENTERING_MAINTENANCE to IN_MAINTENANCE.
@@ -478,12 +479,13 @@ public class BlockManager implements BlockStatsMXBean {
     storagePolicyEnabled =
         conf.getBoolean(DFSConfigKeys.DFS_STORAGE_POLICY_ENABLED_KEY,
             DFSConfigKeys.DFS_STORAGE_POLICY_ENABLED_DEFAULT);
-    spsEnabled =
-        conf.getBoolean(
-            DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_ENABLED_KEY,
-            DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_ENABLED_DEFAULT);
-    sps = new StoragePolicySatisfier(conf);
+    String spsModeVal =
+        conf.get(
+            DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_MODE_KEY,
+            DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_MODE_DEFAULT);
+    spsMode = StoragePolicySatisfierMode.fromString(spsModeVal);
     spsPaths = new SPSPathIds();
+    sps = new StoragePolicySatisfier(conf);
     blockTokenSecretManager = createBlockTokenSecretManager(conf);
 
     providedStorageMap = new ProvidedStorageMap(namesystem, this, conf);
@@ -5031,18 +5033,22 @@ public class BlockManager implements BlockStatsMXBean {
    * Start storage policy satisfier service.
    */
   public void startSPS() {
-    if (!(storagePolicyEnabled && spsEnabled)) {
+    if (!(storagePolicyEnabled && spsMode != StoragePolicySatisfierMode.NONE)) {
       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);
+          DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_MODE_KEY, spsMode);
       return;
     } else if (sps.isRunning()) {
-      LOG.info("Storage policy satisfier is already running.");
+      LOG.info("Storage policy satisfier is already running"
+          + " as internal service.");
       return;
     }
-    sps.start(false);
+    // starting internal SPS service
+    if (spsMode == StoragePolicySatisfierMode.INTERNAL) {
+      sps.start(false, spsMode);
+    }
   }
 
   /**
@@ -5053,11 +5059,13 @@ public class BlockManager implements BlockStatsMXBean {
    *          pending SPS work
    */
   public void stopSPS(boolean forceStop) {
-    if (!(storagePolicyEnabled && spsEnabled)) {
+    if (!(storagePolicyEnabled
+        && (spsMode != StoragePolicySatisfierMode.NONE))) {
       LOG.info("Storage policy satisfier is not enabled.");
       return;
     } else if (!sps.isRunning()) {
-      LOG.info("Storage policy satisfier is already stopped.");
+      removeAllSPSPathIds();
+      LOG.info("Storage policy satisfier is not running.");
       return;
     }
 
@@ -5067,39 +5075,75 @@ public class BlockManager implements BlockStatsMXBean {
   /**
    * Enable storage policy satisfier by starting its service.
    */
-  public void enableSPS() {
+  public void enableInternalSPS() {
     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.");
+      LOG.info("Storage policy satisfier is already running as SPS mode:{}.",
+          spsMode);
       return;
     }
-    sps.start(true);
+    updateSPSMode(StoragePolicySatisfierMode.INTERNAL);
+    sps.start(true, spsMode);
   }
 
   /**
-   * Disable the storage policy satisfier by stopping its services.
+   * Enable storage policy satisfier by starting its service.
    */
-  public void disableSPS() {
-    spsEnabled = false;
-    if (!sps.isRunning()) {
-      LOG.info("Storage policy satisfier is already stopped.");
+  public void enableExternalSPS() {
+    if (!storagePolicyEnabled){
+      LOG.info("Failed to start StoragePolicySatisfier as {} set to {}.",
+          DFSConfigKeys.DFS_STORAGE_POLICY_ENABLED_KEY, storagePolicyEnabled);
+      return;
+    }
+    if (spsMode == StoragePolicySatisfierMode.EXTERNAL) {
+      LOG.info("Storage policy satisfier is already enabled as SPS mode:{}.",
+          spsMode);
       return;
     }
+    updateSPSMode(StoragePolicySatisfierMode.EXTERNAL);
+    sps.stopGracefully();
+  }
 
-    LOG.info("Stopping StoragePolicySatisfier, as admin requested to "
-        + "stop it.");
-    sps.disable(true);
+  private void updateSPSMode(StoragePolicySatisfierMode newSpsMode) {
+    LOG.debug("Updating SPS service status, current mode:{}, new mode:{}",
+        spsMode, newSpsMode);
+    spsMode = newSpsMode;
+  }
+
+  /**
+   * Disable the storage policy satisfier by stopping its services.
+   */
+  public void disableSPS() {
+    switch (spsMode) {
+    case NONE:
+      break;
+    case INTERNAL:
+    case EXTERNAL:
+      if (!sps.isRunning()) {
+        LOG.info("Storage policy satisfier is already stopped.");
+      } else {
+        LOG.info("Stopping StoragePolicySatisfier mode {}, as admin "
+            + "requested to stop it.", spsMode);
+        sps.disable(true);
+      }
+      removeAllSPSPathIds();
+      break;
+    default:
+      // nothing
+      break;
+    }
+    updateSPSMode(StoragePolicySatisfierMode.NONE);
   }
 
   /**
    * Timed wait to stop storage policy satisfier daemon threads.
    */
   public void stopSPSGracefully() {
+    removeAllSPSPathIds();
     sps.stopGracefully();
   }
   /**
@@ -5156,10 +5200,17 @@ public class BlockManager implements BlockStatsMXBean {
   }
 
   /**
-   * @return true if sps enabled.
+   * @return true if sps is running as an internal service or external service.
    */
   public boolean isSPSEnabled() {
-    return spsEnabled;
+    return spsMode == StoragePolicySatisfierMode.INTERNAL
+        || spsMode == StoragePolicySatisfierMode.EXTERNAL;
   }
 
+  /**
+   * @return sps service mode.
+   */
+  public StoragePolicySatisfierMode getSPSMode() {
+    return spsMode;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3b83110d/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 37322e7..c93791b 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
@@ -209,6 +209,7 @@ import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.ReencryptAction;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants.StoragePolicySatisfierMode;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.LastBlockWithStatus;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
@@ -2265,8 +2266,9 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
             DFS_STORAGE_POLICY_ENABLED_KEY));
       }
 
-      if (blockManager.getStoragePolicySatisfier() == null
-          || !blockManager.getStoragePolicySatisfier().isRunning()) {
+      if (!blockManager.isSPSEnabled()
+          || (blockManager.getSPSMode() == StoragePolicySatisfierMode.INTERNAL
+              && !blockManager.getStoragePolicySatisfier().isRunning())) {
         throw new UnsupportedActionException(
             "Cannot request to satisfy storage policy "
                 + "when storage policy satisfier feature has been disabled"

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3b83110d/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
index ff77029..a7d633f 100644
--- 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
@@ -289,7 +289,7 @@ public abstract class FSTreeTraverser {
   /**
    * Check whether current batch can be submitted for the processing.
    *
-   * @return true if batch size meets meet the condition, otherwise false.
+   * @return true if batch size meets the condition, otherwise false.
    */
   protected abstract boolean shouldSubmitCurrentBatch();
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3b83110d/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 2081b6e..a7d8298 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
@@ -45,6 +45,7 @@ import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
 import org.apache.hadoop.hdfs.protocol.ClientProtocol;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants.StoragePolicySatisfierMode;
 import org.apache.hadoop.hdfs.server.aliasmap.InMemoryAliasMap;
 import org.apache.hadoop.hdfs.server.aliasmap.InMemoryLevelDBAliasMapServer;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeManager;
@@ -160,7 +161,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_ENABLED_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_MODE_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;
@@ -295,7 +296,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_ENABLED_KEY));
+          DFS_STORAGE_POLICY_SATISFIER_MODE_KEY));
 
   private static final String USAGE = "Usage: hdfs namenode ["
       + StartupOption.BACKUP.getName() + "] | \n\t["
@@ -2041,7 +2042,7 @@ 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_ENABLED_KEY)) {
+    } else if (property.equals(DFS_STORAGE_POLICY_SATISFIER_MODE_KEY)) {
       return reconfigureSPSEnabled(newVal, property);
     } else {
       throw new ReconfigurationException(property, newVal, getConf().get(
@@ -2128,13 +2129,13 @@ public class NameNode extends ReconfigurableBase implements
 
   String reconfigureSPSEnabled(String newVal, String property)
       throws ReconfigurationException {
-    if (newVal == null || !(newVal.equalsIgnoreCase(Boolean.TRUE.toString())
-        || newVal.equalsIgnoreCase(Boolean.FALSE.toString()))) {
+    if (newVal == null
+        || StoragePolicySatisfierMode.fromString(newVal) == null) {
       throw new ReconfigurationException(property, newVal,
           getConf().get(property),
           new HadoopIllegalArgumentException(
-              "For enabling or disabling storage policy satisfier, "
-                  + "we must pass true/false only"));
+              "For enabling or disabling storage policy satisfier, we must "
+                  + "pass either none/internal/external string value only"));
     }
 
     if (!isActiveState()) {
@@ -2143,12 +2144,21 @@ public class NameNode extends ReconfigurableBase implements
           "Enabling or disabling storage policy satisfier service on "
               + state + " NameNode is not allowed"));
     }
-
-    boolean enableSPS = Boolean.parseBoolean(newVal);
-    if (enableSPS) {
-      namesystem.getBlockManager().enableSPS();
-    } else {
+    StoragePolicySatisfierMode mode = StoragePolicySatisfierMode
+        .fromString(newVal);
+    switch(mode){
+    case NONE:
       namesystem.getBlockManager().disableSPS();
+      break;
+    case INTERNAL:
+      namesystem.getBlockManager().enableInternalSPS();
+      break;
+    case EXTERNAL:
+      namesystem.getBlockManager().enableExternalSPS();
+      break;
+    default:
+      // nothing
+      break;
     }
     return newVal;
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3b83110d/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/BlockStorageMovementNeeded.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/BlockStorageMovementNeeded.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/BlockStorageMovementNeeded.java
index b141502..39c50a7 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/BlockStorageMovementNeeded.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/BlockStorageMovementNeeded.java
@@ -257,7 +257,7 @@ public class BlockStorageMovementNeeded {
     // File's directly added to storageMovementNeeded, So try to remove
     // xAttr for file
     ItemInfo itemInfo;
-    while ((itemInfo = storageMovementNeeded.poll()) != null) {
+    while ((itemInfo = get()) != null) {
       try {
         // Remove xAttr for file
         if (!itemInfo.isDir()) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3b83110d/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/IntraSPSNameNodeContext.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/IntraSPSNameNodeContext.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/IntraSPSNameNodeContext.java
index aed684a..c658812 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/IntraSPSNameNodeContext.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/IntraSPSNameNodeContext.java
@@ -102,6 +102,9 @@ public class IntraSPSNameNodeContext implements Context {
     namesystem.readLock();
     try {
       BlockCollection bc = namesystem.getBlockCollection(inodeID);
+      if (bc == null) {
+        return false;
+      }
       return blockManager.hasLowRedundancyBlocks(bc);
     } finally {
       namesystem.readUnlock();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3b83110d/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/SPSService.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/SPSService.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/SPSService.java
index ecc6ceb..ceec3f3 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/SPSService.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/SPSService.java
@@ -22,6 +22,7 @@ import java.util.List;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants.StoragePolicySatisfierMode;
 import org.apache.hadoop.hdfs.server.protocol.BlocksStorageMoveAttemptFinished;
 
 /**
@@ -55,8 +56,9 @@ public interface SPSService {
    * @param reconfigStart
    *          - to indicate whether the SPS startup requested from
    *          reconfiguration service
+   * @param spsMode sps service mode
    */
-  void start(boolean reconfigStart);
+  void start(boolean reconfigStart, StoragePolicySatisfierMode spsMode);
 
   /**
    * Stops the SPS service gracefully. Timed wait to stop storage policy

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3b83110d/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/StoragePolicySatisfier.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/StoragePolicySatisfier.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/StoragePolicySatisfier.java
index 9ba8af7..33ad6f4 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/StoragePolicySatisfier.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/StoragePolicySatisfier.java
@@ -39,6 +39,7 @@ import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfoWithStorage;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants.StoragePolicySatisfierMode;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.StoragePolicySatisfyPathStatus;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.HdfsLocatedFileStatus;
@@ -82,6 +83,8 @@ public class StoragePolicySatisfier implements SPSService, Runnable {
   private BlockStorageMovementNeeded storageMovementNeeded;
   private BlockStorageMovementAttemptedItems storageMovementsMonitor;
   private volatile boolean isRunning = false;
+  private volatile StoragePolicySatisfierMode spsMode =
+      StoragePolicySatisfierMode.NONE;
   private int spsWorkMultiplier;
   private long blockCount = 0L;
   private int blockMovementMaxRetry;
@@ -152,9 +155,17 @@ public class StoragePolicySatisfier implements SPSService, Runnable {
    * movements monitor for retry the attempts if needed.
    */
   @Override
-  public synchronized void start(boolean reconfigStart) {
+  public synchronized void start(boolean reconfigStart,
+      StoragePolicySatisfierMode serviceMode) {
+    if (serviceMode == StoragePolicySatisfierMode.NONE) {
+      LOG.error("Can't start StoragePolicySatisfier for the given mode:{}",
+          serviceMode);
+      return;
+    }
     isRunning = true;
-    if (ctxt.isMoverRunning()) {
+    this.spsMode = serviceMode;
+    if (spsMode == StoragePolicySatisfierMode.INTERNAL
+        && ctxt.isMoverRunning()) {
       isRunning = false;
       LOG.error(
           "Stopping StoragePolicySatisfier thread " + "as Mover ID file "
@@ -928,7 +939,7 @@ public class StoragePolicySatisfier implements SPSService, Runnable {
 
   @Override
   public void addFileIdToProcess(ItemInfo trackInfo, boolean scanCompleted) {
-    storageMovementNeeded.add(trackInfo);
+    storageMovementNeeded.add(trackInfo, scanCompleted);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3b83110d/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/sps/ExternalSPSFileIDCollector.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/sps/ExternalSPSFileIDCollector.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/sps/ExternalSPSFileIDCollector.java
index 597a7d3..964ee8c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/sps/ExternalSPSFileIDCollector.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/sps/ExternalSPSFileIDCollector.java
@@ -18,6 +18,7 @@ package org.apache.hadoop.hdfs.server.sps;
  */
 
 import java.io.IOException;
+import java.util.ArrayList;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
@@ -48,8 +49,7 @@ public class ExternalSPSFileIDCollector implements FileIdCollector {
   private SPSService service;
   private int maxQueueLimitToScan;
 
-  public ExternalSPSFileIDCollector(Context cxt, SPSService service,
-      int batchSize) {
+  public ExternalSPSFileIDCollector(Context cxt, SPSService service) {
     this.cxt = cxt;
     this.service = service;
     this.maxQueueLimitToScan = service.getConf().getInt(
@@ -74,7 +74,8 @@ public class ExternalSPSFileIDCollector implements FileIdCollector {
    * Recursively scan the given path and add the file info to SPS service for
    * processing.
    */
-  private void processPath(long startID, String fullPath) {
+  private long processPath(long startID, String fullPath) {
+    long pendingWorkCount = 0; // to be satisfied file counter
     for (byte[] lastReturnedName = HdfsFileStatus.EMPTY_NAME;;) {
       final DirectoryListing children;
       try {
@@ -82,14 +83,14 @@ public class ExternalSPSFileIDCollector implements FileIdCollector {
       } catch (IOException e) {
         LOG.warn("Failed to list directory " + fullPath
             + ". Ignore the directory and continue.", e);
-        return;
+        return pendingWorkCount;
       }
       if (children == null) {
         if (LOG.isDebugEnabled()) {
           LOG.debug("The scanning start dir/sub dir " + fullPath
               + " does not have childrens.");
         }
-        return;
+        return pendingWorkCount;
       }
 
       for (HdfsFileStatus child : children.getPartialListing()) {
@@ -97,13 +98,14 @@ public class ExternalSPSFileIDCollector implements FileIdCollector {
           service.addFileIdToProcess(new ItemInfo(startID, child.getFileId()),
               false);
           checkProcessingQueuesFree();
+          pendingWorkCount++; // increment to be satisfied file count
         } else {
           String fullPathStr = child.getFullName(fullPath);
           if (child.isDirectory()) {
             if (!fullPathStr.endsWith(Path.SEPARATOR)) {
               fullPathStr = fullPathStr + Path.SEPARATOR;
             }
-            processPath(startID, fullPathStr);
+            pendingWorkCount += processPath(startID, fullPathStr);
           }
         }
       }
@@ -111,7 +113,7 @@ public class ExternalSPSFileIDCollector implements FileIdCollector {
       if (children.hasMore()) {
         lastReturnedName = children.getLastName();
       } else {
-        return;
+        return pendingWorkCount;
       }
     }
   }
@@ -149,8 +151,20 @@ public class ExternalSPSFileIDCollector implements FileIdCollector {
     if (dfs == null) {
       dfs = getFS(service.getConf());
     }
-    processPath(inodeId, cxt.getFilePath(inodeId));
-    service.markScanCompletedForPath(inodeId);
+    long pendingSatisfyItemsCount = processPath(inodeId,
+        cxt.getFilePath(inodeId));
+    // Check whether the given path contains any item to be tracked
+    // or the no to be satisfied paths. In case of empty list, add the given
+    // inodeId to the 'pendingWorkForDirectory' with empty list so that later
+    // SPSPathIdProcessor#run function will remove the SPS hint considering that
+    // this path is already satisfied the storage policy.
+    if (pendingSatisfyItemsCount <= 0) {
+      LOG.debug("There is no pending items to satisfy the given path "
+          + "inodeId:{}", inodeId);
+      service.addAllFileIdsToProcess(inodeId, new ArrayList<>(), true);
+    } else {
+      service.markScanCompletedForPath(inodeId);
+    }
   }
 
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3b83110d/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 ee90eff..3a2ad48 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
@@ -341,12 +341,15 @@ public class StoragePolicyAdmin extends Configured implements Tool {
     }
   }
 
-  /** Command to check storage policy satisfier status. */
-  private static class IsSatisfierRunningCommand
+  /**
+   * Command to check storage policy satisfier status running internal(inside)
+   * Namenode.
+   */
+  private static class IsInternalSatisfierRunningCommand
       implements AdminHelper.Command {
     @Override
     public String getName() {
-      return "-isSatisfierRunning";
+      return "-isInternalSatisfierRunning";
     }
 
     @Override
@@ -356,8 +359,9 @@ public class StoragePolicyAdmin extends Configured implements Tool {
 
     @Override
     public String getLongUsage() {
-      return getShortUsage() + "\n" +
-          "Check the status of Storage Policy Statisfier.\n\n";
+      return getShortUsage() + "\n"
+          + "Check the status of Storage Policy Statisfier"
+          + " running inside Namenode.\n\n";
     }
 
     @Override
@@ -435,6 +439,6 @@ public class StoragePolicyAdmin extends Configured implements Tool {
       new GetStoragePolicyCommand(),
       new UnsetStoragePolicyCommand(),
       new SatisfyStoragePolicyCommand(),
-      new IsSatisfierRunningCommand()
+      new IsInternalSatisfierRunningCommand()
   };
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3b83110d/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 897ae9a..a32d7c3 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
@@ -4498,11 +4498,16 @@
 <property>
   <name>dfs.storage.policy.satisfier.enabled</name>
   <value>false</value>
+  <name>dfs.storage.policy.satisfier.mode</name>
+  <value>none</value>
   <description>
-    If true, StoragePolicySatisfier will be started along with active namenode.
+    Following values are supported - internal, external, none.
+    If internal, StoragePolicySatisfier will be enabled and started along with active namenode.
+    If external, StoragePolicySatisfier will be enabled and started as an independent service outside namenode.
+    If none, StoragePolicySatisfier is disabled.
     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.
+    Administrator can dynamically change StoragePolicySatisfier mode by using reconfiguration option.
+    Dynamic mode change 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/3b83110d/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 15f5d2c..c10bfc3 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/ArchivalStorage.md
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/ArchivalStorage.md
@@ -108,7 +108,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. 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.
+SPS can be enabled as internal service to Namenode or as an external service outside Namenode or 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)
 
@@ -125,8 +125,9 @@ Detailed design documentation can be found at [Storage Policy Satisfier(SPS) (HD
 
 ####Configurations:
 
-*   **dfs.storage.policy.satisfier.enabled** - Used to enable or disable SPS. Configuring true represents SPS is
-   enabled and vice versa.
+*   **dfs.storage.policy.satisfier.mode** - Used to enable(internal service inside NN or external service outside NN) or disable SPS.
+   Following string values are supported - `internal`, `external`, `none`. Configuring `internal` or `external` value represents SPS is enable and `none` to disable.
+   The default value is `none`.
 
 *   **dfs.storage.policy.satisfier.recheck.timeout.millis** - A timeout to re-check the processed block storage movement
    command results from Datanodes.
@@ -153,7 +154,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 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.
+`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(internal service inside NN or external service outside NN) 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(internal service inside NN or external service outside NN) or disable SPS dynamically.
 
 Storage Policy Commands
 -----------------------
@@ -228,14 +229,14 @@ Schedule blocks to move based on file's/directory's current storage policy.
 
 ### SPS Running Status
 
-Check the running status of Storage Policy Satisfier in namenode. If it is running, return 'yes'. Otherwise return 'no'.
+Check the running status of Storage Policy Satisfier service in namenode. If it is running, return 'yes'. Otherwise return 'no'.
 
 * Command:
 
-        hdfs storagepolicies -isSatisfierRunning
+        hdfs storagepolicies -isInternalSatisfierRunning
 
-### 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
+### Enable(internal service inside NN or external service outside NN) or Disable SPS without restarting Namenode
+If administrator wants to switch modes of SPS feature while Namenode is running, first he/she needs to update the desired value(internal or external or none) for the configuration item `dfs.storage.policy.satisfier.mode` 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/3b83110d/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 3681cae..5f41a86 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,6 +36,7 @@ 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.protocol.HdfsConstants.StoragePolicySatisfierMode;
 import org.apache.hadoop.hdfs.server.protocol.BlockStorageMovementCommand.BlockMovingInfo;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.junit.After;
@@ -66,8 +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_ENABLED_KEY,
-        true);
+    conf.set(DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_MODE_KEY,
+        StoragePolicySatisfierMode.INTERNAL.toString());
   }
 
   @Before

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3b83110d/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 233be45..900dcdb 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
@@ -72,6 +72,7 @@ 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.protocol.HdfsConstants.StoragePolicySatisfierMode;
 import org.apache.hadoop.hdfs.server.balancer.Dispatcher.DBlock;
 import org.apache.hadoop.hdfs.server.balancer.ExitStatus;
 import org.apache.hadoop.hdfs.server.balancer.NameNodeConnector;
@@ -114,8 +115,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_ENABLED_KEY, false);
+    conf.set(DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_MODE_KEY,
+        StoragePolicySatisfierMode.NONE.toString());
   }
 
   static Mover newMover(Configuration conf) throws IOException {
@@ -136,8 +137,8 @@ public class TestMover {
   @Test
   public void testScheduleSameBlock() throws IOException {
     final Configuration conf = new HdfsConfiguration();
-    conf.setBoolean(
-        DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_ENABLED_KEY, false);
+    conf.set(DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_MODE_KEY,
+        StoragePolicySatisfierMode.NONE.toString());
     final MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
         .numDataNodes(4).build();
     try {
@@ -461,8 +462,8 @@ public class TestMover {
   @Test
   public void testMoverCli() throws Exception {
     final Configuration clusterConf = new HdfsConfiguration();
-    clusterConf.setBoolean(
-        DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_ENABLED_KEY, false);
+    clusterConf.set(DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_MODE_KEY,
+        StoragePolicySatisfierMode.NONE.toString());
     final MiniDFSCluster cluster = new MiniDFSCluster
         .Builder(clusterConf).numDataNodes(0).build();
     try {
@@ -496,8 +497,8 @@ public class TestMover {
   @Test
   public void testMoverCliWithHAConf() throws Exception {
     final Configuration conf = new HdfsConfiguration();
-    conf.setBoolean(
-        DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_ENABLED_KEY, false);
+    conf.set(DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_MODE_KEY,
+        StoragePolicySatisfierMode.NONE.toString());
     final MiniDFSCluster cluster = new MiniDFSCluster
         .Builder(conf)
         .nnTopology(MiniDFSNNTopology.simpleHATopology())
@@ -521,15 +522,15 @@ public class TestMover {
   @Test
   public void testMoverCliWithFederation() throws Exception {
     final Configuration clusterConf = new HdfsConfiguration();
-    clusterConf.setBoolean(
-        DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_ENABLED_KEY, false);
+    clusterConf.set(DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_MODE_KEY,
+        StoragePolicySatisfierMode.NONE.toString());
     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_ENABLED_KEY, false);
+    clusterConf.set(DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_MODE_KEY,
+        StoragePolicySatisfierMode.NONE.toString());
     DFSTestUtil.setFederatedConfiguration(cluster, conf);
     try {
       Collection<URI> namenodes = DFSUtil.getInternalNsRpcUris(conf);
@@ -574,15 +575,15 @@ public class TestMover {
   @Test
   public void testMoverCliWithFederationHA() throws Exception {
     final Configuration clusterConf = new HdfsConfiguration();
-    clusterConf.setBoolean(
-        DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_ENABLED_KEY, false);
+    clusterConf.set(DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_MODE_KEY,
+        StoragePolicySatisfierMode.NONE.toString());
     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_ENABLED_KEY, false);
+    clusterConf.set(DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_MODE_KEY,
+        StoragePolicySatisfierMode.NONE.toString());
     DFSTestUtil.setFederatedHAConfiguration(cluster, conf);
     try {
       Collection<URI> namenodes = DFSUtil.getInternalNsRpcUris(conf);
@@ -646,8 +647,8 @@ public class TestMover {
   public void testMoveWhenStoragePolicyNotSatisfying() throws Exception {
     // HDFS-8147
     final Configuration conf = new HdfsConfiguration();
-    conf.setBoolean(
-        DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_ENABLED_KEY, false);
+    conf.set(DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_MODE_KEY,
+        StoragePolicySatisfierMode.NONE.toString());
     final MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
         .numDataNodes(3)
         .storageTypes(
@@ -676,8 +677,8 @@ public class TestMover {
   @Test(timeout = 300000)
   public void testMoveWhenStoragePolicySatisfierIsRunning() throws Exception {
     final Configuration conf = new HdfsConfiguration();
-    conf.setBoolean(
-        DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_ENABLED_KEY, true);
+    conf.set(DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_MODE_KEY,
+        StoragePolicySatisfierMode.INTERNAL.toString());
     final MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
         .numDataNodes(3)
         .storageTypes(
@@ -799,8 +800,8 @@ public class TestMover {
         1L);
     conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_REDUNDANCY_CONSIDERLOAD_KEY,
         false);
-    conf.setBoolean(
-        DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_ENABLED_KEY, false);
+    conf.set(DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_MODE_KEY,
+        StoragePolicySatisfierMode.NONE.toString());
   }
 
   @Test(timeout = 300000)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3b83110d/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 f29aa09..0e9784b 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,8 +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_ENABLED_KEY, false);
+    DEFAULT_CONF.set(DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_MODE_KEY,
+        DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_MODE_DEFAULT);
 
     DEFAULT_POLICIES = BlockStoragePolicySuite.createDefaultSuite();
     HOT = DEFAULT_POLICIES.getPolicy(HdfsConstants.HOT_STORAGE_POLICY_NAME);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3b83110d/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 b4a461a..85a101f 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.protocol.HdfsConstants.StoragePolicySatisfierMode;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
@@ -45,8 +46,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_ENABLED_KEY;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_ENABLED_DEFAULT;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_MODE_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_MODE_DEFAULT;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BLOCK_INVALIDATE_LIMIT_KEY;
 import static org.apache.hadoop.fs.CommonConfigurationKeys.IPC_BACKOFF_ENABLE_DEFAULT;
 
@@ -241,22 +242,25 @@ public class TestNameNodeReconfigure {
     cluster.waitActive();
 
     final NameNode nameNode = cluster.getNameNode();
-    verifySPSEnabled(nameNode, DFS_STORAGE_POLICY_SATISFIER_ENABLED_KEY, false);
+    verifySPSEnabled(nameNode, DFS_STORAGE_POLICY_SATISFIER_MODE_KEY,
+        StoragePolicySatisfierMode.NONE, false);
 
-    // enable SPS
-    nameNode.reconfigureProperty(DFS_STORAGE_POLICY_SATISFIER_ENABLED_KEY,
-        "true");
+    // enable SPS internally by keeping DFS_STORAGE_POLICY_ENABLED_KEY
+    nameNode.reconfigureProperty(DFS_STORAGE_POLICY_SATISFIER_MODE_KEY,
+        StoragePolicySatisfierMode.INTERNAL.toString());
 
     // 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());
+    verifySPSEnabled(nameNode, DFS_STORAGE_POLICY_SATISFIER_MODE_KEY,
+        StoragePolicySatisfierMode.INTERNAL, false);
 
-    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));
+    assertEquals(DFS_STORAGE_POLICY_SATISFIER_MODE_KEY + " has wrong value",
+        StoragePolicySatisfierMode.INTERNAL.toString(), nameNode.getConf()
+            .get(DFS_STORAGE_POLICY_SATISFIER_MODE_KEY,
+            DFS_STORAGE_POLICY_SATISFIER_MODE_DEFAULT));
   }
 
   /**
@@ -267,42 +271,42 @@ public class TestNameNodeReconfigure {
       throws ReconfigurationException {
     final NameNode nameNode = cluster.getNameNode();
 
-    verifySPSEnabled(nameNode, DFS_STORAGE_POLICY_SATISFIER_ENABLED_KEY,
-        false);
+    verifySPSEnabled(nameNode, DFS_STORAGE_POLICY_SATISFIER_MODE_KEY,
+        StoragePolicySatisfierMode.NONE, false);
     // try invalid values
     try {
-      nameNode.reconfigureProperty(DFS_STORAGE_POLICY_SATISFIER_ENABLED_KEY,
+      nameNode.reconfigureProperty(DFS_STORAGE_POLICY_SATISFIER_MODE_KEY,
           "text");
       fail("ReconfigurationException expected");
     } catch (ReconfigurationException e) {
       GenericTestUtils.assertExceptionContains(
-          "For enabling or disabling storage policy satisfier, "
-              + "we must pass true/false only",
+          "For enabling or disabling storage policy satisfier, we must "
+              + "pass either none/internal/external string value only",
           e.getCause());
     }
 
-    // enable SPS
-    nameNode.reconfigureProperty(DFS_STORAGE_POLICY_SATISFIER_ENABLED_KEY,
-        "true");
-
-    verifySPSEnabled(nameNode, DFS_STORAGE_POLICY_SATISFIER_ENABLED_KEY,
-        true);
+    // enable internal SPS
+    nameNode.reconfigureProperty(DFS_STORAGE_POLICY_SATISFIER_MODE_KEY,
+        StoragePolicySatisfierMode.INTERNAL.toString());
+    verifySPSEnabled(nameNode, DFS_STORAGE_POLICY_SATISFIER_MODE_KEY,
+        StoragePolicySatisfierMode.INTERNAL, true);
 
     // disable SPS
-    nameNode.reconfigureProperty(DFS_STORAGE_POLICY_SATISFIER_ENABLED_KEY,
-        "false");
-    verifySPSEnabled(nameNode, DFS_STORAGE_POLICY_SATISFIER_ENABLED_KEY,
-        false);
-
-    // enable SPS
-    nameNode.reconfigureProperty(DFS_STORAGE_POLICY_SATISFIER_ENABLED_KEY,
-        "true");
-    assertEquals(DFS_STORAGE_POLICY_SATISFIER_ENABLED_KEY + " has wrong value",
-        true, nameNode.getNamesystem().getBlockManager()
+    nameNode.reconfigureProperty(DFS_STORAGE_POLICY_SATISFIER_MODE_KEY,
+        StoragePolicySatisfierMode.NONE.toString());
+    verifySPSEnabled(nameNode, DFS_STORAGE_POLICY_SATISFIER_MODE_KEY,
+        StoragePolicySatisfierMode.NONE, false);
+
+    // enable external SPS
+    nameNode.reconfigureProperty(DFS_STORAGE_POLICY_SATISFIER_MODE_KEY,
+        StoragePolicySatisfierMode.EXTERNAL.toString());
+    assertEquals(DFS_STORAGE_POLICY_SATISFIER_MODE_KEY + " has wrong value",
+        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, false));
+    assertEquals(DFS_STORAGE_POLICY_SATISFIER_MODE_KEY + " has wrong value",
+        StoragePolicySatisfierMode.EXTERNAL.toString(),
+        nameNode.getConf().get(DFS_STORAGE_POLICY_SATISFIER_MODE_KEY,
+            DFS_STORAGE_POLICY_SATISFIER_MODE_DEFAULT));
   }
 
   /**
@@ -314,10 +318,10 @@ public class TestNameNodeReconfigure {
     final NameNode nameNode = cluster.getNameNode();
 
     // disable SPS
-    nameNode.reconfigureProperty(DFS_STORAGE_POLICY_SATISFIER_ENABLED_KEY,
-        "false");
-    verifySPSEnabled(nameNode, DFS_STORAGE_POLICY_SATISFIER_ENABLED_KEY,
-        false);
+    nameNode.reconfigureProperty(DFS_STORAGE_POLICY_SATISFIER_MODE_KEY,
+        StoragePolicySatisfierMode.NONE.toString());
+    verifySPSEnabled(nameNode, DFS_STORAGE_POLICY_SATISFIER_MODE_KEY,
+        StoragePolicySatisfierMode.NONE, false);
 
     Path filePath = new Path("/testSPS");
     DistributedFileSystem fileSystem = cluster.getFileSystem();
@@ -334,23 +338,26 @@ public class TestNameNodeReconfigure {
               + "or use Mover tool.", e);
     }
 
-    // revert to default
-    nameNode.reconfigureProperty(DFS_STORAGE_POLICY_SATISFIER_ENABLED_KEY,
-        "true");
-    assertEquals(DFS_STORAGE_POLICY_SATISFIER_ENABLED_KEY + " has wrong value",
+    // start internal
+    nameNode.reconfigureProperty(DFS_STORAGE_POLICY_SATISFIER_MODE_KEY,
+        StoragePolicySatisfierMode.INTERNAL.toString());
+    assertEquals(DFS_STORAGE_POLICY_SATISFIER_MODE_KEY + " has wrong value",
         true, nameNode.getNamesystem().getBlockManager()
             .isStoragePolicySatisfierRunning());
-    assertEquals(DFS_STORAGE_POLICY_SATISFIER_ENABLED_KEY + " has wrong value",
-        true, nameNode.getConf()
-            .getBoolean(DFS_STORAGE_POLICY_SATISFIER_ENABLED_KEY, false));
+    assertEquals(DFS_STORAGE_POLICY_SATISFIER_MODE_KEY + " has wrong value",
+        StoragePolicySatisfierMode.INTERNAL.toString(),
+        nameNode.getConf().get(DFS_STORAGE_POLICY_SATISFIER_MODE_KEY,
+            DFS_STORAGE_POLICY_SATISFIER_MODE_DEFAULT));
   }
 
   void verifySPSEnabled(final NameNode nameNode, String property,
-      boolean expected) {
-    assertEquals(property + " has wrong value", expected, nameNode
+      StoragePolicySatisfierMode expected, boolean isSatisfierRunning) {
+    assertEquals(property + " has wrong value", isSatisfierRunning, nameNode
         .getNamesystem().getBlockManager().isStoragePolicySatisfierRunning());
-    assertEquals(property + " has wrong value", expected, nameNode.getConf()
-        .getBoolean(property, DFS_STORAGE_POLICY_SATISFIER_ENABLED_DEFAULT));
+    String actual = nameNode.getConf().get(property,
+        DFS_STORAGE_POLICY_SATISFIER_MODE_DEFAULT);
+    assertEquals(property + " has wrong value", expected,
+        StoragePolicySatisfierMode.fromString(actual));
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3b83110d/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 c301b8a..b84214c 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
@@ -27,6 +27,7 @@ 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.protocol.HdfsConstants.StoragePolicySatisfierMode;
 import org.apache.hadoop.hdfs.MiniDFSNNTopology;
 import org.apache.hadoop.hdfs.server.namenode.ha.HATestUtil;
 import org.apache.hadoop.test.GenericTestUtils;
@@ -102,8 +103,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);
+    conf.set(DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_MODE_KEY,
+        StoragePolicySatisfierMode.INTERNAL.toString());
     final int dnNumber = storageTypes.length;
     final short replication = 3;
     MiniDFSCluster.Builder clusterBuilder = new MiniDFSCluster.Builder(conf)
@@ -292,8 +293,8 @@ public class TestPersistentStoragePolicySatisfier {
     MiniDFSCluster haCluster = null;
     try {
       conf = new HdfsConfiguration();
-      conf.setBoolean(DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_ENABLED_KEY,
-          true);
+      conf.set(DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_MODE_KEY,
+          StoragePolicySatisfierMode.INTERNAL.toString());
       haCluster = new MiniDFSCluster
           .Builder(conf)
           .nnTopology(MiniDFSNNTopology.simpleHAFederatedTopology(2))

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3b83110d/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 b0fd3af..e89cfa3 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
@@ -31,6 +31,7 @@ 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.protocol.HdfsConstants.StoragePolicySatisfierMode;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
@@ -65,8 +66,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);
+    config.set(DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_MODE_KEY,
+        StoragePolicySatisfierMode.INTERNAL.toString());
     startCluster(config, allDiskTypes, numOfDatanodes, storagesPerDatanode,
         capacity);
     dfs = cluster.getFileSystem(nnIndex);
@@ -133,13 +134,14 @@ public class TestStoragePolicySatisfierWithHA {
 
       try {
         cluster.getNameNode(0).reconfigurePropertyImpl(
-            DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_ENABLED_KEY, "false");
+            DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_MODE_KEY,
+            StoragePolicySatisfierMode.EXTERNAL.toString());
         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_ENABLED_KEY
-            + " from 'true' to 'false'", e);
+            + DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_MODE_KEY
+            + " from 'INTERNAL' to 'EXTERNAL'", e);
         GenericTestUtils.assertExceptionContains(
             "Enabling or disabling storage policy satisfier service on "
                 + "standby NameNode is not allowed", e.getCause());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3b83110d/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/sps/TestStoragePolicySatisfier.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/sps/TestStoragePolicySatisfier.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/sps/TestStoragePolicySatisfier.java
index 8115661..935d4f2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/sps/TestStoragePolicySatisfier.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/sps/TestStoragePolicySatisfier.java
@@ -54,6 +54,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.StoragePolicySatisfierMode;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.StoragePolicySatisfyPathStatus;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
@@ -122,6 +123,13 @@ public class TestStoragePolicySatisfier {
   }
 
   /**
+   * @return hdfs cluster.
+   */
+  public MiniDFSCluster getCluster() {
+    return hdfsCluster;
+  }
+
+  /**
    * Gets distributed file system.
    *
    * @throws IOException
@@ -139,8 +147,6 @@ public class TestStoragePolicySatisfier {
 
   public 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, NUM_OF_DATANODES,
         STORAGES_PER_DATANODE, CAPACITY);
     getFS();
@@ -150,6 +156,8 @@ public class TestStoragePolicySatisfier {
   @Before
   public void setUp() {
     config = new HdfsConfiguration();
+    config.set(DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_MODE_KEY,
+        StoragePolicySatisfierMode.INTERNAL.toString());
   }
 
   @Test(timeout = 300000)
@@ -404,8 +412,7 @@ public class TestStoragePolicySatisfier {
       final String nonExistingFile = "/noneExistingFile";
       hdfsCluster.getConfiguration(0).
           setBoolean(DFSConfigKeys.DFS_STORAGE_POLICY_ENABLED_KEY, false);
-      hdfsCluster.restartNameNodes();
-      hdfsCluster.waitActive();
+      restartNamenode();
       HdfsAdmin hdfsAdmin =
           new HdfsAdmin(FileSystem.getDefaultUri(config), config);
 
@@ -423,8 +430,8 @@ public class TestStoragePolicySatisfier {
 
       hdfsCluster.getConfiguration(0).
           setBoolean(DFSConfigKeys.DFS_STORAGE_POLICY_ENABLED_KEY, true);
-      hdfsCluster.restartNameNodes();
-      hdfsCluster.waitActive();
+      restartNamenode();
+
       hdfsAdmin = new HdfsAdmin(FileSystem.getDefaultUri(config), config);
       try {
         hdfsAdmin.satisfyStoragePolicy(new Path(nonExistingFile));
@@ -552,7 +559,8 @@ public class TestStoragePolicySatisfier {
       createCluster();
       // Stop SPS
       hdfsCluster.getNameNode().reconfigureProperty(
-          DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_ENABLED_KEY, "false");
+          DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_MODE_KEY,
+          StoragePolicySatisfierMode.NONE.toString());
       running = hdfsCluster.getFileSystem()
           .getClient().isStoragePolicySatisfierRunning();
       Assert.assertFalse("SPS should stopped as configured.", running);
@@ -563,7 +571,8 @@ public class TestStoragePolicySatisfier {
 
       // Restart SPS
       hdfsCluster.getNameNode().reconfigureProperty(
-          DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_ENABLED_KEY, "true");
+          DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_MODE_KEY,
+          StoragePolicySatisfierMode.INTERNAL.toString());
 
       running = hdfsCluster.getFileSystem()
           .getClient().isStoragePolicySatisfierRunning();
@@ -578,7 +587,8 @@ public class TestStoragePolicySatisfier {
 
       // Restart SPS again
       hdfsCluster.getNameNode().reconfigureProperty(
-          DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_ENABLED_KEY, "true");
+          DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_MODE_KEY,
+          StoragePolicySatisfierMode.INTERNAL.toString());
       running = hdfsCluster.getFileSystem()
           .getClient().isStoragePolicySatisfierRunning();
       Assert.assertTrue("SPS should be running as "
@@ -588,7 +598,7 @@ public class TestStoragePolicySatisfier {
       doTestWhenStoragePolicySetToCOLD();
     } catch (ReconfigurationException e) {
       throw new IOException("Exception when reconfigure "
-          + DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_ENABLED_KEY, e);
+          + DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_MODE_KEY, e);
     } finally {
       if (out != null) {
         out.close();
@@ -610,7 +620,7 @@ public class TestStoragePolicySatisfier {
       // Simulate the case by creating MOVER_ID file
       DFSTestUtil.createFile(hdfsCluster.getFileSystem(),
           HdfsServerConstants.MOVER_ID_PATH, 0, (short) 1, 0);
-      hdfsCluster.restartNameNode(true);
+      restartNamenode();
       boolean running = hdfsCluster.getFileSystem()
           .getClient().isStoragePolicySatisfierRunning();
       Assert.assertTrue("SPS should be running as "
@@ -630,14 +640,7 @@ public class TestStoragePolicySatisfier {
   public void testMoveWithBlockPinning() throws Exception {
     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();
+      hdfsCluster = startCluster(config, allDiskTypes, 3, 2, CAPACITY);
 
       hdfsCluster.waitActive();
       dfs = hdfsCluster.getFileSystem();
@@ -699,8 +702,6 @@ 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},
@@ -743,8 +744,6 @@ 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, NUM_OF_DATANODES,
           STORAGES_PER_DATANODE, CAPACITY);
@@ -781,8 +780,6 @@ 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,
           STORAGES_PER_DATANODE, CAPACITY);
@@ -816,8 +813,6 @@ public class TestStoragePolicySatisfier {
         {StorageType.DISK, StorageType.ARCHIVE}};
 
     try {
-      config.setBoolean(DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_ENABLED_KEY,
-          true);
       hdfsCluster = startCluster(config, diskTypes, diskTypes.length,
           STORAGES_PER_DATANODE, CAPACITY);
       dfs = hdfsCluster.getFileSystem();
@@ -861,8 +856,6 @@ 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, NUM_OF_DATANODES,
@@ -949,8 +942,6 @@ 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,
           STORAGES_PER_DATANODE, CAPACITY);
@@ -1003,29 +994,25 @@ public class TestStoragePolicySatisfier {
    */
   @Test(timeout = 300000)
   public void testSPSWhenFileLengthIsZero() 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).numDataNodes(0).build();
-      cluster.waitActive();
-      DistributedFileSystem fs = cluster.getFileSystem();
+      hdfsCluster = startCluster(config, allDiskTypes, NUM_OF_DATANODES,
+          STORAGES_PER_DATANODE, CAPACITY);
+      hdfsCluster.waitActive();
+      DistributedFileSystem fs = hdfsCluster.getFileSystem();
       Path filePath = new Path("/zeroSizeFile");
       DFSTestUtil.createFile(fs, filePath, 0, (short) 1, 0);
-      FSEditLog editlog = cluster.getNameNode().getNamesystem().getEditLog();
+      FSEditLog editlog = hdfsCluster.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()
+      INode inode = hdfsCluster.getNameNode().getNamesystem().getFSDirectory()
           .getINode(filePath.toString());
       Assert.assertTrue("XAttrFeature should be null for file",
           inode.getXAttrFeature() == null);
     } finally {
-      if (cluster != null) {
-        cluster.shutdown();
-      }
+      shutdownCluster();
     }
   }
 
@@ -1042,42 +1029,36 @@ public class TestStoragePolicySatisfier {
    */
   @Test(timeout = 300000)
   public void testSPSWhenFileHasLowRedundancyBlocks() throws Exception {
-    MiniDFSCluster cluster = null;
     try {
-      Configuration conf = new Configuration();
-      conf.setBoolean(DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_ENABLED_KEY,
-          true);
-      conf.set(DFSConfigKeys
+      config.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 = startCluster(conf, newtypes, 3, 2, CAPACITY);
-      cluster.waitActive();
-      DistributedFileSystem fs = cluster.getFileSystem();
+      hdfsCluster = startCluster(config, newtypes, 3, 2, CAPACITY);
+      hdfsCluster.waitActive();
+      DistributedFileSystem fs = hdfsCluster.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), false);
-      cluster.restartDataNode(list.get(1), false);
-      cluster.waitActive();
+      list.add(hdfsCluster.stopDataNode(0));
+      list.add(hdfsCluster.stopDataNode(0));
+      list.add(hdfsCluster.stopDataNode(0));
+      restartNamenode();
+      hdfsCluster.restartDataNode(list.get(0), false);
+      hdfsCluster.restartDataNode(list.get(1), false);
+      hdfsCluster.waitActive();
       fs.satisfyStoragePolicy(filePath);
       DFSTestUtil.waitExpectedStorageType(filePath.toString(),
-          StorageType.ARCHIVE, 2, 30000, cluster.getFileSystem());
-      cluster.restartDataNode(list.get(2), false);
+          StorageType.ARCHIVE, 2, 30000, hdfsCluster.getFileSystem());
+      hdfsCluster.restartDataNode(list.get(2), false);
       DFSTestUtil.waitExpectedStorageType(filePath.toString(),
-          StorageType.ARCHIVE, 3, 30000, cluster.getFileSystem());
+          StorageType.ARCHIVE, 3, 30000, hdfsCluster.getFileSystem());
     } finally {
-      if (cluster != null) {
-        cluster.shutdown();
-      }
+      shutdownCluster();
     }
   }
 
@@ -1091,12 +1072,8 @@ public class TestStoragePolicySatisfier {
    */
   @Test(timeout = 300000)
   public void testSPSWhenFileHasExcessRedundancyBlocks() throws Exception {
-    MiniDFSCluster cluster = null;
     try {
-      Configuration conf = new Configuration();
-      conf.setBoolean(DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_ENABLED_KEY,
-          true);
-      conf.set(DFSConfigKeys
+      config.set(DFSConfigKeys
           .DFS_STORAGE_POLICY_SATISFIER_RECHECK_TIMEOUT_MILLIS_KEY,
           "3000");
       StorageType[][] newtypes = new StorageType[][] {
@@ -1105,10 +1082,9 @@ public class TestStoragePolicySatisfier {
           {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();
+      hdfsCluster = startCluster(config, newtypes, 5, 2, CAPACITY);
+      hdfsCluster.waitActive();
+      DistributedFileSystem fs = hdfsCluster.getFileSystem();
       Path filePath = new Path("/zeroSizeFile");
       DFSTestUtil.createFile(fs, filePath, 1024, (short) 5, 0);
       fs.setReplication(filePath, (short) 3);
@@ -1117,13 +1093,11 @@ public class TestStoragePolicySatisfier {
       fs.setStoragePolicy(filePath, "COLD");
       fs.satisfyStoragePolicy(filePath);
       DFSTestUtil.waitExpectedStorageType(filePath.toString(),
-          StorageType.ARCHIVE, 3, 30000, cluster.getFileSystem());
+          StorageType.ARCHIVE, 3, 30000, hdfsCluster.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();
-      }
+      shutdownCluster();
     }
   }
 
@@ -1133,24 +1107,19 @@ public class TestStoragePolicySatisfier {
   @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();
+      hdfsCluster = startCluster(config, allDiskTypes, NUM_OF_DATANODES,
+          STORAGES_PER_DATANODE, CAPACITY);
+      hdfsCluster.waitActive();
+      DistributedFileSystem fs = hdfsCluster.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);
+          XATTR_SATISFY_STORAGE_POLICY, hdfsCluster.getNamesystem(), 30000);
     } finally {
-      if (cluster != null) {
-        cluster.shutdown();
-      }
+      shutdownCluster();
     }
   }
 
@@ -1159,14 +1128,11 @@ public class TestStoragePolicySatisfier {
    */
   @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();
+      hdfsCluster = startCluster(config, allDiskTypes, NUM_OF_DATANODES,
+          STORAGES_PER_DATANODE, CAPACITY);
+      hdfsCluster.waitActive();
+      DistributedFileSystem fs = hdfsCluster.getFileSystem();
       Path emptyDir = new Path("/emptyDir");
       try {
         fs.satisfyStoragePolicy(emptyDir);
@@ -1175,9 +1141,7 @@ public class TestStoragePolicySatisfier {
         // nothing to do
       }
     } finally {
-      if (cluster != null) {
-        cluster.shutdown();
-      }
+      shutdownCluster();
     }
   }
 
@@ -1186,13 +1150,10 @@ public class TestStoragePolicySatisfier {
    */
   @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();
+      hdfsCluster = startCluster(config, allDiskTypes, NUM_OF_DATANODES,
+          STORAGES_PER_DATANODE, CAPACITY);
+      hdfsCluster.waitActive();
       // Create directories
       /*
        *                   root
@@ -1203,7 +1164,7 @@ public class TestStoragePolicySatisfier {
        *                    |
        *                    O
        */
-      DistributedFileSystem fs = cluster.getFileSystem();
+      DistributedFileSystem fs = hdfsCluster.getFileSystem();
       fs.mkdirs(new Path("/root/C/H/O"));
       fs.mkdirs(new Path("/root/A"));
       fs.mkdirs(new Path("/root/D"));
@@ -1212,11 +1173,9 @@ public class TestStoragePolicySatisfier {
       fs.satisfyStoragePolicy(new Path("/root"));
       // Make sure satisfy xattr has been removed.
       DFSTestUtil.waitForXattrRemoved("/root",
-          XATTR_SATISFY_STORAGE_POLICY, cluster.getNamesystem(), 30000);
+          XATTR_SATISFY_STORAGE_POLICY, hdfsCluster.getNamesystem(), 30000);
     } finally {
-      if (cluster != null) {
-        cluster.shutdown();
-      }
+      shutdownCluster();
     }
   }
 
@@ -1232,8 +1191,6 @@ public class TestStoragePolicySatisfier {
           {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,
           STORAGES_PER_DATANODE, CAPACITY);
       dfs = hdfsCluster.getFileSystem();
@@ -1263,8 +1220,6 @@ public class TestStoragePolicySatisfier {
           {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);
@@ -1461,8 +1416,6 @@ public class TestStoragePolicySatisfier {
     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");
@@ -1473,8 +1426,8 @@ public class TestStoragePolicySatisfier {
       StorageType[][] storagetypes = new StorageType[][] {
           {StorageType.ARCHIVE, StorageType.DISK},
           {StorageType.ARCHIVE, StorageType.DISK}};
-      hdfsCluster = new MiniDFSCluster.Builder(config).numDataNodes(2)
-          .storageTypes(storagetypes).build();
+
+      hdfsCluster = startCluster(config, storagetypes, 2, 2, CAPACITY);
       hdfsCluster.waitActive();
       dfs = hdfsCluster.getFileSystem();
 
@@ -1523,8 +1476,6 @@ 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");
@@ -1535,8 +1486,7 @@ public class TestStoragePolicySatisfier {
       StorageType[][] storagetypes = new StorageType[][] {
           {StorageType.ARCHIVE, StorageType.DISK},
           {StorageType.ARCHIVE, StorageType.DISK}};
-      hdfsCluster = new MiniDFSCluster.Builder(config).numDataNodes(2)
-          .storageTypes(storagetypes).build();
+      hdfsCluster = startCluster(config, storagetypes, 2, 2, CAPACITY);
       hdfsCluster.waitActive();
       // BlockStorageMovementNeeded.setStatusClearanceElapsedTimeMs(200000);
       dfs = hdfsCluster.getFileSystem();
@@ -1592,8 +1542,6 @@ 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");
@@ -1603,8 +1551,7 @@ public class TestStoragePolicySatisfier {
       StorageType[][] storagetypes = new StorageType[][] {
           {StorageType.DISK, StorageType.DISK},
           {StorageType.DISK, StorageType.DISK}};
-      hdfsCluster = new MiniDFSCluster.Builder(config).numDataNodes(2)
-          .storageTypes(storagetypes).build();
+      hdfsCluster = startCluster(config, storagetypes, 2, 2, CAPACITY);
       hdfsCluster.waitActive();
       dfs = hdfsCluster.getFileSystem();
 
@@ -1835,4 +1782,9 @@ public class TestStoragePolicySatisfier {
     cluster.waitActive();
     return cluster;
   }
+
+  public void restartNamenode() throws IOException {
+    hdfsCluster.restartNameNodes();
+    hdfsCluster.waitActive();
+  }
 }


---------------------------------------------------------------------
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-12955: [SPS]: Move SPS classes to a separate package. Contributed by Rakesh R.

Posted by um...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/78420719/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/sps/TestStoragePolicySatisfierWithStripedFile.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/sps/TestStoragePolicySatisfierWithStripedFile.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/sps/TestStoragePolicySatisfierWithStripedFile.java
new file mode 100644
index 0000000..c1a2b8b
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/sps/TestStoragePolicySatisfierWithStripedFile.java
@@ -0,0 +1,580 @@
+/**
+ * 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.sps;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+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.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;
+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 StripedFileTestUtil.getDefaultECPolicy();
+  }
+
+  /**
+   * 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();
+    conf.setBoolean(DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_ENABLED_KEY,
+        true);
+    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();
+      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();
+      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,
+          StripedFileTestUtil.getDefaultECPolicy().getName());
+
+      // 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();
+
+      waitForBlocksMovementAttemptReport(cluster, 9, 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();
+    conf.setBoolean(DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_ENABLED_KEY,
+        true);
+    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();
+      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();
+      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,
+          StripedFileTestUtil.getDefaultECPolicy().getName());
+
+      // 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();
+
+      waitForBlocksMovementAttemptReport(cluster, 5, 60000);
+      waitForAttemptedItems(cluster, 1, 30000);
+      // verify storage types and locations.
+      waitExpectedStorageType(cluster, fooFile, fileLen, StorageType.ARCHIVE, 5,
+          9, 60000);
+    } finally {
+      cluster.shutdown();
+    }
+  }
+
+  /**
+   * 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 9 datanodes
+    int numOfDatanodes = 9;
+    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");
+    conf.setBoolean(DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_ENABLED_KEY,
+        true);
+    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}})
+        .storageCapacities(capacities)
+        .build();
+    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
+      fs.setErasureCodingPolicy(barDir,
+          StripedFileTestUtil.getDefaultECPolicy().getName());
+
+      // 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 < 5; i++) {
+        cluster.restartDataNode(list.get(i), false);
+      }
+      cluster.waitActive();
+      fs.satisfyStoragePolicy(fooFile);
+      DFSTestUtil.waitExpectedStorageType(fooFile.toString(),
+          StorageType.ARCHIVE, 5, 30000, cluster.getFileSystem());
+      //Start reaming datanodes
+      for (int i = numOfDatanodes - 1; i >= 5; i--) {
+        cluster.restartDataNode(list.get(i), false);
+      }
+      // 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.
+   *
+   * 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();
+    conf.setBoolean(DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_ENABLED_KEY,
+        true);
+    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();
+      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();
+      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,
+          StripedFileTestUtil.getDefaultECPolicy().getName());
+
+      // 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 attempt report has been arrived at the
+  // Namenode(SPS).
+  private void waitForBlocksMovementAttemptReport(MiniDFSCluster cluster,
+      long expectedMovementFinishedBlocksCount, 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("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


[29/50] [abbrv] hadoop git commit: HDFS-13033: [SPS]: Implement a mechanism to do file block movements for external SPS. Contributed by Rakesh R.

Posted by um...@apache.org.
HDFS-13033: [SPS]: Implement a mechanism to do file block movements for external SPS. 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/b0cb8d9b
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/b0cb8d9b
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/b0cb8d9b

Branch: refs/heads/HDFS-10285
Commit: b0cb8d9bb44c963ae686d2b5c1b70bc76b955e10
Parents: 3159b39
Author: Uma Maheswara Rao G <um...@intel.com>
Authored: Tue Jan 23 16:19:46 2018 -0800
Committer: Uma Maheswara Rao Gangumalla <um...@apache.org>
Committed: Sun Aug 12 03:06:03 2018 -0700

----------------------------------------------------------------------
 .../hdfs/server/balancer/NameNodeConnector.java |   8 +
 .../hdfs/server/common/sps/BlockDispatcher.java | 186 +++++++++++++
 .../sps/BlockMovementAttemptFinished.java       |  80 ++++++
 .../server/common/sps/BlockMovementStatus.java  |  53 ++++
 .../common/sps/BlockStorageMovementTracker.java | 184 +++++++++++++
 .../sps/BlocksMovementsStatusHandler.java       |  95 +++++++
 .../hdfs/server/common/sps/package-info.java    |  27 ++
 .../datanode/BlockStorageMovementTracker.java   | 186 -------------
 .../datanode/StoragePolicySatisfyWorker.java    | 271 ++-----------------
 .../hdfs/server/namenode/FSNamesystem.java      |   4 +-
 .../namenode/sps/BlockMoveTaskHandler.java      |   3 +-
 .../sps/BlockStorageMovementAttemptedItems.java |  12 +-
 .../IntraSPSNameNodeBlockMoveTaskHandler.java   |   3 +-
 .../hdfs/server/namenode/sps/SPSService.java    |  14 +-
 .../namenode/sps/StoragePolicySatisfier.java    |  30 +-
 .../sps/ExternalSPSBlockMoveTaskHandler.java    | 233 ++++++++++++++++
 .../TestBlockStorageMovementAttemptedItems.java |   2 +-
 .../sps/TestStoragePolicySatisfier.java         |   6 +-
 .../sps/TestExternalStoragePolicySatisfier.java |  69 ++++-
 19 files changed, 997 insertions(+), 469 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/b0cb8d9b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/NameNodeConnector.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/NameNodeConnector.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/NameNodeConnector.java
index b0dd779..6bfbbb3 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/NameNodeConnector.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/NameNodeConnector.java
@@ -269,6 +269,14 @@ public class NameNodeConnector implements Closeable {
     }
   }
 
+  /**
+   * Returns fallbackToSimpleAuth. This will be true or false during calls to
+   * indicate if a secure client falls back to simple auth.
+   */
+  public AtomicBoolean getFallbackToSimpleAuth() {
+    return fallbackToSimpleAuth;
+  }
+
   @Override
   public void close() {
     keyManager.close();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b0cb8d9b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/sps/BlockDispatcher.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/sps/BlockDispatcher.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/sps/BlockDispatcher.java
new file mode 100644
index 0000000..f87fcae
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/sps/BlockDispatcher.java
@@ -0,0 +1,186 @@
+/**
+ * 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.common.sps;
+
+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 org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.StorageType;
+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;
+import org.apache.hadoop.hdfs.protocol.datatransfer.sasl.DataEncryptionKeyFactory;
+import org.apache.hadoop.hdfs.protocol.datatransfer.sasl.SaslDataTransferClient;
+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.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Dispatching block replica moves between datanodes to satisfy the storage
+ * policy.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class BlockDispatcher {
+  private static final Logger LOG = LoggerFactory
+      .getLogger(BlockDispatcher.class);
+
+  private final boolean connectToDnViaHostname;
+  private final int socketTimeout;
+  private final int ioFileBufferSize;
+
+  /**
+   * Construct block dispatcher details.
+   *
+   * @param sockTimeout
+   *          soTimeout
+   * @param ioFileBuffSize
+   *          file io buffer size
+   * @param connectToDatanodeViaHostname
+   *          true represents connect via hostname, false otw
+   */
+  public BlockDispatcher(int sockTimeout, int ioFileBuffSize,
+      boolean connectToDatanodeViaHostname) {
+    this.socketTimeout = sockTimeout;
+    this.ioFileBufferSize = ioFileBuffSize;
+    this.connectToDnViaHostname = connectToDatanodeViaHostname;
+  }
+
+  /**
+   * Moves the given block replica to the given target node and wait for the
+   * response.
+   *
+   * @param blkMovingInfo
+   *          block to storage info
+   * @param saslClient
+   *          SASL for DataTransferProtocol on behalf of a client
+   * @param eb
+   *          extended block info
+   * @param sock
+   *          target node's socket
+   * @param km
+   *          for creation of an encryption key
+   * @param accessToken
+   *          connection block access token
+   * @return status of the block movement
+   */
+  public BlockMovementStatus moveBlock(BlockMovingInfo blkMovingInfo,
+      SaslDataTransferClient saslClient, ExtendedBlock eb, Socket sock,
+      DataEncryptionKeyFactory km, Token<BlockTokenIdentifier> accessToken) {
+    LOG.info("Start moving block:{} from src:{} to destin:{} to satisfy "
+        + "storageType, sourceStoragetype:{} and destinStoragetype:{}",
+        blkMovingInfo.getBlock(), blkMovingInfo.getSource(),
+        blkMovingInfo.getTarget(), blkMovingInfo.getSourceStorageType(),
+        blkMovingInfo.getTargetStorageType());
+    DataOutputStream out = null;
+    DataInputStream in = null;
+    try {
+      NetUtils.connect(sock,
+          NetUtils.createSocketAddr(
+              blkMovingInfo.getTarget().getXferAddr(connectToDnViaHostname)),
+          socketTimeout);
+      // Set read timeout so that it doesn't hang forever against
+      // unresponsive nodes. Datanode normally sends IN_PROGRESS response
+      // twice within the client read timeout period (every 30 seconds by
+      // default). Here, we make it give up after "socketTimeout * 5" period
+      // of no response.
+      sock.setSoTimeout(socketTimeout * 5);
+      sock.setKeepAlive(true);
+      OutputStream unbufOut = sock.getOutputStream();
+      InputStream unbufIn = sock.getInputStream();
+      LOG.debug("Connecting to datanode {}", blkMovingInfo.getTarget());
+
+      IOStreamPair saslStreams = saslClient.socketSend(sock, unbufOut,
+          unbufIn, km, accessToken, blkMovingInfo.getTarget());
+      unbufOut = saslStreams.out;
+      unbufIn = saslStreams.in;
+      out = new DataOutputStream(
+          new BufferedOutputStream(unbufOut, ioFileBufferSize));
+      in = new DataInputStream(
+          new BufferedInputStream(unbufIn, ioFileBufferSize));
+      sendRequest(out, eb, accessToken, blkMovingInfo.getSource(),
+          blkMovingInfo.getTargetStorageType());
+      receiveResponse(in);
+
+      LOG.info(
+          "Successfully moved block:{} from src:{} to destin:{} for"
+              + " satisfying storageType:{}",
+          blkMovingInfo.getBlock(), blkMovingInfo.getSource(),
+          blkMovingInfo.getTarget(), blkMovingInfo.getTargetStorageType());
+      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:{}",
+          blkMovingInfo.getBlock(), e);
+      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:{}",
+          blkMovingInfo.getBlock(), blkMovingInfo.getSource(),
+          blkMovingInfo.getTarget(), blkMovingInfo.getTargetStorageType(), e);
+      return BlockMovementStatus.DN_BLK_STORAGE_MOVEMENT_FAILURE;
+    } finally {
+      IOUtils.closeStream(out);
+      IOUtils.closeStream(in);
+      IOUtils.closeSocket(sock);
+    }
+  }
+
+  /** Send a reportedBlock replace request to the output stream. */
+  private static void sendRequest(DataOutputStream out, ExtendedBlock eb,
+      Token<BlockTokenIdentifier> accessToken, DatanodeInfo source,
+      StorageType targetStorageType) throws IOException {
+    new Sender(out).replaceBlock(eb, targetStorageType, accessToken,
+        source.getDatanodeUuid(), source, null);
+  }
+
+  /** Receive a reportedBlock copy response from the input stream. */
+  private static 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/b0cb8d9b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/sps/BlockMovementAttemptFinished.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/sps/BlockMovementAttemptFinished.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/sps/BlockMovementAttemptFinished.java
new file mode 100644
index 0000000..419d806
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/sps/BlockMovementAttemptFinished.java
@@ -0,0 +1,80 @@
+/**
+ * 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.common.sps;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+
+/**
+ * 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.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class BlockMovementAttemptFinished {
+  private final Block block;
+  private final DatanodeInfo src;
+  private final DatanodeInfo target;
+  private final BlockMovementStatus status;
+
+  /**
+   * Construct movement attempt finished info.
+   *
+   * @param block
+   *          block
+   * @param src
+   *          src datanode
+   * @param target
+   *          target datanode
+   * @param status
+   *          movement status
+   */
+  public BlockMovementAttemptFinished(Block block, DatanodeInfo src,
+      DatanodeInfo target, BlockMovementStatus status) {
+    this.block = block;
+    this.src = src;
+    this.target = target;
+    this.status = status;
+  }
+
+  /**
+   * @return details of the block, which attempted to move from src to target
+   *         node.
+   */
+  public Block getBlock() {
+    return block;
+  }
+
+  /**
+   * @return block movement status code.
+   */
+  public BlockMovementStatus getStatus() {
+    return status;
+  }
+
+  @Override
+  public String toString() {
+    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();
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b0cb8d9b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/sps/BlockMovementStatus.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/sps/BlockMovementStatus.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/sps/BlockMovementStatus.java
new file mode 100644
index 0000000..f70d84f
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/sps/BlockMovementStatus.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.common.sps;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * Block movement status code.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public 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;
+
+  BlockMovementStatus(int code) {
+    this.code = code;
+  }
+
+  /**
+   * @return the status code.
+   */
+  int getStatusCode() {
+    return code;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b0cb8d9b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/sps/BlockStorageMovementTracker.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/sps/BlockStorageMovementTracker.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/sps/BlockStorageMovementTracker.java
new file mode 100644
index 0000000..b20d6cf
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/sps/BlockStorageMovementTracker.java
@@ -0,0 +1,184 @@
+/**
+ * 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.common.sps;
+
+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.protocol.Block;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * This class is used to track the completion of block movement future tasks.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class BlockStorageMovementTracker implements Runnable {
+  private static final Logger LOG = LoggerFactory
+      .getLogger(BlockStorageMovementTracker.class);
+  private final CompletionService<BlockMovementAttemptFinished> moverCompletionService;
+  private final BlocksMovementsStatusHandler blksMovementsStatusHandler;
+
+  // 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;
+
+  /**
+   * BlockStorageMovementTracker constructor.
+   *
+   * @param moverCompletionService
+   *          completion service.
+   * @param handler
+   *          blocks movements status handler
+   */
+  public BlockStorageMovementTracker(
+      CompletionService<BlockMovementAttemptFinished> moverCompletionService,
+      BlocksMovementsStatusHandler handler) {
+    this.moverCompletionService = moverCompletionService;
+    this.moverTaskFutures = new HashMap<>();
+    this.blksMovementsStatusHandler = handler;
+    this.movementResults = new HashMap<>();
+  }
+
+  @Override
+  public void run() {
+    while (running) {
+      if (moverTaskFutures.size() <= 0) {
+        try {
+          synchronized (moverTaskFutures) {
+            // Waiting for mover tasks.
+            moverTaskFutures.wait(2000);
+          }
+        } catch (InterruptedException ignore) {
+          // Sets interrupt flag of this thread.
+          Thread.currentThread().interrupt();
+        }
+      }
+      try {
+        Future<BlockMovementAttemptFinished> future =
+            moverCompletionService.take();
+        if (future != null) {
+          BlockMovementAttemptFinished result = future.get();
+          LOG.debug("Completed block movement. {}", result);
+          Block block = result.getBlock();
+          List<Future<BlockMovementAttemptFinished>> blocksMoving =
+              moverTaskFutures.get(block);
+          if (blocksMoving == null) {
+            LOG.warn("Future task doesn't exist for block : {} ", block);
+            continue;
+          }
+          blocksMoving.remove(future);
+
+          List<BlockMovementAttemptFinished> resultPerTrackIdList =
+              addMovementResultToBlockIdList(result);
+
+          // Completed all the scheduled blocks movement under this 'trackId'.
+          if (blocksMoving.isEmpty() || moverTaskFutures.get(block) == null) {
+            synchronized (moverTaskFutures) {
+              moverTaskFutures.remove(block);
+            }
+            if (running) {
+              // handle completed or inprogress blocks movements per trackId.
+              blksMovementsStatusHandler.handle(resultPerTrackIdList);
+            }
+            movementResults.remove(block);
+          }
+        }
+      } 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);
+      }
+    }
+  }
+
+  private List<BlockMovementAttemptFinished> addMovementResultToBlockIdList(
+      BlockMovementAttemptFinished result) {
+    Block block = result.getBlock();
+    List<BlockMovementAttemptFinished> perBlockIdList;
+    synchronized (movementResults) {
+      perBlockIdList = movementResults.get(block);
+      if (perBlockIdList == null) {
+        perBlockIdList = new ArrayList<>();
+        movementResults.put(block, perBlockIdList);
+      }
+      perBlockIdList.add(result);
+    }
+    return perBlockIdList;
+  }
+
+  /**
+   * Add future task to the tracking list to check the completion status of the
+   * block movement.
+   *
+   * @param blockID
+   *          block identifier
+   * @param futureTask
+   *          future task used for moving the respective block
+   */
+  public void addBlock(Block block,
+      Future<BlockMovementAttemptFinished> futureTask) {
+    synchronized (moverTaskFutures) {
+      List<Future<BlockMovementAttemptFinished>> futures =
+          moverTaskFutures.get(block);
+      // null for the first task
+      if (futures == null) {
+        futures = new ArrayList<>();
+        moverTaskFutures.put(block, futures);
+      }
+      futures.add(futureTask);
+      // Notify waiting tracker thread about the newly added tasks.
+      moverTaskFutures.notify();
+    }
+  }
+
+  /**
+   * Clear the pending movement and movement result queues.
+   */
+  public void removeAll() {
+    synchronized (moverTaskFutures) {
+      moverTaskFutures.clear();
+    }
+    synchronized (movementResults) {
+      movementResults.clear();
+    }
+  }
+
+  /**
+   * 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/b0cb8d9b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/sps/BlocksMovementsStatusHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/sps/BlocksMovementsStatusHandler.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/sps/BlocksMovementsStatusHandler.java
new file mode 100644
index 0000000..f9f3954
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/sps/BlocksMovementsStatusHandler.java
@@ -0,0 +1,95 @@
+/**
+ * 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.common.sps;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hdfs.protocol.Block;
+
+/**
+ * Blocks movements status handler, which is used to collect details of the
+ * completed block movements and later these attempted finished(with success or
+ * failure) blocks can be accessed to notify respective listeners, if any.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class BlocksMovementsStatusHandler {
+  private final List<Block> blockIdVsMovementStatus = new ArrayList<>();
+
+  /**
+   * Collect all the storage movement attempt finished blocks. Later this will
+   * be send to namenode via heart beat.
+   *
+   * @param moveAttemptFinishedBlks
+   *          set of storage movement attempt finished blocks
+   */
+  public void handle(
+      List<BlockMovementAttemptFinished> moveAttemptFinishedBlks) {
+    List<Block> blocks = new ArrayList<>();
+
+    for (BlockMovementAttemptFinished item : moveAttemptFinishedBlks) {
+      blocks.add(item.getBlock());
+    }
+    // Adding to the tracking report list. Later this can be accessed to know
+    // the attempted block movements.
+    synchronized (blockIdVsMovementStatus) {
+      blockIdVsMovementStatus.addAll(blocks);
+    }
+  }
+
+  /**
+   * @return unmodifiable list of storage movement attempt finished blocks.
+   */
+  public 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);
+      }
+    }
+    return moveAttemptFinishedBlks;
+  }
+
+  /**
+   * Remove the storage movement attempt finished blocks from the tracking list.
+   *
+   * @param moveAttemptFinishedBlks
+   *          set of storage movement attempt finished blocks
+   */
+  public void remove(List<Block> moveAttemptFinishedBlks) {
+    if (moveAttemptFinishedBlks != null) {
+      blockIdVsMovementStatus.removeAll(moveAttemptFinishedBlks);
+    }
+  }
+
+  /**
+   * Clear the blockID vs movement status tracking map.
+   */
+  public void removeAll() {
+    synchronized (blockIdVsMovementStatus) {
+      blockIdVsMovementStatus.clear();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b0cb8d9b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/sps/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/sps/package-info.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/sps/package-info.java
new file mode 100644
index 0000000..fcffbe9
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/sps/package-info.java
@@ -0,0 +1,27 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+/**
+ * This package provides commonly used classes for the block movement.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+package org.apache.hadoop.hdfs.server.common.sps;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b0cb8d9b/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
deleted file mode 100644
index b3b9fd9..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockStorageMovementTracker.java
+++ /dev/null
@@ -1,186 +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.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.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;
-
-/**
- * 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<BlockMovementAttemptFinished> moverCompletionService;
-  private final BlocksMovementsStatusHandler blksMovementsStatusHandler;
-
-  // 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;
-
-  /**
-   * BlockStorageMovementTracker constructor.
-   *
-   * @param moverCompletionService
-   *          completion service.
-   * @param handler
-   *          blocks movements status handler
-   */
-  public BlockStorageMovementTracker(
-      CompletionService<BlockMovementAttemptFinished> moverCompletionService,
-      BlocksMovementsStatusHandler handler) {
-    this.moverCompletionService = moverCompletionService;
-    this.moverTaskFutures = new HashMap<>();
-    this.blksMovementsStatusHandler = handler;
-    this.movementResults = new HashMap<>();
-  }
-
-  @Override
-  public void run() {
-    while (running) {
-      if (moverTaskFutures.size() <= 0) {
-        try {
-          synchronized (moverTaskFutures) {
-            // Waiting for mover tasks.
-            moverTaskFutures.wait(2000);
-          }
-        } catch (InterruptedException ignore) {
-          // Sets interrupt flag of this thread.
-          Thread.currentThread().interrupt();
-        }
-      }
-      try {
-        Future<BlockMovementAttemptFinished> future =
-            moverCompletionService.take();
-        if (future != null) {
-          BlockMovementAttemptFinished result = future.get();
-          LOG.debug("Completed block movement. {}", result);
-          Block block = result.getBlock();
-          List<Future<BlockMovementAttemptFinished>> blocksMoving =
-              moverTaskFutures.get(block);
-          if (blocksMoving == null) {
-            LOG.warn("Future task doesn't exist for block : {} ", block);
-            continue;
-          }
-          blocksMoving.remove(future);
-
-          List<BlockMovementAttemptFinished> resultPerTrackIdList =
-              addMovementResultToBlockIdList(result);
-
-          // Completed all the scheduled blocks movement under this 'trackId'.
-          if (blocksMoving.isEmpty() || moverTaskFutures.get(block) == null) {
-            synchronized (moverTaskFutures) {
-              moverTaskFutures.remove(block);
-            }
-            if (running) {
-              // handle completed or inprogress blocks movements per trackId.
-              blksMovementsStatusHandler.handle(resultPerTrackIdList);
-            }
-            movementResults.remove(block);
-          }
-        }
-      } 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);
-      }
-    }
-  }
-
-  private List<BlockMovementAttemptFinished> addMovementResultToBlockIdList(
-      BlockMovementAttemptFinished result) {
-    Block block = result.getBlock();
-    List<BlockMovementAttemptFinished> perBlockIdList;
-    synchronized (movementResults) {
-      perBlockIdList = movementResults.get(block);
-      if (perBlockIdList == null) {
-        perBlockIdList = new ArrayList<>();
-        movementResults.put(block, perBlockIdList);
-      }
-      perBlockIdList.add(result);
-    }
-    return perBlockIdList;
-  }
-
-  /**
-   * Add future task to the tracking list to check the completion status of the
-   * block movement.
-   *
-   * @param blockID
-   *          block identifier
-   * @param futureTask
-   *          future task used for moving the respective block
-   */
-  void addBlock(Block block,
-      Future<BlockMovementAttemptFinished> futureTask) {
-    synchronized (moverTaskFutures) {
-      List<Future<BlockMovementAttemptFinished>> futures =
-          moverTaskFutures.get(block);
-      // null for the first task
-      if (futures == null) {
-        futures = new ArrayList<>();
-        moverTaskFutures.put(block, futures);
-      }
-      futures.add(futureTask);
-      // Notify waiting tracker thread about the newly added tasks.
-      moverTaskFutures.notify();
-    }
-  }
-
-  /**
-   * Clear the pending movement and movement result queues.
-   */
-  void removeAll() {
-    synchronized (moverTaskFutures) {
-      moverTaskFutures.clear();
-    }
-    synchronized (movementResults) {
-      movementResults.clear();
-    }
-  }
-
-  /**
-   * 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/b0cb8d9b/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 9a9c7e0..42f2e93 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
@@ -17,21 +17,9 @@
  */
 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.Collection;
-import java.util.Collections;
 import java.util.EnumSet;
-import java.util.List;
 import java.util.concurrent.Callable;
 import java.util.concurrent.CompletionService;
 import java.util.concurrent.ExecutorCompletionService;
@@ -47,20 +35,15 @@ 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.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;
 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.common.sps.BlockMovementAttemptFinished;
+import org.apache.hadoop.hdfs.server.common.sps.BlockMovementStatus;
+import org.apache.hadoop.hdfs.server.common.sps.BlockStorageMovementTracker;
+import org.apache.hadoop.hdfs.server.common.sps.BlocksMovementsStatusHandler;
+import org.apache.hadoop.hdfs.server.common.sps.BlockDispatcher;
 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;
@@ -81,7 +64,6 @@ public class StoragePolicySatisfyWorker {
       .getLogger(StoragePolicySatisfyWorker.class);
 
   private final DataNode datanode;
-  private final int ioFileBufferSize;
 
   private final int moverThreads;
   private final ExecutorService moveExecutor;
@@ -89,10 +71,10 @@ public class StoragePolicySatisfyWorker {
   private final BlocksMovementsStatusHandler handler;
   private final BlockStorageMovementTracker movementTracker;
   private Daemon movementTrackerThread;
+  private final BlockDispatcher blkDispatcher;
 
   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);
@@ -103,7 +85,10 @@ public class StoragePolicySatisfyWorker {
         handler);
     movementTrackerThread = new Daemon(movementTracker);
     movementTrackerThread.setName("BlockStorageMovementTracker");
-
+    DNConf dnConf = datanode.getDnConf();
+    int ioFileBufferSize = DFSUtilClient.getIoFileBufferSize(conf);
+    blkDispatcher = new BlockDispatcher(dnConf.getSocketTimeout(),
+        ioFileBufferSize, dnConf.getConnectToDnViaHostname());
     // TODO: Needs to manage the number of concurrent moves per DataNode.
   }
 
@@ -183,8 +168,7 @@ public class StoragePolicySatisfyWorker {
       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);
+          blkMovingInfo);
       Future<BlockMovementAttemptFinished> moveCallable = moverCompletionService
           .submit(blockMovingTask);
       movementTracker.addBlock(blkMovingInfo.getBlock(),
@@ -199,244 +183,45 @@ public class StoragePolicySatisfyWorker {
   private class BlockMovingTask implements
       Callable<BlockMovementAttemptFinished> {
     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 final BlockMovingInfo blkMovingInfo;
 
-    BlockMovingTask(String blockPoolID, Block block,
-        DatanodeInfo source, DatanodeInfo target,
-        StorageType srcStorageType, StorageType targetStorageType) {
+    BlockMovingTask(String blockPoolID, BlockMovingInfo blkMovInfo) {
       this.blockPoolID = blockPoolID;
-      this.block = block;
-      this.source = source;
-      this.target = target;
-      this.srcStorageType = srcStorageType;
-      this.targetStorageType = targetStorageType;
+      this.blkMovingInfo = blkMovInfo;
     }
 
     @Override
     public BlockMovementAttemptFinished call() {
       BlockMovementStatus status = moveBlock();
-      return new BlockMovementAttemptFinished(block, source, target, status);
+      return new BlockMovementAttemptFinished(blkMovingInfo.getBlock(),
+          blkMovingInfo.getSource(), blkMovingInfo.getTarget(), status);
     }
 
     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;
+      datanode.incrementXmitsInProgress();
+      ExtendedBlock eb = new ExtendedBlock(blockPoolID,
+          blkMovingInfo.getBlock());
       try {
-        datanode.incrementXmitsInProgress();
-
-        ExtendedBlock extendedBlock = new ExtendedBlock(blockPoolID, block);
-        DNConf dnConf = datanode.getDnConf();
-
-        String dnAddr = datanode.getDatanodeId()
-            .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(
-            extendedBlock, EnumSet.of(BlockTokenIdentifier.AccessMode.WRITE),
-            new StorageType[]{targetStorageType}, new String[0]);
-
+            eb, EnumSet.of(BlockTokenIdentifier.AccessMode.WRITE),
+            new StorageType[]{blkMovingInfo.getTargetStorageType()},
+            new String[0]);
         DataEncryptionKeyFactory keyFactory = datanode
-            .getDataEncryptionKeyFactoryForBlock(extendedBlock);
-        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, extendedBlock, accessToken, source, targetStorageType);
-        receiveResponse(in);
+            .getDataEncryptionKeyFactoryForBlock(eb);
 
-        LOG.info(
-            "Successfully moved block:{} from src:{} to destin:{} for"
-                + " 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;
+        return blkDispatcher.moveBlock(blkMovingInfo,
+            datanode.getSaslClient(), eb, datanode.newSocket(),
+            keyFactory, accessToken);
       } 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);
+            blkMovingInfo.getBlock(), blkMovingInfo.getSource(),
+            blkMovingInfo.getTarget(), blkMovingInfo.getTargetStorageType(), e);
         return BlockMovementStatus.DN_BLK_STORAGE_MOVEMENT_FAILURE;
       } finally {
         datanode.decrementXmitsInProgress();
-        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, null);
-    }
-
-    /** 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, true);
-    }
-  }
-
-  /**
-   * Block movement status code.
-   */
-  public 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;
-
-    BlockMovementStatus(int code) {
-      this.code = code;
-    }
-
-    /**
-     * @return the status code.
-     */
-    int getStatusCode() {
-      return code;
-    }
-  }
-
-  /**
-   * 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 BlockMovementAttemptFinished {
-    private final Block block;
-    private final DatanodeInfo src;
-    private final DatanodeInfo target;
-    private final BlockMovementStatus status;
-
-    BlockMovementAttemptFinished(Block block, DatanodeInfo src,
-        DatanodeInfo target, BlockMovementStatus status) {
-      this.block = block;
-      this.src = src;
-      this.target = target;
-      this.status = status;
-    }
-
-    Block getBlock() {
-      return block;
-    }
-
-    BlockMovementStatus getStatus() {
-      return status;
-    }
-
-    @Override
-    public String toString() {
-      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 block movements and it will send these attempted finished(with
-   * success or failure) blocks to the namenode via heartbeat.
-   */
-  public static class BlocksMovementsStatusHandler {
-    private final List<Block> blockIdVsMovementStatus =
-        new ArrayList<>();
-
-    /**
-     * Collect all the storage movement attempt finished blocks. Later this will
-     * be send to namenode via heart beat.
-     *
-     * @param moveAttemptFinishedBlks
-     *          set of storage movement attempt finished blocks
-     */
-    void handle(List<BlockMovementAttemptFinished> moveAttemptFinishedBlks) {
-      List<Block> blocks = new ArrayList<>();
-
-      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 (blockIdVsMovementStatus) {
-        blockIdVsMovementStatus.addAll(blocks);
-      }
-    }
-
-    /**
-     * @return unmodifiable list of storage movement attempt finished blocks.
-     */
-    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);
-        }
-      }
-      return moveAttemptFinishedBlks;
-    }
-
-    /**
-     * Remove the storage movement attempt finished blocks from the tracking
-     * list.
-     *
-     * @param moveAttemptFinishedBlks
-     *          set of storage movement attempt finished blocks
-     */
-    void remove(List<Block> moveAttemptFinishedBlks) {
-      if (moveAttemptFinishedBlks != null) {
-        blockIdVsMovementStatus.removeAll(moveAttemptFinishedBlks);
-      }
-    }
-
-    /**
-     * Clear the blockID vs movement status tracking map.
-     */
-    void removeAll() {
-      synchronized (blockIdVsMovementStatus) {
-        blockIdVsMovementStatus.clear();
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b0cb8d9b/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 ed1c823..37322e7 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
@@ -1299,7 +1299,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
               blockManager.getSPSService()),
           new IntraSPSNameNodeFileIdCollector(getFSDirectory(),
               blockManager.getSPSService()),
-          new IntraSPSNameNodeBlockMoveTaskHandler(blockManager, this));
+          new IntraSPSNameNodeBlockMoveTaskHandler(blockManager, this), null);
       blockManager.startSPS();
     } finally {
       startingActiveService = false;
@@ -3996,7 +3996,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
                     + "  movement attempt finished block info sent by DN");
           }
         } else {
-          sps.handleStorageMovementAttemptFinishedBlks(blksMovementsFinished);
+          sps.notifyStorageMovementAttemptFinishedBlks(blksMovementsFinished);
         }
       }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b0cb8d9b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/BlockMoveTaskHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/BlockMoveTaskHandler.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/BlockMoveTaskHandler.java
index e6f78e1..1b11d01 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/BlockMoveTaskHandler.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/BlockMoveTaskHandler.java
@@ -38,7 +38,6 @@ public interface BlockMoveTaskHandler {
    * contain the required info to move the block, that source location,
    * destination location and storage types.
    */
-  void submitMoveTask(BlockMovingInfo blkMovingInfo,
-      BlockMovementListener blockMoveCompletionListener) throws IOException;
+  void submitMoveTask(BlockMovingInfo blkMovingInfo) throws IOException;
 
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b0cb8d9b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/BlockStorageMovementAttemptedItems.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/BlockStorageMovementAttemptedItems.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/BlockStorageMovementAttemptedItems.java
index 3f0155d..ea7a093 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/BlockStorageMovementAttemptedItems.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/BlockStorageMovementAttemptedItems.java
@@ -46,8 +46,7 @@ import com.google.common.annotations.VisibleForTesting;
  * finished for a longer time period, then such items will retries automatically
  * after timeout. The default timeout would be 5 minutes.
  */
-public class BlockStorageMovementAttemptedItems
-    implements BlockMovementListener {
+public class BlockStorageMovementAttemptedItems{
   private static final Logger LOG =
       LoggerFactory.getLogger(BlockStorageMovementAttemptedItems.class);
 
@@ -59,6 +58,7 @@ public class BlockStorageMovementAttemptedItems
   private final List<Block> movementFinishedBlocks;
   private volatile boolean monitorRunning = true;
   private Daemon timerThread = null;
+  private final BlockMovementListener blkMovementListener;
   //
   // It might take anywhere between 5 to 10 minutes before
   // a request is timed out.
@@ -74,7 +74,8 @@ public class BlockStorageMovementAttemptedItems
   private final SPSService service;
 
   public BlockStorageMovementAttemptedItems(SPSService service,
-      BlockStorageMovementNeeded unsatisfiedStorageMovementFiles) {
+      BlockStorageMovementNeeded unsatisfiedStorageMovementFiles,
+      BlockMovementListener blockMovementListener) {
     this.service = service;
     long recheckTimeout = this.service.getConf().getLong(
         DFS_STORAGE_POLICY_SATISFIER_RECHECK_TIMEOUT_MILLIS_KEY,
@@ -89,6 +90,7 @@ public class BlockStorageMovementAttemptedItems
     this.blockStorageMovementNeeded = unsatisfiedStorageMovementFiles;
     storageMovementAttemptedItems = new ArrayList<>();
     movementFinishedBlocks = new ArrayList<>();
+    this.blkMovementListener = blockMovementListener;
   }
 
   /**
@@ -118,6 +120,10 @@ public class BlockStorageMovementAttemptedItems
     synchronized (movementFinishedBlocks) {
       movementFinishedBlocks.addAll(Arrays.asList(moveAttemptFinishedBlks));
     }
+    // External listener if it is plugged-in
+    if (blkMovementListener != null) {
+      blkMovementListener.notifyMovementTriedBlocks(moveAttemptFinishedBlks);
+    }
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b0cb8d9b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/IntraSPSNameNodeBlockMoveTaskHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/IntraSPSNameNodeBlockMoveTaskHandler.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/IntraSPSNameNodeBlockMoveTaskHandler.java
index b27e8c9..d6e92d2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/IntraSPSNameNodeBlockMoveTaskHandler.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/IntraSPSNameNodeBlockMoveTaskHandler.java
@@ -44,8 +44,7 @@ public class IntraSPSNameNodeBlockMoveTaskHandler
   }
 
   @Override
-  public void submitMoveTask(BlockMovingInfo blkMovingInfo,
-      BlockMovementListener blockMoveCompletionListener) throws IOException {
+  public void submitMoveTask(BlockMovingInfo blkMovingInfo) throws IOException {
     namesystem.readLock();
     try {
       DatanodeDescriptor dn = blockManager.getDatanodeManager()

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b0cb8d9b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/SPSService.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/SPSService.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/SPSService.java
index d74e391..ecc6ceb 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/SPSService.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/SPSService.java
@@ -22,6 +22,7 @@ import java.util.List;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.server.protocol.BlocksStorageMoveAttemptFinished;
 
 /**
  * An interface for SPSService, which exposes life cycle and processing APIs.
@@ -41,9 +42,11 @@ public interface SPSService {
    *          id
    * @param handler
    *          - a helper service for moving the blocks
+   * @param blkMovementListener
+   *          - listener to know about block movement attempt completion
    */
   void init(Context ctxt, FileIdCollector fileIDCollector,
-      BlockMoveTaskHandler handler);
+      BlockMoveTaskHandler handler, BlockMovementListener blkMovementListener);
 
   /**
    * Starts the SPS service. Make sure to initialize the helper services before
@@ -112,4 +115,13 @@ public interface SPSService {
    *          - directory inode id.
    */
   void markScanCompletedForPath(Long inodeId);
+
+  /**
+   * Notify the details of storage movement attempt finished blocks.
+   *
+   * @param moveAttemptFinishedBlks
+   *          - array contains all the blocks that are attempted to move
+   */
+  void notifyStorageMovementAttemptFinishedBlks(
+      BlocksStorageMoveAttemptFinished moveAttemptFinishedBlks);
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b0cb8d9b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/StoragePolicySatisfier.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/StoragePolicySatisfier.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/StoragePolicySatisfier.java
index aafdc65..9ba8af7 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/StoragePolicySatisfier.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/StoragePolicySatisfier.java
@@ -132,13 +132,14 @@ public class StoragePolicySatisfier implements SPSService, Runnable {
   }
 
   public void init(final Context context, final FileIdCollector fileIDCollector,
-      final BlockMoveTaskHandler blockMovementTaskHandler) {
+      final BlockMoveTaskHandler blockMovementTaskHandler,
+      final BlockMovementListener blockMovementListener) {
     this.ctxt = context;
     this.storageMovementNeeded =
         new BlockStorageMovementNeeded(context, fileIDCollector);
     this.storageMovementsMonitor =
         new BlockStorageMovementAttemptedItems(this,
-        storageMovementNeeded);
+        storageMovementNeeded, blockMovementListener);
     this.blockMoveTaskHandler = blockMovementTaskHandler;
     this.spsWorkMultiplier = DFSUtil.getSPSWorkMultiplier(getConf());
     this.blockMovementMaxRetry = getConf().getInt(
@@ -291,6 +292,7 @@ public class StoragePolicySatisfier implements SPSService, Runnable {
                       + " back to retry queue as some of the blocks"
                       + " are low redundant.");
                 }
+                itemInfo.increRetryCount();
                 this.storageMovementNeeded.add(itemInfo);
                 break;
               case BLOCKS_FAILED_TO_MOVE:
@@ -410,15 +412,18 @@ public class StoragePolicySatisfier implements SPSService, Runnable {
             liveDns, ecPolicy);
         if (blocksPaired) {
           status = BlocksMovingAnalysis.Status.BLOCKS_TARGETS_PAIRED;
-        } else {
-          // none of the blocks found its eligible targets for satisfying the
-          // storage policy.
+        } else
+          if (status != BlocksMovingAnalysis.Status.BLOCKS_TARGETS_PAIRED) {
+          // Check if the previous block was successfully paired. Here the
+          // status will set to NO_BLOCKS_TARGETS_PAIRED only when none of the
+          // blocks of a file found its eligible targets to satisfy the storage
+          // policy.
           status = BlocksMovingAnalysis.Status.NO_BLOCKS_TARGETS_PAIRED;
         }
-      } else {
-        if (hasLowRedundancyBlocks) {
-          status = BlocksMovingAnalysis.Status.FEW_LOW_REDUNDANCY_BLOCKS;
-        }
+      } else if (hasLowRedundancyBlocks
+          && status != BlocksMovingAnalysis.Status.BLOCKS_TARGETS_PAIRED) {
+        // Check if the previous block was successfully paired.
+        status = BlocksMovingAnalysis.Status.FEW_LOW_REDUNDANCY_BLOCKS;
       }
     }
 
@@ -426,8 +431,7 @@ public class StoragePolicySatisfier implements SPSService, Runnable {
     for (BlockMovingInfo blkMovingInfo : blockMovingInfos) {
       // Check for at least one block storage movement has been chosen
       try {
-        blockMoveTaskHandler.submitMoveTask(blkMovingInfo,
-            storageMovementsMonitor);
+        blockMoveTaskHandler.submitMoveTask(blkMovingInfo);
         LOG.debug("BlockMovingInfo: {}", blkMovingInfo);
         assignedBlockIds.add(blkMovingInfo.getBlock());
         blockCount++;
@@ -823,7 +827,7 @@ public class StoragePolicySatisfier implements SPSService, Runnable {
    * @param moveAttemptFinishedBlks
    *          set of storage movement attempt finished blocks.
    */
-  public void handleStorageMovementAttemptFinishedBlks(
+  public void notifyStorageMovementAttemptFinishedBlks(
       BlocksStorageMoveAttemptFinished moveAttemptFinishedBlks) {
     if (moveAttemptFinishedBlks.getBlocks().length <= 0) {
       return;
@@ -833,7 +837,7 @@ public class StoragePolicySatisfier implements SPSService, Runnable {
   }
 
   @VisibleForTesting
-  BlockMovementListener getAttemptedItemsMonitor() {
+  BlockStorageMovementAttemptedItems getAttemptedItemsMonitor() {
     return storageMovementsMonitor;
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b0cb8d9b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/sps/ExternalSPSBlockMoveTaskHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/sps/ExternalSPSBlockMoveTaskHandler.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/sps/ExternalSPSBlockMoveTaskHandler.java
new file mode 100644
index 0000000..a1c8eec
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/sps/ExternalSPSBlockMoveTaskHandler.java
@@ -0,0 +1,233 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hdfs.server.sps;
+
+import java.io.IOException;
+import java.net.Socket;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.Callable;
+import java.util.concurrent.CompletionService;
+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.classification.InterfaceStability;
+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.client.HdfsClientConfigKeys;
+import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.protocol.datatransfer.TrustedChannelResolver;
+import org.apache.hadoop.hdfs.protocol.datatransfer.sasl.DataTransferSaslUtil;
+import org.apache.hadoop.hdfs.protocol.datatransfer.sasl.SaslDataTransferClient;
+import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
+import org.apache.hadoop.hdfs.server.balancer.KeyManager;
+import org.apache.hadoop.hdfs.server.balancer.NameNodeConnector;
+import org.apache.hadoop.hdfs.server.common.sps.BlockMovementAttemptFinished;
+import org.apache.hadoop.hdfs.server.common.sps.BlockMovementStatus;
+import org.apache.hadoop.hdfs.server.common.sps.BlockStorageMovementTracker;
+import org.apache.hadoop.hdfs.server.common.sps.BlocksMovementsStatusHandler;
+import org.apache.hadoop.hdfs.server.common.sps.BlockDispatcher;
+import org.apache.hadoop.hdfs.server.namenode.sps.BlockMoveTaskHandler;
+import org.apache.hadoop.hdfs.server.namenode.sps.SPSService;
+import org.apache.hadoop.hdfs.server.protocol.BlockStorageMovementCommand.BlockMovingInfo;
+import org.apache.hadoop.hdfs.server.protocol.BlocksStorageMoveAttemptFinished;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.util.Daemon;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * This class handles the external SPS block movements. This will move the
+ * given block to a target datanode by directly establishing socket connection
+ * to it and invokes function
+ * {@link Sender#replaceBlock(ExtendedBlock, StorageType, Token, String,
+ * DatanodeInfo, String)}.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class ExternalSPSBlockMoveTaskHandler implements BlockMoveTaskHandler {
+  private static final Logger LOG = LoggerFactory
+      .getLogger(ExternalSPSBlockMoveTaskHandler.class);
+
+  private final ExecutorService moveExecutor;
+  private final CompletionService<BlockMovementAttemptFinished> mCompletionServ;
+  private final NameNodeConnector nnc;
+  private final SaslDataTransferClient saslClient;
+  private final BlockStorageMovementTracker blkMovementTracker;
+  private Daemon movementTrackerThread;
+  private final SPSService service;
+  private final BlockDispatcher blkDispatcher;
+
+  public ExternalSPSBlockMoveTaskHandler(Configuration conf,
+      NameNodeConnector nnc, SPSService spsService) {
+    int moverThreads = conf.getInt(DFSConfigKeys.DFS_MOVER_MOVERTHREADS_KEY,
+        DFSConfigKeys.DFS_MOVER_MOVERTHREADS_DEFAULT);
+    moveExecutor = initializeBlockMoverThreadPool(moverThreads);
+    mCompletionServ = new ExecutorCompletionService<>(moveExecutor);
+    this.nnc = nnc;
+    this.saslClient = new SaslDataTransferClient(conf,
+        DataTransferSaslUtil.getSaslPropertiesResolver(conf),
+        TrustedChannelResolver.getInstance(conf),
+        nnc.getFallbackToSimpleAuth());
+    this.blkMovementTracker = new BlockStorageMovementTracker(
+        mCompletionServ, new ExternalBlocksMovementsStatusHandler());
+    this.service = spsService;
+
+    boolean connectToDnViaHostname = conf.getBoolean(
+        HdfsClientConfigKeys.DFS_CLIENT_USE_DN_HOSTNAME,
+        HdfsClientConfigKeys.DFS_CLIENT_USE_DN_HOSTNAME_DEFAULT);
+    int ioFileBufferSize = DFSUtilClient.getIoFileBufferSize(conf);
+    blkDispatcher = new BlockDispatcher(HdfsConstants.READ_TIMEOUT,
+        ioFileBufferSize, connectToDnViaHostname);
+  }
+
+  /**
+   * Initializes block movement tracker daemon and starts the thread.
+   */
+  void init() {
+    movementTrackerThread = new Daemon(this.blkMovementTracker);
+    movementTrackerThread.setName("BlockStorageMovementTracker");
+    movementTrackerThread.start();
+  }
+
+  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;
+  }
+
+  @Override
+  public void submitMoveTask(BlockMovingInfo blkMovingInfo) throws IOException {
+    // TODO: Need to increment scheduled block size on the target node. This
+    // count will be used to calculate the remaining space of target datanode
+    // during block movement assignment logic. In the internal movement,
+    // remaining space is bookkeeping at the DatanodeDescriptor, please refer
+    // IntraSPSNameNodeBlockMoveTaskHandler#submitMoveTask implementation and
+    // updating via the funcation call -
+    // dn.incrementBlocksScheduled(blkMovingInfo.getTargetStorageType());
+    LOG.debug("Received BlockMovingTask {}", blkMovingInfo);
+    BlockMovingTask blockMovingTask = new BlockMovingTask(blkMovingInfo);
+    Future<BlockMovementAttemptFinished> moveCallable = mCompletionServ
+        .submit(blockMovingTask);
+    blkMovementTracker.addBlock(blkMovingInfo.getBlock(), moveCallable);
+  }
+
+  private class ExternalBlocksMovementsStatusHandler
+      extends BlocksMovementsStatusHandler {
+    @Override
+    public void handle(
+        List<BlockMovementAttemptFinished> moveAttemptFinishedBlks) {
+      List<Block> blocks = new ArrayList<>();
+      for (BlockMovementAttemptFinished item : moveAttemptFinishedBlks) {
+        blocks.add(item.getBlock());
+      }
+      BlocksStorageMoveAttemptFinished blkAttempted =
+          new BlocksStorageMoveAttemptFinished(
+          blocks.toArray(new Block[blocks.size()]));
+      service.notifyStorageMovementAttemptFinishedBlks(blkAttempted);
+    }
+  }
+
+  /**
+   * This class encapsulates the process of moving the block replica to the
+   * given target.
+   */
+  private class BlockMovingTask
+      implements Callable<BlockMovementAttemptFinished> {
+    private final BlockMovingInfo blkMovingInfo;
+
+    BlockMovingTask(BlockMovingInfo blkMovingInfo) {
+      this.blkMovingInfo = blkMovingInfo;
+    }
+
+    @Override
+    public BlockMovementAttemptFinished call() {
+      BlockMovementStatus blkMovementStatus = moveBlock();
+      return new BlockMovementAttemptFinished(blkMovingInfo.getBlock(),
+          blkMovingInfo.getSource(), blkMovingInfo.getTarget(),
+          blkMovementStatus);
+    }
+
+    private BlockMovementStatus moveBlock() {
+      ExtendedBlock eb = new ExtendedBlock(nnc.getBlockpoolID(),
+          blkMovingInfo.getBlock());
+
+      final KeyManager km = nnc.getKeyManager();
+      Token<BlockTokenIdentifier> accessToken;
+      try {
+        accessToken = km.getAccessToken(eb,
+            new StorageType[]{blkMovingInfo.getTargetStorageType()},
+            new String[0]);
+      } catch (IOException e) {
+        // TODO: handle failure retries
+        LOG.warn(
+            "Failed to move block:{} from src:{} to destin:{} to satisfy "
+                + "storageType:{}",
+            blkMovingInfo.getBlock(), blkMovingInfo.getSource(),
+            blkMovingInfo.getTarget(), blkMovingInfo.getTargetStorageType(), e);
+        return BlockMovementStatus.DN_BLK_STORAGE_MOVEMENT_FAILURE;
+      }
+      return blkDispatcher.moveBlock(blkMovingInfo, saslClient, eb,
+          new Socket(), km, accessToken);
+    }
+  }
+
+  /**
+   * Cleanup the resources.
+   */
+  void cleanUp() {
+    blkMovementTracker.stopTracking();
+    if (movementTrackerThread != null) {
+      movementTrackerThread.interrupt();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b0cb8d9b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/sps/TestBlockStorageMovementAttemptedItems.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/sps/TestBlockStorageMovementAttemptedItems.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/sps/TestBlockStorageMovementAttemptedItems.java
index 3e2c324..4097339 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/sps/TestBlockStorageMovementAttemptedItems.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/sps/TestBlockStorageMovementAttemptedItems.java
@@ -56,7 +56,7 @@ public class TestBlockStorageMovementAttemptedItems {
     unsatisfiedStorageMovementFiles =
         new BlockStorageMovementNeeded(ctxt, null);
     bsmAttemptedItems = new BlockStorageMovementAttemptedItems(sps,
-        unsatisfiedStorageMovementFiles);
+        unsatisfiedStorageMovementFiles, null);
   }
 
   @After

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b0cb8d9b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/sps/TestStoragePolicySatisfier.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/sps/TestStoragePolicySatisfier.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/sps/TestStoragePolicySatisfier.java
index e0bf410..8115661 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/sps/TestStoragePolicySatisfier.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/sps/TestStoragePolicySatisfier.java
@@ -92,7 +92,7 @@ public class TestStoragePolicySatisfier {
 
   private static final String ONE_SSD = "ONE_SSD";
   private static final String COLD = "COLD";
-  private static final Logger LOG =
+  protected static final Logger LOG =
       LoggerFactory.getLogger(TestStoragePolicySatisfier.class);
   private Configuration config = null;
   private StorageType[][] allDiskTypes =
@@ -1337,7 +1337,7 @@ public class TestStoragePolicySatisfier {
     };
 
     FileIdCollector fileIDCollector = createFileIdCollector(sps, ctxt);
-    sps.init(ctxt, fileIDCollector, null);
+    sps.init(ctxt, fileIDCollector, null, null);
     sps.getStorageMovementQueue().activate();
 
     INode rootINode = fsDir.getINode("/root");
@@ -1404,7 +1404,7 @@ public class TestStoragePolicySatisfier {
       }
     };
     FileIdCollector fileIDCollector = createFileIdCollector(sps, ctxt);
-    sps.init(ctxt, fileIDCollector, null);
+    sps.init(ctxt, fileIDCollector, null, null);
     sps.getStorageMovementQueue().activate();
 
     INode rootINode = fsDir.getINode("/root");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b0cb8d9b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/sps/TestExternalStoragePolicySatisfier.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/sps/TestExternalStoragePolicySatisfier.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/sps/TestExternalStoragePolicySatisfier.java
index 3ced34e..9a401bd 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/sps/TestExternalStoragePolicySatisfier.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/sps/TestExternalStoragePolicySatisfier.java
@@ -18,20 +18,33 @@
 package org.apache.hadoop.hdfs.server.sps;
 
 import java.io.IOException;
+import java.net.URI;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
 
 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.DFSUtil;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.server.balancer.NameNodeConnector;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
+import org.apache.hadoop.hdfs.server.namenode.sps.BlockMovementListener;
 import org.apache.hadoop.hdfs.server.namenode.sps.Context;
 import org.apache.hadoop.hdfs.server.namenode.sps.FileIdCollector;
-import org.apache.hadoop.hdfs.server.namenode.sps.IntraSPSNameNodeBlockMoveTaskHandler;
 import org.apache.hadoop.hdfs.server.namenode.sps.IntraSPSNameNodeContext;
 import org.apache.hadoop.hdfs.server.namenode.sps.SPSService;
 import org.apache.hadoop.hdfs.server.namenode.sps.StoragePolicySatisfier;
 import org.apache.hadoop.hdfs.server.namenode.sps.TestStoragePolicySatisfier;
+import org.junit.Assert;
 import org.junit.Ignore;
 
+import com.google.common.collect.Maps;
+
 /**
  * Tests the external sps service plugins.
  */
@@ -69,23 +82,24 @@ public class TestExternalStoragePolicySatisfier
     cluster.waitActive();
     if (conf.getBoolean(DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_ENABLED_KEY,
         false)) {
-      SPSService spsService = cluster.getNameNode().getNamesystem()
-          .getBlockManager().getSPSService();
+      BlockManager blkMgr = cluster.getNameNode().getNamesystem()
+          .getBlockManager();
+      SPSService spsService = blkMgr.getSPSService();
       spsService.stopGracefully();
 
       IntraSPSNameNodeContext context = new IntraSPSNameNodeContext(
           cluster.getNameNode().getNamesystem(),
-          cluster.getNameNode().getNamesystem().getBlockManager(), cluster
-              .getNameNode().getNamesystem().getBlockManager().getSPSService());
-
+          blkMgr, blkMgr.getSPSService());
+      ExternalBlockMovementListener blkMoveListener =
+          new ExternalBlockMovementListener();
+      ExternalSPSBlockMoveTaskHandler externalHandler =
+          new ExternalSPSBlockMoveTaskHandler(conf, getNameNodeConnector(conf),
+              blkMgr.getSPSService());
+      externalHandler.init();
       spsService.init(context,
-          new ExternalSPSFileIDCollector(context,
-              cluster.getNameNode().getNamesystem().getBlockManager()
-                  .getSPSService(),
-              5),
-          new IntraSPSNameNodeBlockMoveTaskHandler(
-              cluster.getNameNode().getNamesystem().getBlockManager(),
-              cluster.getNameNode().getNamesystem()));
+          new ExternalSPSFileIDCollector(context, blkMgr.getSPSService(), 5),
+          externalHandler,
+          blkMoveListener);
       spsService.start(true);
     }
     return cluster;
@@ -97,6 +111,35 @@ public class TestExternalStoragePolicySatisfier
     return new ExternalSPSFileIDCollector(ctxt, sps, 5);
   }
 
+  private class ExternalBlockMovementListener implements BlockMovementListener {
+
+    private List<Block> actualBlockMovements = new ArrayList<>();
+
+    @Override
+    public void notifyMovementTriedBlocks(Block[] moveAttemptFinishedBlks) {
+      for (Block block : moveAttemptFinishedBlks) {
+        actualBlockMovements.add(block);
+      }
+      LOG.info("Movement attempted blocks", actualBlockMovements);
+    }
+  }
+
+  private NameNodeConnector getNameNodeConnector(Configuration conf)
+      throws IOException {
+    final Collection<URI> namenodes = DFSUtil.getInternalNsRpcUris(conf);
+    Assert.assertEquals(1, namenodes.size());
+    Map<URI, List<Path>> nnMap = Maps.newHashMap();
+    for (URI nn : namenodes) {
+      nnMap.put(nn, null);
+    }
+    final Path externalSPSPathId = new Path("/system/externalSPS.id");
+    final List<NameNodeConnector> nncs = NameNodeConnector
+        .newNameNodeConnectors(nnMap,
+            StoragePolicySatisfier.class.getSimpleName(), externalSPSPathId,
+            conf, NameNodeConnector.DEFAULT_MAX_IDLE_ITERATIONS);
+    return nncs.get(0);
+  }
+
   /**
    * This test need not run as external scan is not a batch based scanning right
    * now.


---------------------------------------------------------------------
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-12152: [SPS]: Re-arrange StoragePolicySatisfyWorker stopping sequence to improve thread cleanup time. Contributed by Rakesh R.

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

Branch: refs/heads/HDFS-10285
Commit: 3b601f2c0e16b84e35ebe5ecdcd06d3277eabb74
Parents: 4bcf61c
Author: Uma Maheswara Rao G <um...@intel.com>
Authored: Wed Jul 19 00:55:26 2017 -0700
Committer: Uma Maheswara Rao Gangumalla <um...@apache.org>
Committed: Sun Aug 12 03:06:00 2018 -0700

----------------------------------------------------------------------
 .../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/3b601f2c/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/3b601f2c/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


[24/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 um...@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/68017e33
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/68017e33
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/68017e33

Branch: refs/heads/HDFS-10285
Commit: 68017e3349e3b71a9c49f2ccea2558231ff8485d
Parents: 5780f06
Author: Rakesh Radhakrishnan <ra...@apache.org>
Authored: Fri Nov 3 08:18:14 2017 +0530
Committer: Uma Maheswara Rao Gangumalla <um...@apache.org>
Committed: Sun Aug 12 03:06:02 2018 -0700

----------------------------------------------------------------------
 .../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/68017e33/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 7337aa2..471ab2c 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.HdfsLocatedFileStatus;
@@ -3169,4 +3170,25 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
     checkOpen();
     return new OpenFilesIterator(namenode, tracer, openFilesTypes, path);
   }
+
+  /**
+   * 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/68017e33/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 81d7c91..360fd63 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
@@ -45,6 +45,7 @@ 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.OpenFilesIterator.OpenFilesType;
+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;
@@ -1764,4 +1765,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/68017e33/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 74efcd2..190a1c6 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
@@ -128,6 +128,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/68017e33/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 d7c32bc..cdc8eac 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.HdfsLocatedFileStatus;
 import org.apache.hadoop.hdfs.protocol.LastBlockWithStatus;
@@ -100,6 +101,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;
@@ -241,6 +244,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;
 
 /**
@@ -1973,4 +1977,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/68017e33/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 4a5a493..9281bff 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
@@ -130,6 +130,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;
@@ -3398,4 +3399,36 @@ public class PBHelperClient {
     }
     return typeProtos;
   }
+
+  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/68017e33/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 519bb01..1de13ca 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
@@ -481,7 +481,6 @@ message RollingUpgradeInfoProto {
 message RollingUpgradeResponseProto {
   optional RollingUpgradeInfoProto rollingUpgradeInfo= 1;
 }
-
 message ListCorruptFileBlocksRequestProto {
   required string path = 1;
   optional string cookie = 2;
@@ -846,6 +845,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);
@@ -1036,4 +1049,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/68017e33/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 f338d4e..09f7ce2 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;
@@ -85,6 +86,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;
@@ -257,7 +260,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;
@@ -1922,4 +1925,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/68017e33/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 b53d946..0957fe2 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;
@@ -5103,4 +5105,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/68017e33/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/68017e33/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 2f621e6..4738bf5 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;
@@ -111,6 +110,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;
@@ -2542,4 +2542,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/68017e33/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/68017e33/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/68017e33/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 5defbd0..cf17e99 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/ArchivalStorage.md
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/ArchivalStorage.md
@@ -217,13 +217,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/68017e33/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/68017e33/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/68017e33/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


[32/50] [abbrv] hadoop git commit: HDFS-13097: [SPS]: Fix the branch review comments(Part1). Contributed by Surendra Singh.

Posted by um...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/4402f3f8/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 85a101f..47ea39f 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
@@ -252,8 +252,8 @@ public class TestNameNodeReconfigure {
     // 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());
+            nameNode.getNamesystem().getBlockManager().getSPSManager()
+            .isInternalSatisfierRunning());
     verifySPSEnabled(nameNode, DFS_STORAGE_POLICY_SATISFIER_MODE_KEY,
         StoragePolicySatisfierMode.INTERNAL, false);
 
@@ -280,8 +280,8 @@ public class TestNameNodeReconfigure {
       fail("ReconfigurationException expected");
     } catch (ReconfigurationException e) {
       GenericTestUtils.assertExceptionContains(
-          "For enabling or disabling storage policy satisfier, we must "
-              + "pass either none/internal/external string value only",
+          "For enabling or disabling storage policy satisfier, must "
+              + "pass either internal/external/none string value only",
           e.getCause());
     }
 
@@ -301,8 +301,8 @@ public class TestNameNodeReconfigure {
     nameNode.reconfigureProperty(DFS_STORAGE_POLICY_SATISFIER_MODE_KEY,
         StoragePolicySatisfierMode.EXTERNAL.toString());
     assertEquals(DFS_STORAGE_POLICY_SATISFIER_MODE_KEY + " has wrong value",
-        false, nameNode.getNamesystem().getBlockManager()
-            .isStoragePolicySatisfierRunning());
+        false, nameNode.getNamesystem().getBlockManager().getSPSManager()
+            .isInternalSatisfierRunning());
     assertEquals(DFS_STORAGE_POLICY_SATISFIER_MODE_KEY + " has wrong value",
         StoragePolicySatisfierMode.EXTERNAL.toString(),
         nameNode.getConf().get(DFS_STORAGE_POLICY_SATISFIER_MODE_KEY,
@@ -342,8 +342,8 @@ public class TestNameNodeReconfigure {
     nameNode.reconfigureProperty(DFS_STORAGE_POLICY_SATISFIER_MODE_KEY,
         StoragePolicySatisfierMode.INTERNAL.toString());
     assertEquals(DFS_STORAGE_POLICY_SATISFIER_MODE_KEY + " has wrong value",
-        true, nameNode.getNamesystem().getBlockManager()
-            .isStoragePolicySatisfierRunning());
+        true, nameNode.getNamesystem().getBlockManager().getSPSManager()
+            .isInternalSatisfierRunning());
     assertEquals(DFS_STORAGE_POLICY_SATISFIER_MODE_KEY + " has wrong value",
         StoragePolicySatisfierMode.INTERNAL.toString(),
         nameNode.getConf().get(DFS_STORAGE_POLICY_SATISFIER_MODE_KEY,
@@ -353,7 +353,8 @@ public class TestNameNodeReconfigure {
   void verifySPSEnabled(final NameNode nameNode, String property,
       StoragePolicySatisfierMode expected, boolean isSatisfierRunning) {
     assertEquals(property + " has wrong value", isSatisfierRunning, nameNode
-        .getNamesystem().getBlockManager().isStoragePolicySatisfierRunning());
+        .getNamesystem().getBlockManager().getSPSManager()
+        .isInternalSatisfierRunning());
     String actual = nameNode.getConf().get(property,
         DFS_STORAGE_POLICY_SATISFIER_MODE_DEFAULT);
     assertEquals(property + " has wrong value", expected,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4402f3f8/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 b84214c..9f98777 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
@@ -389,7 +389,8 @@ public class TestPersistentStoragePolicySatisfier {
       fs.setStoragePolicy(testFile, ONE_SSD);
       fs.satisfyStoragePolicy(testFile);
 
-      cluster.getNamesystem().getBlockManager().disableSPS();
+      cluster.getNamesystem().getBlockManager().getSPSManager()
+          .changeModeEvent(StoragePolicySatisfierMode.NONE);
 
       // Make sure satisfy xattr has been removed.
       DFSTestUtil.waitForXattrRemoved(testFileName,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4402f3f8/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 e89cfa3..b3734d1 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
@@ -103,7 +103,7 @@ public class TestStoragePolicySatisfierWithHA {
       dfs = cluster.getFileSystem(1);
 
       try {
-        dfs.getClient().isStoragePolicySatisfierRunning();
+        dfs.getClient().isInternalSatisfierRunning();
         Assert.fail("Call this function to Standby NN should "
             + "raise an exception.");
       } catch (RemoteException e) {
@@ -115,14 +115,14 @@ public class TestStoragePolicySatisfierWithHA {
 
       cluster.transitionToActive(0);
       dfs = cluster.getFileSystem(0);
-      running = dfs.getClient().isStoragePolicySatisfierRunning();
+      running = dfs.getClient().isInternalSatisfierRunning();
       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 {
-        dfs.getClient().isStoragePolicySatisfierRunning();
+        dfs.getClient().isInternalSatisfierRunning();
         Assert.fail("NN in Standby again, call this function should "
             + "raise an exception.");
       } catch (RemoteException e) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4402f3f8/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/sps/TestStoragePolicySatisfier.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/sps/TestStoragePolicySatisfier.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/sps/TestStoragePolicySatisfier.java
index 9e0a39f..6f7fe89 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/sps/TestStoragePolicySatisfier.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/sps/TestStoragePolicySatisfier.java
@@ -445,15 +445,10 @@ public class TestStoragePolicySatisfier {
       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);
+      } catch (Exception e) {
+        Assert.fail(String.format("Allow to invoke mutlipe times "
+            + "#satisfyStoragePolicy() api for a path %s , internally just "
+            + "skipping addtion to satisfy movement queue.", FILE));
       }
     } finally {
       shutdownCluster();
@@ -563,7 +558,7 @@ public class TestStoragePolicySatisfier {
           DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_MODE_KEY,
           StoragePolicySatisfierMode.NONE.toString());
       running = hdfsCluster.getFileSystem()
-          .getClient().isStoragePolicySatisfierRunning();
+          .getClient().isInternalSatisfierRunning();
       Assert.assertFalse("SPS should stopped as configured.", running);
 
       // Simulate the case by creating MOVER_ID file
@@ -576,7 +571,7 @@ public class TestStoragePolicySatisfier {
           StoragePolicySatisfierMode.INTERNAL.toString());
 
       running = hdfsCluster.getFileSystem()
-          .getClient().isStoragePolicySatisfierRunning();
+          .getClient().isInternalSatisfierRunning();
       Assert.assertFalse("SPS should not be able to run as file "
           + HdfsServerConstants.MOVER_ID_PATH + " is being hold.", running);
 
@@ -591,7 +586,7 @@ public class TestStoragePolicySatisfier {
           DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_MODE_KEY,
           StoragePolicySatisfierMode.INTERNAL.toString());
       running = hdfsCluster.getFileSystem()
-          .getClient().isStoragePolicySatisfierRunning();
+          .getClient().isInternalSatisfierRunning();
       Assert.assertTrue("SPS should be running as "
           + "Mover already exited", running);
 
@@ -623,7 +618,7 @@ public class TestStoragePolicySatisfier {
           HdfsServerConstants.MOVER_ID_PATH, 0, (short) 1, 0);
       restartNamenode();
       boolean running = hdfsCluster.getFileSystem()
-          .getClient().isStoragePolicySatisfierRunning();
+          .getClient().isInternalSatisfierRunning();
       Assert.assertTrue("SPS should be running as "
           + "no Mover really running", running);
     } finally {
@@ -1293,8 +1288,8 @@ public class TestStoragePolicySatisfier {
     sps.getStorageMovementQueue().activate();
 
     INode rootINode = fsDir.getINode("/root");
-    hdfsCluster.getNamesystem().getBlockManager()
-        .addSPSPathId(rootINode.getId());
+    hdfsCluster.getNamesystem().getBlockManager().getSPSManager()
+        .addPathId(rootINode.getId());
 
     //Wait for thread to reach U.
     Thread.sleep(1000);
@@ -1360,8 +1355,8 @@ public class TestStoragePolicySatisfier {
     sps.getStorageMovementQueue().activate();
 
     INode rootINode = fsDir.getINode("/root");
-    hdfsCluster.getNamesystem().getBlockManager()
-        .addSPSPathId(rootINode.getId());
+    hdfsCluster.getNamesystem().getBlockManager().getSPSManager()
+        .addPathId(rootINode.getId());
 
     // Wait for thread to reach U.
     Thread.sleep(1000);
@@ -1704,7 +1699,8 @@ public class TestStoragePolicySatisfier {
   private void waitForAttemptedItems(long expectedBlkMovAttemptedCount,
       int timeout) throws TimeoutException, InterruptedException {
     BlockManager blockManager = hdfsCluster.getNamesystem().getBlockManager();
-    final StoragePolicySatisfier sps = blockManager.getStoragePolicySatisfier();
+    final StoragePolicySatisfier sps = (StoragePolicySatisfier) blockManager
+        .getSPSManager().getInternalSPSService();
     GenericTestUtils.waitFor(new Supplier<Boolean>() {
       @Override
       public Boolean get() {
@@ -1723,7 +1719,8 @@ public class TestStoragePolicySatisfier {
       long expectedMovementFinishedBlocksCount, int timeout)
           throws TimeoutException, InterruptedException {
     BlockManager blockManager = hdfsCluster.getNamesystem().getBlockManager();
-    final StoragePolicySatisfier sps = blockManager.getStoragePolicySatisfier();
+    final StoragePolicySatisfier sps = (StoragePolicySatisfier) blockManager
+        .getSPSManager().getInternalSPSService();
     GenericTestUtils.waitFor(new Supplier<Boolean>() {
       @Override
       public Boolean get() {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4402f3f8/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/sps/TestStoragePolicySatisfierWithStripedFile.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/sps/TestStoragePolicySatisfierWithStripedFile.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/sps/TestStoragePolicySatisfierWithStripedFile.java
index 2257608..ef12300 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/sps/TestStoragePolicySatisfierWithStripedFile.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/sps/TestStoragePolicySatisfierWithStripedFile.java
@@ -495,7 +495,8 @@ public class TestStoragePolicySatisfierWithStripedFile {
       long expectedBlkMovAttemptedCount, int timeout)
           throws TimeoutException, InterruptedException {
     BlockManager blockManager = cluster.getNamesystem().getBlockManager();
-    final StoragePolicySatisfier sps = blockManager.getStoragePolicySatisfier();
+    final StoragePolicySatisfier sps = (StoragePolicySatisfier) blockManager
+        .getSPSManager().getInternalSPSService();
     GenericTestUtils.waitFor(new Supplier<Boolean>() {
       @Override
       public Boolean get() {
@@ -566,7 +567,8 @@ public class TestStoragePolicySatisfierWithStripedFile {
       long expectedMoveFinishedBlks, int timeout)
           throws TimeoutException, InterruptedException {
     BlockManager blockManager = cluster.getNamesystem().getBlockManager();
-    final StoragePolicySatisfier sps = blockManager.getStoragePolicySatisfier();
+    final StoragePolicySatisfier sps = (StoragePolicySatisfier) blockManager
+        .getSPSManager().getInternalSPSService();
     Assert.assertNotNull("Failed to get SPS object reference!", sps);
 
     GenericTestUtils.waitFor(new Supplier<Boolean>() {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4402f3f8/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/sps/TestExternalStoragePolicySatisfier.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/sps/TestExternalStoragePolicySatisfier.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/sps/TestExternalStoragePolicySatisfier.java
index 15a4271..0546f39 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/sps/TestExternalStoragePolicySatisfier.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/sps/TestExternalStoragePolicySatisfier.java
@@ -133,7 +133,7 @@ public class TestExternalStoragePolicySatisfier
 
     BlockManager blkMgr = cluster.getNameNode().getNamesystem()
         .getBlockManager();
-    SPSService spsService = blkMgr.getSPSService();
+    SPSService spsService = blkMgr.getSPSManager().getInternalSPSService();
     spsService.stopGracefully();
 
     ExternalSPSContext context = new ExternalSPSContext(spsService,
@@ -143,12 +143,12 @@ public class TestExternalStoragePolicySatisfier
         new ExternalBlockMovementListener();
     ExternalSPSBlockMoveTaskHandler externalHandler =
         new ExternalSPSBlockMoveTaskHandler(conf, nnc,
-            blkMgr.getSPSService());
+            blkMgr.getSPSManager().getInternalSPSService());
     externalHandler.init();
     spsService.init(context,
-        new ExternalSPSFileIDCollector(context, blkMgr.getSPSService()),
-        externalHandler,
-        blkMoveListener);
+        new ExternalSPSFileIDCollector(context,
+            blkMgr.getSPSManager().getInternalSPSService()),
+        externalHandler, blkMoveListener);
     spsService.start(true, StoragePolicySatisfierMode.EXTERNAL);
     return cluster;
   }
@@ -156,14 +156,14 @@ public class TestExternalStoragePolicySatisfier
   public void restartNamenode() throws IOException{
     BlockManager blkMgr = getCluster().getNameNode().getNamesystem()
         .getBlockManager();
-    SPSService spsService = blkMgr.getSPSService();
+    SPSService spsService = blkMgr.getSPSManager().getInternalSPSService();
     spsService.stopGracefully();
 
     getCluster().restartNameNodes();
     getCluster().waitActive();
     blkMgr = getCluster().getNameNode().getNamesystem()
         .getBlockManager();
-    spsService = blkMgr.getSPSService();
+    spsService = blkMgr.getSPSManager().getInternalSPSService();
     spsService.stopGracefully();
 
     ExternalSPSContext context = new ExternalSPSContext(spsService,
@@ -172,12 +172,12 @@ public class TestExternalStoragePolicySatisfier
         new ExternalBlockMovementListener();
     ExternalSPSBlockMoveTaskHandler externalHandler =
         new ExternalSPSBlockMoveTaskHandler(getConf(), nnc,
-            blkMgr.getSPSService());
+            blkMgr.getSPSManager().getInternalSPSService());
     externalHandler.init();
     spsService.init(context,
-        new ExternalSPSFileIDCollector(context, blkMgr.getSPSService()),
-        externalHandler,
-        blkMoveListener);
+        new ExternalSPSFileIDCollector(context,
+            blkMgr.getSPSManager().getInternalSPSService()),
+        externalHandler, blkMoveListener);
     spsService.start(true, StoragePolicySatisfierMode.EXTERNAL);
   }
 
@@ -323,7 +323,7 @@ public class TestExternalStoragePolicySatisfier
       DistributedFileSystem fs = getFS();
       BlockManager blkMgr = getCluster().getNameNode().getNamesystem()
           .getBlockManager();
-      SPSService spsService = blkMgr.getSPSService();
+      SPSService spsService = blkMgr.getSPSManager().getInternalSPSService();
       spsService.stopGracefully(); // stops SPS
 
       // Creates 4 more files. Send all of them for satisfying the storage


---------------------------------------------------------------------
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-13050: [SPS]: Create start/stop script to start external SPS process. Contributed by Surendra Singh Lilhore.

Posted by um...@apache.org.
HDFS-13050: [SPS]: Create start/stop script to start external SPS process. 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/5845c36c
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/5845c36c
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/5845c36c

Branch: refs/heads/HDFS-10285
Commit: 5845c36c16c423107183287cce3be9357dad7564
Parents: 99594b4
Author: Rakesh Radhakrishnan <ra...@apache.org>
Authored: Mon Jan 29 03:10:48 2018 +0530
Committer: Uma Maheswara Rao Gangumalla <um...@apache.org>
Committed: Sun Aug 12 03:06:04 2018 -0700

----------------------------------------------------------------------
 .../hadoop-hdfs/src/main/bin/hdfs               |   5 +
 .../server/blockmanagement/BlockManager.java    |   9 ++
 .../apache/hadoop/hdfs/server/mover/Mover.java  |   2 +-
 .../hdfs/server/namenode/sps/Context.java       |   5 -
 .../namenode/sps/IntraSPSNameNodeContext.java   |   4 -
 .../sps/IntraSPSNameNodeFileIdCollector.java    |  12 +-
 .../hdfs/server/namenode/sps/SPSPathIds.java    |   1 +
 .../namenode/sps/StoragePolicySatisfier.java    |  83 +++++++-----
 .../sps/ExternalSPSBlockMoveTaskHandler.java    |   2 +-
 .../hdfs/server/sps/ExternalSPSContext.java     |  57 +-------
 .../server/sps/ExternalSPSFileIDCollector.java  |  12 +-
 .../sps/ExternalStoragePolicySatisfier.java     | 130 +++++++++++++++++++
 .../src/site/markdown/ArchivalStorage.md        |  10 +-
 .../sps/TestStoragePolicySatisfier.java         |  22 ++--
 .../sps/TestExternalStoragePolicySatisfier.java |  33 +++--
 15 files changed, 259 insertions(+), 128 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/5845c36c/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 bc6e7a4..94426a5 100755
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs
@@ -63,6 +63,7 @@ function hadoop_usage
   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/satisfyStoragePolicy block storage policies"
+  hadoop_add_subcommand "sps" daemon "run external storagepolicysatisfier"
   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
@@ -201,6 +202,10 @@ function hdfscmd_case
     storagepolicies)
       HADOOP_CLASSNAME=org.apache.hadoop.hdfs.tools.StoragePolicyAdmin
     ;;
+    sps)
+      HADOOP_SUBCMD_SUPPORTDAEMONIZATION="true"
+      HADOOP_CLASSNAME=org.apache.hadoop.hdfs.server.sps.ExternalStoragePolicySatisfier
+    ;;
     version)
       HADOOP_CLASSNAME=org.apache.hadoop.util.VersionInfo
     ;;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5845c36c/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 ac6d44b..4ea64a3 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
@@ -94,6 +94,9 @@ import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.namenode.Namesystem;
 import org.apache.hadoop.hdfs.server.namenode.ha.HAContext;
 import org.apache.hadoop.hdfs.server.namenode.metrics.NameNodeMetrics;
+import org.apache.hadoop.hdfs.server.namenode.sps.IntraSPSNameNodeBlockMoveTaskHandler;
+import org.apache.hadoop.hdfs.server.namenode.sps.IntraSPSNameNodeContext;
+import org.apache.hadoop.hdfs.server.namenode.sps.IntraSPSNameNodeFileIdCollector;
 import org.apache.hadoop.hdfs.server.namenode.sps.SPSPathIds;
 import org.apache.hadoop.hdfs.server.namenode.sps.SPSService;
 import org.apache.hadoop.hdfs.server.namenode.sps.StoragePolicySatisfier;
@@ -5106,9 +5109,15 @@ public class BlockManager implements BlockStatsMXBean {
       return;
     }
     updateSPSMode(StoragePolicySatisfierMode.INTERNAL);
+    sps.init(new IntraSPSNameNodeContext(this.namesystem, this, sps),
+        new IntraSPSNameNodeFileIdCollector(this.namesystem.getFSDirectory(),
+            sps),
+        new IntraSPSNameNodeBlockMoveTaskHandler(this, this.namesystem), null);
     sps.start(true, spsMode);
   }
 
+
+
   /**
    * Enable storage policy satisfier by starting its service.
    */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5845c36c/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 b4e9716..2cc0e27 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
@@ -672,7 +672,7 @@ public class Mover {
           }
           if (spsRunning) {
             System.err.println("Mover failed due to StoragePolicySatisfier"
-                + " is running. Exiting with status "
+                + " service running inside namenode. Exiting with status "
                 + ExitStatus.SKIPPED_DUE_TO_SPS + "... ");
             return ExitStatus.SKIPPED_DUE_TO_SPS.getExitCode();
           }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5845c36c/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/Context.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/Context.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/Context.java
index bddbc1b..ff4ad6b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/Context.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/Context.java
@@ -175,9 +175,4 @@ public interface Context {
    */
   String getFilePath(Long inodeId);
 
-  /**
-   * Close the resources.
-   */
-  void close() throws IOException;
-
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5845c36c/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/IntraSPSNameNodeContext.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/IntraSPSNameNodeContext.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/IntraSPSNameNodeContext.java
index 191886c..ff6cc21 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/IntraSPSNameNodeContext.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/IntraSPSNameNodeContext.java
@@ -196,8 +196,4 @@ public class IntraSPSNameNodeContext implements Context {
     return namesystem.getFilePath(inodeId);
   }
 
-  @Override
-  public void close() throws IOException {
-    // Nothing to clean.
-  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5845c36c/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/IntraSPSNameNodeFileIdCollector.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/IntraSPSNameNodeFileIdCollector.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/IntraSPSNameNodeFileIdCollector.java
index f7cd754..7a44dd9 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/IntraSPSNameNodeFileIdCollector.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/IntraSPSNameNodeFileIdCollector.java
@@ -158,11 +158,15 @@ public class IntraSPSNameNodeFileIdCollector extends FSTreeTraverser
    */
   public synchronized int remainingCapacity() {
     int size = service.processingQueueSize();
-    if (size >= maxQueueLimitToScan) {
-      return 0;
-    } else {
-      return (maxQueueLimitToScan - size);
+    int remainingSize = 0;
+    if (size < maxQueueLimitToScan) {
+      remainingSize = maxQueueLimitToScan - size;
     }
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("SPS processing Q -> maximum capacity:{}, current size:{},"
+          + " remaining size:{}", maxQueueLimitToScan, size, remainingSize);
+    }
+    return remainingSize;
   }
 
   class SPSTraverseInfo extends TraverseInfo {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5845c36c/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/SPSPathIds.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/SPSPathIds.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/SPSPathIds.java
index cd6ad22..e0f4999 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/SPSPathIds.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/SPSPathIds.java
@@ -31,6 +31,7 @@ import org.apache.hadoop.classification.InterfaceStability;
 public class SPSPathIds {
 
   // List of pending dir to satisfy the policy
+  // TODO: Make this bounded queue.
   private final Queue<Long> spsDirsToBeTraveresed = new LinkedList<Long>();
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5845c36c/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/StoragePolicySatisfier.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/StoragePolicySatisfier.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/StoragePolicySatisfier.java
index 89799fc..4ddfe2e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/StoragePolicySatisfier.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/StoragePolicySatisfier.java
@@ -55,6 +55,7 @@ import org.apache.hadoop.hdfs.server.protocol.DatanodeStorageReport;
 import org.apache.hadoop.hdfs.server.protocol.StorageReport;
 import org.apache.hadoop.hdfs.util.StripedBlockUtil;
 import org.apache.hadoop.util.Daemon;
+import org.apache.hadoop.util.StringUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -174,10 +175,11 @@ public class StoragePolicySatisfier implements SPSService, Runnable {
       return;
     }
     if (reconfigStart) {
-      LOG.info("Starting StoragePolicySatisfier, as admin requested to "
-          + "start it.");
+      LOG.info("Starting {} StoragePolicySatisfier, as admin requested to "
+          + "start it.", StringUtils.toLowerCase(spsMode.toString()));
     } else {
-      LOG.info("Starting StoragePolicySatisfier.");
+      LOG.info("Starting {} StoragePolicySatisfier.",
+          StringUtils.toLowerCase(spsMode.toString()));
     }
 
     // Ensure that all the previously submitted block movements(if any) have to
@@ -243,7 +245,14 @@ public class StoragePolicySatisfier implements SPSService, Runnable {
 
   @Override
   public void run() {
-    while (ctxt.isRunning()) {
+    while (isRunning) {
+      // Check if dependent service is running
+      if (!ctxt.isRunning()) {
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("Upstream service is down, skipping the sps work.");
+        }
+        continue;
+      }
       try {
         if (!ctxt.isInSafeMode()) {
           ItemInfo itemInfo = storageMovementNeeded.get();
@@ -284,33 +293,39 @@ public class StoragePolicySatisfier implements SPSService, 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:
+                if (LOG.isDebugEnabled()) {
+                  LOG.debug("Block analysis status:{} for the file path:{}."
+                      + " Adding to attempt monitor queue for the storage "
+                      + "movement attempt finished report",
+                      status.status, fileStatus.getPath());
+                }
                 this.storageMovementsMonitor.add(new AttemptedItemInfo(itemInfo
                     .getStartId(), itemInfo.getFileId(), monotonicNow(),
                     status.assignedBlocks, itemInfo.getRetryCount()));
                 break;
               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.");
+                  LOG.debug("Adding trackID:{} for the file path:{} back to"
+                      + " retry queue as none of the blocks found its eligible"
+                      + " targets.", trackId, fileStatus.getPath());
                 }
                 itemInfo.increRetryCount();
                 this.storageMovementNeeded.add(itemInfo);
                 break;
               case FEW_LOW_REDUNDANCY_BLOCKS:
                 if (LOG.isDebugEnabled()) {
-                  LOG.debug("Adding trackID " + trackId
-                      + " back to retry queue as some of the blocks"
-                      + " are low redundant.");
+                  LOG.debug("Adding trackID:{} for the file path:{} back to "
+                      + "retry queue as some of the blocks are low redundant.",
+                      trackId, fileStatus.getPath());
                 }
                 itemInfo.increRetryCount();
                 this.storageMovementNeeded.add(itemInfo);
                 break;
               case BLOCKS_FAILED_TO_MOVE:
                 if (LOG.isDebugEnabled()) {
-                  LOG.debug("Adding trackID " + trackId
-                      + " back to retry queue as some of the blocks"
-                      + " movement failed.");
+                  LOG.debug("Adding trackID:{} for the file path:{} back to "
+                      + "retry queue as some of the blocks movement failed.",
+                      trackId, fileStatus.getPath());
                 }
                 this.storageMovementNeeded.add(itemInfo);
                 break;
@@ -318,8 +333,9 @@ public class StoragePolicySatisfier implements SPSService, Runnable {
               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.");
+                LOG.info("Block analysis status:{} for the file path:{}."
+                    + " So, Cleaning up the Xattrs.", status.status,
+                    fileStatus.getPath());
                 storageMovementNeeded.removeItemTrackInfo(itemInfo, true);
                 break;
               }
@@ -346,20 +362,20 @@ public class StoragePolicySatisfier implements SPSService, Runnable {
     if (isRunning) {
       synchronized (this) {
         if (isRunning) {
-          isRunning = false;
-          // Stopping monitor thread and clearing queues as well
-          this.clearQueues();
-          this.storageMovementsMonitor.stopGracefully();
-          if (!(t instanceof InterruptedException)) {
-            LOG.info("StoragePolicySatisfier received an exception"
-                + " while shutting down.", t);
+          if (t instanceof InterruptedException) {
+            isRunning = false;
+            LOG.info("Stopping StoragePolicySatisfier.");
+            // Stopping monitor thread and clearing queues as well
+            this.clearQueues();
+            this.storageMovementsMonitor.stopGracefully();
+          } else {
+            LOG.error(
+                "StoragePolicySatisfier thread received runtime exception, "
+                    + "ignoring", t);
           }
-          LOG.info("Stopping StoragePolicySatisfier.");
         }
       }
     }
-    LOG.error("StoragePolicySatisfier thread received runtime exception. "
-        + "Stopping Storage policy satisfier work", t);
     return;
   }
 
@@ -374,9 +390,8 @@ public class StoragePolicySatisfier implements SPSService, Runnable {
     final boolean lastBlkComplete = locatedBlocks.isLastBlockComplete();
     if (!lastBlkComplete) {
       // Postpone, currently file is under construction
-      // 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", fileInfo.getFileId());
+      LOG.info("File: {} is under construction. So, postpone"
+          + " this to the next retry iteration", fileInfo.getPath());
       return new BlocksMovingAnalysis(
           BlocksMovingAnalysis.Status.ANALYSIS_SKIPPED_FOR_RETRY,
           new ArrayList<>());
@@ -384,8 +399,8 @@ public class StoragePolicySatisfier implements SPSService, Runnable {
 
     List<LocatedBlock> blocks = locatedBlocks.getLocatedBlocks();
     if (blocks.size() == 0) {
-      LOG.info("BlockCollectionID: {} file is not having any blocks."
-          + " So, skipping the analysis.", fileInfo.getFileId());
+      LOG.info("File: {} is not having any blocks."
+          + " So, skipping the analysis.", fileInfo.getPath());
       return new BlocksMovingAnalysis(
           BlocksMovingAnalysis.Status.BLOCKS_TARGET_PAIRING_SKIPPED,
           new ArrayList<>());
@@ -970,4 +985,12 @@ public class StoragePolicySatisfier implements SPSService, Runnable {
   public void markScanCompletedForPath(Long inodeId) {
     getStorageMovementQueue().markScanCompletedForDir(inodeId);
   }
+
+  /**
+   * Join main SPS thread.
+   */
+  public void join() throws InterruptedException {
+    //TODO Add join here on SPS rpc server also
+    storagePolicySatisfierThread.join();
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5845c36c/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/sps/ExternalSPSBlockMoveTaskHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/sps/ExternalSPSBlockMoveTaskHandler.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/sps/ExternalSPSBlockMoveTaskHandler.java
index a1c8eec..4a762649 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/sps/ExternalSPSBlockMoveTaskHandler.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/sps/ExternalSPSBlockMoveTaskHandler.java
@@ -110,7 +110,7 @@ public class ExternalSPSBlockMoveTaskHandler implements BlockMoveTaskHandler {
   /**
    * Initializes block movement tracker daemon and starts the thread.
    */
-  void init() {
+  public void init() {
     movementTrackerThread = new Daemon(this.blkMovementTracker);
     movementTrackerThread.setName("BlockStorageMovementTracker");
     movementTrackerThread.start();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5845c36c/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/sps/ExternalSPSContext.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/sps/ExternalSPSContext.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/sps/ExternalSPSContext.java
index e5b04ba..e3b3bbb 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/sps/ExternalSPSContext.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/sps/ExternalSPSContext.java
@@ -19,19 +19,13 @@
 package org.apache.hadoop.hdfs.server.sps;
 
 import java.io.IOException;
-import java.net.URI;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.List;
 
 import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataOutputStream;
 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.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
@@ -57,13 +51,12 @@ public class ExternalSPSContext implements Context {
       LoggerFactory.getLogger(ExternalSPSContext.class);
   private SPSService service;
   private NameNodeConnector nnc = null;
-  private Object nnConnectionLock = new Object();
   private BlockStoragePolicySuite createDefaultSuite =
       BlockStoragePolicySuite.createDefaultSuite();
 
-  public ExternalSPSContext(SPSService service) {
+  public ExternalSPSContext(SPSService service, NameNodeConnector nnc) {
     this.service = service;
-    initializeNamenodeConnector();
+    this.nnc = nnc;
   }
 
   @Override
@@ -73,7 +66,6 @@ public class ExternalSPSContext implements Context {
 
   @Override
   public boolean isInSafeMode() {
-    initializeNamenodeConnector();
     try {
       return nnc != null ? nnc.getDistributedFileSystem().isInSafeMode()
           : false;
@@ -85,7 +77,6 @@ public class ExternalSPSContext implements Context {
 
   @Override
   public boolean isMoverRunning() {
-    initializeNamenodeConnector();
     try {
       FSDataOutputStream out = nnc.getDistributedFileSystem()
           .append(HdfsServerConstants.MOVER_ID_PATH);
@@ -101,7 +92,6 @@ public class ExternalSPSContext implements Context {
   @Override
   public long getFileID(String path) throws UnresolvedLinkException,
       AccessControlException, ParentNotDirectoryException {
-    initializeNamenodeConnector();
     HdfsFileStatus fs = null;
     try {
       fs = (HdfsFileStatus) nnc.getDistributedFileSystem().getFileStatus(
@@ -121,7 +111,6 @@ public class ExternalSPSContext implements Context {
 
   @Override
   public boolean isFileExist(long inodeId) {
-    initializeNamenodeConnector();
     String filePath = null;
     try {
       filePath = getFilePath(inodeId);
@@ -145,14 +134,12 @@ public class ExternalSPSContext implements Context {
 
   @Override
   public void removeSPSHint(long inodeId) throws IOException {
-    initializeNamenodeConnector();
     nnc.getDistributedFileSystem().removeXAttr(new Path(getFilePath(inodeId)),
         HdfsServerConstants.XATTR_SATISFY_STORAGE_POLICY);
   }
 
   @Override
   public int getNumLiveDataNodes() {
-    initializeNamenodeConnector();
     try {
       return nnc.getDistributedFileSystem()
           .getDataNodeStats(DatanodeReportType.LIVE).length;
@@ -164,7 +151,6 @@ public class ExternalSPSContext implements Context {
 
   @Override
   public HdfsFileStatus getFileInfo(long inodeID) throws IOException {
-    initializeNamenodeConnector();
     return nnc.getDistributedFileSystem().getClient()
         .getLocatedFileInfo(getFilePath(inodeID), false);
   }
@@ -172,13 +158,11 @@ public class ExternalSPSContext implements Context {
   @Override
   public DatanodeStorageReport[] getLiveDatanodeStorageReport()
       throws IOException {
-    initializeNamenodeConnector();
     return nnc.getLiveDatanodeStorageReport();
   }
 
   @Override
   public boolean hasLowRedundancyBlocks(long inodeID) {
-    initializeNamenodeConnector();
     try {
       return nnc.getNNProtocolConnection().hasLowRedundancyBlocks(inodeID);
     } catch (IOException e) {
@@ -191,7 +175,6 @@ public class ExternalSPSContext implements Context {
   @Override
   public boolean checkDNSpaceForScheduling(DatanodeInfo dn, StorageType type,
       long estimatedSize) {
-    initializeNamenodeConnector();
     try {
       return nnc.getNNProtocolConnection().checkDNSpaceForScheduling(dn, type,
           estimatedSize);
@@ -204,7 +187,6 @@ public class ExternalSPSContext implements Context {
 
   @Override
   public Long getNextSPSPathId() {
-    initializeNamenodeConnector();
     try {
       return nnc.getNNProtocolConnection().getNextSPSPathId();
     } catch (IOException e) {
@@ -233,39 +215,4 @@ public class ExternalSPSContext implements Context {
       return null;
     }
   }
-
-  @Override
-  public void close() throws IOException {
-    synchronized (nnConnectionLock) {
-      if (nnc != null) {
-        nnc.close();
-      }
-    }
-  }
-
-  private void initializeNamenodeConnector() {
-    synchronized (nnConnectionLock) {
-      if (nnc == null) {
-        try {
-          nnc = getNameNodeConnector(service.getConf());
-        } catch (IOException e) {
-          LOG.warn("Exception while creating Namenode Connector.."
-              + "Namenode might not have started.", e);
-        }
-      }
-    }
-  }
-
-  public static NameNodeConnector getNameNodeConnector(Configuration conf)
-      throws IOException {
-    final Collection<URI> namenodes = DFSUtil.getInternalNsRpcUris(conf);
-    List<NameNodeConnector> nncs = Collections.emptyList();
-    NameNodeConnector.checkOtherInstanceRunning(false);
-    nncs = NameNodeConnector.newNameNodeConnectors(namenodes,
-        ExternalSPSContext.class.getSimpleName(),
-        HdfsServerConstants.MOVER_ID_PATH, conf,
-        NameNodeConnector.DEFAULT_MAX_IDLE_ITERATIONS);
-    return nncs.get(0);
-  }
-
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5845c36c/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/sps/ExternalSPSFileIDCollector.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/sps/ExternalSPSFileIDCollector.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/sps/ExternalSPSFileIDCollector.java
index 964ee8c..ff277ba 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/sps/ExternalSPSFileIDCollector.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/sps/ExternalSPSFileIDCollector.java
@@ -139,11 +139,15 @@ public class ExternalSPSFileIDCollector implements FileIdCollector {
    */
   public int remainingCapacity() {
     int size = service.processingQueueSize();
-    if (size >= maxQueueLimitToScan) {
-      return 0;
-    } else {
-      return (maxQueueLimitToScan - size);
+    int remainingSize = 0;
+    if (size < maxQueueLimitToScan) {
+      remainingSize = maxQueueLimitToScan - size;
+    }
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("SPS processing Q -> maximum capacity:{}, current size:{},"
+          + " remaining size:{}", maxQueueLimitToScan, size, remainingSize);
     }
+    return remainingSize;
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5845c36c/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/sps/ExternalStoragePolicySatisfier.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/sps/ExternalStoragePolicySatisfier.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/sps/ExternalStoragePolicySatisfier.java
new file mode 100644
index 0000000..c64abc3
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/sps/ExternalStoragePolicySatisfier.java
@@ -0,0 +1,130 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.sps;
+
+import static org.apache.hadoop.util.ExitUtil.terminate;
+
+import java.io.IOException;
+import java.net.URI;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.DFSUtil;
+import org.apache.hadoop.hdfs.HdfsConfiguration;
+import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants.StoragePolicySatisfierMode;
+import org.apache.hadoop.hdfs.server.balancer.NameNodeConnector;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
+import org.apache.hadoop.hdfs.server.namenode.sps.BlockMovementListener;
+import org.apache.hadoop.hdfs.server.namenode.sps.StoragePolicySatisfier;
+import org.apache.hadoop.util.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * This class starts and runs external SPS service.
+ */
+@InterfaceAudience.Private
+public class ExternalStoragePolicySatisfier {
+  public static final Logger LOG = LoggerFactory
+      .getLogger(ExternalStoragePolicySatisfier.class);
+
+  /**
+   * Main method to start SPS service.
+   */
+  public static void main(String args[]) throws Exception {
+    NameNodeConnector nnc = null;
+    try {
+      StringUtils.startupShutdownMessage(StoragePolicySatisfier.class, args,
+          LOG);
+      HdfsConfiguration spsConf = new HdfsConfiguration();
+      //TODO : login with SPS keytab
+      StoragePolicySatisfier sps = new StoragePolicySatisfier(spsConf);
+      nnc = getNameNodeConnector(spsConf);
+
+      boolean spsRunning;
+      spsRunning = nnc.getDistributedFileSystem().getClient()
+          .isStoragePolicySatisfierRunning();
+      if (spsRunning) {
+        throw new RuntimeException(
+            "Startup failed due to StoragePolicySatisfier"
+                + " running inside Namenode.");
+      }
+
+      ExternalSPSContext context = new ExternalSPSContext(sps, nnc);
+      ExternalBlockMovementListener blkMoveListener =
+          new ExternalBlockMovementListener();
+      ExternalSPSBlockMoveTaskHandler externalHandler =
+          new ExternalSPSBlockMoveTaskHandler(spsConf, nnc, sps);
+      externalHandler.init();
+      sps.init(context, new ExternalSPSFileIDCollector(context, sps),
+          externalHandler, blkMoveListener);
+      sps.start(true, StoragePolicySatisfierMode.EXTERNAL);
+      if (sps != null) {
+        sps.join();
+      }
+    } catch (Throwable e) {
+      LOG.error("Failed to start storage policy satisfier.", e);
+      terminate(1, e);
+    } finally {
+      if (nnc != null) {
+        nnc.close();
+      }
+    }
+  }
+
+  private static NameNodeConnector getNameNodeConnector(Configuration conf)
+      throws IOException, InterruptedException {
+    final Collection<URI> namenodes = DFSUtil.getInternalNsRpcUris(conf);
+    final Path externalSPSPathId = HdfsServerConstants.MOVER_ID_PATH;
+    while (true) {
+      try {
+        final List<NameNodeConnector> nncs = NameNodeConnector
+            .newNameNodeConnectors(namenodes,
+                StoragePolicySatisfier.class.getSimpleName(),
+                externalSPSPathId, conf,
+                NameNodeConnector.DEFAULT_MAX_IDLE_ITERATIONS);
+        return nncs.get(0);
+      } catch (IOException e) {
+        LOG.warn("Failed to connect with namenode", e);
+        Thread.sleep(3000); // retry the connection after few secs
+      }
+    }
+  }
+
+  /**
+   * It is implementation of BlockMovementListener.
+   */
+  private static class ExternalBlockMovementListener
+      implements BlockMovementListener {
+
+    private List<Block> actualBlockMovements = new ArrayList<>();
+
+    @Override
+    public void notifyMovementTriedBlocks(Block[] moveAttemptFinishedBlks) {
+      for (Block block : moveAttemptFinishedBlks) {
+        actualBlockMovements.add(block);
+      }
+      LOG.info("Movement attempted blocks", actualBlockMovements);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5845c36c/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 c10bfc3..25a6cd9 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/ArchivalStorage.md
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/ArchivalStorage.md
@@ -238,5 +238,13 @@ Check the running status of Storage Policy Satisfier service in namenode. If it
 ### Enable(internal service inside NN or external service outside NN) or Disable SPS without restarting Namenode
 If administrator wants to switch modes of SPS feature while Namenode is running, first he/she needs to update the desired value(internal or external or none) for the configuration item `dfs.storage.policy.satisfier.mode` in configuration file (`hdfs-site.xml`) and then run the following Namenode reconfig command
 
-+       hdfs dfsadmin -reconfig namenode <host:ipc_port> start
+* Command:
+
+       hdfs dfsadmin -reconfig namenode <host:ipc_port> start
+
+### Start External SPS Service.
+If administrator wants to start external sps, first he/she needs to configure property `dfs.storage.policy.satisfier.mode` with `external` value in configuration file (`hdfs-site.xml`) and then run Namenode reconfig command. After this start external sps service using following command
+
+* Command:
 
+      hdfs --daemon start sps

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5845c36c/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/sps/TestStoragePolicySatisfier.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/sps/TestStoragePolicySatisfier.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/sps/TestStoragePolicySatisfier.java
index 935d4f2..135d996 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/sps/TestStoragePolicySatisfier.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/sps/TestStoragePolicySatisfier.java
@@ -603,7 +603,7 @@ public class TestStoragePolicySatisfier {
       if (out != null) {
         out.close();
       }
-      hdfsCluster.shutdown();
+      shutdownCluster();
     }
   }
 
@@ -626,9 +626,7 @@ public class TestStoragePolicySatisfier {
       Assert.assertTrue("SPS should be running as "
           + "no Mover really running", running);
     } finally {
-      if (hdfsCluster != null) {
-        hdfsCluster.shutdown();
-      }
+      shutdownCluster();
     }
   }
 
@@ -672,9 +670,7 @@ public class TestStoragePolicySatisfier {
       DFSTestUtil.waitExpectedStorageType(
           file1, StorageType.DISK, 2, 30000, dfs);
     } finally {
-      if (hdfsCluster != null) {
-        hdfsCluster.shutdown();
-      }
+      shutdownCluster();
     }
   }
 
@@ -1381,7 +1377,11 @@ public class TestStoragePolicySatisfier {
     // 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 = sps.getStorageMovementQueue().get().getFileId();
+      ItemInfo itemInfo = sps.getStorageMovementQueue().get();
+      if (itemInfo == null) {
+        continue;
+      }
+      long trackId = itemInfo.getFileId();
       INode inode = fsDir.getInode(trackId);
       assertTrue("Failed to traverse tree, expected " + path + " but got "
           + inode.getFullPathName(), path.equals(inode.getFullPathName()));
@@ -1392,7 +1392,11 @@ public class TestStoragePolicySatisfier {
     // 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 = sps.getStorageMovementQueue().get().getFileId();
+      ItemInfo itemInfo = sps.getStorageMovementQueue().get();
+      if (itemInfo == null) {
+        continue;
+      }
+      long trackId = itemInfo.getFileId();
       INode inode = fsDir.getInode(trackId);
       assertTrue("Failed to traverse tree, expected " + path + " but got "
           + inode.getFullPathName(), path.equals(inode.getFullPathName()));

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5845c36c/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/sps/TestExternalStoragePolicySatisfier.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/sps/TestExternalStoragePolicySatisfier.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/sps/TestExternalStoragePolicySatisfier.java
index fe08b8f..febc2ea 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/sps/TestExternalStoragePolicySatisfier.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/sps/TestExternalStoragePolicySatisfier.java
@@ -22,7 +22,6 @@ import java.net.URI;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.List;
-import java.util.Map;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
@@ -43,8 +42,6 @@ import org.apache.hadoop.hdfs.server.namenode.sps.TestStoragePolicySatisfier;
 import org.junit.Assert;
 import org.junit.Ignore;
 
-import com.google.common.collect.Maps;
-
 /**
  * Tests the external sps service plugins.
  */
@@ -95,7 +92,8 @@ public class TestExternalStoragePolicySatisfier
     SPSService spsService = blkMgr.getSPSService();
     spsService.stopGracefully();
 
-    ExternalSPSContext context = new ExternalSPSContext(spsService);
+    ExternalSPSContext context = new ExternalSPSContext(spsService,
+        getNameNodeConnector(conf));
 
     ExternalBlockMovementListener blkMoveListener =
         new ExternalBlockMovementListener();
@@ -124,7 +122,8 @@ public class TestExternalStoragePolicySatisfier
     spsService = blkMgr.getSPSService();
     spsService.stopGracefully();
 
-    ExternalSPSContext context = new ExternalSPSContext(spsService);
+    ExternalSPSContext context = new ExternalSPSContext(spsService,
+        getNameNodeConnector(getConf()));
     ExternalBlockMovementListener blkMoveListener =
         new ExternalBlockMovementListener();
     ExternalSPSBlockMoveTaskHandler externalHandler =
@@ -161,16 +160,22 @@ public class TestExternalStoragePolicySatisfier
       throws IOException {
     final Collection<URI> namenodes = DFSUtil.getInternalNsRpcUris(conf);
     Assert.assertEquals(1, namenodes.size());
-    Map<URI, List<Path>> nnMap = Maps.newHashMap();
-    for (URI nn : namenodes) {
-      nnMap.put(nn, null);
-    }
     final Path externalSPSPathId = new Path("/system/tmp.id");
-    final List<NameNodeConnector> nncs = NameNodeConnector
-        .newNameNodeConnectors(nnMap,
-            StoragePolicySatisfier.class.getSimpleName(), externalSPSPathId,
-            conf, NameNodeConnector.DEFAULT_MAX_IDLE_ITERATIONS);
-    return nncs.get(0);
+    NameNodeConnector.checkOtherInstanceRunning(false);
+    while (true) {
+      try {
+        final List<NameNodeConnector> nncs = NameNodeConnector
+            .newNameNodeConnectors(namenodes,
+                StoragePolicySatisfier.class.getSimpleName(),
+                externalSPSPathId, conf,
+                NameNodeConnector.DEFAULT_MAX_IDLE_ITERATIONS);
+        return nncs.get(0);
+      } catch (IOException e) {
+        LOG.warn("Failed to connect with namenode", e);
+        // Ignore
+      }
+
+    }
   }
 
   /**


---------------------------------------------------------------------
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-12982 : [SPS]: Reduce the locking and cleanup the Namesystem access. Contributed by Rakesh R.

Posted by um...@apache.org.
HDFS-12982 : [SPS]: Reduce the locking and cleanup the Namesystem access. 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/05d4daf6
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/05d4daf6
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/05d4daf6

Branch: refs/heads/HDFS-10285
Commit: 05d4daf6ba3e5bd40f46e8003ee12fc7c613453d
Parents: 7842071
Author: Surendra Singh Lilhore <su...@apache.org>
Authored: Mon Jan 8 15:13:11 2018 +0530
Committer: Uma Maheswara Rao Gangumalla <um...@apache.org>
Committed: Sun Aug 12 03:06:02 2018 -0700

----------------------------------------------------------------------
 .../server/blockmanagement/BlockManager.java    |  16 +-
 .../blockmanagement/DatanodeDescriptor.java     |   2 +-
 .../server/blockmanagement/DatanodeManager.java |  22 ++
 .../server/namenode/FSDirStatAndListingOp.java  |   1 +
 .../hdfs/server/namenode/FSNamesystem.java      |  44 ++-
 .../hdfs/server/namenode/IntraNNSPSContext.java |  41 --
 .../hadoop/hdfs/server/namenode/Namesystem.java |  24 ++
 .../sps/BlockStorageMovementAttemptedItems.java |  17 +-
 .../sps/BlockStorageMovementNeeded.java         |  48 ++-
 .../hdfs/server/namenode/sps/Context.java       | 181 +++++++++
 .../namenode/sps/IntraSPSNameNodeContext.java   | 220 +++++++++++
 .../namenode/sps/StoragePolicySatisfier.java    | 374 +++++++++----------
 .../TestBlockStorageMovementAttemptedItems.java |  17 +-
 .../sps/TestStoragePolicySatisfier.java         |  25 +-
 14 files changed, 742 insertions(+), 290 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/05d4daf6/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 ec99a9f..5ee4026 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,11 +89,12 @@ import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
 import org.apache.hadoop.hdfs.server.namenode.CachedBlock;
 import org.apache.hadoop.hdfs.server.namenode.INode.BlocksMapUpdateInfo;
 import org.apache.hadoop.hdfs.server.namenode.INodesInPath;
-import org.apache.hadoop.hdfs.server.namenode.IntraNNSPSContext;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.namenode.Namesystem;
 import org.apache.hadoop.hdfs.server.namenode.ha.HAContext;
 import org.apache.hadoop.hdfs.server.namenode.metrics.NameNodeMetrics;
+import org.apache.hadoop.hdfs.server.namenode.sps.Context;
+import org.apache.hadoop.hdfs.server.namenode.sps.IntraSPSNameNodeContext;
 import org.apache.hadoop.hdfs.server.namenode.sps.StoragePolicySatisfier;
 import org.apache.hadoop.hdfs.server.protocol.BlockCommand;
 import org.apache.hadoop.hdfs.server.protocol.BlockReportContext;
@@ -433,6 +434,7 @@ public class BlockManager implements BlockStatsMXBean {
   private final StoragePolicySatisfier sps;
   private final boolean storagePolicyEnabled;
   private boolean spsEnabled;
+  private Context spsctxt = null;
   /** Minimum live replicas needed for the datanode to be transitioned
    * from ENTERING_MAINTENANCE to IN_MAINTENANCE.
    */
@@ -479,8 +481,8 @@ public class BlockManager implements BlockStatsMXBean {
         conf.getBoolean(
             DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_ENABLED_KEY,
             DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_ENABLED_DEFAULT);
-    StoragePolicySatisfier.Context spsctxt = new IntraNNSPSContext(namesystem);
-    sps = new StoragePolicySatisfier(namesystem, this, conf, spsctxt);
+    spsctxt = new IntraSPSNameNodeContext(namesystem, this, conf);
+    sps = new StoragePolicySatisfier(spsctxt);
     blockTokenSecretManager = createBlockTokenSecretManager(conf);
 
     providedStorageMap = new ProvidedStorageMap(namesystem, this, conf);
@@ -5039,8 +5041,8 @@ public class BlockManager implements BlockStatsMXBean {
       LOG.info("Storage policy satisfier is already running.");
       return;
     }
-
-    sps.start(false);
+    // TODO: FSDirectory will get removed via HDFS-12911 modularization work
+    sps.start(false, namesystem.getFSDirectory());
   }
 
   /**
@@ -5076,8 +5078,8 @@ public class BlockManager implements BlockStatsMXBean {
       LOG.info("Storage policy satisfier is already running.");
       return;
     }
-
-    sps.start(true);
+    // TODO: FSDirectory will get removed via HDFS-12911 modularization work
+    sps.start(true, namesystem.getFSDirectory());
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/05d4daf6/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 f9a76b4..b09d908 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
@@ -802,7 +802,7 @@ public class DatanodeDescriptor extends DatanodeInfo {
   }
 
   /** Increment the number of blocks scheduled. */
-  void incrementBlocksScheduled(StorageType t) {
+  public void incrementBlocksScheduled(StorageType t) {
     currApproxBlocksScheduled.add(t, 1);
   }
   

http://git-wip-us.apache.org/repos/asf/hadoop/blob/05d4daf6/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 6aab5e9..c24a38b 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
@@ -35,6 +35,7 @@ import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.net.DFSNetworkTopology;
 import org.apache.hadoop.hdfs.protocol.*;
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo.DatanodeInfoBuilder;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor.BlockTargetPair;
@@ -2045,5 +2046,26 @@ public class DatanodeManager {
       }
     }
   }
+
+  /**
+   * Generates datanode reports for the given report type.
+   *
+   * @param type
+   *          type of the datanode report
+   * @return array of DatanodeStorageReports
+   */
+  public DatanodeStorageReport[] getDatanodeStorageReport(
+      DatanodeReportType type) {
+    final List<DatanodeDescriptor> datanodes = getDatanodeListForReport(type);
+
+    DatanodeStorageReport[] reports = new DatanodeStorageReport[datanodes
+        .size()];
+    for (int i = 0; i < reports.length; i++) {
+      final DatanodeDescriptor d = datanodes.get(i);
+      reports[i] = new DatanodeStorageReport(
+          new DatanodeInfoBuilder().setFrom(d).build(), d.getStorageReports());
+    }
+    return reports;
+  }
 }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/05d4daf6/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirStatAndListingOp.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirStatAndListingOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirStatAndListingOp.java
index 7e22ae1..709e270 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirStatAndListingOp.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirStatAndListingOp.java
@@ -90,6 +90,7 @@ class FSDirStatAndListingOp {
    * @param srcArg The string representation of the path to the file
    * @param resolveLink whether to throw UnresolvedLinkException
    *        if src refers to a symlink
+   * @param needLocation if blockLocations need to be returned
    *
    * @param needLocation Include {@link LocatedBlocks} in result.
    * @param needBlockToken Include block tokens in {@link LocatedBlocks}.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/05d4daf6/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 2f625ac..e4a495b 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
@@ -3133,6 +3133,29 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
    * @param src The string representation of the path to the file
    * @param resolveLink whether to throw UnresolvedLinkException
    *        if src refers to a symlink
+   * @param needLocation if blockLocations need to be returned
+   *
+   * @throws AccessControlException
+   *           if access is denied
+   * @throws UnresolvedLinkException
+   *           if a symlink is encountered.
+   *
+   * @return object containing information regarding the file or null if file
+   *         not found
+   * @throws StandbyException
+   */
+  @Override
+  public HdfsFileStatus getFileInfo(final String src, boolean resolveLink,
+      boolean needLocation) throws IOException {
+    return getFileInfo(src, resolveLink, needLocation, false);
+  }
+
+  /**
+   * Get the file info for a specific file.
+   *
+   * @param src The string representation of the path to the file
+   * @param resolveLink whether to throw UnresolvedLinkException
+   *        if src refers to a symlink
    *
    * @param needLocation Include {@link LocatedBlocks} in result.
    * @param needBlockToken Include block tokens in {@link LocatedBlocks}
@@ -3167,6 +3190,17 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     return stat;
   }
 
+  @Override
+  public String getFilePath(Long inodeId) {
+    readLock();
+    try {
+      INode inode = getFSDirectory().getInode(inodeId);
+      return inode == null ? null : inode.getFullPathName();
+    } finally {
+      readUnlock();
+    }
+  }
+
   /**
    * Returns true if the file is closed
    */
@@ -4461,15 +4495,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     try {
       checkOperation(OperationCategory.UNCHECKED);
       final DatanodeManager dm = getBlockManager().getDatanodeManager();      
-      final List<DatanodeDescriptor> datanodes = dm.getDatanodeListForReport(type);
-
-      reports = new DatanodeStorageReport[datanodes.size()];
-      for (int i = 0; i < reports.length; i++) {
-        final DatanodeDescriptor d = datanodes.get(i);
-        reports[i] = new DatanodeStorageReport(
-            new DatanodeInfoBuilder().setFrom(d).build(),
-            d.getStorageReports());
-      }
+      reports = dm.getDatanodeStorageReport(type);
     } finally {
       readUnlock("getDatanodeStorageReport");
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/05d4daf6/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/IntraNNSPSContext.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/IntraNNSPSContext.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/IntraNNSPSContext.java
deleted file mode 100644
index 111cabb..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/IntraNNSPSContext.java
+++ /dev/null
@@ -1,41 +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 org.apache.hadoop.hdfs.server.namenode.sps.StoragePolicySatisfier;
-
-/**
- * This class is the Namenode implementation for analyzing the file blocks which
- * are expecting to change its storages and assigning the block storage
- * movements to satisfy the storage policy.
- */
-// TODO: Now, added one API which is required for sps package. Will refine
-// this interface via HDFS-12911.
-public class IntraNNSPSContext implements StoragePolicySatisfier.Context {
-  private final Namesystem namesystem;
-
-  public IntraNNSPSContext(Namesystem namesystem) {
-    this.namesystem = namesystem;
-  }
-
-  @Override
-  public int getNumLiveDataNodes() {
-    return namesystem.getFSDirectory().getBlockManager().getDatanodeManager()
-        .getNumLiveDataNodes();
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/05d4daf6/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 e58fa72..fc933b7 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
@@ -20,6 +20,7 @@ package org.apache.hadoop.hdfs.server.namenode;
 import java.io.IOException;
 
 import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockCollection;
 import org.apache.hadoop.hdfs.server.namenode.ha.HAContext;
 import org.apache.hadoop.hdfs.util.RwLock;
@@ -62,4 +63,27 @@ public interface Namesystem extends RwLock, SafeMode {
    * @throws IOException
    */
   void removeXattr(long id, String xattrName) throws IOException;
+
+  /**
+   * Gets the fileInfo of the given file path.
+   *
+   * @param filePath string representation of the path to the file
+   * @param resolveLink whether to throw UnresolvedLinkException
+   *        if src refers to a symlink
+   * @param needLocation if blockLocations need to be returned
+   *
+   * @return hdfs file status details
+   * @throws IOException
+   */
+  HdfsFileStatus getFileInfo(String filePath, boolean resolveLink,
+      boolean needLocation) throws IOException;
+
+  /**
+   * Gets the file path corresponds to the given file id.
+   *
+   * @param inodeId
+   *          file id
+   * @return string file path
+   */
+  String getFilePath(Long inodeId);
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/05d4daf6/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/BlockStorageMovementAttemptedItems.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/BlockStorageMovementAttemptedItems.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/BlockStorageMovementAttemptedItems.java
index b044f30..1cae027 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/BlockStorageMovementAttemptedItems.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/BlockStorageMovementAttemptedItems.java
@@ -25,6 +25,11 @@ import java.util.Arrays;
 import java.util.Iterator;
 import java.util.List;
 
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_RECHECK_TIMEOUT_MILLIS_DEFAULT;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_RECHECK_TIMEOUT_MILLIS_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_SELF_RETRY_TIMEOUT_MILLIS_DEFAULT;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_SELF_RETRY_TIMEOUT_MILLIS_KEY;
+
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.server.namenode.sps.StoragePolicySatisfier.AttemptedItemInfo;
 import org.apache.hadoop.hdfs.server.namenode.sps.StoragePolicySatisfier.ItemInfo;
@@ -66,15 +71,21 @@ public class BlockStorageMovementAttemptedItems {
   //
   private long minCheckTimeout = 1 * 60 * 1000; // minimum value
   private BlockStorageMovementNeeded blockStorageMovementNeeded;
+  private final Context ctxt;
 
-  public BlockStorageMovementAttemptedItems(long recheckTimeout,
-      long selfRetryTimeout,
+  public BlockStorageMovementAttemptedItems(Context context,
       BlockStorageMovementNeeded unsatisfiedStorageMovementFiles) {
+    this.ctxt = context;
+    long recheckTimeout = ctxt.getConf().getLong(
+        DFS_STORAGE_POLICY_SATISFIER_RECHECK_TIMEOUT_MILLIS_KEY,
+        DFS_STORAGE_POLICY_SATISFIER_RECHECK_TIMEOUT_MILLIS_DEFAULT);
     if (recheckTimeout > 0) {
       this.minCheckTimeout = Math.min(minCheckTimeout, recheckTimeout);
     }
 
-    this.selfRetryTimeout = selfRetryTimeout;
+    this.selfRetryTimeout = ctxt.getConf().getLong(
+        DFS_STORAGE_POLICY_SATISFIER_SELF_RETRY_TIMEOUT_MILLIS_KEY,
+        DFS_STORAGE_POLICY_SATISFIER_SELF_RETRY_TIMEOUT_MILLIS_DEFAULT);
     this.blockStorageMovementNeeded = unsatisfiedStorageMovementFiles;
     storageMovementAttemptedItems = new ArrayList<>();
     movementFinishedBlocks = new ArrayList<>();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/05d4daf6/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/BlockStorageMovementNeeded.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/BlockStorageMovementNeeded.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/BlockStorageMovementNeeded.java
index 5635621..80f1893 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/BlockStorageMovementNeeded.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/BlockStorageMovementNeeded.java
@@ -17,7 +17,8 @@
  */
 package org.apache.hadoop.hdfs.server.namenode.sps;
 
-import static org.apache.hadoop.hdfs.server.common.HdfsServerConstants.XATTR_SATISFY_STORAGE_POLICY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_QUEUE_LIMIT_DEFAULT;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_QUEUE_LIMIT_KEY;
 
 import java.io.IOException;
 import java.util.ArrayList;
@@ -35,10 +36,9 @@ import org.apache.hadoop.hdfs.protocol.HdfsConstants.StoragePolicySatisfyPathSta
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.server.namenode.FSDirectory;
 import org.apache.hadoop.hdfs.server.namenode.FSTreeTraverser;
+import org.apache.hadoop.hdfs.server.namenode.FSTreeTraverser.TraverseInfo;
 import org.apache.hadoop.hdfs.server.namenode.INode;
-import org.apache.hadoop.hdfs.server.namenode.Namesystem;
 import org.apache.hadoop.hdfs.server.namenode.sps.StoragePolicySatisfier.ItemInfo;
-import org.apache.hadoop.hdfs.server.namenode.FSTreeTraverser.TraverseInfo;
 import org.apache.hadoop.util.Daemon;
 import org.apache.hadoop.util.Time;
 import org.slf4j.Logger;
@@ -73,13 +73,11 @@ public class BlockStorageMovementNeeded {
   private final Map<Long, StoragePolicySatisfyPathStatusInfo> spsStatus =
       new ConcurrentHashMap<>();
 
-  private final Namesystem namesystem;
+  private final Context ctxt;
 
   // List of pending dir to satisfy the policy
   private final Queue<Long> spsDirsToBeTraveresed = new LinkedList<Long>();
 
-  private final StoragePolicySatisfier sps;
-
   private Daemon inodeIdCollector;
 
   private final int maxQueuedItem;
@@ -88,11 +86,11 @@ public class BlockStorageMovementNeeded {
   // NOT_AVAILABLE.
   private static long statusClearanceElapsedTimeMs = 300000;
 
-  public BlockStorageMovementNeeded(Namesystem namesystem,
-      StoragePolicySatisfier sps, int queueLimit) {
-    this.namesystem = namesystem;
-    this.sps = sps;
-    this.maxQueuedItem = queueLimit;
+  public BlockStorageMovementNeeded(Context context) {
+    this.ctxt = context;
+    this.maxQueuedItem = ctxt.getConf().getInt(
+                  DFS_STORAGE_POLICY_SATISFIER_QUEUE_LIMIT_KEY,
+                  DFS_STORAGE_POLICY_SATISFIER_QUEUE_LIMIT_DEFAULT);
   }
 
   /**
@@ -188,8 +186,7 @@ public class BlockStorageMovementNeeded {
       // 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) {
+      if (!ctxt.isFileExist(startId)) {
         // directory deleted just remove it.
         this.pendingWorkForDirectory.remove(startId);
         updateStatus(startId, isSuccess);
@@ -198,7 +195,7 @@ public class BlockStorageMovementNeeded {
         if (pendingWork != null) {
           pendingWork.decrementPendingWorkCount();
           if (pendingWork.isDirWorkDone()) {
-            namesystem.removeXattr(startId, XATTR_SATISFY_STORAGE_POLICY);
+            ctxt.removeSPSHint(startId);
             pendingWorkForDirectory.remove(startId);
             pendingWork.setFailure(!isSuccess);
             updateStatus(startId, pendingWork.isPolicySatisfied());
@@ -209,8 +206,7 @@ public class BlockStorageMovementNeeded {
     } else {
       // Remove xAttr if trackID doesn't exist in
       // storageMovementAttemptedItems or file policy satisfied.
-      namesystem.removeXattr(trackInfo.getTrackId(),
-          XATTR_SATISFY_STORAGE_POLICY);
+      ctxt.removeSPSHint(trackInfo.getTrackId());
       updateStatus(trackInfo.getStartId(), isSuccess);
     }
   }
@@ -256,7 +252,7 @@ public class BlockStorageMovementNeeded {
     while ((trackId = spsDirsToBeTraveresed.poll()) != null) {
       try {
         // Remove xAttr for file
-        namesystem.removeXattr(trackId, XATTR_SATISFY_STORAGE_POLICY);
+        ctxt.removeSPSHint(trackId);
       } catch (IOException ie) {
         LOG.warn("Failed to remove SPS xattr for track id " + trackId, ie);
       }
@@ -269,8 +265,7 @@ public class BlockStorageMovementNeeded {
       try {
         // Remove xAttr for file
         if (!itemInfo.isDir()) {
-          namesystem.removeXattr(itemInfo.getTrackId(),
-              XATTR_SATISFY_STORAGE_POLICY);
+          ctxt.removeSPSHint(itemInfo.getTrackId());
         }
       } catch (IOException ie) {
         LOG.warn(
@@ -300,10 +295,9 @@ public class BlockStorageMovementNeeded {
     public void run() {
       LOG.info("Starting FileInodeIdCollector!.");
       long lastStatusCleanTime = 0;
-      while (namesystem.isRunning() && sps.isRunning()) {
+      while (ctxt.isRunning()) {
         try {
-          if (!namesystem.isInSafeMode()) {
-            FSDirectory fsd = namesystem.getFSDirectory();
+          if (!ctxt.isInSafeMode()) {
             Long startINodeId = spsDirsToBeTraveresed.poll();
             if (startINodeId == null) {
               // Waiting for SPS path
@@ -311,7 +305,7 @@ public class BlockStorageMovementNeeded {
                 spsDirsToBeTraveresed.wait(5000);
               }
             } else {
-              INode startInode = fsd.getInode(startINodeId);
+              INode startInode = getFSDirectory().getInode(startINodeId);
               if (startInode != null) {
                 try {
                   remainingCapacity = remainingCapacity();
@@ -333,8 +327,7 @@ public class BlockStorageMovementNeeded {
                 DirPendingWorkInfo dirPendingWorkInfo =
                     pendingWorkForDirectory.get(startInode.getId());
                 if (dirPendingWorkInfo.isDirWorkDone()) {
-                  namesystem.removeXattr(startInode.getId(),
-                      XATTR_SATISFY_STORAGE_POLICY);
+                  ctxt.removeSPSHint(startInode.getId());
                   pendingWorkForDirectory.remove(startInode.getId());
                   updateStatus(startInode.getId(), true);
                 }
@@ -483,9 +476,10 @@ public class BlockStorageMovementNeeded {
     }
   }
 
-  public void init() {
+  // TODO: FSDirectory will get removed via HDFS-12911 modularization work
+  public void init(FSDirectory fsd) {
     inodeIdCollector = new Daemon(new StorageMovementPendingInodeIdCollector(
-        namesystem.getFSDirectory()));
+        fsd));
     inodeIdCollector.setName("FileInodeIdCollector");
     inodeIdCollector.start();
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/05d4daf6/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/Context.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/Context.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/Context.java
new file mode 100644
index 0000000..d11e26f
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/Context.java
@@ -0,0 +1,181 @@
+/**
+ * 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.sps;
+
+import java.io.IOException;
+import java.util.function.Supplier;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.ParentNotDirectoryException;
+import org.apache.hadoop.fs.StorageType;
+import org.apache.hadoop.fs.UnresolvedLinkException;
+import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
+import org.apache.hadoop.hdfs.server.protocol.DatanodeStorageReport;
+import org.apache.hadoop.hdfs.server.protocol.BlockStorageMovementCommand.BlockMovingInfo;
+import org.apache.hadoop.net.NetworkTopology;
+import org.apache.hadoop.security.AccessControlException;
+
+/**
+ * An interface for the communication between NameNode and SPS module.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public interface Context {
+
+  /**
+   * Returns configuration object.
+   */
+  Configuration getConf();
+
+  /**
+   * Returns true if the SPS is running, false otherwise.
+   */
+  boolean isRunning();
+
+  /**
+   * Update the SPS running status.
+   *
+   * @param isSpsRunning
+   *          true represents running, false otherwise
+   */
+  void setSPSRunning(Supplier<Boolean> isSpsRunning);
+
+  /**
+   * Returns true if the Namenode in safe mode, false otherwise.
+   */
+  boolean isInSafeMode();
+
+  /**
+   * Returns true if Mover tool is already running, false otherwise.
+   */
+  boolean isMoverRunning();
+
+  /**
+   * Gets the Inode ID number for the given path.
+   *
+   * @param path
+   *          - file/dir path
+   * @return Inode id number
+   */
+  long getFileID(String path) throws UnresolvedLinkException,
+      AccessControlException, ParentNotDirectoryException;
+
+  /**
+   * Gets the network topology.
+   *
+   * @return network topology
+   */
+  NetworkTopology getNetworkTopology();
+
+  /**
+   * Returns true if the give Inode exists in the Namespace.
+   *
+   * @param inodeId
+   *          - Inode ID
+   * @return true if Inode exists, false otherwise.
+   */
+  boolean isFileExist(long inodeId);
+
+  /**
+   * Gets the storage policy details for the given policy ID.
+   *
+   * @param policyId
+   *          - Storage policy ID
+   * @return the detailed policy object
+   */
+  BlockStoragePolicy getStoragePolicy(byte policyId);
+
+  /**
+   * Drop the SPS work in case if any previous work queued up.
+   */
+  void addDropPreviousSPSWorkAtDNs();
+
+  /**
+   * Remove the hint which was added to track SPS call.
+   *
+   * @param inodeId
+   *          - Inode ID
+   * @throws IOException
+   */
+  void removeSPSHint(long inodeId) throws IOException;
+
+  /**
+   * Gets the number of live datanodes in the cluster.
+   *
+   * @return number of live datanodes
+   */
+  int getNumLiveDataNodes();
+
+  /**
+   * Get the file info for a specific file.
+   *
+   * @param inodeID
+   *          inode identifier
+   * @return file status metadata information
+   */
+  HdfsFileStatus getFileInfo(long inodeID) throws IOException;
+
+  /**
+   * Returns all the live datanodes and its storage details.
+   *
+   * @throws IOException
+   */
+  DatanodeStorageReport[] getLiveDatanodeStorageReport()
+      throws IOException;
+
+  /**
+   * Returns true if the given inode file has low redundancy blocks.
+   *
+   * @param inodeID
+   *          inode identifier
+   * @return true if block collection has low redundancy blocks
+   */
+  boolean hasLowRedundancyBlocks(long inodeID);
+
+  /**
+   * Assign the given block movement task to the target node present in
+   * {@link BlockMovingInfo}.
+   *
+   * @param blkMovingInfo
+   *          block to storage info
+   * @throws IOException
+   */
+  void assignBlockMoveTaskToTargetNode(BlockMovingInfo blkMovingInfo)
+      throws IOException;
+
+  /**
+   * Checks whether the given datanode has sufficient space to occupy the given
+   * blockSize data.
+   *
+   * @param dn
+   *          datanode info
+   * @param type
+   *          storage type
+   * @param blockSize
+   *          blockSize to be scheduled
+   * @return true if the given datanode has sufficient space to occupy blockSize
+   *         data, false otherwise.
+   */
+  boolean verifyTargetDatanodeHasSpaceForScheduling(DatanodeInfo dn,
+      StorageType type, long blockSize);
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/05d4daf6/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/IntraSPSNameNodeContext.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/IntraSPSNameNodeContext.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/IntraSPSNameNodeContext.java
new file mode 100644
index 0000000..6654212
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/IntraSPSNameNodeContext.java
@@ -0,0 +1,220 @@
+/**
+ * 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.sps;
+
+import static org.apache.hadoop.hdfs.server.common.HdfsServerConstants.XATTR_SATISFY_STORAGE_POLICY;
+
+import java.io.IOException;
+import java.util.function.Supplier;
+
+import org.apache.commons.lang.StringUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.ParentNotDirectoryException;
+import org.apache.hadoop.fs.StorageType;
+import org.apache.hadoop.fs.UnresolvedLinkException;
+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.HdfsFileStatus;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockCollection;
+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.namenode.INode;
+import org.apache.hadoop.hdfs.server.namenode.Namesystem;
+import org.apache.hadoop.hdfs.server.protocol.DatanodeStorageReport;
+import org.apache.hadoop.hdfs.server.protocol.BlockStorageMovementCommand.BlockMovingInfo;
+import org.apache.hadoop.net.NetworkTopology;
+import org.apache.hadoop.security.AccessControlException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * This class is the Namenode implementation for analyzing the file blocks which
+ * are expecting to change its storages and assigning the block storage
+ * movements to satisfy the storage policy.
+ */
+public class IntraSPSNameNodeContext implements Context {
+  private static final Logger LOG = LoggerFactory
+      .getLogger(IntraSPSNameNodeContext.class);
+
+  private final Namesystem namesystem;
+  private final BlockManager blockManager;
+  private final Configuration conf;
+  private Supplier<Boolean> isSpsRunning;
+
+  public IntraSPSNameNodeContext(Namesystem namesystem,
+      BlockManager blockManager, Configuration conf) {
+    this.namesystem = namesystem;
+    this.blockManager = blockManager;
+    this.conf = conf;
+    isSpsRunning = () -> false;
+  }
+
+  @Override
+  public int getNumLiveDataNodes() {
+    return blockManager.getDatanodeManager().getNumLiveDataNodes();
+  }
+
+  @Override
+  public HdfsFileStatus getFileInfo(long inodeID) throws IOException {
+    String filePath = namesystem.getFilePath(inodeID);
+    if (StringUtils.isBlank(filePath)) {
+      LOG.debug("File with inodeID:{} doesn't exists!", inodeID);
+      return null;
+    }
+    HdfsFileStatus fileInfo = null;
+    try {
+      fileInfo = namesystem.getFileInfo(filePath, true, true);
+    } catch (IOException e) {
+      LOG.debug("File path:{} doesn't exists!", filePath);
+    }
+    return fileInfo;
+  }
+
+  @Override
+  public DatanodeStorageReport[] getLiveDatanodeStorageReport()
+      throws IOException {
+    namesystem.readLock();
+    try {
+      return blockManager.getDatanodeManager()
+          .getDatanodeStorageReport(DatanodeReportType.LIVE);
+    } finally {
+      namesystem.readUnlock();
+    }
+  }
+
+  @Override
+  public boolean hasLowRedundancyBlocks(long inodeID) {
+    namesystem.readLock();
+    try {
+      BlockCollection bc = namesystem.getBlockCollection(inodeID);
+      return blockManager.hasLowRedundancyBlocks(bc);
+    } finally {
+      namesystem.readUnlock();
+    }
+  }
+
+  @Override
+  public Configuration getConf() {
+    return conf;
+  }
+
+  @Override
+  public boolean isFileExist(long inodeId) {
+    return namesystem.getFSDirectory().getInode(inodeId) != null;
+  }
+
+  @Override
+  public void removeSPSHint(long inodeId) throws IOException {
+    this.namesystem.removeXattr(inodeId, XATTR_SATISFY_STORAGE_POLICY);
+  }
+
+  @Override
+  public boolean isRunning() {
+    // TODO : 'isSpsRunning' flag has been added to avoid the NN lock inside
+    // SPS. Context interface will be further refined as part of HDFS-12911
+    // modularization task. One idea is to introduce a cleaner interface similar
+    // to Namesystem for better abstraction.
+    return namesystem.isRunning() && isSpsRunning.get();
+  }
+
+  @Override
+  public void setSPSRunning(Supplier<Boolean> spsRunningFlag) {
+    this.isSpsRunning = spsRunningFlag;
+  }
+
+  @Override
+  public boolean isInSafeMode() {
+    return namesystem.isInSafeMode();
+  }
+
+  @Override
+  public boolean isMoverRunning() {
+    String moverId = HdfsServerConstants.MOVER_ID_PATH.toString();
+    return namesystem.isFileOpenedForWrite(moverId);
+  }
+
+  @Override
+  public void addDropPreviousSPSWorkAtDNs() {
+    namesystem.readLock();
+    try {
+      blockManager.getDatanodeManager().addDropSPSWorkCommandsToAllDNs();
+    } finally {
+      namesystem.readUnlock();
+    }
+  }
+
+  @Override
+  public BlockStoragePolicy getStoragePolicy(byte policyID) {
+    return blockManager.getStoragePolicy(policyID);
+  }
+
+  @Override
+  public NetworkTopology getNetworkTopology() {
+    return blockManager.getDatanodeManager().getNetworkTopology();
+  }
+
+  @Override
+  public long getFileID(String path) throws UnresolvedLinkException,
+      AccessControlException, ParentNotDirectoryException {
+    namesystem.readLock();
+    try {
+      INode inode = namesystem.getFSDirectory().getINode(path);
+      return inode == null ? -1 : inode.getId();
+    } finally {
+      namesystem.readUnlock();
+    }
+  }
+
+  @Override
+  public void assignBlockMoveTaskToTargetNode(BlockMovingInfo blkMovingInfo)
+      throws IOException {
+    namesystem.readLock();
+    try {
+      DatanodeDescriptor dn = blockManager.getDatanodeManager()
+          .getDatanode(blkMovingInfo.getTarget().getDatanodeUuid());
+      if (dn == null) {
+        throw new IOException("Failed to schedule block movement task:"
+            + blkMovingInfo + " as target datanode: "
+            + blkMovingInfo.getTarget() + " doesn't exists");
+      }
+      dn.addBlocksToMoveStorage(blkMovingInfo);
+      dn.incrementBlocksScheduled(blkMovingInfo.getTargetStorageType());
+    } finally {
+      namesystem.readUnlock();
+    }
+  }
+
+  @Override
+  public boolean verifyTargetDatanodeHasSpaceForScheduling(DatanodeInfo dn,
+      StorageType type, long blockSize) {
+    namesystem.readLock();
+    try {
+      DatanodeDescriptor datanode = blockManager.getDatanodeManager()
+          .getDatanode(dn.getDatanodeUuid());
+      if (datanode == null) {
+        LOG.debug("Target datanode: " + dn + " doesn't exists");
+        return false;
+      }
+      return null != datanode.chooseStorage4Block(type, blockSize);
+    } finally {
+      namesystem.readUnlock();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/05d4daf6/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/StoragePolicySatisfier.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/StoragePolicySatisfier.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/StoragePolicySatisfier.java
index 0d4bb19..b3e6b78 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/StoragePolicySatisfier.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/StoragePolicySatisfier.java
@@ -29,29 +29,28 @@ 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;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
+import org.apache.hadoop.hdfs.protocol.DatanodeInfoWithStorage;
+import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
+import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.StoragePolicySatisfyPathStatus;
+import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
+import org.apache.hadoop.hdfs.protocol.HdfsLocatedFileStatus;
+import org.apache.hadoop.hdfs.protocol.LocatedBlock;
+import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
+import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock;
 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;
 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.namenode.Namesystem;
+import org.apache.hadoop.hdfs.server.namenode.FSDirectory;
 import org.apache.hadoop.hdfs.server.protocol.BlockStorageMovementCommand.BlockMovingInfo;
 import org.apache.hadoop.hdfs.server.protocol.BlocksStorageMoveAttemptFinished;
+import org.apache.hadoop.hdfs.server.protocol.DatanodeStorageReport;
 import org.apache.hadoop.hdfs.server.protocol.StorageReport;
 import org.apache.hadoop.hdfs.util.StripedBlockUtil;
 import org.apache.hadoop.util.Daemon;
@@ -79,8 +78,6 @@ 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;
   private final BlockStorageMovementAttemptedItems storageMovementsMonitor;
   private volatile boolean isRunning = false;
@@ -90,16 +87,6 @@ public class StoragePolicySatisfier implements Runnable {
   private final Context ctxt;
 
   /**
-   * An interface for analyzing and assigning the block storage movements to
-   * worker nodes.
-   */
-  // TODO: Now, added one API which is required for sps package. Will refine
-  // this interface via HDFS-12911.
-  public interface Context {
-    int getNumLiveDataNodes();
-  }
-
-  /**
    * Represents the collective analysis status for all blocks.
    */
   private static class BlocksMovingAnalysis {
@@ -124,7 +111,9 @@ public class StoragePolicySatisfier implements Runnable {
       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
+      FEW_LOW_REDUNDANCY_BLOCKS,
+      // Represents that, movement failures due to unexpected errors.
+      BLOCKS_FAILED_TO_MOVE
     }
 
     private Status status = null;
@@ -136,36 +125,27 @@ public class StoragePolicySatisfier implements Runnable {
     }
   }
 
-  public StoragePolicySatisfier(final Namesystem namesystem,
-      final BlockManager blkManager, Configuration conf, Context ctxt) {
-    this.namesystem = namesystem;
-    this.storageMovementNeeded = new BlockStorageMovementNeeded(namesystem,
-        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(
-            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),
+  public StoragePolicySatisfier(Context ctxt) {
+    this.ctxt = ctxt;
+    this.storageMovementNeeded = new BlockStorageMovementNeeded(ctxt);
+    this.storageMovementsMonitor = new BlockStorageMovementAttemptedItems(ctxt,
         storageMovementNeeded);
-    this.spsWorkMultiplier = DFSUtil.getSPSWorkMultiplier(conf);
-    this.blockMovementMaxRetry = conf.getInt(
+    this.spsWorkMultiplier = DFSUtil.getSPSWorkMultiplier(ctxt.getConf());
+    this.blockMovementMaxRetry = ctxt.getConf().getInt(
         DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_MAX_RETRY_ATTEMPTS_KEY,
         DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_MAX_RETRY_ATTEMPTS_DEFAULT);
-    this.ctxt = ctxt;
   }
 
   /**
    * Start storage policy satisfier demon thread. Also start block storage
    * movements monitor for retry the attempts if needed.
+   *
+   * // TODO: FSDirectory will get removed via HDFS-12911 modularization work.
    */
-  public synchronized void start(boolean reconfigStart) {
+  public synchronized void start(boolean reconfigStart, FSDirectory fsd) {
     isRunning = true;
-    if (checkIfMoverRunning()) {
+    ctxt.setSPSRunning(this::isRunning);
+    if (ctxt.isMoverRunning()) {
       isRunning = false;
       LOG.error(
           "Stopping StoragePolicySatisfier thread " + "as Mover ID file "
@@ -183,7 +163,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.init();
+    storageMovementNeeded.init(fsd);
     storagePolicySatisfierThread = new Daemon(this);
     storagePolicySatisfierThread.setName("StoragePolicySatisfier");
     storagePolicySatisfierThread.start();
@@ -199,7 +179,6 @@ public class StoragePolicySatisfier implements Runnable {
    */
   public synchronized void disable(boolean forceStop) {
     isRunning = false;
-
     if (storagePolicySatisfierThread == null) {
       return;
     }
@@ -242,25 +221,19 @@ public class StoragePolicySatisfier implements Runnable {
     return isRunning;
   }
 
-  // Return true if a Mover instance is running
-  private boolean checkIfMoverRunning() {
-    String moverId = HdfsServerConstants.MOVER_ID_PATH.toString();
-    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();
+    ctxt.addDropPreviousSPSWorkAtDNs();
   }
 
   @Override
   public void run() {
-    while (namesystem.isRunning() && isRunning) {
+    while (ctxt.isRunning()) {
       try {
-        if (!namesystem.isInSafeMode()) {
+        if (!ctxt.isInSafeMode()) {
           ItemInfo itemInfo = storageMovementNeeded.get();
           if (itemInfo != null) {
             if(itemInfo.getRetryCount() >= blockMovementMaxRetry){
@@ -271,25 +244,28 @@ public class StoragePolicySatisfier implements Runnable {
               continue;
             }
             long trackId = itemInfo.getTrackId();
-            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, true);
-              } else {
-                status =
-                    analyseBlocksStorageMovementsAndAssignToDN(
-                        blockCollection);
-              }
-            } finally {
-              namesystem.readUnlock();
-            }
-            if (blockCollection != null) {
+            DatanodeStorageReport[] liveDnReports;
+            BlockStoragePolicy existingStoragePolicy;
+            // TODO: presently, context internally acquire the lock
+            // and returns the result. Need to discuss to move the lock outside?
+            boolean hasLowRedundancyBlocks = ctxt
+                .hasLowRedundancyBlocks(trackId);
+            HdfsFileStatus fileStatus = ctxt.getFileInfo(trackId);
+            // Check path existence.
+            if (fileStatus == null || fileStatus.isDir()) {
+              // File doesn't exists (maybe got deleted) or its a directory,
+              // just remove trackId from the queue
+              storageMovementNeeded.removeItemTrackInfo(itemInfo, true);
+            } else {
+              liveDnReports = ctxt.getLiveDatanodeStorageReport();
+              byte existingStoragePolicyID = fileStatus.getStoragePolicy();
+              existingStoragePolicy = ctxt
+                  .getStoragePolicy(existingStoragePolicyID);
+
+              HdfsLocatedFileStatus file = (HdfsLocatedFileStatus) fileStatus;
+              status = analyseBlocksStorageMovementsAndAssignToDN(file,
+                  hasLowRedundancyBlocks, existingStoragePolicy, liveDnReports);
               switch (status.status) {
               // Just add to monitor, so it will be retried after timeout
               case ANALYSIS_SKIPPED_FOR_RETRY:
@@ -317,6 +293,14 @@ public class StoragePolicySatisfier implements Runnable {
                 }
                 this.storageMovementNeeded.add(itemInfo);
                 break;
+              case BLOCKS_FAILED_TO_MOVE:
+                if (LOG.isDebugEnabled()) {
+                  LOG.debug("Adding trackID " + trackId
+                      + " back to retry queue as some of the blocks"
+                      + " movement failed.");
+                }
+                this.storageMovementNeeded.add(itemInfo);
+                break;
               // Just clean Xattrs
               case BLOCKS_TARGET_PAIRING_SKIPPED:
               case BLOCKS_ALREADY_SATISFIED:
@@ -350,14 +334,11 @@ public class StoragePolicySatisfier implements Runnable {
           // 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);
-            }
-            return;
+          if (!(t instanceof InterruptedException)) {
+            LOG.info("StoragePolicySatisfier received an exception"
+                + " while shutting down.", t);
           }
+          LOG.info("Stopping StoragePolicySatisfier.");
         }
       }
     }
@@ -367,41 +348,43 @@ public class StoragePolicySatisfier implements Runnable {
   }
 
   private BlocksMovingAnalysis analyseBlocksStorageMovementsAndAssignToDN(
-      BlockCollection blockCollection) {
+      HdfsLocatedFileStatus fileInfo, boolean hasLowRedundancyBlocks,
+      BlockStoragePolicy existingStoragePolicy,
+      DatanodeStorageReport[] liveDns) {
     BlocksMovingAnalysis.Status status =
         BlocksMovingAnalysis.Status.BLOCKS_ALREADY_SATISFIED;
-    byte existingStoragePolicyID = blockCollection.getStoragePolicyID();
-    BlockStoragePolicy existingStoragePolicy =
-        blockManager.getStoragePolicy(existingStoragePolicyID);
-    if (!blockCollection.getLastBlock().isComplete()) {
+    final ErasureCodingPolicy ecPolicy = fileInfo.getErasureCodingPolicy();
+    final LocatedBlocks locatedBlocks = fileInfo.getLocatedBlocks();
+    final boolean lastBlkComplete = locatedBlocks.isLastBlockComplete();
+    if (!lastBlkComplete) {
       // Postpone, currently file is under construction
       // 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());
+          + " this to the next retry iteration", fileInfo.getFileId());
       return new BlocksMovingAnalysis(
           BlocksMovingAnalysis.Status.ANALYSIS_SKIPPED_FOR_RETRY,
           new ArrayList<>());
     }
 
-    BlockInfo[] blocks = blockCollection.getBlocks();
-    if (blocks.length == 0) {
+    List<LocatedBlock> blocks = locatedBlocks.getLocatedBlocks();
+    if (blocks.size() == 0) {
       LOG.info("BlockCollectionID: {} file is not having any blocks."
-          + " So, skipping the analysis.", blockCollection.getId());
+          + " So, skipping the analysis.", fileInfo.getFileId());
       return new BlocksMovingAnalysis(
           BlocksMovingAnalysis.Status.BLOCKS_TARGET_PAIRING_SKIPPED,
           new ArrayList<>());
     }
     List<BlockMovingInfo> blockMovingInfos = new ArrayList<BlockMovingInfo>();
 
-    for (int i = 0; i < blocks.length; i++) {
-      BlockInfo blockInfo = blocks[i];
+    for (int i = 0; i < blocks.size(); i++) {
+      LocatedBlock blockInfo = blocks.get(i);
       List<StorageType> expectedStorageTypes;
       if (blockInfo.isStriped()) {
         if (ErasureCodingPolicyManager
             .checkStoragePolicySuitableForECStripedMode(
-                existingStoragePolicyID)) {
+                existingStoragePolicy.getId())) {
           expectedStorageTypes = existingStoragePolicy
-              .chooseStorageTypes((short) blockInfo.getCapacity());
+              .chooseStorageTypes((short) blockInfo.getLocations().length);
         } else {
           // Currently we support only limited policies (HOT, COLD, ALLSSD)
           // for EC striped mode files. SPS will ignore to move the blocks if
@@ -415,22 +398,16 @@ public class StoragePolicySatisfier implements Runnable {
         }
       } else {
         expectedStorageTypes = existingStoragePolicy
-            .chooseStorageTypes(blockInfo.getReplication());
+            .chooseStorageTypes(fileInfo.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));
+      List<StorageType> existing = new LinkedList<StorageType>(
+          Arrays.asList(blockInfo.getStorageTypes()));
       if (!DFSUtil.removeOverlapBetweenStorageTypes(expectedStorageTypes,
           existing, true)) {
         boolean blocksPaired = computeBlockMovingInfos(blockMovingInfos,
-            blockInfo, expectedStorageTypes, existing, storages);
+            blockInfo, expectedStorageTypes, existing, blockInfo.getLocations(),
+            liveDns, ecPolicy);
         if (blocksPaired) {
           status = BlocksMovingAnalysis.Status.BLOCKS_TARGETS_PAIRED;
         } else {
@@ -439,7 +416,7 @@ public class StoragePolicySatisfier implements Runnable {
           status = BlocksMovingAnalysis.Status.NO_BLOCKS_TARGETS_PAIRED;
         }
       } else {
-        if (blockManager.hasLowRedundancyBlocks(blockCollection)) {
+        if (hasLowRedundancyBlocks) {
           status = BlocksMovingAnalysis.Status.FEW_LOW_REDUNDANCY_BLOCKS;
         }
       }
@@ -448,13 +425,15 @@ public class StoragePolicySatisfier implements Runnable {
     List<Block> assignedBlockIds = new ArrayList<Block>();
     for (BlockMovingInfo blkMovingInfo : blockMovingInfos) {
       // 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);
+      try {
+        ctxt.assignBlockMoveTaskToTargetNode(blkMovingInfo);
         LOG.debug("BlockMovingInfo: {}", blkMovingInfo);
         assignedBlockIds.add(blkMovingInfo.getBlock());
         blockCount++;
+      } catch (IOException e) {
+        LOG.warn("Exception while scheduling movement task", e);
+        // failed to move the block.
+        status = BlocksMovingAnalysis.Status.BLOCKS_FAILED_TO_MOVE;
       }
     }
     return new BlocksMovingAnalysis(status, assignedBlockIds);
@@ -481,29 +460,29 @@ public class StoragePolicySatisfier implements Runnable {
    *         satisfy the storage policy, true otherwise
    */
   private boolean computeBlockMovingInfos(
-      List<BlockMovingInfo> blockMovingInfos, BlockInfo blockInfo,
+      List<BlockMovingInfo> blockMovingInfos, LocatedBlock blockInfo,
       List<StorageType> expectedStorageTypes, List<StorageType> existing,
-      DatanodeStorageInfo[] storages) {
+      DatanodeInfo[] storages, DatanodeStorageReport[] liveDns,
+      ErasureCodingPolicy ecPolicy) {
     boolean foundMatchingTargetNodesForBlock = true;
     if (!DFSUtil.removeOverlapBetweenStorageTypes(expectedStorageTypes,
         existing, true)) {
       List<StorageTypeNodePair> sourceWithStorageMap =
           new ArrayList<StorageTypeNodePair>();
-      List<DatanodeStorageInfo> existingBlockStorages =
-          new ArrayList<DatanodeStorageInfo>(Arrays.asList(storages));
+      List<DatanodeInfo> existingBlockStorages = new ArrayList<DatanodeInfo>(
+          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();
+      Iterator<DatanodeInfo> iterator = existingBlockStorages.iterator();
       while (iterator.hasNext()) {
-        DatanodeStorageInfo datanodeStorageInfo = iterator.next();
-        if (checkSourceAndTargetTypeExists(
-            datanodeStorageInfo.getDatanodeDescriptor(), existing,
-            expectedStorageTypes)) {
+        DatanodeInfoWithStorage dnInfo = (DatanodeInfoWithStorage) iterator
+            .next();
+        if (checkSourceAndTargetTypeExists(dnInfo, existing,
+            expectedStorageTypes, liveDns)) {
           sourceWithStorageMap
-              .add(new StorageTypeNodePair(datanodeStorageInfo.getStorageType(),
-                  datanodeStorageInfo.getDatanodeDescriptor()));
+              .add(new StorageTypeNodePair(dnInfo.getStorageType(), dnInfo));
           iterator.remove();
-          existing.remove(datanodeStorageInfo.getStorageType());
+          existing.remove(dnInfo.getStorageType());
         }
       }
 
@@ -511,23 +490,25 @@ public class StoragePolicySatisfier implements Runnable {
       for (StorageType existingType : existing) {
         iterator = existingBlockStorages.iterator();
         while (iterator.hasNext()) {
-          DatanodeStorageInfo datanodeStorageInfo = iterator.next();
-          StorageType storageType = datanodeStorageInfo.getStorageType();
+          DatanodeInfoWithStorage dnStorageInfo =
+              (DatanodeInfoWithStorage) iterator.next();
+          StorageType storageType = dnStorageInfo.getStorageType();
           if (storageType == existingType) {
             iterator.remove();
             sourceWithStorageMap.add(new StorageTypeNodePair(storageType,
-                datanodeStorageInfo.getDatanodeDescriptor()));
+                dnStorageInfo));
             break;
           }
         }
       }
 
       StorageTypeNodeMap locsForExpectedStorageTypes =
-          findTargetsForExpectedStorageTypes(expectedStorageTypes);
+          findTargetsForExpectedStorageTypes(expectedStorageTypes, liveDns);
 
       foundMatchingTargetNodesForBlock |= findSourceAndTargetToMove(
           blockMovingInfos, blockInfo, sourceWithStorageMap,
-          expectedStorageTypes, locsForExpectedStorageTypes);
+          expectedStorageTypes, locsForExpectedStorageTypes,
+          ecPolicy);
     }
     return foundMatchingTargetNodesForBlock;
   }
@@ -550,12 +531,13 @@ public class StoragePolicySatisfier implements Runnable {
    *         satisfy the storage policy
    */
   private boolean findSourceAndTargetToMove(
-      List<BlockMovingInfo> blockMovingInfos, BlockInfo blockInfo,
+      List<BlockMovingInfo> blockMovingInfos, LocatedBlock blockInfo,
       List<StorageTypeNodePair> sourceWithStorageList,
       List<StorageType> expected,
-      StorageTypeNodeMap locsForExpectedStorageTypes) {
+      StorageTypeNodeMap locsForExpectedStorageTypes,
+      ErasureCodingPolicy ecPolicy) {
     boolean foundMatchingTargetNodesForBlock = true;
-    List<DatanodeDescriptor> excludeNodes = new ArrayList<>();
+    List<DatanodeInfo> 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
@@ -566,13 +548,14 @@ public class StoragePolicySatisfier implements Runnable {
       // 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);
+        StorageTypeNodePair chosenTarget = chooseTargetTypeInSameNode(blockInfo,
+            existingTypeNodePair.dn, expected);
         if (chosenTarget != null) {
           if (blockInfo.isStriped()) {
             buildStripedBlockMovingInfos(blockInfo, existingTypeNodePair.dn,
                 existingTypeNodePair.storageType, chosenTarget.dn,
-                chosenTarget.storageType, blockMovingInfos);
+                chosenTarget.storageType, blockMovingInfos,
+                ecPolicy);
           } else {
             buildContinuousBlockMovingInfos(blockInfo, existingTypeNodePair.dn,
                 existingTypeNodePair.storageType, chosenTarget.dn,
@@ -596,7 +579,7 @@ public class StoragePolicySatisfier implements Runnable {
       if (checkIfAlreadyChosen(blockMovingInfos, existingTypeNodePair.dn)) {
         continue;
       }
-      if (chosenTarget == null && blockManager.getDatanodeManager()
+      if (chosenTarget == null && ctxt
           .getNetworkTopology().isNodeGroupAware()) {
         chosenTarget = chooseTarget(blockInfo, existingTypeNodePair.dn,
             expected, Matcher.SAME_NODE_GROUP, locsForExpectedStorageTypes,
@@ -619,7 +602,7 @@ public class StoragePolicySatisfier implements Runnable {
         if (blockInfo.isStriped()) {
           buildStripedBlockMovingInfos(blockInfo, existingTypeNodePair.dn,
               existingTypeNodePair.storageType, chosenTarget.dn,
-              chosenTarget.storageType, blockMovingInfos);
+              chosenTarget.storageType, blockMovingInfos, ecPolicy);
         } else {
           buildContinuousBlockMovingInfos(blockInfo, existingTypeNodePair.dn,
               existingTypeNodePair.storageType, chosenTarget.dn,
@@ -645,7 +628,7 @@ public class StoragePolicySatisfier implements Runnable {
   }
 
   private boolean checkIfAlreadyChosen(List<BlockMovingInfo> blockMovingInfos,
-      DatanodeDescriptor dn) {
+      DatanodeInfo dn) {
     for (BlockMovingInfo blockMovingInfo : blockMovingInfos) {
       if (blockMovingInfo.getSource().equals(dn)) {
         return true;
@@ -654,37 +637,40 @@ public class StoragePolicySatisfier implements Runnable {
     return false;
   }
 
-  private void buildContinuousBlockMovingInfos(BlockInfo blockInfo,
+  private void buildContinuousBlockMovingInfos(LocatedBlock blockInfo,
       DatanodeInfo sourceNode, StorageType sourceStorageType,
       DatanodeInfo targetNode, StorageType targetStorageType,
       List<BlockMovingInfo> blkMovingInfos) {
-    Block blk = new Block(blockInfo.getBlockId(), blockInfo.getNumBytes(),
-        blockInfo.getGenerationStamp());
+    Block blk = ExtendedBlock.getLocalBlock(blockInfo.getBlock());
     BlockMovingInfo blkMovingInfo = new BlockMovingInfo(blk, sourceNode,
         targetNode, sourceStorageType, targetStorageType);
     blkMovingInfos.add(blkMovingInfo);
   }
 
-  private void buildStripedBlockMovingInfos(BlockInfo blockInfo,
+  private void buildStripedBlockMovingInfos(LocatedBlock blockInfo,
       DatanodeInfo sourceNode, StorageType sourceStorageType,
       DatanodeInfo targetNode, StorageType targetStorageType,
-      List<BlockMovingInfo> blkMovingInfos) {
+      List<BlockMovingInfo> blkMovingInfos, ErasureCodingPolicy ecPolicy) {
     // 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();
-        if (sourceNode.equals(dn)) {
+    LocatedStripedBlock sBlockInfo = (LocatedStripedBlock) blockInfo;
+    byte[] indices = sBlockInfo.getBlockIndices();
+    DatanodeInfo[] locations = sBlockInfo.getLocations();
+    for (int i = 0; i < indices.length; i++) {
+      byte blkIndex = indices[i];
+      if (blkIndex >= 0) {
+        // pick block movement only for the given source node.
+        if (sourceNode.equals(locations[i])) {
           // construct internal block
-          long blockId = blockInfo.getBlockId() + si.getBlockIndex();
+          ExtendedBlock extBlock = sBlockInfo.getBlock();
           long numBytes = StripedBlockUtil.getInternalBlockLength(
-              sBlockInfo.getNumBytes(), sBlockInfo.getCellSize(),
-              sBlockInfo.getDataBlockNum(), si.getBlockIndex());
-          Block blk = new Block(blockId, numBytes,
-              blockInfo.getGenerationStamp());
+              extBlock.getNumBytes(), ecPolicy, blkIndex);
+          Block blk = new Block(ExtendedBlock.getLocalBlock(extBlock));
+          long blkId = blk.getBlockId() + blkIndex;
+          blk.setBlockId(blkId);
+          blk.setNumBytes(numBytes);
           BlockMovingInfo blkMovingInfo = new BlockMovingInfo(blk, sourceNode,
               targetNode, sourceStorageType, targetStorageType);
           blkMovingInfos.add(blkMovingInfo);
@@ -703,34 +689,35 @@ public class StoragePolicySatisfier implements Runnable {
    * @param targetTypes
    *          - list of target storage types
    */
-  private StorageTypeNodePair chooseTargetTypeInSameNode(Block block,
-      DatanodeDescriptor source, List<StorageType> targetTypes) {
+  private StorageTypeNodePair chooseTargetTypeInSameNode(LocatedBlock blockInfo,
+      DatanodeInfo source, List<StorageType> targetTypes) {
     for (StorageType t : targetTypes) {
-      DatanodeStorageInfo chooseStorage4Block =
-          source.chooseStorage4Block(t, block.getNumBytes());
-      if (chooseStorage4Block != null) {
+      boolean goodTargetDn = ctxt.verifyTargetDatanodeHasSpaceForScheduling(
+          source, t, blockInfo.getBlockSize());
+      if (goodTargetDn) {
         return new StorageTypeNodePair(t, source);
       }
     }
     return null;
   }
 
-  private StorageTypeNodePair chooseTarget(Block block,
-      DatanodeDescriptor source, List<StorageType> targetTypes, Matcher matcher,
+  private StorageTypeNodePair chooseTarget(LocatedBlock block,
+      DatanodeInfo source, List<StorageType> targetTypes, Matcher matcher,
       StorageTypeNodeMap locsForExpectedStorageTypes,
-      List<DatanodeDescriptor> excludeNodes) {
+      List<DatanodeInfo> excludeNodes) {
     for (StorageType t : targetTypes) {
-      List<DatanodeDescriptor> nodesWithStorages =
-          locsForExpectedStorageTypes.getNodesWithStorages(t);
+      List<DatanodeInfo> 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 (!excludeNodes.contains(target) && matcher.match(
-            blockManager.getDatanodeManager().getNetworkTopology(), source,
-            target)) {
-          if (null != target.chooseStorage4Block(t, block.getNumBytes())) {
+      for (DatanodeInfo target : nodesWithStorages) {
+        if (!excludeNodes.contains(target)
+            && matcher.match(ctxt.getNetworkTopology(), source, target)) {
+          boolean goodTargetDn = ctxt.verifyTargetDatanodeHasSpaceForScheduling(
+              target, t, block.getBlockSize());
+          if (goodTargetDn) {
             return new StorageTypeNodePair(t, target);
           }
         }
@@ -741,27 +728,25 @@ public class StoragePolicySatisfier implements Runnable {
 
   private static class StorageTypeNodePair {
     private StorageType storageType = null;
-    private DatanodeDescriptor dn = null;
+    private DatanodeInfo dn = null;
 
-    StorageTypeNodePair(StorageType storageType, DatanodeDescriptor dn) {
+    StorageTypeNodePair(StorageType storageType, DatanodeInfo dn) {
       this.storageType = storageType;
       this.dn = dn;
     }
   }
 
   private StorageTypeNodeMap findTargetsForExpectedStorageTypes(
-      List<StorageType> expected) {
+      List<StorageType> expected, DatanodeStorageReport[] liveDns) {
     StorageTypeNodeMap targetMap = new StorageTypeNodeMap();
-    List<DatanodeDescriptor> reports = blockManager.getDatanodeManager()
-        .getDatanodeListForReport(DatanodeReportType.LIVE);
-    for (DatanodeDescriptor dn : reports) {
+    for (DatanodeStorageReport dn : liveDns) {
       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);
+            targetMap.add(t, dn.getDatanodeInfo());
           }
         }
       }
@@ -782,32 +767,40 @@ public class StoragePolicySatisfier implements Runnable {
     return max;
   }
 
-  private boolean checkSourceAndTargetTypeExists(DatanodeDescriptor dn,
-      List<StorageType> existing, List<StorageType> expectedStorageTypes) {
-    DatanodeStorageInfo[] allDNStorageInfos = dn.getStorageInfos();
+  private boolean checkSourceAndTargetTypeExists(DatanodeInfo dn,
+      List<StorageType> existing, List<StorageType> expectedStorageTypes,
+      DatanodeStorageReport[] liveDns) {
     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;
+    for (DatanodeStorageReport liveDn : liveDns) {
+      if (dn.equals(liveDn.getDatanodeInfo())) {
+        StorageReport[] storageReports = liveDn.getStorageReports();
+        for (StorageReport eachStorage : storageReports) {
+          StorageType storageType = eachStorage.getStorage().getStorageType();
+          if (existing.contains(storageType)) {
+            isExistingTypeAvailable = true;
+          }
+          if (expectedStorageTypes.contains(storageType)) {
+            isExpectedTypeAvailable = true;
+          }
+          if (isExistingTypeAvailable && isExpectedTypeAvailable) {
+            return true;
+          }
+        }
       }
     }
     return isExistingTypeAvailable && isExpectedTypeAvailable;
   }
 
   private static class StorageTypeNodeMap {
-    private final EnumMap<StorageType, List<DatanodeDescriptor>> typeNodeMap =
-        new EnumMap<StorageType, List<DatanodeDescriptor>>(StorageType.class);
+    private final EnumMap<StorageType, List<DatanodeInfo>> typeNodeMap =
+        new EnumMap<StorageType, List<DatanodeInfo>>(StorageType.class);
 
-    private void add(StorageType t, DatanodeDescriptor dn) {
-      List<DatanodeDescriptor> nodesWithStorages = getNodesWithStorages(t);
-      LinkedList<DatanodeDescriptor> value = null;
+    private void add(StorageType t, DatanodeInfo dn) {
+      List<DatanodeInfo> nodesWithStorages = getNodesWithStorages(t);
+      LinkedList<DatanodeInfo> value = null;
       if (nodesWithStorages == null) {
-        value = new LinkedList<DatanodeDescriptor>();
+        value = new LinkedList<DatanodeInfo>();
         value.add(dn);
         typeNodeMap.put(t, value);
       } else {
@@ -820,7 +813,7 @@ public class StoragePolicySatisfier implements Runnable {
      *          - Storage type
      * @return datanodes which has the given storage type
      */
-    private List<DatanodeDescriptor> getNodesWithStorages(StorageType type) {
+    private List<DatanodeInfo> getNodesWithStorages(StorageType type) {
       return typeNodeMap.get(type);
     }
   }
@@ -982,7 +975,6 @@ public class StoragePolicySatisfier implements Runnable {
 
   public StoragePolicySatisfyPathStatus checkStoragePolicySatisfyPathStatus(
       String path) throws IOException {
-    INode inode = namesystem.getFSDirectory().getINode(path);
-    return storageMovementNeeded.getStatus(inode.getId());
+    return storageMovementNeeded.getStatus(ctxt.getFileID(path));
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/05d4daf6/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/sps/TestBlockStorageMovementAttemptedItems.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/sps/TestBlockStorageMovementAttemptedItems.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/sps/TestBlockStorageMovementAttemptedItems.java
index 62766d9..f9762a8 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/sps/TestBlockStorageMovementAttemptedItems.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/sps/TestBlockStorageMovementAttemptedItems.java
@@ -25,8 +25,9 @@ import static org.junit.Assert.assertTrue;
 import java.util.ArrayList;
 import java.util.List;
 
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.protocol.Block;
-import org.apache.hadoop.hdfs.server.namenode.Namesystem;
 import org.apache.hadoop.hdfs.server.namenode.sps.StoragePolicySatisfier.AttemptedItemInfo;
 import org.apache.hadoop.hdfs.server.namenode.sps.StoragePolicySatisfier.ItemInfo;
 import org.junit.After;
@@ -46,11 +47,15 @@ public class TestBlockStorageMovementAttemptedItems {
 
   @Before
   public void setup() throws Exception {
-    unsatisfiedStorageMovementFiles = new BlockStorageMovementNeeded(
-        Mockito.mock(Namesystem.class),
-        Mockito.mock(StoragePolicySatisfier.class), 100);
-    bsmAttemptedItems = new BlockStorageMovementAttemptedItems(100,
-        selfRetryTimeout, unsatisfiedStorageMovementFiles);
+    Configuration config = new HdfsConfiguration();
+    Context ctxt = Mockito.mock(Context.class);
+    Mockito.when(ctxt.getConf()).thenReturn(config);
+    Mockito.when(ctxt.isRunning()).thenReturn(true);
+    Mockito.when(ctxt.isInSafeMode()).thenReturn(false);
+    Mockito.when(ctxt.isFileExist(Mockito.anyLong())).thenReturn(true);
+    unsatisfiedStorageMovementFiles = new BlockStorageMovementNeeded(ctxt);
+    bsmAttemptedItems = new BlockStorageMovementAttemptedItems(ctxt,
+        unsatisfiedStorageMovementFiles);
   }
 
   @After

http://git-wip-us.apache.org/repos/asf/hadoop/blob/05d4daf6/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/sps/TestStoragePolicySatisfier.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/sps/TestStoragePolicySatisfier.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/sps/TestStoragePolicySatisfier.java
index 8dc52dc..2a7bde5 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/sps/TestStoragePolicySatisfier.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/sps/TestStoragePolicySatisfier.java
@@ -18,6 +18,7 @@
 package org.apache.hadoop.hdfs.server.namenode.sps;
 
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_STORAGE_POLICY_ENABLED_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_QUEUE_LIMIT_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;
@@ -68,6 +69,7 @@ import org.apache.hadoop.hdfs.server.namenode.FSTreeTraverser;
 import org.apache.hadoop.hdfs.server.namenode.INode;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.test.GenericTestUtils.LogCapturer;
+import org.junit.After;
 import org.junit.Assert;
 import org.junit.Test;
 import org.mockito.Mockito;
@@ -105,7 +107,8 @@ public class TestStoragePolicySatisfier {
   private DistributedFileSystem dfs = null;
   private static final int DEFAULT_BLOCK_SIZE = 1024;
 
-  private void shutdownCluster() {
+  @After
+  public void shutdownCluster() {
     if (hdfsCluster != null) {
       hdfsCluster.shutdown();
     }
@@ -1298,11 +1301,17 @@ public class TestStoragePolicySatisfier {
     //entry in queue. After 10 files, traverse control will be on U.
     StoragePolicySatisfier sps = Mockito.mock(StoragePolicySatisfier.class);
     Mockito.when(sps.isRunning()).thenReturn(true);
+    Context ctxt = Mockito.mock(Context.class);
+    config.setInt(DFS_STORAGE_POLICY_SATISFIER_QUEUE_LIMIT_KEY, 10);
+    Mockito.when(ctxt.getConf()).thenReturn(config);
+    Mockito.when(ctxt.isRunning()).thenReturn(true);
+    Mockito.when(ctxt.isInSafeMode()).thenReturn(false);
+    Mockito.when(ctxt.isFileExist(Mockito.anyLong())).thenReturn(true);
     BlockStorageMovementNeeded movmentNeededQueue =
-        new BlockStorageMovementNeeded(hdfsCluster.getNamesystem(), sps, 10);
+        new BlockStorageMovementNeeded(ctxt);
     INode rootINode = fsDir.getINode("/root");
     movmentNeededQueue.addToPendingDirQueue(rootINode.getId());
-    movmentNeededQueue.init();
+    movmentNeededQueue.init(fsDir);
 
     //Wait for thread to reach U.
     Thread.sleep(1000);
@@ -1361,9 +1370,15 @@ public class TestStoragePolicySatisfier {
     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.
+    Context ctxt = Mockito.mock(Context.class);
+    config.setInt(DFS_STORAGE_POLICY_SATISFIER_QUEUE_LIMIT_KEY, 10);
+    Mockito.when(ctxt.getConf()).thenReturn(config);
+    Mockito.when(ctxt.isRunning()).thenReturn(true);
+    Mockito.when(ctxt.isInSafeMode()).thenReturn(false);
+    Mockito.when(ctxt.isFileExist(Mockito.anyLong())).thenReturn(true);
     BlockStorageMovementNeeded movmentNeededQueue =
-        new BlockStorageMovementNeeded(hdfsCluster.getNamesystem(), sps, 10);
-    movmentNeededQueue.init();
+        new BlockStorageMovementNeeded(ctxt);
+    movmentNeededQueue.init(fsDir);
     INode rootINode = fsDir.getINode("/root");
     movmentNeededQueue.addToPendingDirQueue(rootINode.getId());
     // Wait for thread to reach U.


---------------------------------------------------------------------
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-12955: [SPS]: Move SPS classes to a separate package. Contributed by Rakesh R.

Posted by um...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/78420719/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
deleted file mode 100644
index 6991ad2..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStoragePolicySatisfierWithStripedFile.java
+++ /dev/null
@@ -1,580 +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.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-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.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;
-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 StripedFileTestUtil.getDefaultECPolicy();
-  }
-
-  /**
-   * 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();
-    conf.setBoolean(DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_ENABLED_KEY,
-        true);
-    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();
-      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();
-      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,
-          StripedFileTestUtil.getDefaultECPolicy().getName());
-
-      // 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();
-
-      waitForBlocksMovementAttemptReport(cluster, 9, 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();
-    conf.setBoolean(DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_ENABLED_KEY,
-        true);
-    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();
-      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();
-      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,
-          StripedFileTestUtil.getDefaultECPolicy().getName());
-
-      // 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();
-
-      waitForBlocksMovementAttemptReport(cluster, 5, 60000);
-      waitForAttemptedItems(cluster, 1, 30000);
-      // verify storage types and locations.
-      waitExpectedStorageType(cluster, fooFile, fileLen, StorageType.ARCHIVE, 5,
-          9, 60000);
-    } finally {
-      cluster.shutdown();
-    }
-  }
-
-  /**
-   * 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 9 datanodes
-    int numOfDatanodes = 9;
-    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");
-    conf.setBoolean(DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_ENABLED_KEY,
-        true);
-    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}})
-        .storageCapacities(capacities)
-        .build();
-    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
-      fs.setErasureCodingPolicy(barDir,
-          StripedFileTestUtil.getDefaultECPolicy().getName());
-
-      // 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 < 5; i++) {
-        cluster.restartDataNode(list.get(i), false);
-      }
-      cluster.waitActive();
-      fs.satisfyStoragePolicy(fooFile);
-      DFSTestUtil.waitExpectedStorageType(fooFile.toString(),
-          StorageType.ARCHIVE, 5, 30000, cluster.getFileSystem());
-      //Start reaming datanodes
-      for (int i = numOfDatanodes - 1; i >= 5; i--) {
-        cluster.restartDataNode(list.get(i), false);
-      }
-      // 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.
-   *
-   * 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();
-    conf.setBoolean(DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_ENABLED_KEY,
-        true);
-    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();
-      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();
-      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,
-          StripedFileTestUtil.getDefaultECPolicy().getName());
-
-      // 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 attempt report has been arrived at the
-  // Namenode(SPS).
-  private void waitForBlocksMovementAttemptReport(MiniDFSCluster cluster,
-      long expectedMovementFinishedBlocksCount, 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("MovementFinishedBlocks: expectedCount={} actualCount={}",
-            expectedMovementFinishedBlocksCount,
-            sps.getAttemptedItemsMonitor().getMovementFinishedBlocksCount());
-        return sps.getAttemptedItemsMonitor().getMovementFinishedBlocksCount()
-            >= expectedMovementFinishedBlocksCount;
-      }
-    }, 100, timeout);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/78420719/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/sps/TestBlockStorageMovementAttemptedItems.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/sps/TestBlockStorageMovementAttemptedItems.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/sps/TestBlockStorageMovementAttemptedItems.java
new file mode 100644
index 0000000..62766d9
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/sps/TestBlockStorageMovementAttemptedItems.java
@@ -0,0 +1,196 @@
+/**
+ * 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.sps;
+
+import static org.apache.hadoop.util.Time.monotonicNow;
+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.Namesystem;
+import org.apache.hadoop.hdfs.server.namenode.sps.StoragePolicySatisfier.AttemptedItemInfo;
+import org.apache.hadoop.hdfs.server.namenode.sps.StoragePolicySatisfier.ItemInfo;
+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
+ * processed them correctly or not.
+ */
+public class TestBlockStorageMovementAttemptedItems {
+
+  private BlockStorageMovementAttemptedItems bsmAttemptedItems = null;
+  private BlockStorageMovementNeeded unsatisfiedStorageMovementFiles = null;
+  private final int selfRetryTimeout = 500;
+
+  @Before
+  public void setup() throws Exception {
+    unsatisfiedStorageMovementFiles = new BlockStorageMovementNeeded(
+        Mockito.mock(Namesystem.class),
+        Mockito.mock(StoragePolicySatisfier.class), 100);
+    bsmAttemptedItems = new BlockStorageMovementAttemptedItems(100,
+        selfRetryTimeout, unsatisfiedStorageMovementFiles);
+  }
+
+  @After
+  public void teardown() {
+    if (bsmAttemptedItems != null) {
+      bsmAttemptedItems.stop();
+      bsmAttemptedItems.stopGracefully();
+    }
+  }
+
+  private boolean checkItemMovedForRetry(Long item, long retryTimeout)
+      throws InterruptedException {
+    long stopTime = monotonicNow() + (retryTimeout * 2);
+    boolean isItemFound = false;
+    while (monotonicNow() < (stopTime)) {
+      ItemInfo ele = null;
+      while ((ele = unsatisfiedStorageMovementFiles.get()) != null) {
+        if (item == ele.getTrackId()) {
+          isItemFound = true;
+          break;
+        }
+      }
+      if (!isItemFound) {
+        Thread.sleep(100);
+      } else {
+        break;
+      }
+    }
+    return isItemFound;
+  }
+
+  /**
+   * Verify that moved blocks reporting should queued up the block info.
+   */
+  @Test(timeout = 30000)
+  public void testAddReportedMoveAttemptFinishedBlocks() throws Exception {
+    bsmAttemptedItems.start(); // start block movement result monitor thread
+    Long item = new Long(1234);
+    List<Block> blocks = new ArrayList<Block>();
+    blocks.add(new Block(item));
+    bsmAttemptedItems.add(new AttemptedItemInfo(0L, 0L, 0L, blocks, 0));
+    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 testNoBlockMovementAttemptFinishedReportAdded() throws Exception {
+    bsmAttemptedItems.start(); // start block movement report monitor thread
+    Long item = new Long(1234);
+    List<Block> blocks = new ArrayList<>();
+    blocks.add(new Block(item));
+    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,
+        bsmAttemptedItems.getAttemptedItemsCount());
+  }
+
+  /**
+   * 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);
+    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, 0));
+    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(trackID, 5000));
+    assertEquals("Failed to remove from the attempted list", 0,
+        bsmAttemptedItems.getAttemptedItemsCount());
+  }
+
+  /**
+   * Partial block movement. Here, first occurrence is
+   * #blocksStorageMovementUnReportedItemsCheck() and then
+   * #blockStorageMovementReportedItemsCheck().
+   */
+  @Test(timeout = 30000)
+  public void testPartialBlockMovementShouldBeRetried2() throws Exception {
+    Long item = new Long(1234);
+    Long trackID = 0L;
+    List<Block> blocks = new ArrayList<>();
+    blocks.add(new Block(item));
+    bsmAttemptedItems
+        .add(new AttemptedItemInfo(trackID, trackID, 0L, blocks, 0));
+    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.blockStorageMovementReportedItemsCheck();
+
+    assertTrue("Failed to add to the retry list",
+        checkItemMovedForRetry(trackID, 5000));
+    assertEquals("Failed to remove from the attempted list", 0,
+        bsmAttemptedItems.getAttemptedItemsCount());
+  }
+
+  /**
+   * 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);
+    Long trackID = 0L;
+    List<Block> blocks = new ArrayList<>();
+    blocks.add(new Block(item));
+    bsmAttemptedItems
+        .add(new AttemptedItemInfo(trackID, trackID, 0L, blocks, 0));
+    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(trackID, 5000));
+    assertEquals("Failed to remove from the attempted list", 1,
+        bsmAttemptedItems.getAttemptedItemsCount());
+  }
+}


---------------------------------------------------------------------
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-12570: [SPS]: Refactor Co-ordinator datanode logic to track the block storage movements. Contributed by Rakesh R.

Posted by um...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/00eceed2/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/00eceed2/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


[15/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 um...@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/00eceed2
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/00eceed2
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/00eceed2

Branch: refs/heads/HDFS-10285
Commit: 00eceed233d6e80d5c7137bf5b5286746ec4d5fb
Parents: bfd3f8b
Author: Uma Maheswara Rao G <um...@intel.com>
Authored: Thu Oct 12 17:17:51 2017 -0700
Committer: Uma Maheswara Rao Gangumalla <um...@apache.org>
Committed: Sun Aug 12 03:06:01 2018 -0700

----------------------------------------------------------------------
 .../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/00eceed2/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 c90ca33..d577e4c 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
@@ -629,11 +629,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/00eceed2/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/00eceed2/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/00eceed2/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 996b986..38f72c0 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
@@ -42,6 +42,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;
@@ -56,11 +57,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.ProvidedStorageLocationProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.StorageUuidsProto;
@@ -104,8 +105,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;
@@ -971,59 +971,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) {
@@ -1211,34 +1179,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();
   }
@@ -1246,42 +1214,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/00eceed2/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 0311b02..f9a76b4 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;
@@ -212,7 +211,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;
 
@@ -1079,30 +1078,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/00eceed2/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 c8d31fd..b1db377 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,
@@ -1094,13 +1102,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
@@ -1680,19 +1689,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(
@@ -1708,6 +1745,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
@@ -1751,24 +1805,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/00eceed2/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 9308471..1656b16 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
@@ -799,8 +799,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/00eceed2/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/00eceed2/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/00eceed2/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/00eceed2/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/00eceed2/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/00eceed2/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 8b73f1e..b0be914 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
@@ -266,7 +266,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;
@@ -3927,7 +3927,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
@@ -3948,11 +3949,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


[27/50] [abbrv] hadoop git commit: HDFS-12911. [SPS]: Modularize the SPS code and expose necessary interfaces for external/internal implementations. Contributed by Uma Maheswara Rao G

Posted by um...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/8d4f74e7/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/sps/TestStoragePolicySatisfier.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/sps/TestStoragePolicySatisfier.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/sps/TestStoragePolicySatisfier.java
index 2a7bde5..9354044 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/sps/TestStoragePolicySatisfier.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/sps/TestStoragePolicySatisfier.java
@@ -72,7 +72,6 @@ import org.apache.hadoop.test.GenericTestUtils.LogCapturer;
 import org.junit.After;
 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;
@@ -147,12 +146,11 @@ public class TestStoragePolicySatisfier {
     startAdditionalDNs(config, 3, numOfDatanodes, newtypes,
         storagesPerDatanode, capacity, hdfsCluster);
 
-    dfs.satisfyStoragePolicy(new Path(file));
-
     hdfsCluster.triggerHeartbeats();
+    dfs.satisfyStoragePolicy(new Path(file));
     // Wait till namenode notified about the block location details
-    DFSTestUtil.waitExpectedStorageType(
-        file, StorageType.ARCHIVE, 3, 30000, dfs);
+    DFSTestUtil.waitExpectedStorageType(file, StorageType.ARCHIVE, 3, 35000,
+        dfs);
   }
 
   @Test(timeout = 300000)
@@ -1284,6 +1282,7 @@ public class TestStoragePolicySatisfier {
         {StorageType.ARCHIVE, StorageType.SSD},
         {StorageType.DISK, StorageType.DISK}};
     config.setLong("dfs.block.size", DEFAULT_BLOCK_SIZE);
+    config.setInt(DFS_STORAGE_POLICY_SATISFIER_QUEUE_LIMIT_KEY, 10);
     hdfsCluster = startCluster(config, diskTypes, diskTypes.length,
         storagesPerDatanode, capacity);
     dfs = hdfsCluster.getFileSystem();
@@ -1299,19 +1298,28 @@ public class TestStoragePolicySatisfier {
 
     //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);
-    Context ctxt = Mockito.mock(Context.class);
-    config.setInt(DFS_STORAGE_POLICY_SATISFIER_QUEUE_LIMIT_KEY, 10);
-    Mockito.when(ctxt.getConf()).thenReturn(config);
-    Mockito.when(ctxt.isRunning()).thenReturn(true);
-    Mockito.when(ctxt.isInSafeMode()).thenReturn(false);
-    Mockito.when(ctxt.isFileExist(Mockito.anyLong())).thenReturn(true);
-    BlockStorageMovementNeeded movmentNeededQueue =
-        new BlockStorageMovementNeeded(ctxt);
+    StoragePolicySatisfier sps = new StoragePolicySatisfier(config);
+    Context ctxt = new IntraSPSNameNodeContext(hdfsCluster.getNamesystem(),
+        hdfsCluster.getNamesystem().getBlockManager(), sps) {
+      @Override
+      public boolean isInSafeMode() {
+        return false;
+      }
+
+      @Override
+      public boolean isRunning() {
+        return true;
+      }
+    };
+
+    FileIdCollector fileIDCollector =
+        new IntraSPSNameNodeFileIdCollector(fsDir, sps);
+    sps.init(ctxt, fileIDCollector, null);
+    sps.getStorageMovementQueue().activate();
+
     INode rootINode = fsDir.getINode("/root");
-    movmentNeededQueue.addToPendingDirQueue(rootINode.getId());
-    movmentNeededQueue.init(fsDir);
+    hdfsCluster.getNamesystem().getBlockManager()
+        .addSPSPathId(rootINode.getId());
 
     //Wait for thread to reach U.
     Thread.sleep(1000);
@@ -1321,7 +1329,7 @@ public class TestStoragePolicySatisfier {
     // 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();
+      long trackId = sps.getStorageMovementQueue().get().getFileId();
       INode inode = fsDir.getInode(trackId);
       assertTrue("Failed to traverse tree, expected " + path + " but got "
           + inode.getFullPathName(), path.equals(inode.getFullPathName()));
@@ -1332,7 +1340,7 @@ public class TestStoragePolicySatisfier {
     // 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();
+      long trackId = sps.getStorageMovementQueue().get().getFileId();
       INode inode = fsDir.getInode(trackId);
       assertTrue("Failed to traverse tree, expected " + path + " but got "
           + inode.getFullPathName(), path.equals(inode.getFullPathName()));
@@ -1352,6 +1360,7 @@ public class TestStoragePolicySatisfier {
         {StorageType.ARCHIVE, StorageType.SSD},
         {StorageType.DISK, StorageType.DISK}};
     config.setLong("dfs.block.size", DEFAULT_BLOCK_SIZE);
+    config.setInt(DFS_STORAGE_POLICY_SATISFIER_QUEUE_LIMIT_KEY, 10);
     hdfsCluster = startCluster(config, diskTypes, diskTypes.length,
         storagesPerDatanode, capacity);
     dfs = hdfsCluster.getFileSystem();
@@ -1366,21 +1375,33 @@ public class TestStoragePolicySatisfier {
     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.
-    Context ctxt = Mockito.mock(Context.class);
-    config.setInt(DFS_STORAGE_POLICY_SATISFIER_QUEUE_LIMIT_KEY, 10);
-    Mockito.when(ctxt.getConf()).thenReturn(config);
-    Mockito.when(ctxt.isRunning()).thenReturn(true);
-    Mockito.when(ctxt.isInSafeMode()).thenReturn(false);
-    Mockito.when(ctxt.isFileExist(Mockito.anyLong())).thenReturn(true);
-    BlockStorageMovementNeeded movmentNeededQueue =
-        new BlockStorageMovementNeeded(ctxt);
-    movmentNeededQueue.init(fsDir);
+    // StoragePolicySatisfier sps = new StoragePolicySatisfier(config);
+    StoragePolicySatisfier sps = new StoragePolicySatisfier(config);
+    Context ctxt = new IntraSPSNameNodeContext(hdfsCluster.getNamesystem(),
+        hdfsCluster.getNamesystem().getBlockManager(), sps) {
+      @Override
+      public boolean isInSafeMode() {
+        return false;
+      }
+
+      @Override
+      public boolean isRunning() {
+        return true;
+      }
+    };
+
+    FileIdCollector fileIDCollector =
+        new IntraSPSNameNodeFileIdCollector(fsDir, sps);
+    sps.init(ctxt, fileIDCollector, null);
+    sps.getStorageMovementQueue().activate();
+
     INode rootINode = fsDir.getINode("/root");
-    movmentNeededQueue.addToPendingDirQueue(rootINode.getId());
+    hdfsCluster.getNamesystem().getBlockManager()
+        .addSPSPathId(rootINode.getId());
+
     // Wait for thread to reach U.
     Thread.sleep(1000);
 
@@ -1389,7 +1410,7 @@ public class TestStoragePolicySatisfier {
     // 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();
+      long trackId = sps.getStorageMovementQueue().get().getFileId();
       INode inode = fsDir.getInode(trackId);
       assertTrue("Failed to traverse tree, expected " + path + " but got "
           + inode.getFullPathName(), path.equals(inode.getFullPathName()));
@@ -1400,7 +1421,7 @@ public class TestStoragePolicySatisfier {
     // 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();
+      long trackId = sps.getStorageMovementQueue().get().getFileId();
       INode inode = fsDir.getInode(trackId);
       assertTrue("Failed to traverse tree, expected " + path + " but got "
           + inode.getFullPathName(), path.equals(inode.getFullPathName()));
@@ -1502,17 +1523,20 @@ public class TestStoragePolicySatisfier {
       hdfsCluster = new MiniDFSCluster.Builder(config).numDataNodes(2)
           .storageTypes(storagetypes).build();
       hdfsCluster.waitActive();
-      BlockStorageMovementNeeded.setStatusClearanceElapsedTimeMs(20000);
+      // BlockStorageMovementNeeded.setStatusClearanceElapsedTimeMs(200000);
       dfs = hdfsCluster.getFileSystem();
       Path filePath = new Path("/file");
       DFSTestUtil.createFile(dfs, filePath, 1024, (short) 2,
             0);
       dfs.setStoragePolicy(filePath, "COLD");
       dfs.satisfyStoragePolicy(filePath);
+      Thread.sleep(3000);
       StoragePolicySatisfyPathStatus status = dfs.getClient()
           .checkStoragePolicySatisfyPathStatus(filePath.toString());
-      Assert.assertTrue("Status should be IN_PROGRESS",
-          StoragePolicySatisfyPathStatus.IN_PROGRESS.equals(status));
+      Assert.assertTrue(
+          "Status should be IN_PROGRESS/SUCCESS, but status is " + status,
+          StoragePolicySatisfyPathStatus.IN_PROGRESS.equals(status)
+              || StoragePolicySatisfyPathStatus.SUCCESS.equals(status));
       DFSTestUtil.waitExpectedStorageType(filePath.toString(),
           StorageType.ARCHIVE, 2, 30000, dfs);
 
@@ -1530,7 +1554,7 @@ public class TestStoragePolicySatisfier {
           return false;
         }
       }, 100, 60000);
-
+      BlockStorageMovementNeeded.setStatusClearanceElapsedTimeMs(1000);
       // wait till status is NOT_AVAILABLE
       GenericTestUtils.waitFor(new Supplier<Boolean>() {
         @Override
@@ -1719,8 +1743,10 @@ public class TestStoragePolicySatisfier {
       public Boolean get() {
         LOG.info("expectedAttemptedItemsCount={} actualAttemptedItemsCount={}",
             expectedBlkMovAttemptedCount,
-            sps.getAttemptedItemsMonitor().getAttemptedItemsCount());
-        return sps.getAttemptedItemsMonitor()
+            ((BlockStorageMovementAttemptedItems) (sps
+                .getAttemptedItemsMonitor())).getAttemptedItemsCount());
+        return ((BlockStorageMovementAttemptedItems) (sps
+            .getAttemptedItemsMonitor()))
             .getAttemptedItemsCount() == expectedBlkMovAttemptedCount;
       }
     }, 100, timeout);
@@ -1736,8 +1762,11 @@ public class TestStoragePolicySatisfier {
       public Boolean get() {
         LOG.info("MovementFinishedBlocks: expectedCount={} actualCount={}",
             expectedMovementFinishedBlocksCount,
-            sps.getAttemptedItemsMonitor().getMovementFinishedBlocksCount());
-        return sps.getAttemptedItemsMonitor().getMovementFinishedBlocksCount()
+            ((BlockStorageMovementAttemptedItems) (sps
+                .getAttemptedItemsMonitor())).getMovementFinishedBlocksCount());
+        return ((BlockStorageMovementAttemptedItems) (sps
+            .getAttemptedItemsMonitor()))
+                .getMovementFinishedBlocksCount()
             >= expectedMovementFinishedBlocksCount;
       }
     }, 100, timeout);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8d4f74e7/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/sps/TestStoragePolicySatisfierWithStripedFile.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/sps/TestStoragePolicySatisfierWithStripedFile.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/sps/TestStoragePolicySatisfierWithStripedFile.java
index c1a2b8b..0e3a5a3 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/sps/TestStoragePolicySatisfierWithStripedFile.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/sps/TestStoragePolicySatisfierWithStripedFile.java
@@ -500,9 +500,11 @@ public class TestStoragePolicySatisfierWithStripedFile {
       public Boolean get() {
         LOG.info("expectedAttemptedItemsCount={} actualAttemptedItemsCount={}",
             expectedBlkMovAttemptedCount,
-            sps.getAttemptedItemsMonitor().getAttemptedItemsCount());
-        return sps.getAttemptedItemsMonitor()
-            .getAttemptedItemsCount() == expectedBlkMovAttemptedCount;
+            ((BlockStorageMovementAttemptedItems) sps
+                .getAttemptedItemsMonitor()).getAttemptedItemsCount());
+        return ((BlockStorageMovementAttemptedItems) sps
+            .getAttemptedItemsMonitor())
+                .getAttemptedItemsCount() == expectedBlkMovAttemptedCount;
       }
     }, 100, timeout);
   }
@@ -560,7 +562,7 @@ public class TestStoragePolicySatisfierWithStripedFile {
   // Check whether the block movement attempt report has been arrived at the
   // Namenode(SPS).
   private void waitForBlocksMovementAttemptReport(MiniDFSCluster cluster,
-      long expectedMovementFinishedBlocksCount, int timeout)
+      long expectedMoveFinishedBlks, int timeout)
           throws TimeoutException, InterruptedException {
     BlockManager blockManager = cluster.getNamesystem().getBlockManager();
     final StoragePolicySatisfier sps = blockManager.getStoragePolicySatisfier();
@@ -570,10 +572,11 @@ public class TestStoragePolicySatisfierWithStripedFile {
       @Override
       public Boolean get() {
         LOG.info("MovementFinishedBlocks: expectedCount={} actualCount={}",
-            expectedMovementFinishedBlocksCount,
-            sps.getAttemptedItemsMonitor().getMovementFinishedBlocksCount());
-        return sps.getAttemptedItemsMonitor().getMovementFinishedBlocksCount()
-            >= expectedMovementFinishedBlocksCount;
+            expectedMoveFinishedBlks, ((BlockStorageMovementAttemptedItems) sps
+                .getAttemptedItemsMonitor()).getMovementFinishedBlocksCount());
+        return ((BlockStorageMovementAttemptedItems) sps
+            .getAttemptedItemsMonitor())
+                .getMovementFinishedBlocksCount() >= expectedMoveFinishedBlks;
       }
     }, 100, timeout);
   }


---------------------------------------------------------------------
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-12556: [SPS] : Block movement analysis should be done in read lock.

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

Branch: refs/heads/HDFS-10285
Commit: 5780f0624de2531194bc98eb25a928f7a483b992
Parents: 00eceed
Author: Surendra Singh Lilhore <su...@apache.org>
Authored: Sat Oct 14 15:11:26 2017 +0530
Committer: Uma Maheswara Rao Gangumalla <um...@apache.org>
Committed: Sun Aug 12 03:06:01 2018 -0700

----------------------------------------------------------------------
 .../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/5780f062/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/5780f062/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


[05/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 um...@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/5eb24ef7
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/5eb24ef7
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/5eb24ef7

Branch: refs/heads/HDFS-10285
Commit: 5eb24ef7e7b8fb61a5f5b88bae3596b30aaeb60b
Parents: 0b360b1
Author: Uma Maheswara Rao G <um...@intel.com>
Authored: Wed Jul 12 17:56:56 2017 -0700
Committer: Uma Maheswara Rao Gangumalla <um...@apache.org>
Committed: Sun Aug 12 03:06:00 2018 -0700

----------------------------------------------------------------------
 .../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/5eb24ef7/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/5eb24ef7/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/5eb24ef7/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


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

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

Branch: refs/heads/HDFS-10285
Commit: 7ea24fc06c081e2ba6f5f66d212abb14b80c9064
Parents: 0e820f1
Author: Uma Maheswara Rao G <um...@intel.com>
Authored: Wed Aug 23 15:37:03 2017 -0700
Committer: Uma Maheswara Rao Gangumalla <um...@apache.org>
Committed: Sun Aug 12 03:06:01 2018 -0700

----------------------------------------------------------------------
 .../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/7ea24fc0/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 bcc07cc..b53d946 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;
@@ -431,9 +430,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.
    */
@@ -480,8 +476,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);
 
     providedStorageMap = new ProvidedStorageMap(namesystem, this, conf);
@@ -5017,20 +5012,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/7ea24fc0/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 2d7c80e..c8d31fd 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
@@ -1751,6 +1751,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();
@@ -1762,13 +1769,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/7ea24fc0/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 79109b7..9308471 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
@@ -837,6 +837,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/7ea24fc0/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/7ea24fc0/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/7ea24fc0/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/7ea24fc0/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 3c6f837..459e697 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
@@ -206,6 +206,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;
     }
@@ -297,8 +305,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/7ea24fc0/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 1a06105..35341d7 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
@@ -1415,7 +1415,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/7ea24fc0/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 f730731..8b73f1e 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
@@ -1322,7 +1322,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/7ea24fc0/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/7ea24fc0/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/7ea24fc0/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/7ea24fc0/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/7ea24fc0/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


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

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

Branch: refs/heads/HDFS-10285
Commit: e53f89ccc361615b254e3ecd270728573908c071
Parents: 5ce332d
Author: Uma Maheswara Rao G <um...@intel.com>
Authored: Tue May 30 18:12:17 2017 -0700
Committer: Uma Maheswara Rao Gangumalla <um...@apache.org>
Committed: Sun Aug 12 03:05:59 2018 -0700

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


http://git-wip-us.apache.org/repos/asf/hadoop/blob/e53f89cc/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


[08/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 um...@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/0b360b16
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/0b360b16
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/0b360b16

Branch: refs/heads/HDFS-10285
Commit: 0b360b16ab8759e3db606ada3420f4e2f56235f3
Parents: 00cf207
Author: Uma Maheswara Rao G <um...@intel.com>
Authored: Mon Jul 10 18:00:58 2017 -0700
Committer: Uma Maheswara Rao Gangumalla <um...@apache.org>
Committed: Sun Aug 12 03:06:00 2018 -0700

----------------------------------------------------------------------
 .../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/0b360b16/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 3385af6..988067c 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
@@ -4324,6 +4324,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/0b360b16/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/0b360b16/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/0b360b16/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


[30/50] [abbrv] hadoop git commit: HDFS-13075. [SPS]: Provide External Context implementation. Contributed by Uma Maheswara Rao G.

Posted by um...@apache.org.
HDFS-13075. [SPS]: Provide External Context implementation. 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/99594b48
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/99594b48
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/99594b48

Branch: refs/heads/HDFS-10285
Commit: 99594b48b8e040ab5a0939d7c3dbcfb34400e6fc
Parents: 3b83110
Author: Surendra Singh Lilhore <su...@apache.org>
Authored: Sun Jan 28 20:46:56 2018 +0530
Committer: Uma Maheswara Rao Gangumalla <um...@apache.org>
Committed: Sun Aug 12 03:06:04 2018 -0700

----------------------------------------------------------------------
 .../NamenodeProtocolServerSideTranslatorPB.java |  67 +++++
 .../NamenodeProtocolTranslatorPB.java           |  58 ++++
 .../hdfs/server/balancer/NameNodeConnector.java |  28 +-
 .../server/blockmanagement/BlockManager.java    |  19 ++
 .../server/blockmanagement/DatanodeManager.java |  18 ++
 .../hdfs/server/common/HdfsServerConstants.java |   3 +-
 .../hdfs/server/namenode/NameNodeRpcServer.java |  53 +++-
 .../sps/BlockStorageMovementNeeded.java         |   8 +-
 .../hdfs/server/namenode/sps/Context.java       |   9 +-
 .../namenode/sps/IntraSPSNameNodeContext.java   |  23 +-
 .../namenode/sps/StoragePolicySatisfier.java    |  15 +-
 .../hdfs/server/protocol/NamenodeProtocol.java  |  46 +++-
 .../hdfs/server/sps/ExternalSPSContext.java     | 271 +++++++++++++++++++
 .../src/main/proto/NamenodeProtocol.proto       |  57 ++++
 .../sps/TestExternalStoragePolicySatisfier.java |  31 +--
 15 files changed, 652 insertions(+), 54 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/99594b48/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/NamenodeProtocolServerSideTranslatorPB.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/NamenodeProtocolServerSideTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/NamenodeProtocolServerSideTranslatorPB.java
index 90c2c49..25eafdf 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/NamenodeProtocolServerSideTranslatorPB.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/NamenodeProtocolServerSideTranslatorPB.java
@@ -23,6 +23,8 @@ import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo.DatanodeInfoBuilder;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsServerProtos.VersionRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsServerProtos.VersionResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.CheckDNSpaceRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.CheckDNSpaceResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.EndCheckpointRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.EndCheckpointResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.ErrorReportRequestProto;
@@ -33,10 +35,16 @@ import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetBlocksReq
 import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetBlocksResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetEditLogManifestRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetEditLogManifestResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetFilePathRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetFilePathResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetMostRecentCheckpointTxIdRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetMostRecentCheckpointTxIdResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetNextSPSPathIdRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetNextSPSPathIdResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetTransactionIdRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetTransactionIdResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.HasLowRedundancyBlocksRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.HasLowRedundancyBlocksResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.IsRollingUpgradeRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.IsRollingUpgradeResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.IsUpgradeFinalizedRequestProto;
@@ -257,4 +265,63 @@ public class NamenodeProtocolServerSideTranslatorPB implements
     return IsRollingUpgradeResponseProto.newBuilder()
         .setIsRollingUpgrade(isRollingUpgrade).build();
   }
+
+  @Override
+  public GetNextSPSPathIdResponseProto getNextSPSPathId(
+      RpcController controller, GetNextSPSPathIdRequestProto request)
+          throws ServiceException {
+    try {
+      Long nextSPSPathId = impl.getNextSPSPathId();
+      if (nextSPSPathId == null) {
+        return GetNextSPSPathIdResponseProto.newBuilder().build();
+      }
+      return GetNextSPSPathIdResponseProto.newBuilder().setFileId(nextSPSPathId)
+          .build();
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+  }
+
+  @Override
+  public GetFilePathResponseProto getFilePath(RpcController controller,
+      GetFilePathRequestProto request) throws ServiceException {
+    try {
+      return GetFilePathResponseProto.newBuilder()
+          .setSrcPath(impl.getFilePath(request.getFileId())).build();
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+  }
+
+  @Override
+  public CheckDNSpaceResponseProto checkDNSpaceForScheduling(
+      RpcController controller, CheckDNSpaceRequestProto request)
+          throws ServiceException {
+    try {
+      CheckDNSpaceResponseProto build = CheckDNSpaceResponseProto.newBuilder()
+          .setIsGoodDatanodeWithSpace(impl.checkDNSpaceForScheduling(
+              PBHelperClient.convert(request.getDnInfo()),
+              PBHelperClient.convertStorageType(request.getStorageType()),
+              request.getEstimatedSize()))
+          .build();
+      return build;
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+  }
+
+  @Override
+  public HasLowRedundancyBlocksResponseProto hasLowRedundancyBlocks(
+      RpcController controller, HasLowRedundancyBlocksRequestProto request)
+          throws ServiceException {
+    try {
+      return HasLowRedundancyBlocksResponseProto.newBuilder()
+          .setHasLowRedundancyBlocks(
+              impl.hasLowRedundancyBlocks(request.getInodeId()))
+          .build();
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/99594b48/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/NamenodeProtocolTranslatorPB.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/NamenodeProtocolTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/NamenodeProtocolTranslatorPB.java
index 632f8b7..8bff499 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/NamenodeProtocolTranslatorPB.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/NamenodeProtocolTranslatorPB.java
@@ -22,18 +22,24 @@ import java.io.IOException;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsServerProtos.NamenodeCommandProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsServerProtos.VersionRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.CheckDNSpaceRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.EndCheckpointRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.ErrorReportRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetBlockKeysRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetBlockKeysResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetBlocksRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetEditLogManifestRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetFilePathRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetMostRecentCheckpointTxIdRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetNextSPSPathIdRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetNextSPSPathIdResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetTransactionIdRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.HasLowRedundancyBlocksRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.IsRollingUpgradeRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.IsRollingUpgradeResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.IsUpgradeFinalizedRequestProto;
@@ -263,4 +269,56 @@ public class NamenodeProtocolTranslatorPB implements NamenodeProtocol,
       throw ProtobufHelper.getRemoteException(e);
     }
   }
+
+  @Override
+  public Long getNextSPSPathId() throws IOException {
+    GetNextSPSPathIdRequestProto req =
+        GetNextSPSPathIdRequestProto.newBuilder().build();
+    try {
+      GetNextSPSPathIdResponseProto nextSPSPathId =
+          rpcProxy.getNextSPSPathId(NULL_CONTROLLER, req);
+      return nextSPSPathId.hasFileId() ? nextSPSPathId.getFileId() : null;
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public String getFilePath(Long inodeId) throws IOException {
+    GetFilePathRequestProto req =
+        GetFilePathRequestProto.newBuilder().setFileId(inodeId).build();
+    try {
+      return rpcProxy.getFilePath(NULL_CONTROLLER, req).getSrcPath();
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public boolean checkDNSpaceForScheduling(DatanodeInfo dn, StorageType type,
+      long estimatedSize) throws IOException {
+    CheckDNSpaceRequestProto req = CheckDNSpaceRequestProto.newBuilder()
+        .setDnInfo(PBHelperClient.convert(dn))
+        .setStorageType(PBHelperClient.convertStorageType(type))
+        .setEstimatedSize(estimatedSize).build();
+    try {
+      return rpcProxy.checkDNSpaceForScheduling(NULL_CONTROLLER, req)
+          .getIsGoodDatanodeWithSpace();
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public boolean hasLowRedundancyBlocks(long inodeId) throws IOException {
+    HasLowRedundancyBlocksRequestProto req = HasLowRedundancyBlocksRequestProto
+        .newBuilder().setInodeId(inodeId).build();
+    try {
+      return rpcProxy.hasLowRedundancyBlocks(NULL_CONTROLLER, req)
+          .getHasLowRedundancyBlocks();
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/99594b48/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/NameNodeConnector.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/NameNodeConnector.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/NameNodeConnector.java
index 6bfbbb3..2b3c193 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/NameNodeConnector.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/NameNodeConnector.java
@@ -66,7 +66,8 @@ public class NameNodeConnector implements Closeable {
 
   public static final int DEFAULT_MAX_IDLE_ITERATIONS = 5;
   private static boolean write2IdFile = true;
-  
+  private static boolean checkOtherInstanceRunning = true;
+
   /** Create {@link NameNodeConnector} for the given namenodes. */
   public static List<NameNodeConnector> newNameNodeConnectors(
       Collection<URI> namenodes, String name, Path idPath, Configuration conf,
@@ -101,6 +102,11 @@ public class NameNodeConnector implements Closeable {
     NameNodeConnector.write2IdFile = write2IdFile;
   }
 
+  @VisibleForTesting
+  public static void checkOtherInstanceRunning(boolean toCheck) {
+    NameNodeConnector.checkOtherInstanceRunning = toCheck;
+  }
+
   private final URI nameNodeUri;
   private final String blockpoolID;
 
@@ -111,7 +117,7 @@ public class NameNodeConnector implements Closeable {
 
   private final DistributedFileSystem fs;
   private final Path idPath;
-  private final OutputStream out;
+  private OutputStream out;
   private final List<Path> targetPaths;
   private final AtomicLong bytesMoved = new AtomicLong();
 
@@ -141,10 +147,12 @@ public class NameNodeConnector implements Closeable {
     this.keyManager = new KeyManager(blockpoolID, namenode,
         defaults.getEncryptDataTransfer(), conf);
     // if it is for test, we do not create the id file
-    out = checkAndMarkRunning();
-    if (out == null) {
-      // Exit if there is another one running.
-      throw new IOException("Another " + name + " is running.");
+    if (checkOtherInstanceRunning) {
+      out = checkAndMarkRunning();
+      if (out == null) {
+        // Exit if there is another one running.
+        throw new IOException("Another " + name + " is running.");
+      }
     }
   }
 
@@ -285,13 +293,19 @@ public class NameNodeConnector implements Closeable {
     IOUtils.closeStream(out); 
     if (fs != null) {
       try {
-        fs.delete(idPath, true);
+        if (checkOtherInstanceRunning) {
+          fs.delete(idPath, true);
+        }
       } catch(IOException ioe) {
         LOG.warn("Failed to delete " + idPath, ioe);
       }
     }
   }
 
+  public NamenodeProtocol getNNProtocolConnection() {
+    return this.namenode;
+  }
+
   @Override
   public String toString() {
     return getClass().getSimpleName() + "[namenodeUri=" + nameNodeUri

http://git-wip-us.apache.org/repos/asf/hadoop/blob/99594b48/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 9ef1045..ac6d44b 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
@@ -5021,6 +5021,25 @@ public class BlockManager implements BlockStatsMXBean {
   }
 
   /**
+   * Check whether file id has low redundancy blocks.
+   *
+   * @param inodeID
+   *          - inode id
+   */
+  public boolean hasLowRedundancyBlocks(long inodeID) {
+    namesystem.readLock();
+    try {
+      BlockCollection bc = namesystem.getBlockCollection(inodeID);
+      if (bc == null) {
+        return false;
+      }
+      return hasLowRedundancyBlocks(bc);
+    } finally {
+      namesystem.readUnlock();
+    }
+  }
+
+  /**
    * Gets the storage policy satisfier instance.
    *
    * @return sps

http://git-wip-us.apache.org/repos/asf/hadoop/blob/99594b48/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 c24a38b..3542864 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
@@ -30,6 +30,7 @@ import org.apache.hadoop.HadoopIllegalArgumentException;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 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.HdfsConfiguration;
@@ -2067,5 +2068,22 @@ public class DatanodeManager {
     }
     return reports;
   }
+
+  public boolean verifyTargetDatanodeHasSpaceForScheduling(DatanodeInfo dn,
+      StorageType type, long estimatedSize) {
+    namesystem.readLock();
+    try {
+      DatanodeDescriptor datanode =
+          blockManager.getDatanodeManager().getDatanode(dn.getDatanodeUuid());
+      if (datanode == null) {
+        LOG.debug("Target datanode: " + dn + " doesn't exists");
+        return false;
+      }
+      return null != datanode.chooseStorage4Block(type, estimatedSize);
+    } finally {
+      namesystem.readUnlock();
+    }
+  }
+
 }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/99594b48/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 42a2fc6..1378de2 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,8 +365,7 @@ public interface HdfsServerConstants {
   String XATTR_ERASURECODING_POLICY =
       "system.hdfs.erasurecoding.policy";
 
-  String XATTR_SATISFY_STORAGE_POLICY =
-      "system.hdfs.satisfy.storage.policy";
+  String XATTR_SATISFY_STORAGE_POLICY = "user.hdfs.sps.xattr";
 
   Path MOVER_ID_PATH = new Path("/system/mover.id");
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/99594b48/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 4738bf5..0e50965 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
@@ -2537,10 +2537,15 @@ public class NameNodeRpcServer implements NamenodeProtocols {
   @Override
   public boolean isStoragePolicySatisfierRunning() throws IOException {
     checkNNStartup();
+    String operationName = "isStoragePolicySatisfierRunning";
+    namesystem.checkSuperuserPrivilege(operationName);
     if (nn.isStandbyState()) {
       throw new StandbyException("Not supported by Standby Namenode.");
     }
-    return namesystem.getBlockManager().isStoragePolicySatisfierRunning();
+    boolean isSPSRunning =
+        namesystem.getBlockManager().isStoragePolicySatisfierRunning();
+    namesystem.logAuditEvent(true, operationName, null);
+    return isSPSRunning;
   }
 
   @Override
@@ -2553,4 +2558,50 @@ public class NameNodeRpcServer implements NamenodeProtocols {
     return namesystem.getBlockManager().checkStoragePolicySatisfyPathStatus(
         path);
   }
+
+  @Override
+  public String getFilePath(Long inodeId) throws IOException {
+    checkNNStartup();
+    String operationName = "getFilePath";
+    namesystem.checkSuperuserPrivilege(operationName);
+    if (nn.isStandbyState()) {
+      throw new StandbyException("Not supported by Standby Namenode.");
+    }
+    return namesystem.getFilePath(inodeId);
+  }
+
+  @Override
+  public Long getNextSPSPathId() throws IOException {
+    checkNNStartup();
+    String operationName = "getNextSPSPathId";
+    namesystem.checkSuperuserPrivilege(operationName);
+    if (nn.isStandbyState()) {
+      throw new StandbyException("Not supported by Standby Namenode.");
+    }
+    return namesystem.getBlockManager().getNextSPSPathId();
+  }
+
+  @Override
+  public boolean checkDNSpaceForScheduling(DatanodeInfo dn,
+      StorageType type, long estimatedSize) throws IOException {
+    checkNNStartup();
+    String operationName = "checkDNSpaceForScheduling";
+    namesystem.checkSuperuserPrivilege(operationName);
+    if (nn.isStandbyState()) {
+      throw new StandbyException("Not supported by Standby Namenode.");
+    }
+    return namesystem.getBlockManager().getDatanodeManager()
+        .verifyTargetDatanodeHasSpaceForScheduling(dn, type, estimatedSize);
+  }
+
+  @Override
+  public boolean hasLowRedundancyBlocks(long inodeId) throws IOException {
+    checkNNStartup();
+    String operationName = "hasLowRedundancyBlocks";
+    namesystem.checkSuperuserPrivilege(operationName);
+    if (nn.isStandbyState()) {
+      throw new StandbyException("Not supported by Standby Namenode.");
+    }
+    return namesystem.getBlockManager().hasLowRedundancyBlocks(inodeId);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/99594b48/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/BlockStorageMovementNeeded.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/BlockStorageMovementNeeded.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/BlockStorageMovementNeeded.java
index 39c50a7..8a10183 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/BlockStorageMovementNeeded.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/BlockStorageMovementNeeded.java
@@ -319,12 +319,16 @@ public class BlockStorageMovementNeeded {
           String reClass = t.getClass().getName();
           if (InterruptedException.class.getName().equals(reClass)) {
             LOG.info("SPSPathIdProcessor thread is interrupted. Stopping..");
-            Thread.currentThread().interrupt();
             break;
           }
           LOG.warn("Exception while scanning file inodes to satisfy the policy",
               t);
-          // TODO: may be we should retry the current inode id?
+          try {
+            Thread.sleep(3000);
+          } catch (InterruptedException e) {
+            LOG.info("Interrupted while waiting in SPSPathIdProcessor", t);
+            break;
+          }
         }
       }
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/99594b48/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/Context.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/Context.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/Context.java
index f103dfe..bddbc1b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/Context.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/Context.java
@@ -149,8 +149,8 @@ public interface Context {
    * @return true if the given datanode has sufficient space to occupy blockSize
    *         data, false otherwise.
    */
-  boolean verifyTargetDatanodeHasSpaceForScheduling(DatanodeInfo dn,
-      StorageType type, long blockSize);
+  boolean checkDNSpaceForScheduling(DatanodeInfo dn, StorageType type,
+      long blockSize);
 
   /**
    * @return next SPS path id to process.
@@ -175,4 +175,9 @@ public interface Context {
    */
   String getFilePath(Long inodeId);
 
+  /**
+   * Close the resources.
+   */
+  void close() throws IOException;
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/99594b48/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/IntraSPSNameNodeContext.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/IntraSPSNameNodeContext.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/IntraSPSNameNodeContext.java
index c658812..191886c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/IntraSPSNameNodeContext.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/IntraSPSNameNodeContext.java
@@ -30,7 +30,6 @@ 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.HdfsFileStatus;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockCollection;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
@@ -98,17 +97,8 @@ public class IntraSPSNameNodeContext implements Context {
   }
 
   @Override
-  public boolean hasLowRedundancyBlocks(long inodeID) {
-    namesystem.readLock();
-    try {
-      BlockCollection bc = namesystem.getBlockCollection(inodeID);
-      if (bc == null) {
-        return false;
-      }
-      return blockManager.hasLowRedundancyBlocks(bc);
-    } finally {
-      namesystem.readUnlock();
-    }
+  public boolean hasLowRedundancyBlocks(long inodeId) {
+    return blockManager.hasLowRedundancyBlocks(inodeId);
   }
 
   @Override
@@ -170,8 +160,8 @@ public class IntraSPSNameNodeContext implements Context {
   }
 
   @Override
-  public boolean verifyTargetDatanodeHasSpaceForScheduling(DatanodeInfo dn,
-      StorageType type, long blockSize) {
+  public boolean checkDNSpaceForScheduling(DatanodeInfo dn, StorageType type,
+      long blockSize) {
     namesystem.readLock();
     try {
       DatanodeDescriptor datanode = blockManager.getDatanodeManager()
@@ -205,4 +195,9 @@ public class IntraSPSNameNodeContext implements Context {
   public String getFilePath(Long inodeId) {
     return namesystem.getFilePath(inodeId);
   }
+
+  @Override
+  public void close() throws IOException {
+    // Nothing to clean.
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/99594b48/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/StoragePolicySatisfier.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/StoragePolicySatisfier.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/StoragePolicySatisfier.java
index 33ad6f4..89799fc 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/StoragePolicySatisfier.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/StoragePolicySatisfier.java
@@ -325,6 +325,9 @@ public class StoragePolicySatisfier implements SPSService, Runnable {
               }
             }
           }
+        } else {
+          LOG.info("Namenode is in safemode. It will retry again.");
+          Thread.sleep(3000);
         }
         int numLiveDn = ctxt.getNumLiveDataNodes();
         if (storageMovementNeeded.size() == 0
@@ -706,8 +709,8 @@ public class StoragePolicySatisfier implements SPSService, Runnable {
   private StorageTypeNodePair chooseTargetTypeInSameNode(LocatedBlock blockInfo,
       DatanodeInfo source, List<StorageType> targetTypes) {
     for (StorageType t : targetTypes) {
-      boolean goodTargetDn = ctxt.verifyTargetDatanodeHasSpaceForScheduling(
-          source, t, blockInfo.getBlockSize());
+      boolean goodTargetDn =
+          ctxt.checkDNSpaceForScheduling(source, t, blockInfo.getBlockSize());
       if (goodTargetDn) {
         return new StorageTypeNodePair(t, source);
       }
@@ -720,8 +723,8 @@ public class StoragePolicySatisfier implements SPSService, Runnable {
       StorageTypeNodeMap locsForExpectedStorageTypes,
       List<DatanodeInfo> excludeNodes) {
     for (StorageType t : targetTypes) {
-      List<DatanodeInfo> nodesWithStorages = locsForExpectedStorageTypes
-          .getNodesWithStorages(t);
+      List<DatanodeInfo> nodesWithStorages =
+          locsForExpectedStorageTypes.getNodesWithStorages(t);
       if (nodesWithStorages == null || nodesWithStorages.isEmpty()) {
         continue; // no target nodes with the required storage type.
       }
@@ -729,8 +732,8 @@ public class StoragePolicySatisfier implements SPSService, Runnable {
       for (DatanodeInfo target : nodesWithStorages) {
         if (!excludeNodes.contains(target)
             && matcher.match(ctxt.getNetworkTopology(), source, target)) {
-          boolean goodTargetDn = ctxt.verifyTargetDatanodeHasSpaceForScheduling(
-              target, t, block.getBlockSize());
+          boolean goodTargetDn =
+              ctxt.checkDNSpaceForScheduling(target, t, block.getBlockSize());
           if (goodTargetDn) {
             return new StorageTypeNodePair(t, target);
           }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/99594b48/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/NamenodeProtocol.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/NamenodeProtocol.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/NamenodeProtocol.java
index 0c8adc6..9f5cadd 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/NamenodeProtocol.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/NamenodeProtocol.java
@@ -21,6 +21,7 @@ package org.apache.hadoop.hdfs.server.protocol;
 import java.io.IOException;
 
 import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.security.token.block.ExportedBlockKeys;
@@ -31,7 +32,8 @@ import org.apache.hadoop.security.KerberosInfo;
 
 /*****************************************************************************
  * Protocol that a secondary NameNode uses to communicate with the NameNode.
- * It's used to get part of the name node state
+ * Also used by external storage policy satisfier. It's used to get part of the
+ * name node state
  *****************************************************************************/
 @KerberosInfo(
     serverPrincipal = DFSConfigKeys.DFS_NAMENODE_KERBEROS_PRINCIPAL_KEY)
@@ -202,5 +204,47 @@ public interface NamenodeProtocol {
    */
   @Idempotent
   boolean isRollingUpgrade() throws IOException;
+
+  /**
+   * Gets the file path for the given file id. This API used by External SPS.
+   *
+   * @param inodeId
+   *          - file inode id.
+   * @return path
+   */
+  @Idempotent
+  String getFilePath(Long inodeId) throws IOException;
+
+  /**
+   * @return Gets the next available sps path id, otherwise null. This API used
+   *         by External SPS.
+   */
+  @AtMostOnce
+  Long getNextSPSPathId() throws IOException;
+
+  /**
+   * Verifies whether the given Datanode has the enough estimated size with
+   * given storage type for scheduling the block. This API used by External SPS.
+   *
+   * @param dn
+   *          - datanode
+   * @param type
+   *          - storage type
+   * @param estimatedSize
+   *          - size
+   */
+  @Idempotent
+  boolean checkDNSpaceForScheduling(DatanodeInfo dn, StorageType type,
+      long estimatedSize) throws IOException;
+
+  /**
+   * Check if any low redundancy blocks for given file id. This API used by
+   * External SPS.
+   *
+   * @param inodeID
+   *          - inode id.
+   */
+  @Idempotent
+  boolean hasLowRedundancyBlocks(long inodeID) throws IOException;
 }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/99594b48/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/sps/ExternalSPSContext.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/sps/ExternalSPSContext.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/sps/ExternalSPSContext.java
new file mode 100644
index 0000000..e5b04ba
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/sps/ExternalSPSContext.java
@@ -0,0 +1,271 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hdfs.server.sps;
+
+import java.io.IOException;
+import java.net.URI;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+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.hdfs.DFSUtil;
+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.HdfsFileStatus;
+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.sps.Context;
+import org.apache.hadoop.hdfs.server.namenode.sps.SPSService;
+import org.apache.hadoop.hdfs.server.protocol.DatanodeStorageReport;
+import org.apache.hadoop.net.NetworkTopology;
+import org.apache.hadoop.security.AccessControlException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * This class used to connect to Namenode and gets the required information to
+ * SPS from Namenode state.
+ */
+@InterfaceAudience.Private
+public class ExternalSPSContext implements Context {
+  public static final Logger LOG =
+      LoggerFactory.getLogger(ExternalSPSContext.class);
+  private SPSService service;
+  private NameNodeConnector nnc = null;
+  private Object nnConnectionLock = new Object();
+  private BlockStoragePolicySuite createDefaultSuite =
+      BlockStoragePolicySuite.createDefaultSuite();
+
+  public ExternalSPSContext(SPSService service) {
+    this.service = service;
+    initializeNamenodeConnector();
+  }
+
+  @Override
+  public boolean isRunning() {
+    return service.isRunning();
+  }
+
+  @Override
+  public boolean isInSafeMode() {
+    initializeNamenodeConnector();
+    try {
+      return nnc != null ? nnc.getDistributedFileSystem().isInSafeMode()
+          : false;
+    } catch (IOException e) {
+      LOG.warn("Exception while creating Namenode Connector..", e);
+      return false;
+    }
+  }
+
+  @Override
+  public boolean isMoverRunning() {
+    initializeNamenodeConnector();
+    try {
+      FSDataOutputStream out = nnc.getDistributedFileSystem()
+          .append(HdfsServerConstants.MOVER_ID_PATH);
+      out.close();
+      return false;
+    } catch (IOException ioe) {
+      LOG.warn("Exception while checking mover is running..", ioe);
+      return true;
+    }
+
+  }
+
+  @Override
+  public long getFileID(String path) throws UnresolvedLinkException,
+      AccessControlException, ParentNotDirectoryException {
+    initializeNamenodeConnector();
+    HdfsFileStatus fs = null;
+    try {
+      fs = (HdfsFileStatus) nnc.getDistributedFileSystem().getFileStatus(
+          new Path(path));
+      LOG.info("Fetched the fileID:{} for the path:{}", fs.getFileId(), path);
+    } catch (IllegalArgumentException | IOException e) {
+      LOG.warn("Exception while getting file is for the given path:{}.", path,
+          e);
+    }
+    return fs != null ? fs.getFileId() : 0;
+  }
+
+  @Override
+  public NetworkTopology getNetworkTopology() {
+    return NetworkTopology.getInstance(service.getConf());
+  }
+
+  @Override
+  public boolean isFileExist(long inodeId) {
+    initializeNamenodeConnector();
+    String filePath = null;
+    try {
+      filePath = getFilePath(inodeId);
+      return nnc.getDistributedFileSystem().exists(new Path(filePath));
+    } catch (IllegalArgumentException | IOException e) {
+      LOG.warn("Exception while getting file is for the given path:{} "
+          + "and fileId:{}", filePath, inodeId, e);
+    }
+    return false;
+  }
+
+  @Override
+  public BlockStoragePolicy getStoragePolicy(byte policyId) {
+    return createDefaultSuite.getPolicy(policyId);
+  }
+
+  @Override
+  public void addDropPreviousSPSWorkAtDNs() {
+    // Nothing todo
+  }
+
+  @Override
+  public void removeSPSHint(long inodeId) throws IOException {
+    initializeNamenodeConnector();
+    nnc.getDistributedFileSystem().removeXAttr(new Path(getFilePath(inodeId)),
+        HdfsServerConstants.XATTR_SATISFY_STORAGE_POLICY);
+  }
+
+  @Override
+  public int getNumLiveDataNodes() {
+    initializeNamenodeConnector();
+    try {
+      return nnc.getDistributedFileSystem()
+          .getDataNodeStats(DatanodeReportType.LIVE).length;
+    } catch (IOException e) {
+      LOG.warn("Exception while getting number of live datanodes.", e);
+    }
+    return 0;
+  }
+
+  @Override
+  public HdfsFileStatus getFileInfo(long inodeID) throws IOException {
+    initializeNamenodeConnector();
+    return nnc.getDistributedFileSystem().getClient()
+        .getLocatedFileInfo(getFilePath(inodeID), false);
+  }
+
+  @Override
+  public DatanodeStorageReport[] getLiveDatanodeStorageReport()
+      throws IOException {
+    initializeNamenodeConnector();
+    return nnc.getLiveDatanodeStorageReport();
+  }
+
+  @Override
+  public boolean hasLowRedundancyBlocks(long inodeID) {
+    initializeNamenodeConnector();
+    try {
+      return nnc.getNNProtocolConnection().hasLowRedundancyBlocks(inodeID);
+    } catch (IOException e) {
+      LOG.warn("Failed to check whether fileid:{} has low redundancy blocks.",
+          inodeID, e);
+      return false;
+    }
+  }
+
+  @Override
+  public boolean checkDNSpaceForScheduling(DatanodeInfo dn, StorageType type,
+      long estimatedSize) {
+    initializeNamenodeConnector();
+    try {
+      return nnc.getNNProtocolConnection().checkDNSpaceForScheduling(dn, type,
+          estimatedSize);
+    } catch (IOException e) {
+      LOG.warn("Verify the given datanode:{} is good and has "
+          + "estimated space in it.", dn, e);
+      return false;
+    }
+  }
+
+  @Override
+  public Long getNextSPSPathId() {
+    initializeNamenodeConnector();
+    try {
+      return nnc.getNNProtocolConnection().getNextSPSPathId();
+    } catch (IOException e) {
+      LOG.warn("Exception while getting next sps path id from Namenode.", e);
+      return null;
+    }
+  }
+
+  @Override
+  public void removeSPSPathId(long pathId) {
+    // We need not specifically implement for external.
+  }
+
+  @Override
+  public void removeAllSPSPathIds() {
+    // We need not specifically implement for external.
+  }
+
+  @Override
+  public String getFilePath(Long inodeId) {
+    try {
+      return nnc.getNNProtocolConnection().getFilePath(inodeId);
+    } catch (IOException e) {
+      LOG.warn("Exception while getting file path id:{} from Namenode.",
+          inodeId, e);
+      return null;
+    }
+  }
+
+  @Override
+  public void close() throws IOException {
+    synchronized (nnConnectionLock) {
+      if (nnc != null) {
+        nnc.close();
+      }
+    }
+  }
+
+  private void initializeNamenodeConnector() {
+    synchronized (nnConnectionLock) {
+      if (nnc == null) {
+        try {
+          nnc = getNameNodeConnector(service.getConf());
+        } catch (IOException e) {
+          LOG.warn("Exception while creating Namenode Connector.."
+              + "Namenode might not have started.", e);
+        }
+      }
+    }
+  }
+
+  public static NameNodeConnector getNameNodeConnector(Configuration conf)
+      throws IOException {
+    final Collection<URI> namenodes = DFSUtil.getInternalNsRpcUris(conf);
+    List<NameNodeConnector> nncs = Collections.emptyList();
+    NameNodeConnector.checkOtherInstanceRunning(false);
+    nncs = NameNodeConnector.newNameNodeConnectors(namenodes,
+        ExternalSPSContext.class.getSimpleName(),
+        HdfsServerConstants.MOVER_ID_PATH, conf,
+        NameNodeConnector.DEFAULT_MAX_IDLE_ITERATIONS);
+    return nncs.get(0);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/99594b48/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/NamenodeProtocol.proto
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/NamenodeProtocol.proto b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/NamenodeProtocol.proto
index 683dc80..b0e900d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/NamenodeProtocol.proto
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/NamenodeProtocol.proto
@@ -206,6 +206,39 @@ message IsRollingUpgradeResponseProto {
   required bool isRollingUpgrade = 1;
 }
 
+message GetFilePathRequestProto {
+  required uint64 fileId = 1;
+}
+
+message GetFilePathResponseProto {
+  required string srcPath = 1;
+}
+
+message GetNextSPSPathIdRequestProto {
+}
+
+message GetNextSPSPathIdResponseProto {
+  optional uint64 fileId = 1;
+}
+
+message CheckDNSpaceRequestProto {
+  required DatanodeInfoProto dnInfo = 1;
+  required StorageTypeProto storageType = 2;
+  required uint64 estimatedSize = 3;
+}
+
+message CheckDNSpaceResponseProto {
+  required bool isGoodDatanodeWithSpace = 1;
+}
+
+message HasLowRedundancyBlocksRequestProto {
+  required uint64 inodeId = 1;
+}
+
+message HasLowRedundancyBlocksResponseProto {
+  required bool hasLowRedundancyBlocks = 1;
+}
+
 /**
  * Protocol used by the sub-ordinate namenode to send requests
  * the active/primary namenode.
@@ -287,4 +320,28 @@ service NamenodeProtocolService {
    */
   rpc isRollingUpgrade(IsRollingUpgradeRequestProto)
       returns (IsRollingUpgradeResponseProto);
+
+  /**
+   * Return the corresponding file path for give file id
+   */
+  rpc getFilePath(GetFilePathRequestProto)
+      returns (GetFilePathResponseProto);
+
+  /**
+   * Return the sps path id from namenode
+   */
+  rpc getNextSPSPathId(GetNextSPSPathIdRequestProto)
+      returns (GetNextSPSPathIdResponseProto);
+
+  /**
+   * Return the sps path id from namenode
+   */
+  rpc checkDNSpaceForScheduling(CheckDNSpaceRequestProto)
+      returns (CheckDNSpaceResponseProto);
+
+  /**
+   * check whether given file id has low redundancy blocks.
+   */
+  rpc hasLowRedundancyBlocks(HasLowRedundancyBlocksRequestProto)
+      returns (HasLowRedundancyBlocksResponseProto);
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/99594b48/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/sps/TestExternalStoragePolicySatisfier.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/sps/TestExternalStoragePolicySatisfier.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/sps/TestExternalStoragePolicySatisfier.java
index 42b04da..fe08b8f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/sps/TestExternalStoragePolicySatisfier.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/sps/TestExternalStoragePolicySatisfier.java
@@ -37,7 +37,6 @@ import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.namenode.sps.BlockMovementListener;
 import org.apache.hadoop.hdfs.server.namenode.sps.Context;
 import org.apache.hadoop.hdfs.server.namenode.sps.FileIdCollector;
-import org.apache.hadoop.hdfs.server.namenode.sps.IntraSPSNameNodeContext;
 import org.apache.hadoop.hdfs.server.namenode.sps.SPSService;
 import org.apache.hadoop.hdfs.server.namenode.sps.StoragePolicySatisfier;
 import org.apache.hadoop.hdfs.server.namenode.sps.TestStoragePolicySatisfier;
@@ -96,14 +95,8 @@ public class TestExternalStoragePolicySatisfier
     SPSService spsService = blkMgr.getSPSService();
     spsService.stopGracefully();
 
-    // TODO: Since External is not fully implemented, just used INTERNAL now.
-    // Need to set External context here.
-    IntraSPSNameNodeContext context = new IntraSPSNameNodeContext(
-        cluster.getNameNode().getNamesystem(), blkMgr, blkMgr.getSPSService()) {
-      public boolean isRunning() {
-        return true;
-      };
-    };
+    ExternalSPSContext context = new ExternalSPSContext(spsService);
+
     ExternalBlockMovementListener blkMoveListener =
         new ExternalBlockMovementListener();
     ExternalSPSBlockMoveTaskHandler externalHandler =
@@ -131,15 +124,7 @@ public class TestExternalStoragePolicySatisfier
     spsService = blkMgr.getSPSService();
     spsService.stopGracefully();
 
-    // TODO: Since External is not fully implemented, just used INTERNAL now.
-    // Need to set External context here.
-    IntraSPSNameNodeContext context = new IntraSPSNameNodeContext(
-        getCluster().getNameNode().getNamesystem(), blkMgr,
-        blkMgr.getSPSService()) {
-      public boolean isRunning() {
-        return true;
-      };
-    };
+    ExternalSPSContext context = new ExternalSPSContext(spsService);
     ExternalBlockMovementListener blkMoveListener =
         new ExternalBlockMovementListener();
     ExternalSPSBlockMoveTaskHandler externalHandler =
@@ -180,7 +165,7 @@ public class TestExternalStoragePolicySatisfier
     for (URI nn : namenodes) {
       nnMap.put(nn, null);
     }
-    final Path externalSPSPathId = new Path("/system/externalSPS.id");
+    final Path externalSPSPathId = new Path("/system/tmp.id");
     final List<NameNodeConnector> nncs = NameNodeConnector
         .newNameNodeConnectors(nnMap,
             StoragePolicySatisfier.class.getSimpleName(), externalSPSPathId,
@@ -205,6 +190,14 @@ public class TestExternalStoragePolicySatisfier
   }
 
   /**
+   * This test case is more specific to internal.
+   */
+  @Ignore("This test is specific to internal, so skipping here.")
+  public void testWhenMoverIsAlreadyRunningBeforeStoragePolicySatisfier()
+      throws Exception {
+  }
+
+  /**
    * Status won't be supported for external SPS, now. So, ignoring it.
    */
   @Ignore("Status is not supported for external SPS. So, ignoring it.")


---------------------------------------------------------------------
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-12214: [SPS]: Fix review comments of StoragePolicySatisfier feature. Contributed by Rakesh R.

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

Branch: refs/heads/HDFS-10285
Commit: 0e820f16af309cc8476edba448dd548686431133
Parents: 3b601f2
Author: Uma Maheswara Rao G <um...@intel.com>
Authored: Thu Aug 17 13:21:07 2017 -0700
Committer: Uma Maheswara Rao Gangumalla <um...@apache.org>
Committed: Sun Aug 12 03:06:01 2018 -0700

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


http://git-wip-us.apache.org/repos/asf/hadoop/blob/0e820f16/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/0e820f16/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 b5a2a5a..e66806f 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
@@ -614,10 +614,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/0e820f16/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 8b7abaa..bcc07cc 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
@@ -429,6 +429,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();
 
@@ -440,15 +442,9 @@ public class BlockManager implements BlockStatsMXBean {
   /** Storages accessible from multiple DNs. */
   private final ProvidedStorageMap providedStorageMap;
 
-  /**
-   * 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);
@@ -477,24 +473,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);
 
     providedStorageMap = new ProvidedStorageMap(namesystem, this, conf);
@@ -719,15 +706,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();
@@ -5058,46 +5040,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/0e820f16/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/0e820f16/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 16ebe18..f730731 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
@@ -1291,7 +1291,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
             edekCacheLoaderDelay, edekCacheLoaderInterval);
       }
 
-      blockManager.activateSPS();
+      blockManager.startSPS();
     } finally {
       startingActiveService = false;
       blockManager.checkSafeMode();
@@ -1322,7 +1322,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     writeLock();
     try {
       if (blockManager != null) {
-        blockManager.deactivateSPS();
+        blockManager.stopSPS(true);
       }
       stopSecretManager();
       leaseManager.stopMonitor();
@@ -2241,6 +2241,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 {
@@ -2258,16 +2260,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/0e820f16/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 2eb2f04..2081b6e 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
@@ -160,7 +160,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;
@@ -295,7 +295,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["
@@ -2041,8 +2041,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));
@@ -2126,29 +2126,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/0e820f16/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/0e820f16/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/0e820f16/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/0e820f16/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 fab6ece..17f7795 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
@@ -4496,13 +4496,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/0e820f16/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 9098616..f6bbd10 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/ArchivalStorage.md
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/ArchivalStorage.md
@@ -108,7 +108,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)
 
@@ -125,8 +125,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.
@@ -153,7 +153,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
 -----------------------
@@ -232,10 +232,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/0e820f16/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 8234930..c2a6ae1 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSCommands.md
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSCommands.md
@@ -616,7 +616,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/0e820f16/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/0e820f16/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 f0f264c..233be45 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
@@ -115,7 +115,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 {
@@ -137,7 +137,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 {
@@ -462,7 +462,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 {
@@ -497,7 +497,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())
@@ -522,14 +522,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);
@@ -575,14 +575,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);
@@ -647,7 +647,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(
@@ -677,7 +677,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(
@@ -800,7 +800,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/0e820f16/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/0e820f16/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/0e820f16/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/0e820f16/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/0e820f16/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/0e820f16/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/0e820f16/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/0e820f16/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


[48/50] [abbrv] hadoop git commit: HDFS-13381 : [SPS]: Use DFSUtilClient#makePathFromFileId() to prepare satisfier file path. Contributed by Rakesh R.

Posted by um...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/66e8f9b3/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/sps/TestStoragePolicySatisfier.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/sps/TestStoragePolicySatisfier.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/sps/TestStoragePolicySatisfier.java
index b05717a..ec5307b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/sps/TestStoragePolicySatisfier.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/sps/TestStoragePolicySatisfier.java
@@ -108,8 +108,6 @@ public class TestStoragePolicySatisfier {
   public static final long CAPACITY = 2 * 256 * 1024 * 1024;
   public static final String FILE = "/testMoveToSatisfyStoragePolicy";
   public static final int DEFAULT_BLOCK_SIZE = 1024;
-  private ExternalBlockMovementListener blkMoveListener =
-      new ExternalBlockMovementListener();
 
   /**
    * Sets hdfs cluster.
@@ -1282,8 +1280,8 @@ public class TestStoragePolicySatisfier {
 
     //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<Long> sps = new StoragePolicySatisfier<Long>(config);
-    Context<Long> ctxt = new IntraSPSNameNodeContext(
+    StoragePolicySatisfier sps = new StoragePolicySatisfier(config);
+    Context ctxt = new IntraSPSNameNodeContext(
         hdfsCluster.getNamesystem(),
         hdfsCluster.getNamesystem().getBlockManager(), sps) {
       @Override
@@ -1297,8 +1295,7 @@ public class TestStoragePolicySatisfier {
       }
     };
 
-    FileCollector<Long> fileIDCollector = createFileIdCollector(sps, ctxt);
-    sps.init(ctxt, fileIDCollector, null, null);
+    sps.init(ctxt);
     sps.getStorageMovementQueue().activate();
 
     INode rootINode = fsDir.getINode("/root");
@@ -1314,13 +1311,6 @@ public class TestStoragePolicySatisfier {
     dfs.delete(new Path("/root"), true);
   }
 
-  public FileCollector<Long> createFileIdCollector(
-      StoragePolicySatisfier<Long> sps, Context<Long> ctxt) {
-    FileCollector<Long> fileIDCollector = new IntraSPSNameNodeFileIdCollector(
-        hdfsCluster.getNamesystem().getFSDirectory(), sps);
-    return fileIDCollector;
-  }
-
   /**
    *  Test traverse when root parent got deleted.
    *  1. Delete L when traversing Q
@@ -1351,8 +1341,8 @@ public class TestStoragePolicySatisfier {
 
     // 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<Long> sps = new StoragePolicySatisfier<Long>(config);
-    Context<Long> ctxt = new IntraSPSNameNodeContext(
+    StoragePolicySatisfier sps = new StoragePolicySatisfier(config);
+    Context ctxt = new IntraSPSNameNodeContext(
         hdfsCluster.getNamesystem(),
         hdfsCluster.getNamesystem().getBlockManager(), sps) {
       @Override
@@ -1365,8 +1355,7 @@ public class TestStoragePolicySatisfier {
         return true;
       }
     };
-    FileCollector<Long> fileIDCollector = createFileIdCollector(sps, ctxt);
-    sps.init(ctxt, fileIDCollector, null, null);
+    sps.init(ctxt);
     sps.getStorageMovementQueue().activate();
 
     INode rootINode = fsDir.getINode("/root");
@@ -1383,12 +1372,12 @@ public class TestStoragePolicySatisfier {
   }
 
   private void assertTraversal(List<String> expectedTraverseOrder,
-      FSDirectory fsDir, StoragePolicySatisfier<Long> sps)
+      FSDirectory fsDir, StoragePolicySatisfier sps)
           throws InterruptedException {
     // Remove 10 element and make queue free, So other traversing will start.
     for (int i = 0; i < 10; i++) {
       String path = expectedTraverseOrder.remove(0);
-      ItemInfo<Long> itemInfo = sps.getStorageMovementQueue().get();
+      ItemInfo itemInfo = sps.getStorageMovementQueue().get();
       if (itemInfo == null) {
         continue;
       }
@@ -1403,7 +1392,7 @@ public class TestStoragePolicySatisfier {
     // 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) {
-      ItemInfo<Long> itemInfo = sps.getStorageMovementQueue().get();
+      ItemInfo itemInfo = sps.getStorageMovementQueue().get();
       if (itemInfo == null) {
         continue;
       }
@@ -1717,17 +1706,17 @@ public class TestStoragePolicySatisfier {
   public void waitForAttemptedItems(long expectedBlkMovAttemptedCount,
       int timeout) throws TimeoutException, InterruptedException {
     BlockManager blockManager = hdfsCluster.getNamesystem().getBlockManager();
-    final StoragePolicySatisfier<Long> sps =
-        (StoragePolicySatisfier<Long>) blockManager.getSPSManager()
+    final StoragePolicySatisfier sps =
+        (StoragePolicySatisfier) blockManager.getSPSManager()
         .getInternalSPSService();
     GenericTestUtils.waitFor(new Supplier<Boolean>() {
       @Override
       public Boolean get() {
         LOG.info("expectedAttemptedItemsCount={} actualAttemptedItemsCount={}",
             expectedBlkMovAttemptedCount,
-            ((BlockStorageMovementAttemptedItems<Long>) (sps
+            ((BlockStorageMovementAttemptedItems) (sps
                 .getAttemptedItemsMonitor())).getAttemptedItemsCount());
-        return ((BlockStorageMovementAttemptedItems<Long>) (sps
+        return ((BlockStorageMovementAttemptedItems) (sps
             .getAttemptedItemsMonitor()))
             .getAttemptedItemsCount() == expectedBlkMovAttemptedCount;
       }
@@ -1737,15 +1726,17 @@ public class TestStoragePolicySatisfier {
   public void waitForBlocksMovementAttemptReport(
       long expectedMovementFinishedBlocksCount, int timeout)
           throws TimeoutException, InterruptedException {
-    Assert.assertNotNull("Didn't set external block move listener",
-        blkMoveListener);
+    BlockManager blockManager = hdfsCluster.getNamesystem().getBlockManager();
+    final StoragePolicySatisfier sps =
+        (StoragePolicySatisfier) blockManager.getSPSManager()
+        .getInternalSPSService();
     GenericTestUtils.waitFor(new Supplier<Boolean>() {
       @Override
       public Boolean get() {
-        int actualCount = blkMoveListener.getActualBlockMovements().size();
+        int actualCount = ((BlockStorageMovementAttemptedItems) (sps
+            .getAttemptedItemsMonitor())).getAttemptedItemsCount();
         LOG.info("MovementFinishedBlocks: expectedCount={} actualCount={}",
-            expectedMovementFinishedBlocksCount,
-            actualCount);
+            expectedMovementFinishedBlocksCount, actualCount);
         return actualCount
             >= expectedMovementFinishedBlocksCount;
       }
@@ -1798,29 +1789,12 @@ public class TestStoragePolicySatisfier {
         .numDataNodes(numberOfDatanodes).storagesPerDatanode(storagesPerDn)
         .storageTypes(storageTypes).storageCapacities(capacities).build();
     cluster.waitActive();
-
-    // Sets external listener for assertion.
-    blkMoveListener.clear();
-    BlockManager blockManager = cluster.getNamesystem().getBlockManager();
-    final StoragePolicySatisfier<Long> sps =
-        (StoragePolicySatisfier<Long>) blockManager
-        .getSPSManager().getInternalSPSService();
-    sps.setBlockMovementListener(blkMoveListener);
     return cluster;
   }
 
   public void restartNamenode() throws IOException {
     hdfsCluster.restartNameNodes();
     hdfsCluster.waitActive();
-    BlockManager blockManager = hdfsCluster.getNamesystem().getBlockManager();
-    StoragePolicySatisfyManager spsMgr = blockManager.getSPSManager();
-    if (spsMgr != null && spsMgr.isInternalSatisfierRunning()) {
-      // Sets external listener for assertion.
-      blkMoveListener.clear();
-      final StoragePolicySatisfier<Long> sps =
-          (StoragePolicySatisfier<Long>) spsMgr.getInternalSPSService();
-      sps.setBlockMovementListener(blkMoveListener);
-    }
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/66e8f9b3/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/sps/TestStoragePolicySatisfierWithStripedFile.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/sps/TestStoragePolicySatisfierWithStripedFile.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/sps/TestStoragePolicySatisfierWithStripedFile.java
index 857bd6c..8a25a5e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/sps/TestStoragePolicySatisfierWithStripedFile.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/sps/TestStoragePolicySatisfierWithStripedFile.java
@@ -43,7 +43,6 @@ import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.StoragePolicySatisfierMode;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
-import org.apache.hadoop.hdfs.server.namenode.sps.TestStoragePolicySatisfier.ExternalBlockMovementListener;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.junit.Assert;
 import org.junit.Before;
@@ -71,8 +70,6 @@ public class TestStoragePolicySatisfierWithStripedFile {
   private int cellSize;
   private int defaultStripeBlockSize;
   private Configuration conf;
-  private ExternalBlockMovementListener blkMoveListener =
-      new ExternalBlockMovementListener();
 
   private ErasureCodingPolicy getEcPolicy() {
     return StripedFileTestUtil.getDefaultECPolicy();
@@ -94,6 +91,8 @@ public class TestStoragePolicySatisfierWithStripedFile {
     // Reduced refresh cycle to update latest datanodes.
     conf.setLong(DFSConfigKeys.DFS_SPS_DATANODE_CACHE_REFRESH_INTERVAL_MS,
         1000);
+    conf.setInt(
+        DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_MAX_RETRY_ATTEMPTS_KEY, 30);
     initConfWithStripe(conf, defaultStripeBlockSize);
   }
 
@@ -135,14 +134,6 @@ public class TestStoragePolicySatisfierWithStripedFile {
     try {
       cluster.waitActive();
 
-      // Sets external listener for assertion.
-      blkMoveListener.clear();
-      BlockManager blockManager = cluster.getNamesystem().getBlockManager();
-      final StoragePolicySatisfier<Long> sps =
-          (StoragePolicySatisfier<Long>) blockManager
-          .getSPSManager().getInternalSPSService();
-      sps.setBlockMovementListener(blkMoveListener);
-
       DistributedFileSystem dfs = cluster.getFileSystem();
       dfs.enableErasureCodingPolicy(
           StripedFileTestUtil.getDefaultECPolicy().getName());
@@ -253,14 +244,6 @@ public class TestStoragePolicySatisfierWithStripedFile {
     try {
       cluster.waitActive();
 
-      // Sets external listener for assertion.
-      blkMoveListener.clear();
-      BlockManager blockManager = cluster.getNamesystem().getBlockManager();
-      final StoragePolicySatisfier<Long> sps =
-          (StoragePolicySatisfier<Long>) blockManager
-          .getSPSManager().getInternalSPSService();
-      sps.setBlockMovementListener(blkMoveListener);
-
       DistributedFileSystem dfs = cluster.getFileSystem();
       dfs.enableErasureCodingPolicy(
           StripedFileTestUtil.getDefaultECPolicy().getName());
@@ -400,10 +383,11 @@ public class TestStoragePolicySatisfierWithStripedFile {
       fs.satisfyStoragePolicy(fooFile);
       DFSTestUtil.waitExpectedStorageType(fooFile.toString(),
           StorageType.ARCHIVE, 5, 30000, cluster.getFileSystem());
-      //Start reaming datanodes
+      //Start remaining datanodes
       for (int i = numOfDatanodes - 1; i >= 5; i--) {
         cluster.restartDataNode(list.get(i), false);
       }
+      cluster.waitActive();
       // verify storage types and locations.
       waitExpectedStorageType(cluster, fooFile.toString(), fileLen,
           StorageType.ARCHIVE, 9, 9, 60000);
@@ -511,17 +495,17 @@ public class TestStoragePolicySatisfierWithStripedFile {
       long expectedBlkMovAttemptedCount, int timeout)
           throws TimeoutException, InterruptedException {
     BlockManager blockManager = cluster.getNamesystem().getBlockManager();
-    final StoragePolicySatisfier<Long> sps =
-        (StoragePolicySatisfier<Long>) blockManager
+    final StoragePolicySatisfier sps =
+        (StoragePolicySatisfier) blockManager
         .getSPSManager().getInternalSPSService();
     GenericTestUtils.waitFor(new Supplier<Boolean>() {
       @Override
       public Boolean get() {
         LOG.info("expectedAttemptedItemsCount={} actualAttemptedItemsCount={}",
             expectedBlkMovAttemptedCount,
-            ((BlockStorageMovementAttemptedItems<Long>) sps
+            ((BlockStorageMovementAttemptedItems) sps
                 .getAttemptedItemsMonitor()).getAttemptedItemsCount());
-        return ((BlockStorageMovementAttemptedItems<Long>) sps
+        return ((BlockStorageMovementAttemptedItems) sps
             .getAttemptedItemsMonitor())
                 .getAttemptedItemsCount() == expectedBlkMovAttemptedCount;
       }
@@ -583,12 +567,15 @@ public class TestStoragePolicySatisfierWithStripedFile {
   private void waitForBlocksMovementAttemptReport(MiniDFSCluster cluster,
       long expectedMoveFinishedBlks, int timeout)
           throws TimeoutException, InterruptedException {
-    Assert.assertNotNull("Didn't set external block move listener",
-        blkMoveListener);
+    BlockManager blockManager = cluster.getNamesystem().getBlockManager();
+    final StoragePolicySatisfier sps =
+        (StoragePolicySatisfier) blockManager.getSPSManager()
+        .getInternalSPSService();
     GenericTestUtils.waitFor(new Supplier<Boolean>() {
       @Override
       public Boolean get() {
-        int actualCount = blkMoveListener.getActualBlockMovements().size();
+        int actualCount = ((BlockStorageMovementAttemptedItems) (sps
+            .getAttemptedItemsMonitor())).getMovementFinishedBlocksCount();
         LOG.info("MovementFinishedBlocks: expectedCount={} actualCount={}",
             expectedMoveFinishedBlks,
             actualCount);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/66e8f9b3/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/sps/TestExternalStoragePolicySatisfier.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/sps/TestExternalStoragePolicySatisfier.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/sps/TestExternalStoragePolicySatisfier.java
index be243cb..18acb50 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/sps/TestExternalStoragePolicySatisfier.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/sps/TestExternalStoragePolicySatisfier.java
@@ -88,10 +88,8 @@ public class TestExternalStoragePolicySatisfier
   private String principal;
   private MiniKdc kdc;
   private File baseDir;
-  private StoragePolicySatisfier<String> externalSps;
+  private StoragePolicySatisfier externalSps;
   private ExternalSPSContext externalCtxt;
-  private ExternalBlockMovementListener blkMoveListener =
-      new ExternalBlockMovementListener();
 
   @After
   public void destroy() throws Exception {
@@ -143,16 +141,10 @@ public class TestExternalStoragePolicySatisfier
 
     nnc = getNameNodeConnector(getConf());
 
-    externalSps = new StoragePolicySatisfier<String>(getConf());
+    externalSps = new StoragePolicySatisfier(getConf());
     externalCtxt = new ExternalSPSContext(externalSps, nnc);
 
-    blkMoveListener.clear();
-    ExternalSPSBlockMoveTaskHandler externalHandler =
-        new ExternalSPSBlockMoveTaskHandler(conf, nnc,
-            externalSps);
-    externalSps.init(externalCtxt,
-        new ExternalSPSFilePathCollector(externalSps), externalHandler,
-        blkMoveListener);
+    externalSps.init(externalCtxt);
     externalSps.start(true, StoragePolicySatisfierMode.EXTERNAL);
     return cluster;
   }
@@ -164,16 +156,10 @@ public class TestExternalStoragePolicySatisfier
 
     getCluster().restartNameNodes();
     getCluster().waitActive();
-    externalSps = new StoragePolicySatisfier<>(getConf());
+    externalSps = new StoragePolicySatisfier(getConf());
 
     externalCtxt = new ExternalSPSContext(externalSps, nnc);
-    blkMoveListener.clear();
-    ExternalSPSBlockMoveTaskHandler externalHandler =
-        new ExternalSPSBlockMoveTaskHandler(getConf(), nnc,
-            externalSps);
-    externalSps.init(externalCtxt,
-        new ExternalSPSFilePathCollector(externalSps), externalHandler,
-        blkMoveListener);
+    externalSps.init(externalCtxt);
     externalSps.start(true, StoragePolicySatisfierMode.EXTERNAL);
   }
 
@@ -206,11 +192,11 @@ public class TestExternalStoragePolicySatisfier
       public Boolean get() {
         LOG.info("expectedAttemptedItemsCount={} actualAttemptedItemsCount={}",
             expectedBlkMovAttemptedCount,
-            ((BlockStorageMovementAttemptedItems<String>) (externalSps
+            ((BlockStorageMovementAttemptedItems) (externalSps
                 .getAttemptedItemsMonitor())).getAttemptedItemsCount());
-        return ((BlockStorageMovementAttemptedItems<String>) (externalSps
+        return ((BlockStorageMovementAttemptedItems) (externalSps
             .getAttemptedItemsMonitor()))
-            .getAttemptedItemsCount() == expectedBlkMovAttemptedCount;
+                .getAttemptedItemsCount() == expectedBlkMovAttemptedCount;
       }
     }, 100, timeout);
   }
@@ -218,12 +204,11 @@ public class TestExternalStoragePolicySatisfier
   public void waitForBlocksMovementAttemptReport(
       long expectedMovementFinishedBlocksCount, int timeout)
           throws TimeoutException, InterruptedException {
-    Assert.assertNotNull("Didn't set external block move listener",
-        blkMoveListener);
     GenericTestUtils.waitFor(new Supplier<Boolean>() {
       @Override
       public Boolean get() {
-        int actualCount = blkMoveListener.getActualBlockMovements().size();
+        int actualCount = externalSps.getAttemptedItemsMonitor()
+            .getAttemptedItemsCount();
         LOG.info("MovementFinishedBlocks: expectedCount={} actualCount={}",
             expectedMovementFinishedBlocksCount, actualCount);
         return actualCount


---------------------------------------------------------------------
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-13110: [SPS]: Reduce the number of APIs in NamenodeProtocol used by external satisfier. Contributed by Rakesh R.

Posted by um...@apache.org.
HDFS-13110: [SPS]: Reduce the number of APIs in NamenodeProtocol used by external satisfier. 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/8467ec24
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/8467ec24
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/8467ec24

Branch: refs/heads/HDFS-10285
Commit: 8467ec24fb74f30371d5a13e893fc56309ee9372
Parents: 4402f3f
Author: Rakesh Radhakrishnan <ra...@apache.org>
Authored: Fri Feb 16 17:01:38 2018 +0530
Committer: Uma Maheswara Rao Gangumalla <um...@apache.org>
Committed: Sun Aug 12 03:06:05 2018 -0700

----------------------------------------------------------------------
 .../NamenodeProtocolServerSideTranslatorPB.java |  46 +----
 .../NamenodeProtocolTranslatorPB.java           |  42 +----
 .../hdfs/server/namenode/FSTreeTraverser.java   |   2 +-
 .../hdfs/server/namenode/NameNodeRpcServer.java |  32 +---
 .../server/namenode/ReencryptionHandler.java    |   2 +-
 .../sps/BlockStorageMovementAttemptedItems.java |  42 +++--
 .../sps/BlockStorageMovementNeeded.java         | 119 +++++++------
 .../hdfs/server/namenode/sps/Context.java       |  55 +++---
 .../hdfs/server/namenode/sps/FileCollector.java |  48 +++++
 .../server/namenode/sps/FileIdCollector.java    |  43 -----
 .../namenode/sps/IntraSPSNameNodeContext.java   |  39 ++---
 .../sps/IntraSPSNameNodeFileIdCollector.java    |  23 +--
 .../hdfs/server/namenode/sps/ItemInfo.java      |  39 +++--
 .../hdfs/server/namenode/sps/SPSService.java    |  32 ++--
 .../namenode/sps/StoragePolicySatisfier.java    | 129 +++++++++-----
 .../sps/StoragePolicySatisfyManager.java        |   6 +-
 .../hdfs/server/protocol/NamenodeProtocol.java  |  24 +--
 .../sps/ExternalSPSBlockMoveTaskHandler.java    |   4 +-
 .../hdfs/server/sps/ExternalSPSContext.java     |  60 +++----
 .../server/sps/ExternalSPSFileIDCollector.java  | 174 -------------------
 .../sps/ExternalSPSFilePathCollector.java       | 172 ++++++++++++++++++
 .../sps/ExternalStoragePolicySatisfier.java     |   7 +-
 .../src/main/proto/NamenodeProtocol.proto       |  27 +--
 .../TestBlockStorageMovementAttemptedItems.java |  27 ++-
 .../sps/TestStoragePolicySatisfier.java         |  52 +++---
 ...stStoragePolicySatisfierWithStripedFile.java |  15 +-
 .../sps/TestExternalStoragePolicySatisfier.java | 148 +++++++++++-----
 27 files changed, 701 insertions(+), 708 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/8467ec24/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/NamenodeProtocolServerSideTranslatorPB.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/NamenodeProtocolServerSideTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/NamenodeProtocolServerSideTranslatorPB.java
index 25eafdf..ed176cc 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/NamenodeProtocolServerSideTranslatorPB.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/NamenodeProtocolServerSideTranslatorPB.java
@@ -35,16 +35,12 @@ import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetBlocksReq
 import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetBlocksResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetEditLogManifestRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetEditLogManifestResponseProto;
-import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetFilePathRequestProto;
-import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetFilePathResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetMostRecentCheckpointTxIdRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetMostRecentCheckpointTxIdResponseProto;
-import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetNextSPSPathIdRequestProto;
-import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetNextSPSPathIdResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetNextSPSPathRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetNextSPSPathResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetTransactionIdRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetTransactionIdResponseProto;
-import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.HasLowRedundancyBlocksRequestProto;
-import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.HasLowRedundancyBlocksResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.IsRollingUpgradeRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.IsRollingUpgradeResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.IsUpgradeFinalizedRequestProto;
@@ -267,15 +263,15 @@ public class NamenodeProtocolServerSideTranslatorPB implements
   }
 
   @Override
-  public GetNextSPSPathIdResponseProto getNextSPSPathId(
-      RpcController controller, GetNextSPSPathIdRequestProto request)
+  public GetNextSPSPathResponseProto getNextSPSPath(
+      RpcController controller, GetNextSPSPathRequestProto request)
           throws ServiceException {
     try {
-      Long nextSPSPathId = impl.getNextSPSPathId();
-      if (nextSPSPathId == null) {
-        return GetNextSPSPathIdResponseProto.newBuilder().build();
+      String nextSPSPath = impl.getNextSPSPath();
+      if (nextSPSPath == null) {
+        return GetNextSPSPathResponseProto.newBuilder().build();
       }
-      return GetNextSPSPathIdResponseProto.newBuilder().setFileId(nextSPSPathId)
+      return GetNextSPSPathResponseProto.newBuilder().setSpsPath(nextSPSPath)
           .build();
     } catch (IOException e) {
       throw new ServiceException(e);
@@ -283,17 +279,6 @@ public class NamenodeProtocolServerSideTranslatorPB implements
   }
 
   @Override
-  public GetFilePathResponseProto getFilePath(RpcController controller,
-      GetFilePathRequestProto request) throws ServiceException {
-    try {
-      return GetFilePathResponseProto.newBuilder()
-          .setSrcPath(impl.getFilePath(request.getFileId())).build();
-    } catch (IOException e) {
-      throw new ServiceException(e);
-    }
-  }
-
-  @Override
   public CheckDNSpaceResponseProto checkDNSpaceForScheduling(
       RpcController controller, CheckDNSpaceRequestProto request)
           throws ServiceException {
@@ -309,19 +294,4 @@ public class NamenodeProtocolServerSideTranslatorPB implements
       throw new ServiceException(e);
     }
   }
-
-  @Override
-  public HasLowRedundancyBlocksResponseProto hasLowRedundancyBlocks(
-      RpcController controller, HasLowRedundancyBlocksRequestProto request)
-          throws ServiceException {
-    try {
-      return HasLowRedundancyBlocksResponseProto.newBuilder()
-          .setHasLowRedundancyBlocks(
-              impl.hasLowRedundancyBlocks(request.getInodeId()))
-          .build();
-    } catch (IOException e) {
-      throw new ServiceException(e);
-    }
-  }
-
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8467ec24/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/NamenodeProtocolTranslatorPB.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/NamenodeProtocolTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/NamenodeProtocolTranslatorPB.java
index 8bff499..d2e97a2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/NamenodeProtocolTranslatorPB.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/NamenodeProtocolTranslatorPB.java
@@ -34,12 +34,10 @@ import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetBlockKeys
 import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetBlockKeysResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetBlocksRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetEditLogManifestRequestProto;
-import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetFilePathRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetMostRecentCheckpointTxIdRequestProto;
-import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetNextSPSPathIdRequestProto;
-import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetNextSPSPathIdResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetNextSPSPathRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetNextSPSPathResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetTransactionIdRequestProto;
-import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.HasLowRedundancyBlocksRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.IsRollingUpgradeRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.IsRollingUpgradeResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.IsUpgradeFinalizedRequestProto;
@@ -271,24 +269,13 @@ public class NamenodeProtocolTranslatorPB implements NamenodeProtocol,
   }
 
   @Override
-  public Long getNextSPSPathId() throws IOException {
-    GetNextSPSPathIdRequestProto req =
-        GetNextSPSPathIdRequestProto.newBuilder().build();
+  public String getNextSPSPath() throws IOException {
+    GetNextSPSPathRequestProto req =
+        GetNextSPSPathRequestProto.newBuilder().build();
     try {
-      GetNextSPSPathIdResponseProto nextSPSPathId =
-          rpcProxy.getNextSPSPathId(NULL_CONTROLLER, req);
-      return nextSPSPathId.hasFileId() ? nextSPSPathId.getFileId() : null;
-    } catch (ServiceException e) {
-      throw ProtobufHelper.getRemoteException(e);
-    }
-  }
-
-  @Override
-  public String getFilePath(Long inodeId) throws IOException {
-    GetFilePathRequestProto req =
-        GetFilePathRequestProto.newBuilder().setFileId(inodeId).build();
-    try {
-      return rpcProxy.getFilePath(NULL_CONTROLLER, req).getSrcPath();
+      GetNextSPSPathResponseProto nextSPSPath =
+          rpcProxy.getNextSPSPath(NULL_CONTROLLER, req);
+      return nextSPSPath.hasSpsPath() ? nextSPSPath.getSpsPath() : null;
     } catch (ServiceException e) {
       throw ProtobufHelper.getRemoteException(e);
     }
@@ -308,17 +295,4 @@ public class NamenodeProtocolTranslatorPB implements NamenodeProtocol,
       throw ProtobufHelper.getRemoteException(e);
     }
   }
-
-  @Override
-  public boolean hasLowRedundancyBlocks(long inodeId) throws IOException {
-    HasLowRedundancyBlocksRequestProto req = HasLowRedundancyBlocksRequestProto
-        .newBuilder().setInodeId(inodeId).build();
-    try {
-      return rpcProxy.hasLowRedundancyBlocks(NULL_CONTROLLER, req)
-          .getHasLowRedundancyBlocks();
-    } catch (ServiceException e) {
-      throw ProtobufHelper.getRemoteException(e);
-    }
-  }
-
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8467ec24/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
index a7d633f..2acbda4 100644
--- 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
@@ -310,7 +310,7 @@ public abstract class FSTreeTraverser {
    * @throws IOException
    * @throws InterruptedException
    */
-  protected abstract void submitCurrentBatch(long startId)
+  protected abstract void submitCurrentBatch(Long startId)
       throws IOException, InterruptedException;
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8467ec24/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 97f38c7..6fe38d6 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
@@ -2561,20 +2561,9 @@ public class NameNodeRpcServer implements NamenodeProtocols {
   }
 
   @Override
-  public String getFilePath(Long inodeId) throws IOException {
+  public String getNextSPSPath() throws IOException {
     checkNNStartup();
-    String operationName = "getFilePath";
-    namesystem.checkSuperuserPrivilege(operationName);
-    if (nn.isStandbyState()) {
-      throw new StandbyException("Not supported by Standby Namenode.");
-    }
-    return namesystem.getFilePath(inodeId);
-  }
-
-  @Override
-  public Long getNextSPSPathId() throws IOException {
-    checkNNStartup();
-    String operationName = "getNextSPSPathId";
+    String operationName = "getNextSPSPath";
     namesystem.checkSuperuserPrivilege(operationName);
     if (nn.isStandbyState()) {
       throw new StandbyException("Not supported by Standby Namenode.");
@@ -2588,7 +2577,11 @@ public class NameNodeRpcServer implements NamenodeProtocols {
           + " inside namenode, so external SPS is not allowed to fetch"
           + " the path Ids");
     }
-    return namesystem.getBlockManager().getSPSManager().getNextPathId();
+    Long pathId = namesystem.getBlockManager().getSPSManager().getNextPathId();
+    if (pathId == null) {
+      return null;
+    }
+    return namesystem.getFilePath(pathId);
   }
 
   @Override
@@ -2603,15 +2596,4 @@ public class NameNodeRpcServer implements NamenodeProtocols {
     return namesystem.getBlockManager().getDatanodeManager()
         .verifyTargetDatanodeHasSpaceForScheduling(dn, type, estimatedSize);
   }
-
-  @Override
-  public boolean hasLowRedundancyBlocks(long inodeId) throws IOException {
-    checkNNStartup();
-    String operationName = "hasLowRedundancyBlocks";
-    namesystem.checkSuperuserPrivilege(operationName);
-    if (nn.isStandbyState()) {
-      throw new StandbyException("Not supported by Standby Namenode.");
-    }
-    return namesystem.getBlockManager().hasLowRedundancyBlocks(inodeId);
-  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8467ec24/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 feacd74..c8c8d68 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
@@ -702,7 +702,7 @@ public class ReencryptionHandler implements Runnable {
      * @throws InterruptedException
      */
     @Override
-    protected void submitCurrentBatch(final long zoneId) throws IOException,
+    protected void submitCurrentBatch(final Long zoneId) throws IOException,
         InterruptedException {
       if (currentBatch.isEmpty()) {
         return;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8467ec24/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/BlockStorageMovementAttemptedItems.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/BlockStorageMovementAttemptedItems.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/BlockStorageMovementAttemptedItems.java
index ea7a093..d2f0bb2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/BlockStorageMovementAttemptedItems.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/BlockStorageMovementAttemptedItems.java
@@ -45,8 +45,13 @@ import com.google.common.annotations.VisibleForTesting;
  * 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.
+ *
+ * @param <T>
+ *          is identifier of inode or full path name of inode. Internal sps will
+ *          use the file inodeId for the block movement. External sps will use
+ *          file string path representation for the block movement.
  */
-public class BlockStorageMovementAttemptedItems{
+public class BlockStorageMovementAttemptedItems<T> {
   private static final Logger LOG =
       LoggerFactory.getLogger(BlockStorageMovementAttemptedItems.class);
 
@@ -54,7 +59,7 @@ public class BlockStorageMovementAttemptedItems{
    * A map holds the items which are already taken for blocks movements
    * processing and sent to DNs.
    */
-  private final List<AttemptedItemInfo> storageMovementAttemptedItems;
+  private final List<AttemptedItemInfo<T>> storageMovementAttemptedItems;
   private final List<Block> movementFinishedBlocks;
   private volatile boolean monitorRunning = true;
   private Daemon timerThread = null;
@@ -70,11 +75,11 @@ public class BlockStorageMovementAttemptedItems{
   // a request is timed out.
   //
   private long minCheckTimeout = 1 * 60 * 1000; // minimum value
-  private BlockStorageMovementNeeded blockStorageMovementNeeded;
-  private final SPSService service;
+  private BlockStorageMovementNeeded<T> blockStorageMovementNeeded;
+  private final SPSService<T> service;
 
-  public BlockStorageMovementAttemptedItems(SPSService service,
-      BlockStorageMovementNeeded unsatisfiedStorageMovementFiles,
+  public BlockStorageMovementAttemptedItems(SPSService<T> service,
+      BlockStorageMovementNeeded<T> unsatisfiedStorageMovementFiles,
       BlockMovementListener blockMovementListener) {
     this.service = service;
     long recheckTimeout = this.service.getConf().getLong(
@@ -100,7 +105,7 @@ public class BlockStorageMovementAttemptedItems{
    * @param itemInfo
    *          - tracking info
    */
-  public void add(AttemptedItemInfo itemInfo) {
+  public void add(AttemptedItemInfo<T> itemInfo) {
     synchronized (storageMovementAttemptedItems) {
       storageMovementAttemptedItems.add(itemInfo);
     }
@@ -190,25 +195,24 @@ public class BlockStorageMovementAttemptedItems{
   @VisibleForTesting
   void blocksStorageMovementUnReportedItemsCheck() {
     synchronized (storageMovementAttemptedItems) {
-      Iterator<AttemptedItemInfo> iter = storageMovementAttemptedItems
+      Iterator<AttemptedItemInfo<T>> iter = storageMovementAttemptedItems
           .iterator();
       long now = monotonicNow();
       while (iter.hasNext()) {
-        AttemptedItemInfo itemInfo = iter.next();
+        AttemptedItemInfo<T> itemInfo = iter.next();
         if (now > itemInfo.getLastAttemptedOrReportedTime()
             + selfRetryTimeout) {
-          Long blockCollectionID = itemInfo.getFileId();
+          T file = itemInfo.getFile();
           synchronized (movementFinishedBlocks) {
-            ItemInfo candidate = new ItemInfo(itemInfo.getStartId(),
-                blockCollectionID, itemInfo.getRetryCount() + 1);
+            ItemInfo<T> candidate = new ItemInfo<T>(itemInfo.getStartPath(),
+                file, itemInfo.getRetryCount() + 1);
             blockStorageMovementNeeded.add(candidate);
             iter.remove();
             LOG.info("TrackID: {} becomes timed out and moved to needed "
-                + "retries queue for next iteration.", blockCollectionID);
+                + "retries queue for next iteration.", file);
           }
         }
       }
-
     }
   }
 
@@ -219,17 +223,17 @@ public class BlockStorageMovementAttemptedItems{
       while (finishedBlksIter.hasNext()) {
         Block blk = finishedBlksIter.next();
         synchronized (storageMovementAttemptedItems) {
-          Iterator<AttemptedItemInfo> iterator = storageMovementAttemptedItems
-              .iterator();
+          Iterator<AttemptedItemInfo<T>> iterator =
+              storageMovementAttemptedItems.iterator();
           while (iterator.hasNext()) {
-            AttemptedItemInfo attemptedItemInfo = iterator.next();
+            AttemptedItemInfo<T> 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.add(new ItemInfo(attemptedItemInfo
-                  .getStartId(), attemptedItemInfo.getFileId(),
+              blockStorageMovementNeeded.add(new ItemInfo<T>(attemptedItemInfo
+                  .getStartPath(), attemptedItemInfo.getFile(),
                   attemptedItemInfo.getRetryCount() + 1));
               iterator.remove();
             }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8467ec24/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/BlockStorageMovementNeeded.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/BlockStorageMovementNeeded.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/BlockStorageMovementNeeded.java
index c683a63..a194876 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/BlockStorageMovementNeeded.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/BlockStorageMovementNeeded.java
@@ -43,31 +43,36 @@ import com.google.common.annotations.VisibleForTesting;
  * 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.
+ *
+ * @param <T>
+ *          is identifier of inode or full path name of inode. Internal sps will
+ *          use the file inodeId for the block movement. External sps will use
+ *          file string path representation for the block movement.
  */
 @InterfaceAudience.Private
-public class BlockStorageMovementNeeded {
+public class BlockStorageMovementNeeded<T> {
 
   public static final Logger LOG =
       LoggerFactory.getLogger(BlockStorageMovementNeeded.class);
 
-  private final Queue<ItemInfo> storageMovementNeeded =
-      new LinkedList<ItemInfo>();
+  private final Queue<ItemInfo<T>> storageMovementNeeded =
+      new LinkedList<ItemInfo<T>>();
 
   /**
-   * Map of startId and number of child's. Number of child's indicate the
+   * Map of startPath and number of child's. Number of child's indicate the
    * number of files pending to satisfy the policy.
    */
-  private final Map<Long, DirPendingWorkInfo> pendingWorkForDirectory =
-      new HashMap<Long, DirPendingWorkInfo>();
+  private final Map<T, DirPendingWorkInfo> pendingWorkForDirectory =
+      new HashMap<>();
 
-  private final Map<Long, StoragePolicySatisfyPathStatusInfo> spsStatus =
+  private final Map<T, StoragePolicySatisfyPathStatusInfo> spsStatus =
       new ConcurrentHashMap<>();
 
-  private final Context ctxt;
+  private final Context<T> ctxt;
 
   private Daemon pathIdCollector;
 
-  private FileIdCollector fileIDCollector;
+  private FileCollector<T> fileCollector;
 
   private SPSPathIdProcessor pathIDProcessor;
 
@@ -75,10 +80,10 @@ public class BlockStorageMovementNeeded {
   // NOT_AVAILABLE.
   private static long statusClearanceElapsedTimeMs = 300000;
 
-  public BlockStorageMovementNeeded(Context context,
-      FileIdCollector fileIDCollector) {
+  public BlockStorageMovementNeeded(Context<T> context,
+      FileCollector<T> fileCollector) {
     this.ctxt = context;
-    this.fileIDCollector = fileIDCollector;
+    this.fileCollector = fileCollector;
     pathIDProcessor = new SPSPathIdProcessor();
   }
 
@@ -89,8 +94,8 @@ public class BlockStorageMovementNeeded {
    * @param trackInfo
    *          - track info for satisfy the policy
    */
-  public synchronized void add(ItemInfo trackInfo) {
-    spsStatus.put(trackInfo.getStartId(),
+  public synchronized void add(ItemInfo<T> trackInfo) {
+    spsStatus.put(trackInfo.getFile(),
         new StoragePolicySatisfyPathStatusInfo(
             StoragePolicySatisfyPathStatus.IN_PROGRESS));
     storageMovementNeeded.add(trackInfo);
@@ -100,8 +105,8 @@ public class BlockStorageMovementNeeded {
    * Add the itemInfo list to tracking list for which storage movement expected
    * if necessary.
    *
-   * @param startId
-   *          - start id
+   * @param startPath
+   *          - start path
    * @param itemInfoList
    *          - List of child in the directory
    * @param scanCompleted
@@ -109,10 +114,10 @@ public class BlockStorageMovementNeeded {
    *          scan.
    */
   @VisibleForTesting
-  public synchronized void addAll(long startId, List<ItemInfo> itemInfoList,
+  public synchronized void addAll(T startPath, List<ItemInfo<T>> itemInfoList,
       boolean scanCompleted) {
     storageMovementNeeded.addAll(itemInfoList);
-    updatePendingDirScanStats(startId, itemInfoList.size(), scanCompleted);
+    updatePendingDirScanStats(startPath, itemInfoList.size(), scanCompleted);
   }
 
   /**
@@ -126,22 +131,22 @@ public class BlockStorageMovementNeeded {
    *          elements to scan.
    */
   @VisibleForTesting
-  public synchronized void add(ItemInfo itemInfo, boolean scanCompleted) {
+  public synchronized void add(ItemInfo<T> itemInfo, boolean scanCompleted) {
     storageMovementNeeded.add(itemInfo);
     // This represents sps start id is file, so no need to update pending dir
     // stats.
-    if (itemInfo.getStartId() == itemInfo.getFileId()) {
+    if (itemInfo.getStartPath() == itemInfo.getFile()) {
       return;
     }
-    updatePendingDirScanStats(itemInfo.getStartId(), 1, scanCompleted);
+    updatePendingDirScanStats(itemInfo.getFile(), 1, scanCompleted);
   }
 
-  private void updatePendingDirScanStats(long startId, int numScannedFiles,
+  private void updatePendingDirScanStats(T startPath, int numScannedFiles,
       boolean scanCompleted) {
-    DirPendingWorkInfo pendingWork = pendingWorkForDirectory.get(startId);
+    DirPendingWorkInfo pendingWork = pendingWorkForDirectory.get(startPath);
     if (pendingWork == null) {
       pendingWork = new DirPendingWorkInfo();
-      pendingWorkForDirectory.put(startId, pendingWork);
+      pendingWorkForDirectory.put(startPath, pendingWork);
     }
     pendingWork.addPendingWorkCount(numScannedFiles);
     if (scanCompleted) {
@@ -150,12 +155,12 @@ public class BlockStorageMovementNeeded {
   }
 
   /**
-   * Gets the block collection id for which storage movements check necessary
+   * Gets the satisfier files for which block storage movements check necessary
    * and make the movement if required.
    *
-   * @return block collection ID
+   * @return satisfier files
    */
-  public synchronized ItemInfo get() {
+  public synchronized ItemInfo<T> get() {
     return storageMovementNeeded.poll();
   }
 
@@ -176,12 +181,12 @@ 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,
+  public synchronized void removeItemTrackInfo(ItemInfo<T> trackInfo,
       boolean isSuccess) throws IOException {
     if (trackInfo.isDir()) {
       // If track is part of some start inode then reduce the pending
       // directory work count.
-      long startId = trackInfo.getStartId();
+      T startId = trackInfo.getStartPath();
       if (!ctxt.isFileExist(startId)) {
         // directory deleted just remove it.
         this.pendingWorkForDirectory.remove(startId);
@@ -202,17 +207,17 @@ public class BlockStorageMovementNeeded {
     } else {
       // Remove xAttr if trackID doesn't exist in
       // storageMovementAttemptedItems or file policy satisfied.
-      ctxt.removeSPSHint(trackInfo.getFileId());
-      updateStatus(trackInfo.getStartId(), isSuccess);
+      ctxt.removeSPSHint(trackInfo.getFile());
+      updateStatus(trackInfo.getFile(), isSuccess);
     }
   }
 
-  public synchronized void clearQueue(long trackId) {
+  public synchronized void clearQueue(T trackId) {
     ctxt.removeSPSPathId(trackId);
-    Iterator<ItemInfo> iterator = storageMovementNeeded.iterator();
+    Iterator<ItemInfo<T>> iterator = storageMovementNeeded.iterator();
     while (iterator.hasNext()) {
-      ItemInfo next = iterator.next();
-      if (next.getStartId() == trackId) {
+      ItemInfo<T> next = iterator.next();
+      if (next.getFile() == trackId) {
         iterator.remove();
       }
     }
@@ -222,7 +227,7 @@ public class BlockStorageMovementNeeded {
   /**
    * Mark inode status as SUCCESS in map.
    */
-  private void updateStatus(long startId, boolean isSuccess){
+  private void updateStatus(T startId, boolean isSuccess){
     StoragePolicySatisfyPathStatusInfo spsStatusInfo =
         spsStatus.get(startId);
     if (spsStatusInfo == null) {
@@ -244,8 +249,8 @@ public class BlockStorageMovementNeeded {
    */
   public synchronized void clearQueuesWithNotification() {
     // Remove xAttr from directories
-    Long trackId;
-    while ((trackId = ctxt.getNextSPSPathId()) != null) {
+    T trackId;
+    while ((trackId = ctxt.getNextSPSPath()) != null) {
       try {
         // Remove xAttr for file
         ctxt.removeSPSHint(trackId);
@@ -256,17 +261,17 @@ public class BlockStorageMovementNeeded {
 
     // File's directly added to storageMovementNeeded, So try to remove
     // xAttr for file
-    ItemInfo itemInfo;
+    ItemInfo<T> itemInfo;
     while ((itemInfo = get()) != null) {
       try {
         // Remove xAttr for file
         if (!itemInfo.isDir()) {
-          ctxt.removeSPSHint(itemInfo.getFileId());
+          ctxt.removeSPSHint(itemInfo.getFile());
         }
       } catch (IOException ie) {
         LOG.warn(
             "Failed to remove SPS xattr for track id "
-                + itemInfo.getFileId(), ie);
+                + itemInfo.getFile(), ie);
       }
     }
     this.clearAll();
@@ -282,29 +287,29 @@ public class BlockStorageMovementNeeded {
     public void run() {
       LOG.info("Starting SPSPathIdProcessor!.");
       long lastStatusCleanTime = 0;
-      Long startINodeId = null;
+      T startINode = null;
       while (ctxt.isRunning()) {
         try {
           if (!ctxt.isInSafeMode()) {
-            if (startINodeId == null) {
-              startINodeId = ctxt.getNextSPSPathId();
+            if (startINode == null) {
+              startINode = ctxt.getNextSPSPath();
             } // else same id will be retried
-            if (startINodeId == null) {
+            if (startINode == null) {
               // Waiting for SPS path
               Thread.sleep(3000);
             } else {
-              spsStatus.put(startINodeId,
+              spsStatus.put(startINode,
                   new StoragePolicySatisfyPathStatusInfo(
                       StoragePolicySatisfyPathStatus.IN_PROGRESS));
-              fileIDCollector.scanAndCollectFileIds(startINodeId);
+              fileCollector.scanAndCollectFiles(startINode);
               // check if directory was empty and no child added to queue
               DirPendingWorkInfo dirPendingWorkInfo =
-                  pendingWorkForDirectory.get(startINodeId);
+                  pendingWorkForDirectory.get(startINode);
               if (dirPendingWorkInfo != null
                   && dirPendingWorkInfo.isDirWorkDone()) {
-                ctxt.removeSPSHint(startINodeId);
-                pendingWorkForDirectory.remove(startINodeId);
-                updateStatus(startINodeId, true);
+                ctxt.removeSPSHint(startINode);
+                pendingWorkForDirectory.remove(startINode);
+                updateStatus(startINode, true);
               }
             }
             //Clear the SPS status if status is in SUCCESS more than 5 min.
@@ -313,7 +318,7 @@ public class BlockStorageMovementNeeded {
               lastStatusCleanTime = Time.monotonicNow();
               cleanSPSStatus();
             }
-            startINodeId = null; // Current inode id successfully scanned.
+            startINode = null; // Current inode successfully scanned.
           }
         } catch (Throwable t) {
           String reClass = t.getClass().getName();
@@ -334,9 +339,9 @@ public class BlockStorageMovementNeeded {
     }
 
     private synchronized void cleanSPSStatus() {
-      for (Iterator<Entry<Long, StoragePolicySatisfyPathStatusInfo>> it =
-          spsStatus.entrySet().iterator(); it.hasNext();) {
-        Entry<Long, StoragePolicySatisfyPathStatusInfo> entry = it.next();
+      for (Iterator<Entry<T, StoragePolicySatisfyPathStatusInfo>> it = spsStatus
+          .entrySet().iterator(); it.hasNext();) {
+        Entry<T, StoragePolicySatisfyPathStatusInfo> entry = it.next();
         if (entry.getValue().canRemove()) {
           it.remove();
         }
@@ -472,8 +477,8 @@ public class BlockStorageMovementNeeded {
     return statusClearanceElapsedTimeMs;
   }
 
-  public void markScanCompletedForDir(Long inodeId) {
-    DirPendingWorkInfo pendingWork = pendingWorkForDirectory.get(inodeId);
+  public void markScanCompletedForDir(T inode) {
+    DirPendingWorkInfo pendingWork = pendingWorkForDirectory.get(inode);
     if (pendingWork != null) {
       pendingWork.markScanCompleted();
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8467ec24/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/Context.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/Context.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/Context.java
index ff4ad6b..84a969d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/Context.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/Context.java
@@ -33,11 +33,16 @@ import org.apache.hadoop.net.NetworkTopology;
 import org.apache.hadoop.security.AccessControlException;
 
 /**
- * An interface for the communication between NameNode and SPS module.
+ * An interface for the communication between SPS and Namenode module.
+ *
+ * @param <T>
+ *          is identifier of inode or full path name of inode. Internal sps will
+ *          use the file inodeId for the block movement. External sps will use
+ *          file string path representation for the block movement.
  */
 @InterfaceAudience.Private
 @InterfaceStability.Evolving
-public interface Context {
+public interface Context<T> {
 
   /**
    * Returns true if the SPS is running, false otherwise.
@@ -72,13 +77,13 @@ public interface Context {
   NetworkTopology getNetworkTopology();
 
   /**
-   * Returns true if the give Inode exists in the Namespace.
+   * Returns true if the give file exists in the Namespace.
    *
-   * @param inodeId
-   *          - Inode ID
-   * @return true if Inode exists, false otherwise.
+   * @param filePath
+   *          - file info
+   * @return true if the given file exists, false otherwise.
    */
-  boolean isFileExist(long inodeId);
+  boolean isFileExist(T filePath);
 
   /**
    * Gets the storage policy details for the given policy ID.
@@ -97,11 +102,11 @@ public interface Context {
   /**
    * Remove the hint which was added to track SPS call.
    *
-   * @param inodeId
-   *          - Inode ID
+   * @param spsPath
+   *          - user invoked satisfier path
    * @throws IOException
    */
-  void removeSPSHint(long inodeId) throws IOException;
+  void removeSPSHint(T spsPath) throws IOException;
 
   /**
    * Gets the number of live datanodes in the cluster.
@@ -113,11 +118,11 @@ public interface Context {
   /**
    * Get the file info for a specific file.
    *
-   * @param inodeID
-   *          inode identifier
+   * @param file
+   *          file path
    * @return file status metadata information
    */
-  HdfsFileStatus getFileInfo(long inodeID) throws IOException;
+  HdfsFileStatus getFileInfo(T file) throws IOException;
 
   /**
    * Returns all the live datanodes and its storage details.
@@ -128,15 +133,6 @@ public interface Context {
       throws IOException;
 
   /**
-   * Returns true if the given inode file has low redundancy blocks.
-   *
-   * @param inodeID
-   *          inode identifier
-   * @return true if block collection has low redundancy blocks
-   */
-  boolean hasLowRedundancyBlocks(long inodeID);
-
-  /**
    * Checks whether the given datanode has sufficient space to occupy the given
    * blockSize data.
    *
@@ -153,26 +149,17 @@ public interface Context {
       long blockSize);
 
   /**
-   * @return next SPS path id to process.
+   * @return next SPS path info to process.
    */
-  Long getNextSPSPathId();
+  T getNextSPSPath();
 
   /**
    * Removes the SPS path id.
    */
-  void removeSPSPathId(long pathId);
+  void removeSPSPathId(T pathId);
 
   /**
    * Removes all SPS path ids.
    */
   void removeAllSPSPathIds();
-
-  /**
-   * Gets the file path for a given inode id.
-   *
-   * @param inodeId
-   *          - path inode id.
-   */
-  String getFilePath(Long inodeId);
-
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8467ec24/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/FileCollector.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/FileCollector.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/FileCollector.java
new file mode 100644
index 0000000..dceb5fa
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/FileCollector.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.namenode.sps;
+
+import java.io.IOException;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * An interface for scanning the directory recursively and collect files
+ * under the given directory.
+ *
+ * @param <T>
+ *          is identifier of inode or full path name of inode. Internal sps will
+ *          use the file inodeId for the block movement. External sps will use
+ *          file string path representation for the block movement.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public interface FileCollector<T> {
+
+  /**
+   * This method can be used to scan and collects the files under that
+   * directory and adds to the given BlockStorageMovementNeeded.
+   *
+   * @param filePath
+   *          - file path
+   */
+  void scanAndCollectFiles(T filePath)
+      throws IOException, InterruptedException;
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8467ec24/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/FileIdCollector.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/FileIdCollector.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/FileIdCollector.java
deleted file mode 100644
index 7cf77f0..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/FileIdCollector.java
+++ /dev/null
@@ -1,43 +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.sps;
-
-import java.io.IOException;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-
-/**
- * An interface for scanning the directory recursively and collect file ids
- * under the given directory.
- */
-@InterfaceAudience.Private
-@InterfaceStability.Evolving
-public interface FileIdCollector {
-
-  /**
-   * Scans the given inode directory and collects the file ids under that
-   * directory and adds to the given BlockStorageMovementNeeded.
-   *
-   * @param inodeID
-   *          - The directory ID
-   */
-  void scanAndCollectFileIds(Long inodeId)
-      throws IOException, InterruptedException;
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8467ec24/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/IntraSPSNameNodeContext.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/IntraSPSNameNodeContext.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/IntraSPSNameNodeContext.java
index 495d1c4..f6b6d95 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/IntraSPSNameNodeContext.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/IntraSPSNameNodeContext.java
@@ -47,17 +47,17 @@ import org.slf4j.LoggerFactory;
  * movements to satisfy the storage policy.
  */
 @InterfaceAudience.Private
-public class IntraSPSNameNodeContext implements Context {
+public class IntraSPSNameNodeContext implements Context<Long> {
   private static final Logger LOG = LoggerFactory
       .getLogger(IntraSPSNameNodeContext.class);
 
   private final Namesystem namesystem;
   private final BlockManager blockManager;
 
-  private SPSService service;
+  private SPSService<Long> service;
 
   public IntraSPSNameNodeContext(Namesystem namesystem,
-      BlockManager blockManager, SPSService service) {
+      BlockManager blockManager, SPSService<Long> service) {
     this.namesystem = namesystem;
     this.blockManager = blockManager;
     this.service = service;
@@ -68,20 +68,18 @@ public class IntraSPSNameNodeContext implements Context {
     return blockManager.getDatanodeManager().getNumLiveDataNodes();
   }
 
+  /**
+   * @return object containing information regarding the file or null if file
+   *         not found.
+   */
   @Override
-  public HdfsFileStatus getFileInfo(long inodeID) throws IOException {
+  public HdfsFileStatus getFileInfo(Long inodeID) throws IOException {
     String filePath = namesystem.getFilePath(inodeID);
     if (StringUtils.isBlank(filePath)) {
       LOG.debug("File with inodeID:{} doesn't exists!", inodeID);
       return null;
     }
-    HdfsFileStatus fileInfo = null;
-    try {
-      fileInfo = namesystem.getFileInfo(filePath, true, true);
-    } catch (IOException e) {
-      LOG.debug("File path:{} doesn't exists!", filePath);
-    }
-    return fileInfo;
+    return namesystem.getFileInfo(filePath, true, true);
   }
 
   @Override
@@ -97,17 +95,12 @@ public class IntraSPSNameNodeContext implements Context {
   }
 
   @Override
-  public boolean hasLowRedundancyBlocks(long inodeId) {
-    return blockManager.hasLowRedundancyBlocks(inodeId);
-  }
-
-  @Override
-  public boolean isFileExist(long inodeId) {
+  public boolean isFileExist(Long inodeId) {
     return namesystem.getFSDirectory().getInode(inodeId) != null;
   }
 
   @Override
-  public void removeSPSHint(long inodeId) throws IOException {
+  public void removeSPSHint(Long inodeId) throws IOException {
     this.namesystem.removeXattr(inodeId, XATTR_SATISFY_STORAGE_POLICY);
   }
 
@@ -177,12 +170,12 @@ public class IntraSPSNameNodeContext implements Context {
   }
 
   @Override
-  public Long getNextSPSPathId() {
+  public Long getNextSPSPath() {
     return blockManager.getSPSManager().getNextPathId();
   }
 
   @Override
-  public void removeSPSPathId(long trackId) {
+  public void removeSPSPathId(Long trackId) {
     blockManager.getSPSManager().removePathId(trackId);
   }
 
@@ -190,10 +183,4 @@ public class IntraSPSNameNodeContext implements Context {
   public void removeAllSPSPathIds() {
     blockManager.getSPSManager().removeAllPathIds();
   }
-
-  @Override
-  public String getFilePath(Long inodeId) {
-    return namesystem.getFilePath(inodeId);
-  }
-
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8467ec24/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/IntraSPSNameNodeFileIdCollector.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/IntraSPSNameNodeFileIdCollector.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/IntraSPSNameNodeFileIdCollector.java
index 7a44dd9..27d9e7d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/IntraSPSNameNodeFileIdCollector.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/IntraSPSNameNodeFileIdCollector.java
@@ -35,15 +35,16 @@ import org.apache.hadoop.hdfs.server.namenode.INode;
  */
 @InterfaceAudience.Private
 public class IntraSPSNameNodeFileIdCollector extends FSTreeTraverser
-    implements FileIdCollector {
+    implements FileCollector<Long> {
   private int maxQueueLimitToScan;
-  private final SPSService service;
+  private final SPSService <Long> service;
 
   private int remainingCapacity = 0;
 
-  private List<ItemInfo> currentBatch;
+  private List<ItemInfo<Long>> currentBatch;
 
-  public IntraSPSNameNodeFileIdCollector(FSDirectory dir, SPSService service) {
+  public IntraSPSNameNodeFileIdCollector(FSDirectory dir,
+      SPSService<Long> service) {
     super(dir);
     this.service = service;
     this.maxQueueLimitToScan = service.getConf().getInt(
@@ -63,7 +64,7 @@ public class IntraSPSNameNodeFileIdCollector extends FSTreeTraverser
       return false;
     }
     if (inode.isFile() && inode.asFile().numBlocks() != 0) {
-      currentBatch.add(new ItemInfo(
+      currentBatch.add(new ItemInfo<Long>(
           ((SPSTraverseInfo) traverseInfo).getStartId(), inode.getId()));
       remainingCapacity--;
     }
@@ -83,10 +84,10 @@ public class IntraSPSNameNodeFileIdCollector extends FSTreeTraverser
   }
 
   @Override
-  protected void submitCurrentBatch(long startId)
+  protected void submitCurrentBatch(Long startId)
       throws IOException, InterruptedException {
     // Add current child's to queue
-    service.addAllFileIdsToProcess(startId,
+    service.addAllFilesToProcess(startId,
         currentBatch, false);
     currentBatch.clear();
   }
@@ -119,7 +120,7 @@ public class IntraSPSNameNodeFileIdCollector extends FSTreeTraverser
   }
 
   @Override
-  public void scanAndCollectFileIds(final Long startINodeId)
+  public void scanAndCollectFiles(final Long startINodeId)
       throws IOException, InterruptedException {
     FSDirectory fsd = getFSDirectory();
     INode startInode = fsd.getInode(startINodeId);
@@ -129,9 +130,9 @@ public class IntraSPSNameNodeFileIdCollector extends FSTreeTraverser
         throttle();
       }
       if (startInode.isFile()) {
-        currentBatch.add(new ItemInfo(startInode.getId(), startInode.getId()));
+        currentBatch
+            .add(new ItemInfo<Long>(startInode.getId(), startInode.getId()));
       } else {
-
         readLock();
         // NOTE: this lock will not be held for full directory scanning. It is
         // basically a sliced locking. Once it collects a batch size( at max the
@@ -148,7 +149,7 @@ public class IntraSPSNameNodeFileIdCollector extends FSTreeTraverser
         }
       }
       // Mark startInode traverse is done, this is last-batch
-      service.addAllFileIdsToProcess(startInode.getId(), currentBatch, true);
+      service.addAllFilesToProcess(startInode.getId(), currentBatch, true);
       currentBatch.clear();
     }
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8467ec24/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/ItemInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/ItemInfo.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/ItemInfo.java
index 47c64cc..bd8ab92 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/ItemInfo.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/ItemInfo.java
@@ -21,48 +21,51 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 
 /**
- * ItemInfo is a file info object for which need to satisfy the policy.
+ * ItemInfo is a file info object for which need to satisfy the policy. For
+ * internal satisfier service, it uses inode id which is Long datatype. For the
+ * external satisfier service, it uses the full string representation of the
+ * path.
  */
 @InterfaceAudience.Private
 @InterfaceStability.Evolving
-public class ItemInfo {
-  private long startId;
-  private long fileId;
+public class ItemInfo<T> {
+  private T startPath;
+  private T file;
   private int retryCount;
 
-  public ItemInfo(long startId, long fileId) {
-    this.startId = startId;
-    this.fileId = fileId;
+  public ItemInfo(T startPath, T file) {
+    this.startPath = startPath;
+    this.file = file;
     // set 0 when item is getting added first time in queue.
     this.retryCount = 0;
   }
 
-  public ItemInfo(final long startId, final long fileId, final int retryCount) {
-    this.startId = startId;
-    this.fileId = fileId;
+  public ItemInfo(final T startPath, final T file, final int retryCount) {
+    this.startPath = startPath;
+    this.file = file;
     this.retryCount = retryCount;
   }
 
   /**
-   * Return the start inode id of the current track Id. This indicates that SPS
-   * was invoked on this inode id.
+   * Returns the start path of the current file. This indicates that SPS
+   * was invoked on this path.
    */
-  public long getStartId() {
-    return startId;
+  public T getStartPath() {
+    return startPath;
   }
 
   /**
-   * Return the File inode Id for which needs to satisfy the policy.
+   * Returns the file for which needs to satisfy the policy.
    */
-  public long getFileId() {
-    return fileId;
+  public T getFile() {
+    return file;
   }
 
   /**
    * Returns true if the tracking path is a directory, false otherwise.
    */
   public boolean isDir() {
-    return (startId != fileId);
+    return !startPath.equals(file);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8467ec24/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/SPSService.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/SPSService.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/SPSService.java
index da6e365..71d8fd1 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/SPSService.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/SPSService.java
@@ -27,10 +27,15 @@ import org.apache.hadoop.hdfs.server.protocol.BlocksStorageMoveAttemptFinished;
 
 /**
  * An interface for SPSService, which exposes life cycle and processing APIs.
+ *
+ * @param <T>
+ *          is identifier of inode or full path name of inode. Internal sps will
+ *          use the file inodeId for the block movement. External sps will use
+ *          file string path representation for the block movement.
  */
 @InterfaceAudience.Private
 @InterfaceStability.Evolving
-public interface SPSService {
+public interface SPSService<T> {
 
   /**
    * Initializes the helper services.
@@ -38,7 +43,7 @@ public interface SPSService {
    * @param ctxt
    *          - context is an helper service to provide communication channel
    *          between NN and SPS
-   * @param fileIDCollector
+   * @param fileCollector
    *          - a helper service for scanning the files under a given directory
    *          id
    * @param handler
@@ -46,7 +51,7 @@ public interface SPSService {
    * @param blkMovementListener
    *          - listener to know about block movement attempt completion
    */
-  void init(Context ctxt, FileIdCollector fileIDCollector,
+  void init(Context<T> ctxt, FileCollector<T> fileCollector,
       BlockMoveTaskHandler handler, BlockMovementListener blkMovementListener);
 
   /**
@@ -82,23 +87,24 @@ public interface SPSService {
   boolean isRunning();
 
   /**
-   * Adds the Item information(file id etc) to processing queue.
+   * Adds the Item information(file etc) to processing queue.
    *
    * @param itemInfo
+   *          file info object for which need to satisfy the policy
    */
-  void addFileIdToProcess(ItemInfo itemInfo, boolean scanCompleted);
+  void addFileToProcess(ItemInfo<T> itemInfo, boolean scanCompleted);
 
   /**
-   * Adds all the Item information(file id etc) to processing queue.
+   * Adds all the Item information(file etc) to processing queue.
    *
-   * @param startId
-   *          - directory/file id, on which SPS was called.
+   * @param startPath
+   *          - directory/file, on which SPS was called.
    * @param itemInfoList
    *          - list of item infos
    * @param scanCompleted
    *          - whether the scanning of directory fully done with itemInfoList
    */
-  void addAllFileIdsToProcess(long startId, List<ItemInfo> itemInfoList,
+  void addAllFilesToProcess(T startPath, List<ItemInfo<T>> itemInfoList,
       boolean scanCompleted);
 
   /**
@@ -109,7 +115,7 @@ public interface SPSService {
   /**
    * Clear inodeId present in the processing queue.
    */
-  void clearQueue(long inodeId);
+  void clearQueue(T spsPath);
 
   /**
    * @return the configuration.
@@ -119,10 +125,10 @@ public interface SPSService {
   /**
    * Marks the scanning of directory if finished.
    *
-   * @param inodeId
-   *          - directory inode id.
+   * @param spsPath
+   *          - satisfier path
    */
-  void markScanCompletedForPath(Long inodeId);
+  void markScanCompletedForPath(T spsPath);
 
   /**
    * Notify the details of storage movement attempt finished blocks.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8467ec24/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/StoragePolicySatisfier.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/StoragePolicySatisfier.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/StoragePolicySatisfier.java
index 6b449aa..08a26e1 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/StoragePolicySatisfier.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/StoragePolicySatisfier.java
@@ -66,7 +66,7 @@ import com.google.common.base.Preconditions;
  * 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 a dedidated daemon (can
+ * where users may want to physically move the blocks by a dedicated daemon (can
  * run inside Namenode or stand alone) 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
@@ -77,19 +77,19 @@ import com.google.common.base.Preconditions;
  * physical block movements.
  */
 @InterfaceAudience.Private
-public class StoragePolicySatisfier implements SPSService, Runnable {
+public class StoragePolicySatisfier<T> implements SPSService<T>, Runnable {
   public static final Logger LOG =
       LoggerFactory.getLogger(StoragePolicySatisfier.class);
   private Daemon storagePolicySatisfierThread;
-  private BlockStorageMovementNeeded storageMovementNeeded;
-  private BlockStorageMovementAttemptedItems storageMovementsMonitor;
+  private BlockStorageMovementNeeded<T> storageMovementNeeded;
+  private BlockStorageMovementAttemptedItems<T> storageMovementsMonitor;
   private volatile boolean isRunning = false;
   private volatile StoragePolicySatisfierMode spsMode =
       StoragePolicySatisfierMode.NONE;
   private int spsWorkMultiplier;
   private long blockCount = 0L;
   private int blockMovementMaxRetry;
-  private Context ctxt;
+  private Context<T> ctxt;
   private BlockMoveTaskHandler blockMoveTaskHandler;
   private final Configuration conf;
 
@@ -135,15 +135,15 @@ public class StoragePolicySatisfier implements SPSService, Runnable {
     }
   }
 
-  public void init(final Context context, final FileIdCollector fileIDCollector,
+  public void init(final Context<T> context,
+      final FileCollector<T> fileIDCollector,
       final BlockMoveTaskHandler blockMovementTaskHandler,
       final BlockMovementListener blockMovementListener) {
     this.ctxt = context;
-    this.storageMovementNeeded =
-        new BlockStorageMovementNeeded(context, fileIDCollector);
-    this.storageMovementsMonitor =
-        new BlockStorageMovementAttemptedItems(this,
-        storageMovementNeeded, blockMovementListener);
+    this.storageMovementNeeded = new BlockStorageMovementNeeded<T>(context,
+        fileIDCollector);
+    this.storageMovementsMonitor = new BlockStorageMovementAttemptedItems<T>(
+        this, storageMovementNeeded, blockMovementListener);
     this.blockMoveTaskHandler = blockMovementTaskHandler;
     this.spsWorkMultiplier = getSPSWorkMultiplier(getConf());
     this.blockMovementMaxRetry = getConf().getInt(
@@ -257,24 +257,24 @@ public class StoragePolicySatisfier implements SPSService, Runnable {
         continue;
       }
       try {
+        ItemInfo<T> itemInfo = null;
+        boolean retryItem = false;
         if (!ctxt.isInSafeMode()) {
-          ItemInfo itemInfo = storageMovementNeeded.get();
+          itemInfo = storageMovementNeeded.get();
           if (itemInfo != null) {
             if(itemInfo.getRetryCount() >= blockMovementMaxRetry){
               LOG.info("Failed to satisfy the policy after "
                   + blockMovementMaxRetry + " retries. Removing inode "
-                  + itemInfo.getFileId() + " from the queue");
+                  + itemInfo.getFile() + " from the queue");
               storageMovementNeeded.removeItemTrackInfo(itemInfo, false);
               continue;
             }
-            long trackId = itemInfo.getFileId();
+            T trackId = itemInfo.getFile();
             BlocksMovingAnalysis status = null;
             DatanodeStorageReport[] liveDnReports;
             BlockStoragePolicy existingStoragePolicy;
             // TODO: presently, context internally acquire the lock
             // and returns the result. Need to discuss to move the lock outside?
-            boolean hasLowRedundancyBlocks = ctxt
-                .hasLowRedundancyBlocks(trackId);
             HdfsFileStatus fileStatus = ctxt.getFileInfo(trackId);
             // Check path existence.
             if (fileStatus == null || fileStatus.isDir()) {
@@ -289,7 +289,7 @@ public class StoragePolicySatisfier implements SPSService, Runnable {
 
               HdfsLocatedFileStatus file = (HdfsLocatedFileStatus) fileStatus;
               status = analyseBlocksStorageMovementsAndAssignToDN(file,
-                  hasLowRedundancyBlocks, existingStoragePolicy, liveDnReports);
+                  existingStoragePolicy, liveDnReports);
               switch (status.status) {
               // Just add to monitor, so it will be retried after timeout
               case ANALYSIS_SKIPPED_FOR_RETRY:
@@ -302,8 +302,8 @@ public class StoragePolicySatisfier implements SPSService, Runnable {
                       + "movement attempt finished report",
                       status.status, fileStatus.getPath());
                 }
-                this.storageMovementsMonitor.add(new AttemptedItemInfo(itemInfo
-                    .getStartId(), itemInfo.getFileId(), monotonicNow(),
+                this.storageMovementsMonitor.add(new AttemptedItemInfo<T>(
+                    itemInfo.getStartPath(), itemInfo.getFile(), monotonicNow(),
                     status.assignedBlocks, itemInfo.getRetryCount()));
                 break;
               case NO_BLOCKS_TARGETS_PAIRED:
@@ -312,8 +312,7 @@ public class StoragePolicySatisfier implements SPSService, Runnable {
                       + " retry queue as none of the blocks found its eligible"
                       + " targets.", trackId, fileStatus.getPath());
                 }
-                itemInfo.increRetryCount();
-                this.storageMovementNeeded.add(itemInfo);
+                retryItem = true;
                 break;
               case FEW_LOW_REDUNDANCY_BLOCKS:
                 if (LOG.isDebugEnabled()) {
@@ -321,8 +320,7 @@ public class StoragePolicySatisfier implements SPSService, Runnable {
                       + "retry queue as some of the blocks are low redundant.",
                       trackId, fileStatus.getPath());
                 }
-                itemInfo.increRetryCount();
-                this.storageMovementNeeded.add(itemInfo);
+                retryItem = true;
                 break;
               case BLOCKS_FAILED_TO_MOVE:
                 if (LOG.isDebugEnabled()) {
@@ -330,7 +328,7 @@ public class StoragePolicySatisfier implements SPSService, Runnable {
                       + "retry queue as some of the blocks movement failed.",
                       trackId, fileStatus.getPath());
                 }
-                this.storageMovementNeeded.add(itemInfo);
+                retryItem = true;
                 break;
               // Just clean Xattrs
               case BLOCKS_TARGET_PAIRING_SKIPPED:
@@ -354,6 +352,10 @@ public class StoragePolicySatisfier implements SPSService, Runnable {
           Thread.sleep(3000);
           blockCount = 0L;
         }
+        if (retryItem) {
+          itemInfo.increRetryCount();
+          this.storageMovementNeeded.add(itemInfo);
+        }
       } catch (IOException e) {
         LOG.error("Exception during StoragePolicySatisfier execution - "
             + "will continue next cycle", e);
@@ -377,7 +379,7 @@ public class StoragePolicySatisfier implements SPSService, Runnable {
   }
 
   private BlocksMovingAnalysis analyseBlocksStorageMovementsAndAssignToDN(
-      HdfsLocatedFileStatus fileInfo, boolean hasLowRedundancyBlocks,
+      HdfsLocatedFileStatus fileInfo,
       BlockStoragePolicy existingStoragePolicy,
       DatanodeStorageReport[] liveDns) {
     BlocksMovingAnalysis.Status status =
@@ -403,9 +405,17 @@ public class StoragePolicySatisfier implements SPSService, Runnable {
           new ArrayList<>());
     }
     List<BlockMovingInfo> blockMovingInfos = new ArrayList<BlockMovingInfo>();
-
+    boolean hasLowRedundancyBlocks = false;
+    int replication = fileInfo.getReplication();
     for (int i = 0; i < blocks.size(); i++) {
       LocatedBlock blockInfo = blocks.get(i);
+
+      // Block is considered as low redundancy when the block locations array
+      // length is less than expected replication factor. If any of the block is
+      // low redundant, then hasLowRedundancyBlocks will be marked as true.
+      hasLowRedundancyBlocks |= isLowRedundancyBlock(blockInfo, replication,
+          ecPolicy);
+
       List<StorageType> expectedStorageTypes;
       if (blockInfo.isStriped()) {
         if (ErasureCodingPolicyManager
@@ -446,13 +456,15 @@ public class StoragePolicySatisfier implements SPSService, Runnable {
           // policy.
           status = BlocksMovingAnalysis.Status.NO_BLOCKS_TARGETS_PAIRED;
         }
-      } else if (hasLowRedundancyBlocks
-          && status != BlocksMovingAnalysis.Status.BLOCKS_TARGETS_PAIRED) {
-        // Check if the previous block was successfully paired.
-        status = BlocksMovingAnalysis.Status.FEW_LOW_REDUNDANCY_BLOCKS;
       }
     }
 
+    // If there is no block paired and few blocks are low redundant, so marking
+    // the status as FEW_LOW_REDUNDANCY_BLOCKS.
+    if (hasLowRedundancyBlocks
+        && status == BlocksMovingAnalysis.Status.NO_BLOCKS_TARGETS_PAIRED) {
+      status = BlocksMovingAnalysis.Status.FEW_LOW_REDUNDANCY_BLOCKS;
+    }
     List<Block> assignedBlockIds = new ArrayList<Block>();
     for (BlockMovingInfo blkMovingInfo : blockMovingInfos) {
       // Check for at least one block storage movement has been chosen
@@ -471,6 +483,33 @@ public class StoragePolicySatisfier implements SPSService, Runnable {
   }
 
   /**
+   * The given block is considered as low redundancy when the block locations
+   * length is less than expected replication factor. For EC blocks, redundancy
+   * is the summation of data + parity blocks.
+   *
+   * @param blockInfo
+   *          block
+   * @param replication
+   *          replication factor of the given file block
+   * @param ecPolicy
+   *          erasure coding policy of the given file block
+   * @return true if the given block is low redundant.
+   */
+  private boolean isLowRedundancyBlock(LocatedBlock blockInfo, int replication,
+      ErasureCodingPolicy ecPolicy) {
+    boolean hasLowRedundancyBlock = false;
+    if (blockInfo.isStriped()) {
+      // For EC blocks, redundancy is the summation of data + parity blocks.
+      replication = ecPolicy.getNumDataUnits() + ecPolicy.getNumParityUnits();
+    }
+    // block is considered as low redundancy when the block locations length is
+    // less than expected replication factor.
+    hasLowRedundancyBlock = blockInfo.getLocations().length < replication ? true
+        : false;
+    return hasLowRedundancyBlock;
+  }
+
+  /**
    * 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
@@ -863,7 +902,7 @@ public class StoragePolicySatisfier implements SPSService, Runnable {
   }
 
   @VisibleForTesting
-  BlockStorageMovementAttemptedItems getAttemptedItemsMonitor() {
+  public BlockStorageMovementAttemptedItems<T> getAttemptedItemsMonitor() {
     return storageMovementsMonitor;
   }
 
@@ -880,7 +919,7 @@ public class StoragePolicySatisfier implements SPSService, Runnable {
   /**
    * Clear queues for given track id.
    */
-  public void clearQueue(long trackId) {
+  public void clearQueue(T trackId) {
     storageMovementNeeded.clearQueue(trackId);
   }
 
@@ -889,7 +928,7 @@ public class StoragePolicySatisfier implements SPSService, Runnable {
    * attempted or reported time stamp. This is used by
    * {@link BlockStorageMovementAttemptedItems#storageMovementAttemptedItems}.
    */
-  final static class AttemptedItemInfo extends ItemInfo {
+  final static class AttemptedItemInfo<T> extends ItemInfo<T> {
     private long lastAttemptedOrReportedTime;
     private final List<Block> blocks;
 
@@ -903,7 +942,7 @@ public class StoragePolicySatisfier implements SPSService, Runnable {
      * @param lastAttemptedOrReportedTime
      *          last attempted or reported time
      */
-    AttemptedItemInfo(long rootId, long trackId,
+    AttemptedItemInfo(T rootId, T trackId,
         long lastAttemptedOrReportedTime,
         List<Block> blocks, int retryCount) {
       super(rootId, trackId, retryCount);
@@ -932,24 +971,33 @@ public class StoragePolicySatisfier implements SPSService, Runnable {
 
   }
 
+  /**
+   * Returns sps invoked path status. This method is used by internal satisfy
+   * storage policy service.
+   *
+   * @param path
+   *          sps path
+   * @return storage policy satisfy path status
+   * @throws IOException
+   */
   public StoragePolicySatisfyPathStatus checkStoragePolicySatisfyPathStatus(
       String path) throws IOException {
     return storageMovementNeeded.getStatus(ctxt.getFileID(path));
   }
 
   @Override
-  public void addFileIdToProcess(ItemInfo trackInfo, boolean scanCompleted) {
+  public void addFileToProcess(ItemInfo<T> trackInfo, boolean scanCompleted) {
     storageMovementNeeded.add(trackInfo, scanCompleted);
     if (LOG.isDebugEnabled()) {
       LOG.debug("Added track info for inode {} to block "
-          + "storageMovementNeeded queue", trackInfo.getFileId());
+          + "storageMovementNeeded queue", trackInfo.getFile());
     }
   }
 
   @Override
-  public void addAllFileIdsToProcess(long startId, List<ItemInfo> itemInfoList,
+  public void addAllFilesToProcess(T startPath, List<ItemInfo<T>> itemInfoList,
       boolean scanCompleted) {
-    getStorageMovementQueue().addAll(startId, itemInfoList, scanCompleted);
+    getStorageMovementQueue().addAll(startPath, itemInfoList, scanCompleted);
   }
 
   @Override
@@ -963,12 +1011,12 @@ public class StoragePolicySatisfier implements SPSService, Runnable {
   }
 
   @VisibleForTesting
-  public BlockStorageMovementNeeded getStorageMovementQueue() {
+  public BlockStorageMovementNeeded<T> getStorageMovementQueue() {
     return storageMovementNeeded;
   }
 
   @Override
-  public void markScanCompletedForPath(Long inodeId) {
+  public void markScanCompletedForPath(T inodeId) {
     getStorageMovementQueue().markScanCompletedForDir(inodeId);
   }
 
@@ -976,7 +1024,6 @@ public class StoragePolicySatisfier implements SPSService, Runnable {
    * Join main SPS thread.
    */
   public void join() throws InterruptedException {
-    //TODO Add join here on SPS rpc server also
     storagePolicySatisfierThread.join();
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8467ec24/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/StoragePolicySatisfyManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/StoragePolicySatisfyManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/StoragePolicySatisfyManager.java
index 5bdf6ae..5ec0372 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/StoragePolicySatisfyManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/StoragePolicySatisfyManager.java
@@ -60,7 +60,7 @@ import org.slf4j.LoggerFactory;
 public class StoragePolicySatisfyManager {
   private static final Logger LOG = LoggerFactory
       .getLogger(StoragePolicySatisfyManager.class);
-  private final StoragePolicySatisfier spsService;
+  private final StoragePolicySatisfier<Long> spsService;
   private final boolean storagePolicyEnabled;
   private volatile StoragePolicySatisfierMode mode;
   private final Queue<Long> pathsToBeTraveresed;
@@ -84,7 +84,7 @@ public class StoragePolicySatisfyManager {
     pathsToBeTraveresed = new LinkedList<Long>();
     // instantiate SPS service by just keeps config reference and not starting
     // any supporting threads.
-    spsService = new StoragePolicySatisfier(conf);
+    spsService = new StoragePolicySatisfier<Long>(conf);
     this.namesystem = namesystem;
     this.blkMgr = blkMgr;
   }
@@ -309,7 +309,7 @@ public class StoragePolicySatisfyManager {
   /**
    * @return internal SPS service instance.
    */
-  public SPSService getInternalSPSService() {
+  public SPSService<Long> getInternalSPSService() {
     return this.spsService;
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8467ec24/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/NamenodeProtocol.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/NamenodeProtocol.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/NamenodeProtocol.java
index 9f5cadd..615e297 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/NamenodeProtocol.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/NamenodeProtocol.java
@@ -206,21 +206,11 @@ public interface NamenodeProtocol {
   boolean isRollingUpgrade() throws IOException;
 
   /**
-   * Gets the file path for the given file id. This API used by External SPS.
-   *
-   * @param inodeId
-   *          - file inode id.
-   * @return path
-   */
-  @Idempotent
-  String getFilePath(Long inodeId) throws IOException;
-
-  /**
-   * @return Gets the next available sps path id, otherwise null. This API used
+   * @return Gets the next available sps path, otherwise null. This API used
    *         by External SPS.
    */
   @AtMostOnce
-  Long getNextSPSPathId() throws IOException;
+  String getNextSPSPath() throws IOException;
 
   /**
    * Verifies whether the given Datanode has the enough estimated size with
@@ -236,15 +226,5 @@ public interface NamenodeProtocol {
   @Idempotent
   boolean checkDNSpaceForScheduling(DatanodeInfo dn, StorageType type,
       long estimatedSize) throws IOException;
-
-  /**
-   * Check if any low redundancy blocks for given file id. This API used by
-   * External SPS.
-   *
-   * @param inodeID
-   *          - inode id.
-   */
-  @Idempotent
-  boolean hasLowRedundancyBlocks(long inodeID) throws IOException;
 }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8467ec24/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/sps/ExternalSPSBlockMoveTaskHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/sps/ExternalSPSBlockMoveTaskHandler.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/sps/ExternalSPSBlockMoveTaskHandler.java
index 4a762649..7580ba9 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/sps/ExternalSPSBlockMoveTaskHandler.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/sps/ExternalSPSBlockMoveTaskHandler.java
@@ -81,11 +81,11 @@ public class ExternalSPSBlockMoveTaskHandler implements BlockMoveTaskHandler {
   private final SaslDataTransferClient saslClient;
   private final BlockStorageMovementTracker blkMovementTracker;
   private Daemon movementTrackerThread;
-  private final SPSService service;
+  private final SPSService<String> service;
   private final BlockDispatcher blkDispatcher;
 
   public ExternalSPSBlockMoveTaskHandler(Configuration conf,
-      NameNodeConnector nnc, SPSService spsService) {
+      NameNodeConnector nnc, SPSService<String> spsService) {
     int moverThreads = conf.getInt(DFSConfigKeys.DFS_MOVER_MOVERTHREADS_KEY,
         DFSConfigKeys.DFS_MOVER_MOVERTHREADS_DEFAULT);
     moveExecutor = initializeBlockMoverThreadPool(moverThreads);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8467ec24/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/sps/ExternalSPSContext.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/sps/ExternalSPSContext.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/sps/ExternalSPSContext.java
index c309209..5d0aee6 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/sps/ExternalSPSContext.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/sps/ExternalSPSContext.java
@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.hdfs.server.sps;
 
+import java.io.FileNotFoundException;
 import java.io.IOException;
 
 import org.apache.hadoop.classification.InterfaceAudience;
@@ -30,6 +31,7 @@ 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.HdfsFileStatus;
+import org.apache.hadoop.hdfs.protocol.HdfsLocatedFileStatus;
 import org.apache.hadoop.hdfs.server.balancer.NameNodeConnector;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
@@ -46,15 +48,15 @@ import org.slf4j.LoggerFactory;
  * SPS from Namenode state.
  */
 @InterfaceAudience.Private
-public class ExternalSPSContext implements Context {
+public class ExternalSPSContext implements Context<String> {
   public static final Logger LOG =
       LoggerFactory.getLogger(ExternalSPSContext.class);
-  private SPSService service;
+  private SPSService<String> service;
   private NameNodeConnector nnc = null;
   private BlockStoragePolicySuite createDefaultSuite =
       BlockStoragePolicySuite.createDefaultSuite();
 
-  public ExternalSPSContext(SPSService service, NameNodeConnector nnc) {
+  public ExternalSPSContext(SPSService<String> service, NameNodeConnector nnc) {
     this.service = service;
     this.nnc = nnc;
   }
@@ -110,14 +112,12 @@ public class ExternalSPSContext implements Context {
   }
 
   @Override
-  public boolean isFileExist(long inodeId) {
-    String filePath = null;
+  public boolean isFileExist(String filePath) {
     try {
-      filePath = getFilePath(inodeId);
       return nnc.getDistributedFileSystem().exists(new Path(filePath));
     } catch (IllegalArgumentException | IOException e) {
-      LOG.warn("Exception while getting file is for the given path:{} "
-          + "and fileId:{}", filePath, inodeId, e);
+      LOG.warn("Exception while getting file is for the given path:{}",
+          filePath, e);
     }
     return false;
   }
@@ -133,8 +133,8 @@ public class ExternalSPSContext implements Context {
   }
 
   @Override
-  public void removeSPSHint(long inodeId) throws IOException {
-    nnc.getDistributedFileSystem().removeXAttr(new Path(getFilePath(inodeId)),
+  public void removeSPSHint(String inodeId) throws IOException {
+    nnc.getDistributedFileSystem().removeXAttr(new Path(inodeId),
         HdfsServerConstants.XATTR_SATISFY_STORAGE_POLICY);
   }
 
@@ -150,9 +150,15 @@ public class ExternalSPSContext implements Context {
   }
 
   @Override
-  public HdfsFileStatus getFileInfo(long inodeID) throws IOException {
-    return nnc.getDistributedFileSystem().getClient()
-        .getLocatedFileInfo(getFilePath(inodeID), false);
+  public HdfsFileStatus getFileInfo(String path) throws IOException {
+    HdfsLocatedFileStatus fileInfo = null;
+    try {
+      fileInfo = nnc.getDistributedFileSystem().getClient()
+          .getLocatedFileInfo(path, false);
+    } catch (FileNotFoundException e) {
+      LOG.debug("Path:{} doesn't exists!", path, e);
+    }
+    return fileInfo;
   }
 
   @Override
@@ -162,17 +168,6 @@ public class ExternalSPSContext implements Context {
   }
 
   @Override
-  public boolean hasLowRedundancyBlocks(long inodeID) {
-    try {
-      return nnc.getNNProtocolConnection().hasLowRedundancyBlocks(inodeID);
-    } catch (IOException e) {
-      LOG.warn("Failed to check whether fileid:{} has low redundancy blocks.",
-          inodeID, e);
-      return false;
-    }
-  }
-
-  @Override
   public boolean checkDNSpaceForScheduling(DatanodeInfo dn, StorageType type,
       long estimatedSize) {
     // TODO: Instead of calling namenode for checking the available space, it
@@ -190,9 +185,9 @@ public class ExternalSPSContext implements Context {
   }
 
   @Override
-  public Long getNextSPSPathId() {
+  public String getNextSPSPath() {
     try {
-      return nnc.getNNProtocolConnection().getNextSPSPathId();
+      return nnc.getNNProtocolConnection().getNextSPSPath();
     } catch (IOException e) {
       LOG.warn("Exception while getting next sps path id from Namenode.", e);
       return null;
@@ -200,7 +195,7 @@ public class ExternalSPSContext implements Context {
   }
 
   @Override
-  public void removeSPSPathId(long pathId) {
+  public void removeSPSPathId(String pathId) {
     // We need not specifically implement for external.
   }
 
@@ -208,15 +203,4 @@ public class ExternalSPSContext implements Context {
   public void removeAllSPSPathIds() {
     // We need not specifically implement for external.
   }
-
-  @Override
-  public String getFilePath(Long inodeId) {
-    try {
-      return nnc.getNNProtocolConnection().getFilePath(inodeId);
-    } catch (IOException e) {
-      LOG.warn("Exception while getting file path id:{} from Namenode.",
-          inodeId, e);
-      return null;
-    }
-  }
 }


---------------------------------------------------------------------
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-12955: [SPS]: Move SPS classes to a separate package. Contributed by Rakesh R.

Posted by um...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/78420719/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
deleted file mode 100644
index 9f733ff..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStoragePolicySatisfier.java
+++ /dev/null
@@ -1,1775 +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 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 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;
-import java.net.InetSocketAddress;
-import java.util.ArrayList;
-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;
-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.MiniDFSCluster.DataNodeProperties;
-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;
-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;
-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;
-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;
-
-/**
- * Tests that StoragePolicySatisfier daemon is able to check the blocks to be
- * 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 =
-      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 dfs = null;
-  private static final int DEFAULT_BLOCK_SIZE = 1024;
-
-  private void shutdownCluster() {
-    if (hdfsCluster != null) {
-      hdfsCluster.shutdown();
-    }
-  }
-
-  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();
-    writeContent(file);
-  }
-
-  @Test(timeout = 300000)
-  public void testWhenStoragePolicySetToCOLD()
-      throws Exception {
-
-    try {
-      createCluster();
-      doTestWhenStoragePolicySetToCOLD();
-    } finally {
-      shutdownCluster();
-    }
-  }
-
-  private void doTestWhenStoragePolicySetToCOLD() throws Exception {
-    // 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);
-
-    dfs.satisfyStoragePolicy(new Path(file));
-
-    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 {
-    try {
-      createCluster();
-      // Change policy to ALL_SSD
-      dfs.setStoragePolicy(new Path(file), "ALL_SSD");
-
-      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);
-      dfs.satisfyStoragePolicy(new Path(file));
-      hdfsCluster.triggerHeartbeats();
-      // Wait till StorgePolicySatisfier Identified that block to move to SSD
-      // areas
-      DFSTestUtil.waitExpectedStorageType(
-          file, StorageType.SSD, 3, 30000, dfs);
-    } finally {
-      shutdownCluster();
-    }
-  }
-
-  @Test(timeout = 300000)
-  public void testWhenStoragePolicySetToONESSD()
-      throws Exception {
-    try {
-      createCluster();
-      // Change policy to ONE_SSD
-      dfs.setStoragePolicy(new Path(file), ONE_SSD);
-
-      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);
-      dfs.satisfyStoragePolicy(new Path(file));
-      hdfsCluster.triggerHeartbeats();
-      // Wait till StorgePolicySatisfier Identified that block to move to SSD
-      // areas
-      DFSTestUtil.waitExpectedStorageType(
-          file, StorageType.SSD, 1, 30000, dfs);
-      DFSTestUtil.waitExpectedStorageType(
-          file, StorageType.DISK, 2, 30000, dfs);
-    } finally {
-      shutdownCluster();
-    }
-  }
-
-  /**
-   * Tests to verify that the block storage movement report will be propagated
-   * to Namenode via datanode heartbeat.
-   */
-  @Test(timeout = 300000)
-  public void testBlksStorageMovementAttemptFinishedReport() throws Exception {
-    try {
-      createCluster();
-      // Change policy to ONE_SSD
-      dfs.setStoragePolicy(new Path(file), ONE_SSD);
-
-      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);
-      dfs.satisfyStoragePolicy(new Path(file));
-      hdfsCluster.triggerHeartbeats();
-
-      // Wait till the block is moved to SSD areas
-      DFSTestUtil.waitExpectedStorageType(
-          file, StorageType.SSD, 1, 30000, dfs);
-      DFSTestUtil.waitExpectedStorageType(
-          file, StorageType.DISK, 2, 30000, dfs);
-
-      waitForBlocksMovementAttemptReport(1, 30000);
-    } finally {
-      shutdownCluster();
-    }
-  }
-
-  /**
-   * 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 {
-    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);
-      }
-      // Change policy to ONE_SSD
-      for (String fileName : files) {
-        dfs.setStoragePolicy(new Path(fileName), ONE_SSD);
-        dfs.satisfyStoragePolicy(new Path(fileName));
-      }
-
-      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);
-      hdfsCluster.triggerHeartbeats();
-
-      for (String fileName : files) {
-        // Wait till the block is moved to SSD areas
-        DFSTestUtil.waitExpectedStorageType(
-            fileName, StorageType.SSD, 1, 30000, dfs);
-        DFSTestUtil.waitExpectedStorageType(
-            fileName, StorageType.DISK, 2, 30000, dfs);
-      }
-
-      waitForBlocksMovementAttemptReport(files.size(), 30000);
-    } finally {
-      shutdownCluster();
-    }
-  }
-
-  /**
-   * Tests to verify hdfsAdmin.satisfyStoragePolicy works well for file.
-   * @throws Exception
-   */
-  @Test(timeout = 300000)
-  public void testSatisfyFileWithHdfsAdmin() throws Exception {
-    try {
-      createCluster();
-      HdfsAdmin hdfsAdmin =
-          new HdfsAdmin(FileSystem.getDefaultUri(config), config);
-      // Change policy to COLD
-      dfs.setStoragePolicy(new Path(file), COLD);
-
-      StorageType[][] newtypes =
-          new StorageType[][]{{StorageType.DISK, StorageType.ARCHIVE},
-              {StorageType.DISK, StorageType.ARCHIVE},
-              {StorageType.DISK, 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
-      DFSTestUtil.waitExpectedStorageType(
-          file, StorageType.ARCHIVE, 3, 30000, dfs);
-    } finally {
-      shutdownCluster();
-    }
-  }
-
-  /**
-   * Tests to verify hdfsAdmin.satisfyStoragePolicy works well for dir.
-   * @throws Exception
-   */
-  @Test(timeout = 300000)
-  public void testSatisfyDirWithHdfsAdmin() throws Exception {
-    try {
-      createCluster();
-      HdfsAdmin hdfsAdmin =
-          new HdfsAdmin(FileSystem.getDefaultUri(config), config);
-      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.
-      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.
-      DFSTestUtil.waitExpectedStorageType(
-          subFile2, StorageType.SSD, 1, 30000, dfs);
-      DFSTestUtil.waitExpectedStorageType(
-          subFile2, StorageType.DISK, 2, 30000, dfs);
-    } finally {
-      shutdownCluster();
-    }
-  }
-
-  /**
-   * Tests to verify hdfsAdmin.satisfyStoragePolicy exceptions.
-   * @throws Exception
-   */
-  @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);
-      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) {
-
-      }
-
-      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();
-    }
-  }
-
-  /**
-   * 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 {
-      createCluster();
-      // Change policy to COLD
-      dfs.setStoragePolicy(new Path(file), COLD);
-
-      StorageType[][] newtypes =
-          new StorageType[][]{{StorageType.ARCHIVE, StorageType.ARCHIVE}};
-
-      // Adding ARCHIVE based datanodes.
-      startAdditionalDNs(config, 1, numOfDatanodes, newtypes,
-          storagesPerDatanode, capacity, hdfsCluster);
-
-      dfs.satisfyStoragePolicy(new Path(file));
-      hdfsCluster.triggerHeartbeats();
-      // Wait till StorgePolicySatisfier identified that block to move to
-      // ARCHIVE area.
-      DFSTestUtil.waitExpectedStorageType(
-          file, StorageType.ARCHIVE, 1, 30000, dfs);
-      DFSTestUtil.waitExpectedStorageType(
-          file, StorageType.DISK, 2, 30000, dfs);
-
-      waitForBlocksMovementAttemptReport(1, 30000);
-    } finally {
-      shutdownCluster();
-    }
-  }
-
-  /**
-   * 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 {
-      createCluster();
-      // Change policy to COLD
-      dfs.setStoragePolicy(new Path(file), COLD);
-
-      StorageType[][] newtypes =
-          new StorageType[][]{{StorageType.DISK, StorageType.DISK}};
-      // Adding DISK based datanodes
-      startAdditionalDNs(config, 1, numOfDatanodes, newtypes,
-          storagesPerDatanode, capacity, hdfsCluster);
-
-      dfs.satisfyStoragePolicy(new Path(file));
-      hdfsCluster.triggerHeartbeats();
-
-      // 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);
-      // Since there is no target node the item will get timed out and then
-      // re-attempted.
-      waitForAttemptedItems(1, 30000);
-    } finally {
-      shutdownCluster();
-    }
-  }
-
-  /**
-   * Tests to verify that SPS should not start when a Mover instance
-   * is running.
-   */
-  @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_ENABLED_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_ENABLED_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_ENABLED_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_ENABLED_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 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.assertTrue("SPS should be running as "
-          + "no Mover really running", running);
-    } finally {
-      if (hdfsCluster != null) {
-        hdfsCluster.shutdown();
-      }
-    }
-  }
-
-  /**
-   * 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 {
-    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();
-
-      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);
-
-      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);
-
-      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);
-      waitForBlocksMovementAttemptReport(1, 30000);
-      DFSTestUtil.waitExpectedStorageType(
-          file1, StorageType.ARCHIVE, 1, 30000, dfs);
-      DFSTestUtil.waitExpectedStorageType(
-          file1, StorageType.DISK, 2, 30000, dfs);
-    } finally {
-      if (hdfsCluster != null) {
-        hdfsCluster.shutdown();
-      }
-    }
-  }
-
-  /**
-   * 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);
-      config.setBoolean(DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_ENABLED_KEY,
-          true);
-      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);
-
-      dfs.satisfyStoragePolicy(new Path(file));
-      hdfsCluster.triggerHeartbeats();
-      // Wait till StorgePolicySatisfier identified that block to move to
-      // ARCHIVE area.
-      DFSTestUtil.waitExpectedStorageType(
-          file, StorageType.ARCHIVE, 2, 30000, dfs);
-      DFSTestUtil.waitExpectedStorageType(
-          file, StorageType.DISK, 3, 30000, dfs);
-
-      waitForBlocksMovementAttemptReport(1, 30000);
-    } finally {
-      shutdownCluster();
-    }
-  }
-
-  /**
-   * 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);
-    config.setBoolean(DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_ENABLED_KEY,
-        true);
-    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);
-
-      dfs.satisfyStoragePolicy(new Path(file));
-      hdfsCluster.triggerHeartbeats();
-      DFSTestUtil.waitExpectedStorageType(
-          file, StorageType.SSD, 1, 30000, dfs);
-      DFSTestUtil.waitExpectedStorageType(
-          file, StorageType.DISK, 2, 30000, dfs);
-
-    } 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);
-    config.setBoolean(DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_ENABLED_KEY,
-        true);
-    try {
-      hdfsCluster = startCluster(config, diskTypes, diskTypes.length,
-          storagesPerDatanode, capacity);
-      dfs = hdfsCluster.getFileSystem();
-      writeContent(file);
-
-      // Change policy to WARM
-      dfs.setStoragePolicy(new Path(file), "WARM");
-      dfs.satisfyStoragePolicy(new Path(file));
-      hdfsCluster.triggerHeartbeats();
-
-      DFSTestUtil.waitExpectedStorageType(
-          file, StorageType.DISK, 1, 30000, dfs);
-      DFSTestUtil.waitExpectedStorageType(
-          file, StorageType.ARCHIVE, 2, 30000, dfs);
-    } finally {
-      shutdownCluster();
-    }
-  }
-
-  /**
-   * 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 {
-      config.setBoolean(DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_ENABLED_KEY,
-          true);
-      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.
-   */
-  @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);
-    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,
-          storagesPerDatanode, dnCapacity);
-      dfs = hdfsCluster.getFileSystem();
-      writeContent(file);
-
-      // Change policy to ONE_SSD
-      dfs.setStoragePolicy(new Path(file), ONE_SSD);
-      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);
-      }
-      dfs.satisfyStoragePolicy(new Path(file));
-
-      // 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(1));
-      }
-
-      // 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();
-    }
-  }
-
-  /**
-   * 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 =
-        StripedFileTestUtil.getDefaultECPolicy().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);
-    config.setBoolean(DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_ENABLED_KEY,
-        true);
-    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,
-          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,
-          StripedFileTestUtil.getDefaultECPolicy().getName());
-
-      // write file to fooDir
-      final String testFile = "/foo/bar";
-      long fileLen = 20 * defaultStripedBlockSize;
-      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();
-    }
-  }
-
-  /**
-   * 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 {
-      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");
-      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();
-      }
-    }
-  }
-
-  /**
-   * 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.setBoolean(DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_ENABLED_KEY,
-          true);
-      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), false);
-      cluster.restartDataNode(list.get(1), false);
-      cluster.waitActive();
-      fs.satisfyStoragePolicy(filePath);
-      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 {
-      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.setBoolean(DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_ENABLED_KEY,
-          true);
-      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();
-      }
-    }
-  }
-
-  /**
-   * 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 = 3000000)
-  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);
-      }
-      waitForBlocksMovementAttemptReport(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(timeout = 300000)
-  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(timeout = 300000)
-  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);
-  }
-
-  /**
-   * 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_LOW_MAX_STREAMS_PREFERENCE_KEY,
-          false);
-
-      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();
-    }
-  }
-
-  @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_LOW_MAX_STREAMS_PREFERENCE_KEY,
-          false);
-
-      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();
-    }
-  }
-
-  @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
-    /*
-     *                           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();
-    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());
-      InternalDataNodeTestUtils.mockDatanodeBlkPinning(dn, true);
-      favoredNodesCount--;
-      if (favoredNodesCount <= 0) {
-        break; // marked favoredNodesCount number of pinned block location
-      }
-    }
-    return file1;
-  }
-
-  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 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("MovementFinishedBlocks: expectedCount={} actualCount={}",
-            expectedMovementFinishedBlocksCount,
-            sps.getAttemptedItemsMonitor().getMovementFinishedBlocksCount());
-        return sps.getAttemptedItemsMonitor().getMovementFinishedBlocksCount()
-            >= expectedMovementFinishedBlocksCount;
-      }
-    }, 100, timeout);
-  }
-
-  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),
-        replicatonFactor);
-    for (int i = 0; i < 1024; i++) {
-      out.write(i);
-    }
-    out.close();
-  }
-
-  private void startAdditionalDNs(final Configuration conf,
-      int newNodesRequired, int existingNodesNum, StorageType[][] newTypes,
-      int storagesPerDn, long nodeCapacity, final MiniDFSCluster cluster)
-          throws IOException {
-    long[][] capacities;
-    existingNodesNum += newNodesRequired;
-    capacities = new long[newNodesRequired][storagesPerDn];
-    for (int i = 0; i < newNodesRequired; i++) {
-      for (int j = 0; j < storagesPerDn; j++) {
-        capacities[i][j] = nodeCapacity;
-      }
-    }
-
-    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;
-  }
-}


---------------------------------------------------------------------
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-13076: [SPS]: Cleanup work for HDFS-10285 merge. Contributed by Rakesh R.

Posted by um...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/39ed3a66/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/sps/TestExternalStoragePolicySatisfier.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/sps/TestExternalStoragePolicySatisfier.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/sps/TestExternalStoragePolicySatisfier.java
index 18acb50..d9a93fd 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/sps/TestExternalStoragePolicySatisfier.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/sps/TestExternalStoragePolicySatisfier.java
@@ -32,34 +32,57 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_SPS_ADDRESS_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_SPS_KERBEROS_PRINCIPAL_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_SPS_KEYTAB_FILE_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_SPS_MAX_OUTSTANDING_PATHS_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_STORAGE_POLICY_ENABLED_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_WEB_AUTHENTICATION_KERBEROS_PRINCIPAL_KEY;
 import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_DATA_TRANSFER_PROTECTION_KEY;
+import static org.apache.hadoop.hdfs.server.common.HdfsServerConstants.XATTR_SATISFY_STORAGE_POLICY;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.fail;
 
 import java.io.File;
+import java.io.FileNotFoundException;
 import java.io.IOException;
-import java.net.URI;
+import java.net.InetSocketAddress;
 import java.security.PrivilegedExceptionAction;
 import java.util.ArrayList;
-import java.util.Collection;
 import java.util.List;
 import java.util.Properties;
 import java.util.concurrent.TimeoutException;
 
+import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileUtil;
 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.DFSUtil;
 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.MiniDFSCluster.DataNodeProperties;
+import org.apache.hadoop.hdfs.client.HdfsAdmin;
+import org.apache.hadoop.hdfs.protocol.Block;
+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.protocol.HdfsConstants.StoragePolicySatisfierMode;
 import org.apache.hadoop.hdfs.server.balancer.NameNodeConnector;
 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.hdfs.server.namenode.FSEditLog;
+import org.apache.hadoop.hdfs.server.namenode.INode;
+import org.apache.hadoop.hdfs.server.namenode.sps.BlockMovementListener;
 import org.apache.hadoop.hdfs.server.namenode.sps.BlockStorageMovementAttemptedItems;
 import org.apache.hadoop.hdfs.server.namenode.sps.StoragePolicySatisfier;
-import org.apache.hadoop.hdfs.server.namenode.sps.TestStoragePolicySatisfier;
 import org.apache.hadoop.http.HttpConfig;
 import org.apache.hadoop.minikdc.MiniKdc;
 import org.apache.hadoop.security.SecurityUtil;
@@ -67,29 +90,57 @@ import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.authentication.util.KerberosName;
 import org.apache.hadoop.security.ssl.KeyStoreTestUtil;
 import org.apache.hadoop.test.GenericTestUtils;
+import org.apache.hadoop.test.GenericTestUtils.LogCapturer;
 import org.junit.After;
 import org.junit.Assert;
+import org.junit.Before;
 import org.junit.Ignore;
 import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import com.google.common.base.Supplier;
 
 /**
  * Tests the external sps service plugins.
  */
-public class TestExternalStoragePolicySatisfier
-    extends TestStoragePolicySatisfier {
+public class TestExternalStoragePolicySatisfier {
+  private static final String ONE_SSD = "ONE_SSD";
+  private static final String COLD = "COLD";
   private StorageType[][] allDiskTypes =
       new StorageType[][]{{StorageType.DISK, StorageType.DISK},
           {StorageType.DISK, StorageType.DISK},
           {StorageType.DISK, StorageType.DISK}};
-  private NameNodeConnector nnc;
   private File keytabFile;
   private String principal;
   private MiniKdc kdc;
   private File baseDir;
+  private NameNodeConnector nnc;
   private StoragePolicySatisfier externalSps;
   private ExternalSPSContext externalCtxt;
+  private DistributedFileSystem dfs = null;
+  private MiniDFSCluster hdfsCluster = null;
+  private Configuration config = null;
+  private static final int NUM_OF_DATANODES = 3;
+  private static final int STORAGES_PER_DATANODE = 2;
+  private static final long CAPACITY = 2 * 256 * 1024 * 1024;
+  private static final String FILE = "/testMoveToSatisfyStoragePolicy";
+  private static final int DEFAULT_BLOCK_SIZE = 1024;
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TestExternalStoragePolicySatisfier.class);
+
+  @Before
+  public void setUp() {
+    config = new HdfsConfiguration();
+    config.set(DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_MODE_KEY,
+        StoragePolicySatisfierMode.EXTERNAL.toString());
+    // Most of the tests are restarting DNs and NN. So, reduced refresh cycle to
+    // update latest datanodes.
+    config.setLong(DFSConfigKeys.DFS_SPS_DATANODE_CACHE_REFRESH_INTERVAL_MS,
+        3000);
+    config.set(DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_MODE_KEY,
+        StoragePolicySatisfierMode.EXTERNAL.toString());
+  }
 
   @After
   public void destroy() throws Exception {
@@ -97,26 +148,49 @@ public class TestExternalStoragePolicySatisfier
       kdc.stop();
       FileUtil.fullyDelete(baseDir);
     }
+    if (hdfsCluster != null) {
+      hdfsCluster.shutdown();
+    }
   }
 
-  @Override
-  public void shutdownCluster() {
-    if (externalSps != null) {
-      externalSps.stopGracefully();
-    }
-    super.shutdownCluster();
+  /**
+   * Sets hdfs cluster.
+   */
+  private void setCluster(MiniDFSCluster cluster) {
+    this.hdfsCluster = cluster;
   }
 
-  @Override
-  public void setUp() {
-    super.setUp();
+  /**
+   * @return conf.
+   */
+  private Configuration getConf() {
+    return this.config;
+  }
 
-    getConf().set(DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_MODE_KEY,
-        StoragePolicySatisfierMode.EXTERNAL.toString());
+  /**
+   * @return hdfs cluster.
+   */
+  private MiniDFSCluster getCluster() {
+    return hdfsCluster;
+  }
+
+  /**
+   * Gets distributed file system.
+   *
+   * @throws IOException
+   */
+  private DistributedFileSystem getFS() throws IOException {
+    this.dfs = hdfsCluster.getFileSystem();
+    return this.dfs;
+  }
+
+  private void shutdownCluster() {
+    if (externalSps != null) {
+      externalSps.stopGracefully();
+    }
   }
 
-  @Override
-  public void createCluster() throws IOException {
+  private void createCluster() throws IOException {
     getConf().setLong("dfs.block.size", DEFAULT_BLOCK_SIZE);
     setCluster(startCluster(getConf(), allDiskTypes, NUM_OF_DATANODES,
         STORAGES_PER_DATANODE, CAPACITY));
@@ -124,8 +198,7 @@ public class TestExternalStoragePolicySatisfier
     writeContent(FILE);
   }
 
-  @Override
-  public MiniDFSCluster startCluster(final Configuration conf,
+  private MiniDFSCluster startCluster(final Configuration conf,
       StorageType[][] storageTypes, int numberOfDatanodes, int storagesPerDn,
       long nodeCapacity) throws IOException {
     long[][] capacities = new long[numberOfDatanodes][storagesPerDn];
@@ -139,7 +212,8 @@ public class TestExternalStoragePolicySatisfier
         .storageTypes(storageTypes).storageCapacities(capacities).build();
     cluster.waitActive();
 
-    nnc = getNameNodeConnector(getConf());
+    nnc = DFSTestUtil.getNameNodeConnector(getConf(),
+        HdfsServerConstants.MOVER_ID_PATH, 1, false);
 
     externalSps = new StoragePolicySatisfier(getConf());
     externalCtxt = new ExternalSPSContext(externalSps, nnc);
@@ -149,7 +223,7 @@ public class TestExternalStoragePolicySatisfier
     return cluster;
   }
 
-  public void restartNamenode() throws IOException{
+  private void restartNamenode() throws IOException{
     if (externalSps != null) {
       externalSps.stopGracefully();
     }
@@ -163,60 +237,6 @@ public class TestExternalStoragePolicySatisfier
     externalSps.start(true, StoragePolicySatisfierMode.EXTERNAL);
   }
 
-  private NameNodeConnector getNameNodeConnector(Configuration conf)
-      throws IOException {
-    final Collection<URI> namenodes = DFSUtil.getInternalNsRpcUris(conf);
-    Assert.assertEquals(1, namenodes.size());
-    final Path externalSPSPathId = HdfsServerConstants.MOVER_ID_PATH;
-    NameNodeConnector.checkOtherInstanceRunning(false);
-    while (true) {
-      try {
-        final List<NameNodeConnector> nncs = NameNodeConnector
-            .newNameNodeConnectors(namenodes,
-                StoragePolicySatisfier.class.getSimpleName(),
-                externalSPSPathId, conf,
-                NameNodeConnector.DEFAULT_MAX_IDLE_ITERATIONS);
-        return nncs.get(0);
-      } catch (IOException e) {
-        LOG.warn("Failed to connect with namenode", e);
-        // Ignore
-      }
-
-    }
-  }
-
-  public void waitForAttemptedItems(long expectedBlkMovAttemptedCount,
-      int timeout) throws TimeoutException, InterruptedException {
-    GenericTestUtils.waitFor(new Supplier<Boolean>() {
-      @Override
-      public Boolean get() {
-        LOG.info("expectedAttemptedItemsCount={} actualAttemptedItemsCount={}",
-            expectedBlkMovAttemptedCount,
-            ((BlockStorageMovementAttemptedItems) (externalSps
-                .getAttemptedItemsMonitor())).getAttemptedItemsCount());
-        return ((BlockStorageMovementAttemptedItems) (externalSps
-            .getAttemptedItemsMonitor()))
-                .getAttemptedItemsCount() == expectedBlkMovAttemptedCount;
-      }
-    }, 100, timeout);
-  }
-
-  public void waitForBlocksMovementAttemptReport(
-      long expectedMovementFinishedBlocksCount, int timeout)
-          throws TimeoutException, InterruptedException {
-    GenericTestUtils.waitFor(new Supplier<Boolean>() {
-      @Override
-      public Boolean get() {
-        int actualCount = externalSps.getAttemptedItemsMonitor()
-            .getAttemptedItemsCount();
-        LOG.info("MovementFinishedBlocks: expectedCount={} actualCount={}",
-            expectedMovementFinishedBlocksCount, actualCount);
-        return actualCount
-            >= expectedMovementFinishedBlocksCount;
-      }
-    }, 100, timeout);
-  }
-
   private void initSecureConf(Configuration conf) throws Exception {
     String username = "externalSPS";
     baseDir = GenericTestUtils
@@ -344,22 +364,6 @@ public class TestExternalStoragePolicySatisfier
   }
 
   /**
-   * Test verifies status check when Satisfier is not running inside namenode.
-   */
-  @Test(timeout = 90000)
-  public void testStoragePolicySatisfyPathStatus() throws Exception {
-    createCluster();
-    DistributedFileSystem fs = getFS();
-    try {
-      fs.getClient().checkStoragePolicySatisfyPathStatus(FILE);
-      Assert.fail("Should throw exception as SPS is not running inside NN!");
-    } catch (IOException e) {
-      GenericTestUtils.assertExceptionContains("Satisfier is not running"
-          + " inside namenode, so status can't be returned.", e);
-    }
-  }
-
-  /**
    * 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.
@@ -399,17 +403,9 @@ public class TestExternalStoragePolicySatisfier
   }
 
   /**
-   * Status won't be supported for external SPS, now. So, ignoring it.
-   */
-  @Ignore("Status is not supported for external SPS. So, ignoring it.")
-  public void testMaxRetryForFailedBlock() throws Exception {
-  }
-
-  /**
    * This test is specific to internal SPS. So, ignoring it.
    */
   @Ignore("This test is specific to internal SPS. So, ignoring it.")
-  @Override
   public void testTraverseWhenParentDeleted() throws Exception {
   }
 
@@ -417,7 +413,1238 @@ public class TestExternalStoragePolicySatisfier
    * This test is specific to internal SPS. So, ignoring it.
    */
   @Ignore("This test is specific to internal SPS. So, ignoring it.")
-  @Override
   public void testTraverseWhenRootParentDeleted() throws Exception {
   }
+
+
+  @Test(timeout = 300000)
+  public void testWhenStoragePolicySetToCOLD()
+      throws Exception {
+
+    try {
+      createCluster();
+      doTestWhenStoragePolicySetToCOLD();
+    } finally {
+      shutdownCluster();
+    }
+  }
+
+  private void doTestWhenStoragePolicySetToCOLD() throws Exception {
+    // 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, NUM_OF_DATANODES, newtypes,
+        STORAGES_PER_DATANODE, CAPACITY, hdfsCluster);
+
+    hdfsCluster.triggerHeartbeats();
+    dfs.satisfyStoragePolicy(new Path(FILE));
+    // Wait till namenode notified about the block location details
+    DFSTestUtil.waitExpectedStorageType(FILE, StorageType.ARCHIVE, 3, 35000,
+        dfs);
+  }
+
+  @Test(timeout = 300000)
+  public void testWhenStoragePolicySetToALLSSD()
+      throws Exception {
+    try {
+      createCluster();
+      // Change policy to ALL_SSD
+      dfs.setStoragePolicy(new Path(FILE), "ALL_SSD");
+
+      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, NUM_OF_DATANODES, newtypes,
+          STORAGES_PER_DATANODE, CAPACITY, hdfsCluster);
+      dfs.satisfyStoragePolicy(new Path(FILE));
+      hdfsCluster.triggerHeartbeats();
+      // Wait till StorgePolicySatisfier Identified that block to move to SSD
+      // areas
+      DFSTestUtil.waitExpectedStorageType(FILE, StorageType.SSD, 3, 30000, dfs);
+    } finally {
+      shutdownCluster();
+    }
+  }
+
+  @Test(timeout = 300000)
+  public void testWhenStoragePolicySetToONESSD()
+      throws Exception {
+    try {
+      createCluster();
+      // Change policy to ONE_SSD
+      dfs.setStoragePolicy(new Path(FILE), ONE_SSD);
+
+      StorageType[][] newtypes =
+          new StorageType[][]{{StorageType.SSD, StorageType.DISK}};
+
+      // Making sure SDD based nodes added to cluster. Adding SSD based
+      // datanodes.
+      startAdditionalDNs(config, 1, NUM_OF_DATANODES, newtypes,
+          STORAGES_PER_DATANODE, CAPACITY, hdfsCluster);
+      dfs.satisfyStoragePolicy(new Path(FILE));
+      hdfsCluster.triggerHeartbeats();
+      // Wait till StorgePolicySatisfier Identified that block to move to SSD
+      // areas
+      DFSTestUtil.waitExpectedStorageType(FILE, StorageType.SSD, 1, 30000, dfs);
+      DFSTestUtil.waitExpectedStorageType(FILE, StorageType.DISK, 2, 30000,
+          dfs);
+    } finally {
+      shutdownCluster();
+    }
+  }
+
+  /**
+   * Tests to verify that the block storage movement report will be propagated
+   * to Namenode via datanode heartbeat.
+   */
+  @Test(timeout = 300000)
+  public void testBlksStorageMovementAttemptFinishedReport() throws Exception {
+    try {
+      createCluster();
+      // Change policy to ONE_SSD
+      dfs.setStoragePolicy(new Path(FILE), ONE_SSD);
+
+      StorageType[][] newtypes =
+          new StorageType[][]{{StorageType.SSD, StorageType.DISK}};
+
+      // Making sure SDD based nodes added to cluster. Adding SSD based
+      // datanodes.
+      startAdditionalDNs(config, 1, NUM_OF_DATANODES, newtypes,
+          STORAGES_PER_DATANODE, CAPACITY, hdfsCluster);
+      dfs.satisfyStoragePolicy(new Path(FILE));
+      hdfsCluster.triggerHeartbeats();
+
+      // Wait till the block is moved to SSD areas
+      DFSTestUtil.waitExpectedStorageType(FILE, StorageType.SSD, 1, 30000, dfs);
+      DFSTestUtil.waitExpectedStorageType(FILE, StorageType.DISK, 2, 30000,
+          dfs);
+
+      waitForBlocksMovementAttemptReport(1, 30000);
+    } finally {
+      shutdownCluster();
+    }
+  }
+
+  /**
+   * 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 {
+    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);
+      }
+      // Change policy to ONE_SSD
+      for (String fileName : files) {
+        dfs.setStoragePolicy(new Path(fileName), ONE_SSD);
+        dfs.satisfyStoragePolicy(new Path(fileName));
+      }
+
+      StorageType[][] newtypes =
+          new StorageType[][]{{StorageType.SSD, StorageType.DISK}};
+
+      // Making sure SDD based nodes added to cluster. Adding SSD based
+      // datanodes.
+      startAdditionalDNs(config, 1, NUM_OF_DATANODES, newtypes,
+          STORAGES_PER_DATANODE, CAPACITY, hdfsCluster);
+      hdfsCluster.triggerHeartbeats();
+
+      for (String fileName : files) {
+        // Wait till the block is moved to SSD areas
+        DFSTestUtil.waitExpectedStorageType(
+            fileName, StorageType.SSD, 1, 30000, dfs);
+        DFSTestUtil.waitExpectedStorageType(
+            fileName, StorageType.DISK, 2, 30000, dfs);
+      }
+
+      waitForBlocksMovementAttemptReport(files.size(), 30000);
+    } finally {
+      shutdownCluster();
+    }
+  }
+
+  /**
+   * Tests to verify hdfsAdmin.satisfyStoragePolicy works well for file.
+   * @throws Exception
+   */
+  @Test(timeout = 300000)
+  public void testSatisfyFileWithHdfsAdmin() throws Exception {
+    try {
+      createCluster();
+      HdfsAdmin hdfsAdmin =
+          new HdfsAdmin(FileSystem.getDefaultUri(config), config);
+      // Change policy to COLD
+      dfs.setStoragePolicy(new Path(FILE), COLD);
+
+      StorageType[][] newtypes =
+          new StorageType[][]{{StorageType.DISK, StorageType.ARCHIVE},
+              {StorageType.DISK, StorageType.ARCHIVE},
+              {StorageType.DISK, StorageType.ARCHIVE}};
+      startAdditionalDNs(config, 3, NUM_OF_DATANODES, newtypes,
+          STORAGES_PER_DATANODE, CAPACITY, hdfsCluster);
+
+      hdfsAdmin.satisfyStoragePolicy(new Path(FILE));
+
+      hdfsCluster.triggerHeartbeats();
+      // Wait till namenode notified about the block location details
+      DFSTestUtil.waitExpectedStorageType(FILE, StorageType.ARCHIVE, 3, 30000,
+          dfs);
+    } finally {
+      shutdownCluster();
+    }
+  }
+
+  /**
+   * Tests to verify hdfsAdmin.satisfyStoragePolicy works well for dir.
+   * @throws Exception
+   */
+  @Test(timeout = 300000)
+  public void testSatisfyDirWithHdfsAdmin() throws Exception {
+    try {
+      createCluster();
+      HdfsAdmin hdfsAdmin =
+          new HdfsAdmin(FileSystem.getDefaultUri(config), config);
+      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, NUM_OF_DATANODES, newtypes,
+          STORAGES_PER_DATANODE, CAPACITY, hdfsCluster);
+
+      hdfsAdmin.satisfyStoragePolicy(new Path(subDir));
+
+      hdfsCluster.triggerHeartbeats();
+
+      // take effect for the file in the directory.
+      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.
+      DFSTestUtil.waitExpectedStorageType(
+          subFile2, StorageType.SSD, 1, 30000, dfs);
+      DFSTestUtil.waitExpectedStorageType(
+          subFile2, StorageType.DISK, 2, 30000, dfs);
+    } finally {
+      shutdownCluster();
+    }
+  }
+
+  /**
+   * Tests to verify hdfsAdmin.satisfyStoragePolicy exceptions.
+   * @throws Exception
+   */
+  @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);
+      restartNamenode();
+      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) {
+        GenericTestUtils.assertExceptionContains(String.format(
+            "Failed to satisfy storage policy since %s is set to false.",
+            DFS_STORAGE_POLICY_ENABLED_KEY), e);
+      }
+
+      hdfsCluster.getConfiguration(0).
+          setBoolean(DFSConfigKeys.DFS_STORAGE_POLICY_ENABLED_KEY, true);
+      restartNamenode();
+
+      hdfsAdmin = new HdfsAdmin(FileSystem.getDefaultUri(config), config);
+      try {
+        hdfsAdmin.satisfyStoragePolicy(new Path(nonExistingFile));
+        Assert.fail("Should throw FileNotFoundException for " +
+            nonExistingFile);
+      } catch (FileNotFoundException e) {
+
+      }
+
+      try {
+        hdfsAdmin.satisfyStoragePolicy(new Path(FILE));
+        hdfsAdmin.satisfyStoragePolicy(new Path(FILE));
+      } catch (Exception e) {
+        Assert.fail(String.format("Allow to invoke mutlipe times "
+            + "#satisfyStoragePolicy() api for a path %s , internally just "
+            + "skipping addtion to satisfy movement queue.", FILE));
+      }
+    } finally {
+      shutdownCluster();
+    }
+  }
+
+  /**
+   * 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 {
+      createCluster();
+      // Change policy to COLD
+      dfs.setStoragePolicy(new Path(FILE), COLD);
+
+      StorageType[][] newtypes =
+          new StorageType[][]{{StorageType.ARCHIVE, StorageType.ARCHIVE}};
+
+      // Adding ARCHIVE based datanodes.
+      startAdditionalDNs(config, 1, NUM_OF_DATANODES, newtypes,
+          STORAGES_PER_DATANODE, CAPACITY, hdfsCluster);
+
+      dfs.satisfyStoragePolicy(new Path(FILE));
+      hdfsCluster.triggerHeartbeats();
+      // Wait till StorgePolicySatisfier identified that block to move to
+      // ARCHIVE area.
+      DFSTestUtil.waitExpectedStorageType(FILE, StorageType.ARCHIVE, 1, 30000,
+          dfs);
+      DFSTestUtil.waitExpectedStorageType(FILE, StorageType.DISK, 2, 30000,
+          dfs);
+
+      waitForBlocksMovementAttemptReport(1, 30000);
+    } finally {
+      shutdownCluster();
+    }
+  }
+
+  /**
+   * 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 {
+      createCluster();
+      // Change policy to COLD
+      dfs.setStoragePolicy(new Path(FILE), COLD);
+
+      StorageType[][] newtypes =
+          new StorageType[][]{{StorageType.DISK, StorageType.DISK}};
+      // Adding DISK based datanodes
+      startAdditionalDNs(config, 1, NUM_OF_DATANODES, newtypes,
+          STORAGES_PER_DATANODE, CAPACITY, hdfsCluster);
+
+      dfs.satisfyStoragePolicy(new Path(FILE));
+      hdfsCluster.triggerHeartbeats();
+
+      // 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);
+      // Since there is no target node the item will get timed out and then
+      // re-attempted.
+      waitForAttemptedItems(1, 30000);
+    } finally {
+      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 {
+    try{
+      config.setBoolean(DFSConfigKeys.DFS_DATANODE_BLOCK_PINNING_ENABLED, true);
+      hdfsCluster = startCluster(config, allDiskTypes, 3, 2, CAPACITY);
+
+      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);
+
+      StorageType[][] newtypes =
+          new StorageType[][]{{StorageType.ARCHIVE, StorageType.ARCHIVE},
+              {StorageType.ARCHIVE, StorageType.ARCHIVE},
+              {StorageType.ARCHIVE, StorageType.ARCHIVE}};
+      // Adding DISK based datanodes
+      startAdditionalDNs(config, 3, NUM_OF_DATANODES, newtypes,
+          STORAGES_PER_DATANODE, CAPACITY, hdfsCluster);
+
+      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);
+      waitForBlocksMovementAttemptReport(1, 30000);
+      DFSTestUtil.waitExpectedStorageType(
+          file1, StorageType.ARCHIVE, 1, 30000, dfs);
+      DFSTestUtil.waitExpectedStorageType(
+          file1, StorageType.DISK, 2, 30000, dfs);
+    } finally {
+      shutdownCluster();
+    }
+  }
+
+  /**
+   * 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,
+          STORAGES_PER_DATANODE, CAPACITY);
+      dfs = hdfsCluster.getFileSystem();
+      writeContent(FILE, (short) 5);
+
+      // Change policy to COLD
+      dfs.setStoragePolicy(new Path(FILE), COLD);
+
+      dfs.satisfyStoragePolicy(new Path(FILE));
+      hdfsCluster.triggerHeartbeats();
+      // Wait till StorgePolicySatisfier identified that block to move to
+      // ARCHIVE area.
+      DFSTestUtil.waitExpectedStorageType(FILE, StorageType.ARCHIVE, 2, 30000,
+          dfs);
+      DFSTestUtil.waitExpectedStorageType(FILE, StorageType.DISK, 3, 30000,
+          dfs);
+
+      waitForBlocksMovementAttemptReport(1, 30000);
+    } finally {
+      shutdownCluster();
+    }
+  }
+
+  /**
+   * 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, NUM_OF_DATANODES,
+          STORAGES_PER_DATANODE, CAPACITY);
+      dfs = hdfsCluster.getFileSystem();
+      writeContent(FILE);
+
+      // Change policy to ONE_SSD
+      dfs.setStoragePolicy(new Path(FILE), ONE_SSD);
+
+      dfs.satisfyStoragePolicy(new Path(FILE));
+      hdfsCluster.triggerHeartbeats();
+      DFSTestUtil.waitExpectedStorageType(FILE, StorageType.SSD, 1, 30000, dfs);
+      DFSTestUtil.waitExpectedStorageType(FILE, StorageType.DISK, 2, 30000,
+          dfs);
+
+    } 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,
+          STORAGES_PER_DATANODE, CAPACITY);
+      dfs = hdfsCluster.getFileSystem();
+      writeContent(FILE);
+
+      // Change policy to WARM
+      dfs.setStoragePolicy(new Path(FILE), "WARM");
+      dfs.satisfyStoragePolicy(new Path(FILE));
+      hdfsCluster.triggerHeartbeats();
+
+      DFSTestUtil.waitExpectedStorageType(FILE, StorageType.DISK, 1, 30000,
+          dfs);
+      DFSTestUtil.waitExpectedStorageType(FILE, StorageType.ARCHIVE, 2, 30000,
+          dfs);
+    } finally {
+      shutdownCluster();
+    }
+  }
+
+  /**
+   * 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,
+          STORAGES_PER_DATANODE, 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.
+   */
+  @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, NUM_OF_DATANODES,
+          STORAGES_PER_DATANODE, dnCapacity);
+      dfs = hdfsCluster.getFileSystem();
+      writeContent(FILE);
+
+      // Change policy to ONE_SSD
+      dfs.setStoragePolicy(new Path(FILE), ONE_SSD);
+      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);
+      }
+      dfs.satisfyStoragePolicy(new Path(FILE));
+
+      // Wait for items to be processed
+      waitForAttemptedItems(1, 30000);
+
+      // 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();
+    }
+  }
+
+  /**
+   * 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 =
+        StripedFileTestUtil.getDefaultECPolicy().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,
+          STORAGES_PER_DATANODE, CAPACITY);
+      dfs = hdfsCluster.getFileSystem();
+      dfs.enableErasureCodingPolicy(
+          StripedFileTestUtil.getDefaultECPolicy().getName());
+
+      // 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,
+          StripedFileTestUtil.getDefaultECPolicy().getName());
+
+      // write file to fooDir
+      final String testFile = "/foo/bar";
+      long fileLen = 20 * defaultStripedBlockSize;
+      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();
+    }
+  }
+
+  /**
+   * 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 {
+    try {
+      hdfsCluster = startCluster(config, allDiskTypes, NUM_OF_DATANODES,
+          STORAGES_PER_DATANODE, CAPACITY);
+      hdfsCluster.waitActive();
+      DistributedFileSystem fs = hdfsCluster.getFileSystem();
+      Path filePath = new Path("/zeroSizeFile");
+      DFSTestUtil.createFile(fs, filePath, 0, (short) 1, 0);
+      FSEditLog editlog = hdfsCluster.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 = hdfsCluster.getNameNode().getNamesystem().getFSDirectory()
+          .getINode(filePath.toString());
+      Assert.assertTrue("XAttrFeature should be null for file",
+          inode.getXAttrFeature() == null);
+    } finally {
+      shutdownCluster();
+    }
+  }
+
+  /**
+   * 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 {
+    try {
+      config.set(DFSConfigKeys
+          .DFS_STORAGE_POLICY_SATISFIER_RECHECK_TIMEOUT_MILLIS_KEY,
+          "3000");
+      config.set(DFSConfigKeys
+          .DFS_STORAGE_POLICY_SATISFIER_SELF_RETRY_TIMEOUT_MILLIS_KEY,
+          "5000");
+      StorageType[][] newtypes = new StorageType[][] {
+          {StorageType.ARCHIVE, StorageType.DISK},
+          {StorageType.ARCHIVE, StorageType.DISK},
+          {StorageType.ARCHIVE, StorageType.DISK}};
+      hdfsCluster = startCluster(config, newtypes, 3, 2, CAPACITY);
+      hdfsCluster.waitActive();
+      DistributedFileSystem fs = hdfsCluster.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(hdfsCluster.stopDataNode(0));
+      list.add(hdfsCluster.stopDataNode(0));
+      list.add(hdfsCluster.stopDataNode(0));
+      restartNamenode();
+      hdfsCluster.restartDataNode(list.get(0), false);
+      hdfsCluster.restartDataNode(list.get(1), false);
+      hdfsCluster.waitActive();
+      fs.satisfyStoragePolicy(filePath);
+      DFSTestUtil.waitExpectedStorageType(filePath.toString(),
+          StorageType.ARCHIVE, 2, 30000, hdfsCluster.getFileSystem());
+      hdfsCluster.restartDataNode(list.get(2), false);
+      DFSTestUtil.waitExpectedStorageType(filePath.toString(),
+          StorageType.ARCHIVE, 3, 30000, hdfsCluster.getFileSystem());
+    } finally {
+      shutdownCluster();
+    }
+  }
+
+  /**
+   * 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 {
+    try {
+      config.set(DFSConfigKeys
+          .DFS_STORAGE_POLICY_SATISFIER_RECHECK_TIMEOUT_MILLIS_KEY,
+          "3000");
+      config.set(DFSConfigKeys
+          .DFS_STORAGE_POLICY_SATISFIER_SELF_RETRY_TIMEOUT_MILLIS_KEY,
+          "5000");
+      StorageType[][] newtypes = new StorageType[][] {
+          {StorageType.ARCHIVE, StorageType.DISK},
+          {StorageType.ARCHIVE, StorageType.DISK},
+          {StorageType.ARCHIVE, StorageType.DISK},
+          {StorageType.ARCHIVE, StorageType.DISK},
+          {StorageType.ARCHIVE, StorageType.DISK}};
+      hdfsCluster = startCluster(config, newtypes, 5, 2, CAPACITY);
+      hdfsCluster.waitActive();
+      DistributedFileSystem fs = hdfsCluster.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, 60000, hdfsCluster.getFileSystem());
+      assertFalse("Log output does not contain expected log message: ",
+          logs.getOutput().contains("some of the blocks are low redundant"));
+    } finally {
+      shutdownCluster();
+    }
+  }
+
+  /**
+   * Test SPS for empty directory, xAttr should be removed.
+   */
+  @Test(timeout = 300000)
+  public void testSPSForEmptyDirectory() throws IOException, TimeoutException,
+      InterruptedException {
+    try {
+      hdfsCluster = startCluster(config, allDiskTypes, NUM_OF_DATANODES,
+          STORAGES_PER_DATANODE, CAPACITY);
+      hdfsCluster.waitActive();
+      DistributedFileSystem fs = hdfsCluster.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, hdfsCluster.getNamesystem(), 30000);
+    } finally {
+      shutdownCluster();
+    }
+  }
+
+  /**
+   * Test SPS for not exist directory.
+   */
+  @Test(timeout = 300000)
+  public void testSPSForNonExistDirectory() throws Exception {
+    try {
+      hdfsCluster = startCluster(config, allDiskTypes, NUM_OF_DATANODES,
+          STORAGES_PER_DATANODE, CAPACITY);
+      hdfsCluster.waitActive();
+      DistributedFileSystem fs = hdfsCluster.getFileSystem();
+      Path emptyDir = new Path("/emptyDir");
+      try {
+        fs.satisfyStoragePolicy(emptyDir);
+        fail("FileNotFoundException should throw");
+      } catch (FileNotFoundException e) {
+        // nothing to do
+      }
+    } finally {
+      shutdownCluster();
+    }
+  }
+
+  /**
+   * Test SPS for directory tree which doesn't have files.
+   */
+  @Test(timeout = 300000)
+  public void testSPSWithDirectoryTreeWithoutFile() throws Exception {
+    try {
+      hdfsCluster = startCluster(config, allDiskTypes, NUM_OF_DATANODES,
+          STORAGES_PER_DATANODE, CAPACITY);
+      hdfsCluster.waitActive();
+      // Create directories
+      /*
+       *                   root
+       *                    |
+       *           A--------C--------D
+       *                    |
+       *               G----H----I
+       *                    |
+       *                    O
+       */
+      DistributedFileSystem fs = hdfsCluster.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, hdfsCluster.getNamesystem(), 30000);
+    } finally {
+      shutdownCluster();
+    }
+  }
+
+  /**
+   * 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);
+      hdfsCluster = startCluster(config, diskTypes, diskTypes.length,
+          STORAGES_PER_DATANODE, 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 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.set(DFSConfigKeys
+          .DFS_STORAGE_POLICY_SATISFIER_RECHECK_TIMEOUT_MILLIS_KEY,
+          "3000");
+      config.set(DFSConfigKeys
+          .DFS_STORAGE_POLICY_SATISFIER_SELF_RETRY_TIMEOUT_MILLIS_KEY,
+          "5000");
+
+      StorageType[][] storagetypes = new StorageType[][] {
+          {StorageType.ARCHIVE, StorageType.DISK},
+          {StorageType.ARCHIVE, StorageType.DISK}};
+
+      hdfsCluster = startCluster(config, storagetypes, 2, 2, CAPACITY);
+      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, NUM_OF_DATANODES, newtypes,
+          STORAGES_PER_DATANODE, 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, 60000, 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
+    /*
+     *                           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();
+    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());
+      InternalDataNodeTestUtils.mockDatanodeBlkPinning(dn, true);
+      favoredNodesCount--;
+      if (favoredNodesCount <= 0) {
+        break; // marked favoredNodesCount number of pinned block location
+      }
+    }
+    return file1;
+  }
+
+  public void waitForAttemptedItems(long expectedBlkMovAttemptedCount,
+      int timeout) throws TimeoutException, InterruptedException {
+    GenericTestUtils.waitFor(new Supplier<Boolean>() {
+      @Override
+      public Boolean get() {
+        LOG.info("expectedAttemptedItemsCount={} actualAttemptedItemsCount={}",
+            expectedBlkMovAttemptedCount,
+            ((BlockStorageMovementAttemptedItems) (externalSps
+                .getAttemptedItemsMonitor())).getAttemptedItemsCount());
+        return ((BlockStorageMovementAttemptedItems) (externalSps
+            .getAttemptedItemsMonitor()))
+                .getAttemptedItemsCount() == expectedBlkMovAttemptedCount;
+      }
+    }, 100, timeout);
+  }
+
+  public void waitForBlocksMovementAttemptReport(
+      long expectedMovementFinishedBlocksCount, int timeout)
+          throws TimeoutException, InterruptedException {
+    GenericTestUtils.waitFor(new Supplier<Boolean>() {
+      @Override
+      public Boolean get() {
+        int actualCount = externalSps.getAttemptedItemsMonitor()
+            .getAttemptedItemsCount();
+        LOG.info("MovementFinishedBlocks: expectedCount={} actualCount={}",
+            expectedMovementFinishedBlocksCount, actualCount);
+        return actualCount
+            >= expectedMovementFinishedBlocksCount;
+      }
+    }, 100, timeout);
+  }
+
+  public 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),
+        replicatonFactor);
+    for (int i = 0; i < 1024; i++) {
+      out.write(i);
+    }
+    out.close();
+  }
+
+  private void startAdditionalDNs(final Configuration conf,
+      int newNodesRequired, int existingNodesNum, StorageType[][] newTypes,
+      int storagesPerDn, long nodeCapacity, final MiniDFSCluster cluster)
+          throws IOException {
+    long[][] capacities;
+    existingNodesNum += newNodesRequired;
+    capacities = new long[newNodesRequired][storagesPerDn];
+    for (int i = 0; i < newNodesRequired; i++) {
+      for (int j = 0; j < storagesPerDn; j++) {
+        capacities[i][j] = nodeCapacity;
+      }
+    }
+
+    cluster.startDataNodes(conf, newNodesRequired, newTypes, true, null, null,
+        null, capacities, null, false, false, false, null);
+    cluster.triggerHeartbeats();
+  }
+
+  /**
+   * Implementation of listener callback, where it collects all the sps move
+   * attempted blocks for assertion.
+   */
+  public static final class ExternalBlockMovementListener
+      implements BlockMovementListener {
+
+    private List<Block> actualBlockMovements = new ArrayList<>();
+
+    @Override
+    public void notifyMovementTriedBlocks(Block[] moveAttemptFinishedBlks) {
+      for (Block block : moveAttemptFinishedBlks) {
+        actualBlockMovements.add(block);
+      }
+      LOG.info("Movement attempted blocks:{}", actualBlockMovements);
+    }
+
+    public List<Block> getActualBlockMovements() {
+      return actualBlockMovements;
+    }
+
+    public void clear() {
+      actualBlockMovements.clear();
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/39ed3a66/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 28838a6..ad77684 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
@@ -50,7 +50,7 @@ public class TestStoragePolicyCommands {
   public void clusterSetUp() throws IOException, URISyntaxException {
     conf = new HdfsConfiguration();
     conf.set(DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_MODE_KEY,
-        StoragePolicySatisfierMode.INTERNAL.toString());
+        StoragePolicySatisfierMode.EXTERNAL.toString());
     StorageType[][] newtypes = new StorageType[][] {
         {StorageType.ARCHIVE, StorageType.DISK}};
     cluster = new MiniDFSCluster.Builder(conf).numDataNodes(REPL)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/39ed3a66/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
index 8a62e0e..1ab7788 100644
--- 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
@@ -29,6 +29,11 @@ import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.StoragePolicySatisfierMode;
+import org.apache.hadoop.hdfs.server.balancer.NameNodeConnector;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
+import org.apache.hadoop.hdfs.server.namenode.sps.Context;
+import org.apache.hadoop.hdfs.server.namenode.sps.StoragePolicySatisfier;
+import org.apache.hadoop.hdfs.server.sps.ExternalSPSContext;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
@@ -43,12 +48,13 @@ public class TestStoragePolicySatisfyAdminCommands {
   private Configuration conf = null;
   private MiniDFSCluster cluster = null;
   private DistributedFileSystem dfs = null;
+  private StoragePolicySatisfier externalSps = null;
 
   @Before
   public void clusterSetUp() throws IOException, URISyntaxException {
     conf = new HdfsConfiguration();
     conf.set(DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_MODE_KEY,
-        StoragePolicySatisfierMode.INTERNAL.toString());
+        StoragePolicySatisfierMode.EXTERNAL.toString());
     // Reduced refresh cycle to update latest datanodes.
     conf.setLong(DFSConfigKeys.DFS_SPS_DATANODE_CACHE_REFRESH_INTERVAL_MS,
         1000);
@@ -58,6 +64,14 @@ public class TestStoragePolicySatisfyAdminCommands {
         .storageTypes(newtypes).build();
     cluster.waitActive();
     dfs = cluster.getFileSystem();
+    NameNodeConnector nnc = DFSTestUtil.getNameNodeConnector(conf,
+        HdfsServerConstants.MOVER_ID_PATH, 1, false);
+
+    StoragePolicySatisfier externalSps = new StoragePolicySatisfier(conf);
+    Context externalCtxt = new ExternalSPSContext(externalSps, nnc);
+
+    externalSps.init(externalCtxt);
+    externalSps.start(true, StoragePolicySatisfierMode.EXTERNAL);
   }
 
   @After
@@ -70,6 +84,9 @@ public class TestStoragePolicySatisfyAdminCommands {
       cluster.shutdown();
       cluster = null;
     }
+    if (externalSps != null) {
+      externalSps.stopGracefully();
+    }
   }
 
   @Test(timeout = 30000)
@@ -92,41 +109,4 @@ public class TestStoragePolicySatisfyAdminCommands {
     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, "-isInternalSatisfierRunning", 0, "yes");
-
-    cluster.getNameNode().reconfigureProperty(
-        DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_MODE_KEY,
-        StoragePolicySatisfierMode.NONE.toString());
-    cluster.waitActive();
-
-    DFSTestUtil.toolRun(admin, "-isInternalSatisfierRunning", 0, "no");
-
-    // Test with unnecessary args
-    DFSTestUtil.toolRun(admin, "-isInternalSatisfierRunning 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


[14/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 um...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/00eceed2/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 d3c5cb1..2f621e6 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
@@ -156,7 +156,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;
@@ -1517,14 +1517,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/00eceed2/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/00eceed2/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/00eceed2/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/00eceed2/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/00eceed2/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/00eceed2/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/00eceed2/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 41a74a7..2fbdcc9 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
@@ -4534,24 +4534,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/00eceed2/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 c8a9466..5defbd0 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.
 ### <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.
 
@@ -129,10 +129,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/00eceed2/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 9530e20..f247370 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/00eceed2/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/00eceed2/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 3d006e0..0fa1696 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
@@ -49,7 +49,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;
@@ -161,7 +161,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/00eceed2/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 a05fdfd..052eb87 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
@@ -93,7 +93,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;
@@ -234,7 +234,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/00eceed2/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/00eceed2/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/00eceed2/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/00eceed2/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/00eceed2/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/00eceed2/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/00eceed2/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 4584add..899bb82 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;
@@ -132,7 +132,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/00eceed2/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/00eceed2/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


[39/50] [abbrv] hadoop git commit: HDFS-13165: [SPS]: Collects successfully moved block details via IBR. Contributed by Rakesh R.

Posted by um...@apache.org.
HDFS-13165: [SPS]: Collects successfully moved block details via IBR. 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/2acc50b8
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/2acc50b8
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/2acc50b8

Branch: refs/heads/HDFS-10285
Commit: 2acc50b826fa8b00f2b09d9546c4b3215b89d46d
Parents: 75ccc13
Author: Rakesh Radhakrishnan <ra...@apache.org>
Authored: Sun Apr 29 11:06:59 2018 +0530
Committer: Uma Maheswara Rao Gangumalla <um...@apache.org>
Committed: Sun Aug 12 03:06:05 2018 -0700

----------------------------------------------------------------------
 .../DatanodeProtocolClientSideTranslatorPB.java |  11 +-
 .../DatanodeProtocolServerSideTranslatorPB.java |   4 +-
 .../apache/hadoop/hdfs/protocolPB/PBHelper.java |  25 ---
 .../server/blockmanagement/BlockManager.java    |  86 +++++++++-
 .../sps/BlockMovementAttemptFinished.java       |  24 ++-
 .../common/sps/BlockStorageMovementTracker.java | 109 +-----------
 .../sps/BlocksMovementsStatusHandler.java       |  70 +-------
 .../hdfs/server/datanode/BPServiceActor.java    |  14 +-
 .../hadoop/hdfs/server/datanode/DataNode.java   |   7 +-
 .../datanode/StoragePolicySatisfyWorker.java    |  48 ++----
 .../namenode/FSDirSatisfyStoragePolicyOp.java   |  13 +-
 .../hdfs/server/namenode/FSDirXAttrOp.java      |   8 +-
 .../hdfs/server/namenode/FSDirectory.java       |   5 +-
 .../hdfs/server/namenode/FSNamesystem.java      |  30 ++--
 .../hadoop/hdfs/server/namenode/NameNode.java   |  19 ++-
 .../hdfs/server/namenode/NameNodeRpcServer.java |  46 +++--
 .../sps/BlockStorageMovementAttemptedItems.java | 167 +++++++++++++------
 .../hdfs/server/namenode/sps/SPSService.java    |  19 ++-
 .../namenode/sps/StoragePolicySatisfier.java    | 154 +++++++++++------
 .../hdfs/server/protocol/DatanodeProtocol.java  |   4 +-
 .../sps/ExternalSPSBlockMoveTaskHandler.java    |  32 ++--
 .../sps/ExternalStoragePolicySatisfier.java     |   3 +-
 .../src/main/proto/DatanodeProtocol.proto       |   9 -
 .../src/main/resources/hdfs-default.xml         |  41 +++++
 .../TestNameNodePrunesMissingStorages.java      |   4 +-
 .../datanode/InternalDataNodeTestUtils.java     |   4 +-
 .../SimpleBlocksMovementsStatusHandler.java     |  88 ++++++++++
 .../server/datanode/TestBPOfferService.java     |  12 +-
 .../hdfs/server/datanode/TestBlockRecovery.java |   4 +-
 .../server/datanode/TestDataNodeLifeline.java   |   7 +-
 .../TestDatanodeProtocolRetryPolicy.java        |   4 +-
 .../server/datanode/TestFsDatasetCache.java     |   4 +-
 .../TestStoragePolicySatisfyWorker.java         |  76 +--------
 .../hdfs/server/datanode/TestStorageReport.java |   4 +-
 .../server/namenode/NNThroughputBenchmark.java  |   9 +-
 .../hdfs/server/namenode/NameNodeAdapter.java   |   4 +-
 .../hdfs/server/namenode/TestDeadDatanode.java  |   5 +-
 .../namenode/TestNameNodeReconfigure.java       |  17 +-
 .../TestBlockStorageMovementAttemptedItems.java |  88 ++++++----
 .../sps/TestStoragePolicySatisfier.java         |  73 ++++++--
 ...stStoragePolicySatisfierWithStripedFile.java |  40 +++--
 .../sps/TestExternalStoragePolicySatisfier.java |  44 ++---
 42 files changed, 776 insertions(+), 659 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/2acc50b8/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 dcc0705..e4125dc 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,6 @@ 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.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;
@@ -139,8 +138,7 @@ public class DatanodeProtocolClientSideTranslatorPB implements
       VolumeFailureSummary volumeFailureSummary,
       boolean requestFullBlockReportLease,
       @Nonnull SlowPeerReports slowPeers,
-      @Nonnull SlowDiskReports slowDisks,
-      BlocksStorageMoveAttemptFinished storageMovementFinishedBlks)
+      @Nonnull SlowDiskReports slowDisks)
           throws IOException {
     HeartbeatRequestProto.Builder builder = HeartbeatRequestProto.newBuilder()
         .setRegistration(PBHelper.convert(registration))
@@ -165,13 +163,6 @@ public class DatanodeProtocolClientSideTranslatorPB implements
       builder.addAllSlowDisks(PBHelper.convertSlowDiskInfo(slowDisks));
     }
 
-    // Adding blocks movement results to the heart beat request.
-    if (storageMovementFinishedBlks != null
-        && storageMovementFinishedBlks.getBlocks() != null) {
-      builder.setStorageMoveAttemptFinishedBlks(
-          PBHelper.convertBlksMovReport(storageMovementFinishedBlks));
-    }
-
     HeartbeatResponseProto resp;
     try {
       resp = rpcProxy.sendHeartbeat(NULL_CONTROLLER, builder.build());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2acc50b8/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 b5bb80a..5cba284 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,9 +122,7 @@ public class DatanodeProtocolServerSideTranslatorPB implements
           request.getXceiverCount(), request.getFailedVolumes(),
           volumeFailureSummary, request.getRequestFullBlockReportLease(),
           PBHelper.convertSlowPeerInfo(request.getSlowPeersList()),
-          PBHelper.convertSlowDiskInfo(request.getSlowDisksList()),
-          PBHelper.convertBlksMovReport(
-              request.getStorageMoveAttemptFinishedBlks()));
+          PBHelper.convertSlowDiskInfo(request.getSlowDisksList()));
     } catch (IOException e) {
       throw new ServiceException(e);
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2acc50b8/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 38f72c0..f51f839 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
@@ -57,7 +57,6 @@ 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.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;
@@ -105,7 +104,6 @@ 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.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;
@@ -971,29 +969,6 @@ public class PBHelper {
     return SlowDiskReports.create(slowDisksMap);
   }
 
-  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 new BlocksStorageMoveAttemptFinished(blocks);
-  }
-
-  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 builder.build();
-  }
-
   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/2acc50b8/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 e7979b4..42e246c 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
@@ -69,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.StoragePolicySatisfierMode;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock;
@@ -92,6 +93,7 @@ import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.namenode.Namesystem;
 import org.apache.hadoop.hdfs.server.namenode.ha.HAContext;
 import org.apache.hadoop.hdfs.server.namenode.metrics.NameNodeMetrics;
+import org.apache.hadoop.hdfs.server.namenode.sps.SPSService;
 import org.apache.hadoop.hdfs.server.namenode.sps.StoragePolicySatisfyManager;
 import org.apache.hadoop.hdfs.server.protocol.BlockCommand;
 import org.apache.hadoop.hdfs.server.protocol.BlockReportContext;
@@ -427,8 +429,11 @@ public class BlockManager implements BlockStatsMXBean {
 
   private final BlockIdManager blockIdManager;
 
-  /** For satisfying block storage policies. */
-  private final StoragePolicySatisfyManager spsManager;
+  /**
+   * For satisfying block storage policies. Instantiates if sps is enabled
+   * internally or externally.
+   */
+  private StoragePolicySatisfyManager spsManager;
 
   /** Minimum live replicas needed for the datanode to be transitioned
    * from ENTERING_MAINTENANCE to IN_MAINTENANCE.
@@ -469,8 +474,7 @@ public class BlockManager implements BlockStatsMXBean {
         DFSConfigKeys.DFS_NAMENODE_RECONSTRUCTION_PENDING_TIMEOUT_SEC_DEFAULT)
         * 1000L);
 
-    // sps manager manages the user invoked sps paths and does the movement.
-    spsManager = new StoragePolicySatisfyManager(conf, namesystem, this);
+    createSPSManager(conf);
 
     blockTokenSecretManager = createBlockTokenSecretManager(conf);
 
@@ -699,7 +703,9 @@ public class BlockManager implements BlockStatsMXBean {
   }
 
   public void close() {
-    getSPSManager().stop();
+    if (getSPSManager() != null) {
+      getSPSManager().stop();
+    }
     bmSafeMode.close();
     try {
       redundancyThread.interrupt();
@@ -713,7 +719,9 @@ public class BlockManager implements BlockStatsMXBean {
     datanodeManager.close();
     pendingReconstruction.stop();
     blocksMap.close();
-    getSPSManager().stopGracefully();
+    if (getSPSManager() != null) {
+      getSPSManager().stopGracefully();
+    }
   }
 
   /** @return the datanodeManager */
@@ -3881,6 +3889,21 @@ public class BlockManager implements BlockStatsMXBean {
     }
     processAndHandleReportedBlock(storageInfo, block, ReplicaState.FINALIZED,
         delHintNode);
+
+    // notify SPS about the reported block
+    notifyStorageMovementAttemptFinishedBlk(storageInfo, block);
+  }
+
+  private void notifyStorageMovementAttemptFinishedBlk(
+      DatanodeStorageInfo storageInfo, Block block) {
+    if (getSPSManager() != null) {
+      SPSService<Long> sps = getSPSManager().getInternalSPSService();
+      if (sps.isRunning()) {
+        sps.notifyStorageMovementAttemptFinishedBlk(
+            storageInfo.getDatanodeDescriptor(), storageInfo.getStorageType(),
+            block);
+      }
+    }
   }
   
   private void processAndHandleReportedBlock(
@@ -5026,6 +5049,57 @@ public class BlockManager implements BlockStatsMXBean {
   }
 
   /**
+   * Create SPS manager instance. It manages the user invoked sps paths and does
+   * the movement.
+   *
+   * @param conf
+   *          configuration
+   * @return true if the instance is successfully created, false otherwise.
+   */
+  private boolean createSPSManager(final Configuration conf) {
+    return createSPSManager(conf, null);
+  }
+
+  /**
+   * Create SPS manager instance. It manages the user invoked sps paths and does
+   * the movement.
+   *
+   * @param conf
+   *          configuration
+   * @param spsMode
+   *          satisfier mode
+   * @return true if the instance is successfully created, false otherwise.
+   */
+  public boolean createSPSManager(final Configuration conf,
+      final String spsMode) {
+    // sps manager manages the user invoked sps paths and does the movement.
+    // StoragePolicySatisfier(SPS) configs
+    boolean storagePolicyEnabled = conf.getBoolean(
+        DFSConfigKeys.DFS_STORAGE_POLICY_ENABLED_KEY,
+        DFSConfigKeys.DFS_STORAGE_POLICY_ENABLED_DEFAULT);
+    String modeVal = spsMode;
+    if (org.apache.commons.lang.StringUtils.isBlank(modeVal)) {
+      modeVal = conf.get(DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_MODE_KEY,
+          DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_MODE_DEFAULT);
+    }
+    StoragePolicySatisfierMode mode = StoragePolicySatisfierMode
+        .fromString(modeVal);
+    if (!storagePolicyEnabled || mode == StoragePolicySatisfierMode.NONE) {
+      LOG.info("Storage policy satisfier is disabled");
+      return false;
+    }
+    spsManager = new StoragePolicySatisfyManager(conf, namesystem, this);
+    return true;
+  }
+
+  /**
+   * Nullify SPS manager as this feature is disabled fully.
+   */
+  public void disableSPS() {
+    spsManager = null;
+  }
+
+  /**
    * @return sps manager.
    */
   public StoragePolicySatisfyManager getSPSManager() {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2acc50b8/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/sps/BlockMovementAttemptFinished.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/sps/BlockMovementAttemptFinished.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/sps/BlockMovementAttemptFinished.java
index 419d806..29c5e9c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/sps/BlockMovementAttemptFinished.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/sps/BlockMovementAttemptFinished.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.hdfs.server.common.sps;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 
@@ -33,6 +34,7 @@ public class BlockMovementAttemptFinished {
   private final Block block;
   private final DatanodeInfo src;
   private final DatanodeInfo target;
+  private final StorageType targetType;
   private final BlockMovementStatus status;
 
   /**
@@ -44,14 +46,17 @@ public class BlockMovementAttemptFinished {
    *          src datanode
    * @param target
    *          target datanode
+   * @param targetType
+   *          target storage type
    * @param status
    *          movement status
    */
   public BlockMovementAttemptFinished(Block block, DatanodeInfo src,
-      DatanodeInfo target, BlockMovementStatus status) {
+      DatanodeInfo target, StorageType targetType, BlockMovementStatus status) {
     this.block = block;
     this.src = src;
     this.target = target;
+    this.targetType = targetType;
     this.status = status;
   }
 
@@ -64,6 +69,20 @@ public class BlockMovementAttemptFinished {
   }
 
   /**
+   * @return the target datanode where it moved the block.
+   */
+  public DatanodeInfo getTargetDatanode() {
+    return target;
+  }
+
+  /**
+   * @return target storage type.
+   */
+  public StorageType getTargetType() {
+    return targetType;
+  }
+
+  /**
    * @return block movement status code.
    */
   public BlockMovementStatus getStatus() {
@@ -74,7 +93,8 @@ public class BlockMovementAttemptFinished {
   public String toString() {
     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(" target node: ").append(target).append(" target type: ")
+        .append(targetType).append(" movement status: ")
         .append(status).append(")").toString();
   }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2acc50b8/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/sps/BlockStorageMovementTracker.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/sps/BlockStorageMovementTracker.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/sps/BlockStorageMovementTracker.java
index b20d6cf..4ee415e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/sps/BlockStorageMovementTracker.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/sps/BlockStorageMovementTracker.java
@@ -17,17 +17,12 @@
  */
 package org.apache.hadoop.hdfs.server.common.sps;
 
-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.protocol.Block;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -39,13 +34,10 @@ import org.slf4j.LoggerFactory;
 public class BlockStorageMovementTracker implements Runnable {
   private static final Logger LOG = LoggerFactory
       .getLogger(BlockStorageMovementTracker.class);
-  private final CompletionService<BlockMovementAttemptFinished> moverCompletionService;
+  private final CompletionService<BlockMovementAttemptFinished>
+      moverCompletionService;
   private final BlocksMovementsStatusHandler blksMovementsStatusHandler;
 
-  // 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;
 
   /**
@@ -60,53 +52,21 @@ public class BlockStorageMovementTracker implements Runnable {
       CompletionService<BlockMovementAttemptFinished> moverCompletionService,
       BlocksMovementsStatusHandler handler) {
     this.moverCompletionService = moverCompletionService;
-    this.moverTaskFutures = new HashMap<>();
     this.blksMovementsStatusHandler = handler;
-    this.movementResults = new HashMap<>();
   }
 
   @Override
   public void run() {
     while (running) {
-      if (moverTaskFutures.size() <= 0) {
-        try {
-          synchronized (moverTaskFutures) {
-            // Waiting for mover tasks.
-            moverTaskFutures.wait(2000);
-          }
-        } catch (InterruptedException ignore) {
-          // Sets interrupt flag of this thread.
-          Thread.currentThread().interrupt();
-        }
-      }
       try {
-        Future<BlockMovementAttemptFinished> future =
-            moverCompletionService.take();
+        Future<BlockMovementAttemptFinished> future = moverCompletionService
+            .take();
         if (future != null) {
           BlockMovementAttemptFinished result = future.get();
           LOG.debug("Completed block movement. {}", result);
-          Block block = result.getBlock();
-          List<Future<BlockMovementAttemptFinished>> blocksMoving =
-              moverTaskFutures.get(block);
-          if (blocksMoving == null) {
-            LOG.warn("Future task doesn't exist for block : {} ", block);
-            continue;
-          }
-          blocksMoving.remove(future);
-
-          List<BlockMovementAttemptFinished> resultPerTrackIdList =
-              addMovementResultToBlockIdList(result);
-
-          // Completed all the scheduled blocks movement under this 'trackId'.
-          if (blocksMoving.isEmpty() || moverTaskFutures.get(block) == null) {
-            synchronized (moverTaskFutures) {
-              moverTaskFutures.remove(block);
-            }
-            if (running) {
-              // handle completed or inprogress blocks movements per trackId.
-              blksMovementsStatusHandler.handle(resultPerTrackIdList);
-            }
-            movementResults.remove(block);
+          if (running && blksMovementsStatusHandler != null) {
+            // handle completed block movement.
+            blksMovementsStatusHandler.handle(result);
           }
         }
       } catch (InterruptedException e) {
@@ -122,63 +82,10 @@ public class BlockStorageMovementTracker implements Runnable {
     }
   }
 
-  private List<BlockMovementAttemptFinished> addMovementResultToBlockIdList(
-      BlockMovementAttemptFinished result) {
-    Block block = result.getBlock();
-    List<BlockMovementAttemptFinished> perBlockIdList;
-    synchronized (movementResults) {
-      perBlockIdList = movementResults.get(block);
-      if (perBlockIdList == null) {
-        perBlockIdList = new ArrayList<>();
-        movementResults.put(block, perBlockIdList);
-      }
-      perBlockIdList.add(result);
-    }
-    return perBlockIdList;
-  }
-
-  /**
-   * Add future task to the tracking list to check the completion status of the
-   * block movement.
-   *
-   * @param blockID
-   *          block identifier
-   * @param futureTask
-   *          future task used for moving the respective block
-   */
-  public void addBlock(Block block,
-      Future<BlockMovementAttemptFinished> futureTask) {
-    synchronized (moverTaskFutures) {
-      List<Future<BlockMovementAttemptFinished>> futures =
-          moverTaskFutures.get(block);
-      // null for the first task
-      if (futures == null) {
-        futures = new ArrayList<>();
-        moverTaskFutures.put(block, futures);
-      }
-      futures.add(futureTask);
-      // Notify waiting tracker thread about the newly added tasks.
-      moverTaskFutures.notify();
-    }
-  }
-
-  /**
-   * Clear the pending movement and movement result queues.
-   */
-  public void removeAll() {
-    synchronized (moverTaskFutures) {
-      moverTaskFutures.clear();
-    }
-    synchronized (movementResults) {
-      movementResults.clear();
-    }
-  }
-
   /**
-   * Sets running flag to false and clear the pending movement result queues.
+   * Sets running flag to false.
    */
   public void stopTracking() {
     running = false;
-    removeAll();
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2acc50b8/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/sps/BlocksMovementsStatusHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/sps/BlocksMovementsStatusHandler.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/sps/BlocksMovementsStatusHandler.java
index f9f3954..ab67424 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/sps/BlocksMovementsStatusHandler.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/sps/BlocksMovementsStatusHandler.java
@@ -18,78 +18,22 @@
 
 package org.apache.hadoop.hdfs.server.common.sps;
 
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.hdfs.protocol.Block;
 
 /**
- * Blocks movements status handler, which is used to collect details of the
- * completed block movements and later these attempted finished(with success or
- * failure) blocks can be accessed to notify respective listeners, if any.
+ * Blocks movements status handler, which can be used to collect details of the
+ * completed block movements.
  */
 @InterfaceAudience.Private
 @InterfaceStability.Evolving
-public class BlocksMovementsStatusHandler {
-  private final List<Block> blockIdVsMovementStatus = new ArrayList<>();
-
-  /**
-   * Collect all the storage movement attempt finished blocks. Later this will
-   * be send to namenode via heart beat.
-   *
-   * @param moveAttemptFinishedBlks
-   *          set of storage movement attempt finished blocks
-   */
-  public void handle(
-      List<BlockMovementAttemptFinished> moveAttemptFinishedBlks) {
-    List<Block> blocks = new ArrayList<>();
-
-    for (BlockMovementAttemptFinished item : moveAttemptFinishedBlks) {
-      blocks.add(item.getBlock());
-    }
-    // Adding to the tracking report list. Later this can be accessed to know
-    // the attempted block movements.
-    synchronized (blockIdVsMovementStatus) {
-      blockIdVsMovementStatus.addAll(blocks);
-    }
-  }
+public interface BlocksMovementsStatusHandler {
 
   /**
-   * @return unmodifiable list of storage movement attempt finished blocks.
-   */
-  public 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);
-      }
-    }
-    return moveAttemptFinishedBlks;
-  }
-
-  /**
-   * Remove the storage movement attempt finished blocks from the tracking list.
+   * Collect all the storage movement attempt finished blocks.
    *
-   * @param moveAttemptFinishedBlks
-   *          set of storage movement attempt finished blocks
-   */
-  public void remove(List<Block> moveAttemptFinishedBlks) {
-    if (moveAttemptFinishedBlks != null) {
-      blockIdVsMovementStatus.removeAll(moveAttemptFinishedBlks);
-    }
-  }
-
-  /**
-   * Clear the blockID vs movement status tracking map.
+   * @param moveAttemptFinishedBlk
+   *          storage movement attempt finished block
    */
-  public void removeAll() {
-    synchronized (blockIdVsMovementStatus) {
-      blockIdVsMovementStatus.clear();
-    }
-  }
+  void handle(BlockMovementAttemptFinished moveAttemptFinishedBlk);
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2acc50b8/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 b7beda4..dab8ae9 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
@@ -514,12 +514,6 @@ class BPServiceActor implements Runnable {
             SlowDiskReports.create(dn.getDiskMetrics().getDiskOutliersStats()) :
             SlowDiskReports.EMPTY_REPORT;
 
-    // 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,
         dn.getFSDataset().getCacheCapacity(),
@@ -530,19 +524,13 @@ class BPServiceActor implements Runnable {
         volumeFailureSummary,
         requestBlockReportLease,
         slowPeers,
-        slowDisks,
-        storageMoveAttemptFinishedBlks);
+        slowDisks);
 
     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().getBlocksMovementsStatusHandler()
-        .remove(results);
-
     return response;
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2acc50b8/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 4ee364b..a714602 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
@@ -1427,7 +1427,7 @@ public class DataNode extends ReconfigurableBase
     ecWorker = new ErasureCodingWorker(getConf(), this);
     blockRecoveryWorker = new BlockRecoveryWorker(this);
     storagePolicySatisfyWorker =
-        new StoragePolicySatisfyWorker(getConf(), this);
+        new StoragePolicySatisfyWorker(getConf(), this, null);
     storagePolicySatisfyWorker.start();
 
     blockPoolManager = new BlockPoolManager(this);
@@ -2137,11 +2137,6 @@ 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/2acc50b8/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 af6137c..0157205 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
@@ -24,7 +24,6 @@ import java.util.concurrent.Callable;
 import java.util.concurrent.CompletionService;
 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;
@@ -38,19 +37,17 @@ import org.apache.hadoop.hdfs.DFSUtilClient;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.datatransfer.sasl.DataEncryptionKeyFactory;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
+import org.apache.hadoop.hdfs.server.common.sps.BlockDispatcher;
 import org.apache.hadoop.hdfs.server.common.sps.BlockMovementAttemptFinished;
 import org.apache.hadoop.hdfs.server.common.sps.BlockMovementStatus;
 import org.apache.hadoop.hdfs.server.common.sps.BlockStorageMovementTracker;
 import org.apache.hadoop.hdfs.server.common.sps.BlocksMovementsStatusHandler;
-import org.apache.hadoop.hdfs.server.common.sps.BlockDispatcher;
 import org.apache.hadoop.hdfs.server.protocol.BlockStorageMovementCommand.BlockMovingInfo;
 import org.apache.hadoop.security.token.Token;
 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
@@ -67,19 +64,19 @@ public class StoragePolicySatisfyWorker {
 
   private final int moverThreads;
   private final ExecutorService moveExecutor;
-  private final CompletionService<BlockMovementAttemptFinished> moverCompletionService;
-  private final BlocksMovementsStatusHandler handler;
+  private final CompletionService<BlockMovementAttemptFinished>
+      moverCompletionService;
   private final BlockStorageMovementTracker movementTracker;
   private Daemon movementTrackerThread;
   private final BlockDispatcher blkDispatcher;
 
-  public StoragePolicySatisfyWorker(Configuration conf, DataNode datanode) {
+  public StoragePolicySatisfyWorker(Configuration conf, DataNode datanode,
+      BlocksMovementsStatusHandler handler) {
     this.datanode = datanode;
-    // Defaulting to 10. This is to minimise the number of move ops.
+    // Defaulting to 10. This is to minimize the number of move ops.
     moverThreads = conf.getInt(DFSConfigKeys.DFS_MOVER_MOVERTHREADS_KEY, 10);
     moveExecutor = initializeBlockMoverThreadPool(moverThreads);
     moverCompletionService = new ExecutorCompletionService<>(moveExecutor);
-    handler = new BlocksMovementsStatusHandler();
     movementTracker = new BlockStorageMovementTracker(moverCompletionService,
         handler);
     movementTrackerThread = new Daemon(movementTracker);
@@ -88,7 +85,6 @@ public class StoragePolicySatisfyWorker {
     int ioFileBufferSize = DFSUtilClient.getIoFileBufferSize(conf);
     blkDispatcher = new BlockDispatcher(dnConf.getSocketTimeout(),
         ioFileBufferSize, dnConf.getConnectToDnViaHostname());
-    // TODO: Needs to manage the number of concurrent moves per DataNode.
   }
 
   /**
@@ -100,22 +96,17 @@ public class StoragePolicySatisfyWorker {
   }
 
   /**
-   * Stop StoragePolicySatisfyWorker, which will stop block movement tracker
-   * thread.
+   * Stop StoragePolicySatisfyWorker, which will terminate executor service and
+   * stop block movement tracker thread.
    */
   void stop() {
     movementTracker.stopTracking();
     movementTrackerThread.interrupt();
-  }
-
-  /**
-   * Timed wait to stop BlockStorageMovement tracker daemon thread.
-   */
-  void waitToFinishWorkerThread() {
+    moveExecutor.shutdown();
     try {
-      movementTrackerThread.join(3000);
-    } catch (InterruptedException ignore) {
-      // ignore
+      moveExecutor.awaitTermination(500, TimeUnit.MILLISECONDS);
+    } catch (InterruptedException e) {
+      LOG.error("Interrupted while waiting for mover thread to terminate", e);
     }
   }
 
@@ -160,10 +151,7 @@ public class StoragePolicySatisfyWorker {
           : "Source and Target storage type shouldn't be same!";
       BlockMovingTask blockMovingTask = new BlockMovingTask(blockPoolID,
           blkMovingInfo);
-      Future<BlockMovementAttemptFinished> moveCallable = moverCompletionService
-          .submit(blockMovingTask);
-      movementTracker.addBlock(blkMovingInfo.getBlock(),
-          moveCallable);
+      moverCompletionService.submit(blockMovingTask);
     }
   }
 
@@ -185,7 +173,8 @@ public class StoragePolicySatisfyWorker {
     public BlockMovementAttemptFinished call() {
       BlockMovementStatus status = moveBlock();
       return new BlockMovementAttemptFinished(blkMovingInfo.getBlock(),
-          blkMovingInfo.getSource(), blkMovingInfo.getTarget(), status);
+          blkMovingInfo.getSource(), blkMovingInfo.getTarget(),
+          blkMovingInfo.getTargetStorageType(), status);
     }
 
     private BlockMovementStatus moveBlock() {
@@ -217,11 +206,6 @@ public class StoragePolicySatisfyWorker {
     }
   }
 
-  @VisibleForTesting
-  BlocksMovementsStatusHandler getBlocksMovementsStatusHandler() {
-    return handler;
-  }
-
   /**
    * Drop the in-progress SPS work queues.
    */
@@ -229,7 +213,5 @@ public class StoragePolicySatisfyWorker {
     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/2acc50b8/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 45d6218..3f873d7 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
@@ -31,6 +31,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.sps.StoragePolicySatisfyManager;
 
 import com.google.common.collect.Lists;
 
@@ -102,7 +103,11 @@ final class FSDirSatisfyStoragePolicyOp {
 
         // Adding directory in the pending queue, so FileInodeIdCollector
         // process directory child in batch and recursively
-        fsd.getBlockManager().getSPSManager().addPathId(inode.getId());
+        StoragePolicySatisfyManager spsManager =
+            fsd.getBlockManager().getSPSManager();
+        if (spsManager != null) {
+          spsManager.addPathId(inode.getId());
+        }
       }
     } finally {
       fsd.writeUnlock();
@@ -116,7 +121,11 @@ final class FSDirSatisfyStoragePolicyOp {
     } else {
       // Adding directory in the pending queue, so FileInodeIdCollector process
       // directory child in batch and recursively
-      fsd.getBlockManager().getSPSManager().addPathId(inode.getId());
+      StoragePolicySatisfyManager spsManager =
+          fsd.getBlockManager().getSPSManager();
+      if (spsManager != null) {
+        spsManager.addPathId(inode.getId());
+      }
       return true;
     }
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2acc50b8/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 1150a72..3b68979 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
@@ -32,6 +32,7 @@ import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ReencryptionInfoProto;
 import org.apache.hadoop.hdfs.protocolPB.PBHelperClient;
 import org.apache.hadoop.hdfs.server.namenode.FSDirectory.DirOp;
+import org.apache.hadoop.hdfs.server.namenode.sps.StoragePolicySatisfyManager;
 import org.apache.hadoop.security.AccessControlException;
 
 import java.io.FileNotFoundException;
@@ -209,8 +210,11 @@ class FSDirXAttrOp {
       for (XAttr xattr : toRemove) {
         if (XATTR_SATISFY_STORAGE_POLICY
             .equals(XAttrHelper.getPrefixedName(xattr))) {
-          fsd.getBlockManager().getSPSManager().getInternalSPSService()
-              .clearQueue(inode.getId());
+          StoragePolicySatisfyManager spsManager =
+              fsd.getBlockManager().getSPSManager();
+          if (spsManager != null) {
+            spsManager.getInternalSPSService().clearQueue(inode.getId());
+          }
           break;
         }
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2acc50b8/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 6539b51..2a976d2 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
@@ -58,6 +58,7 @@ import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.namenode.INode.BlocksMapUpdateInfo.UpdatedReplicationInfo;
+import org.apache.hadoop.hdfs.server.namenode.sps.StoragePolicySatisfyManager;
 import org.apache.hadoop.hdfs.util.ByteArray;
 import org.apache.hadoop.hdfs.util.EnumCounters;
 import org.apache.hadoop.hdfs.util.ReadOnlyList;
@@ -1401,7 +1402,9 @@ public class FSDirectory implements Closeable {
       if (!inode.isSymlink()) {
         final XAttrFeature xaf = inode.getXAttrFeature();
         addEncryptionZone((INodeWithAdditionalFields) inode, xaf);
-        if (namesystem.getBlockManager().getSPSManager().isEnabled()) {
+        StoragePolicySatisfyManager spsManager =
+            namesystem.getBlockManager().getSPSManager();
+        if (spsManager != null && spsManager.isEnabled()) {
           addStoragePolicySatisfier((INodeWithAdditionalFields) inode, xaf);
         }
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2acc50b8/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 8c5a410..e1ceecd 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
@@ -259,7 +259,6 @@ import org.apache.hadoop.hdfs.server.namenode.metrics.NameNodeMetrics;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.DirectorySnapshottableFeature;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotManager;
-import org.apache.hadoop.hdfs.server.namenode.sps.SPSService;
 import org.apache.hadoop.hdfs.server.namenode.startupprogress.Phase;
 import org.apache.hadoop.hdfs.server.namenode.startupprogress.StartupProgress;
 import org.apache.hadoop.hdfs.server.namenode.startupprogress.Step;
@@ -268,7 +267,6 @@ 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.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;
@@ -1292,7 +1290,9 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
         FSDirEncryptionZoneOp.warmUpEdekCache(edekCacheLoader, dir,
             edekCacheLoaderDelay, edekCacheLoaderInterval);
       }
-      blockManager.getSPSManager().start();
+      if (blockManager.getSPSManager() != null) {
+        blockManager.getSPSManager().start();
+      }
     } finally {
       startingActiveService = false;
       blockManager.checkSafeMode();
@@ -1322,7 +1322,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     LOG.info("Stopping services started for active state");
     writeLock();
     try {
-      if (blockManager != null) {
+      if (blockManager != null && blockManager.getSPSManager() != null) {
         blockManager.getSPSManager().stop();
       }
       stopSecretManager();
@@ -1363,7 +1363,9 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
         // Don't want to keep replication queues when not in Active.
         blockManager.clearQueues();
         blockManager.setInitializedReplQueues(false);
-        blockManager.getSPSManager().stopGracefully();
+        if (blockManager.getSPSManager() != null) {
+          blockManager.getSPSManager().stopGracefully();
+        }
       }
     } finally {
       writeUnlock("stopActiveServices");
@@ -2272,7 +2274,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
           DFS_STORAGE_POLICY_ENABLED_KEY));
     }
     // checks sps status
-    if (!blockManager.getSPSManager().isEnabled() || (blockManager
+    boolean disabled = (blockManager.getSPSManager() == null);
+    if (disabled || (blockManager
         .getSPSManager().getMode() == StoragePolicySatisfierMode.INTERNAL
         && !blockManager.getSPSManager().isInternalSatisfierRunning())) {
       throw new UnsupportedActionException(
@@ -3970,8 +3973,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       VolumeFailureSummary volumeFailureSummary,
       boolean requestFullBlockReportLease,
       @Nonnull SlowPeerReports slowPeers,
-      @Nonnull SlowDiskReports slowDisks,
-      BlocksStorageMoveAttemptFinished blksMovementsFinished)
+      @Nonnull SlowDiskReports slowDisks)
           throws IOException {
     readLock();
     try {
@@ -3987,18 +3989,6 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
         blockReportLeaseId =  blockManager.requestBlockReportLeaseId(nodeReg);
       }
 
-      // Handle blocks movement results sent by the coordinator datanode.
-      SPSService sps = blockManager.getSPSManager().getInternalSPSService();
-      if (!sps.isRunning()) {
-        if (LOG.isDebugEnabled()) {
-          LOG.debug(
-              "Storage policy satisfier is not running. So, ignoring storage"
-                  + "  movement attempt finished block info sent by DN");
-        }
-      } else {
-        sps.notifyStorageMovementAttemptFinishedBlks(blksMovementsFinished);
-      }
-
       //create ha status
       final NNHAStatusHeartbeat haState = new NNHAStatusHeartbeat(
           haContext.getState().getServiceState(),

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2acc50b8/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 4e3a3ba..7f78d2f 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
@@ -2147,7 +2147,24 @@ public class NameNode extends ReconfigurableBase implements
     }
     StoragePolicySatisfierMode mode = StoragePolicySatisfierMode
         .fromString(newVal);
-    namesystem.getBlockManager().getSPSManager().changeModeEvent(mode);
+    if (mode == StoragePolicySatisfierMode.NONE) {
+      // disabling sps service
+      if (namesystem.getBlockManager().getSPSManager() != null) {
+        namesystem.getBlockManager().getSPSManager().changeModeEvent(mode);
+        namesystem.getBlockManager().disableSPS();
+      }
+    } else {
+      // enabling sps service
+      boolean spsCreated = (namesystem.getBlockManager()
+          .getSPSManager() != null);
+      if (!spsCreated) {
+        spsCreated = namesystem.getBlockManager().createSPSManager(getConf(),
+            newVal);
+      }
+      if (spsCreated) {
+        namesystem.getBlockManager().getSPSManager().changeModeEvent(mode);
+      }
+    }
     return newVal;
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2acc50b8/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 1590423..57e827d 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
@@ -156,8 +156,8 @@ import org.apache.hadoop.hdfs.server.common.HttpGetFailedException;
 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.namenode.sps.StoragePolicySatisfyManager;
 import org.apache.hadoop.hdfs.server.protocol.BlockReportContext;
-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;
@@ -1517,16 +1517,14 @@ public class NameNodeRpcServer implements NamenodeProtocols {
       int failedVolumes, VolumeFailureSummary volumeFailureSummary,
       boolean requestFullBlockReportLease,
       @Nonnull SlowPeerReports slowPeers,
-      @Nonnull SlowDiskReports slowDisks,
-      BlocksStorageMoveAttemptFinished storageMovementFinishedBlks)
+      @Nonnull SlowDiskReports slowDisks)
           throws IOException {
     checkNNStartup();
     verifyRequest(nodeReg);
     return namesystem.handleHeartbeat(nodeReg, report,
         dnCacheCapacity, dnCacheUsed, xceiverCount, xmitsInProgress,
         failedVolumes, volumeFailureSummary, requestFullBlockReportLease,
-        slowPeers, slowDisks,
-        storageMovementFinishedBlks);
+        slowPeers, slowDisks);
   }
 
   @Override // DatanodeProtocol
@@ -2543,10 +2541,12 @@ public class NameNodeRpcServer implements NamenodeProtocols {
     if (nn.isStandbyState()) {
       throw new StandbyException("Not supported by Standby Namenode.");
     }
-    boolean isSPSRunning = namesystem.getBlockManager().getSPSManager()
-        .isInternalSatisfierRunning();
+    StoragePolicySatisfyManager spsMgr =
+        namesystem.getBlockManager().getSPSManager();
+    boolean isInternalSatisfierRunning = (spsMgr != null
+        ? spsMgr.isInternalSatisfierRunning() : false);
     namesystem.logAuditEvent(true, operationName, null);
-    return isSPSRunning;
+    return isInternalSatisfierRunning;
   }
 
   @Override
@@ -2556,6 +2556,14 @@ public class NameNodeRpcServer implements NamenodeProtocols {
     if (nn.isStandbyState()) {
       throw new StandbyException("Not supported by Standby Namenode.");
     }
+    if (namesystem.getBlockManager().getSPSManager() == null) {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Satisfier is not running inside namenode, so status "
+            + "can't be returned.");
+      }
+      throw new IOException("Satisfier is not running inside namenode, "
+          + "so status can't be returned.");
+    }
     return namesystem.getBlockManager().getSPSManager()
         .checkStoragePolicySatisfyPathStatus(path);
   }
@@ -2568,16 +2576,20 @@ public class NameNodeRpcServer implements NamenodeProtocols {
     if (nn.isStandbyState()) {
       throw new StandbyException("Not supported by Standby Namenode.");
     }
-    // Check that SPS daemon service is running inside namenode
-    if (namesystem.getBlockManager().getSPSManager()
-        .getMode() == StoragePolicySatisfierMode.INTERNAL) {
-      LOG.debug("SPS service is internally enabled and running inside "
-          + "namenode, so external SPS is not allowed to fetch the path Ids");
-      throw new IOException("SPS service is internally enabled and running"
-          + " inside namenode, so external SPS is not allowed to fetch"
-          + " the path Ids");
+    // Check that SPS is enabled externally
+    StoragePolicySatisfyManager spsMgr =
+        namesystem.getBlockManager().getSPSManager();
+    StoragePolicySatisfierMode spsMode = (spsMgr != null ? spsMgr.getMode()
+        : StoragePolicySatisfierMode.NONE);
+    if (spsMode != StoragePolicySatisfierMode.EXTERNAL) {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("SPS service mode is {}, so external SPS service is "
+            + "not allowed to fetch the path Ids", spsMode);
+      }
+      throw new IOException("SPS service mode is " + spsMode + ", so "
+          + "external SPS service is not allowed to fetch the path Ids");
     }
-    Long pathId = namesystem.getBlockManager().getSPSManager().getNextPathId();
+    Long pathId = spsMgr.getNextPathId();
     if (pathId == null) {
       return null;
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2acc50b8/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/BlockStorageMovementAttemptedItems.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/BlockStorageMovementAttemptedItems.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/BlockStorageMovementAttemptedItems.java
index d2f0bb2..5b25491 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/BlockStorageMovementAttemptedItems.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/BlockStorageMovementAttemptedItems.java
@@ -17,21 +17,28 @@
  */
 package org.apache.hadoop.hdfs.server.namenode.sps;
 
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_RECHECK_TIMEOUT_MILLIS_DEFAULT;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_RECHECK_TIMEOUT_MILLIS_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_SELF_RETRY_TIMEOUT_MILLIS_DEFAULT;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_SELF_RETRY_TIMEOUT_MILLIS_KEY;
 import static org.apache.hadoop.util.Time.monotonicNow;
 
 import java.io.IOException;
 import java.util.ArrayList;
-import java.util.Arrays;
+import java.util.Collection;
+import java.util.HashMap;
 import java.util.Iterator;
 import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.LinkedBlockingQueue;
 
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_RECHECK_TIMEOUT_MILLIS_DEFAULT;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_RECHECK_TIMEOUT_MILLIS_KEY;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_SELF_RETRY_TIMEOUT_MILLIS_DEFAULT;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_SELF_RETRY_TIMEOUT_MILLIS_KEY;
-
+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.server.namenode.sps.StoragePolicySatisfier.AttemptedItemInfo;
+import org.apache.hadoop.hdfs.server.namenode.sps.StoragePolicySatisfier.StorageTypeNodePair;
 import org.apache.hadoop.util.Daemon;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -60,10 +67,13 @@ public class BlockStorageMovementAttemptedItems<T> {
    * processing and sent to DNs.
    */
   private final List<AttemptedItemInfo<T>> storageMovementAttemptedItems;
-  private final List<Block> movementFinishedBlocks;
+  private Map<Block, Set<StorageTypeNodePair>> scheduledBlkLocs;
+  // Maintains separate Queue to keep the movement finished blocks. This Q
+  // is used to update the storageMovementAttemptedItems list asynchronously.
+  private final BlockingQueue<Block> movementFinishedBlocks;
   private volatile boolean monitorRunning = true;
   private Daemon timerThread = null;
-  private final BlockMovementListener blkMovementListener;
+  private BlockMovementListener blkMovementListener;
   //
   // It might take anywhere between 5 to 10 minutes before
   // a request is timed out.
@@ -94,7 +104,8 @@ public class BlockStorageMovementAttemptedItems<T> {
         DFS_STORAGE_POLICY_SATISFIER_SELF_RETRY_TIMEOUT_MILLIS_DEFAULT);
     this.blockStorageMovementNeeded = unsatisfiedStorageMovementFiles;
     storageMovementAttemptedItems = new ArrayList<>();
-    movementFinishedBlocks = new ArrayList<>();
+    scheduledBlkLocs = new HashMap<>();
+    movementFinishedBlocks = new LinkedBlockingQueue<>();
     this.blkMovementListener = blockMovementListener;
   }
 
@@ -105,29 +116,67 @@ public class BlockStorageMovementAttemptedItems<T> {
    * @param itemInfo
    *          - tracking info
    */
-  public void add(AttemptedItemInfo<T> itemInfo) {
+  public void add(T startPath, T file, long monotonicNow,
+      Map<Block, Set<StorageTypeNodePair>> assignedBlocks, int retryCount) {
+    AttemptedItemInfo<T> itemInfo = new AttemptedItemInfo<T>(startPath, file,
+        monotonicNow, assignedBlocks.keySet(), retryCount);
     synchronized (storageMovementAttemptedItems) {
       storageMovementAttemptedItems.add(itemInfo);
     }
+    synchronized (scheduledBlkLocs) {
+      scheduledBlkLocs.putAll(assignedBlocks);
+    }
   }
 
   /**
-   * Add the storage movement attempt finished blocks to
-   * storageMovementFinishedBlocks.
+   * Notify the storage movement attempt finished block.
    *
-   * @param moveAttemptFinishedBlks
-   *          storage movement attempt finished blocks
+   * @param reportedDn
+   *          reported datanode
+   * @param type
+   *          storage type
+   * @param reportedBlock
+   *          reported block
    */
-  public void notifyMovementTriedBlocks(Block[] moveAttemptFinishedBlks) {
-    if (moveAttemptFinishedBlks.length == 0) {
-      return;
+  public void notifyReportedBlock(DatanodeInfo reportedDn, StorageType type,
+      Block reportedBlock) {
+    synchronized (scheduledBlkLocs) {
+      if (scheduledBlkLocs.size() <= 0) {
+        return;
+      }
+      matchesReportedBlock(reportedDn, type, reportedBlock);
     }
-    synchronized (movementFinishedBlocks) {
-      movementFinishedBlocks.addAll(Arrays.asList(moveAttemptFinishedBlks));
+  }
+
+  private void matchesReportedBlock(DatanodeInfo reportedDn, StorageType type,
+      Block reportedBlock) {
+    Set<StorageTypeNodePair> blkLocs = scheduledBlkLocs.get(reportedBlock);
+    if (blkLocs == null) {
+      return; // unknown block, simply skip.
     }
-    // External listener if it is plugged-in
-    if (blkMovementListener != null) {
-      blkMovementListener.notifyMovementTriedBlocks(moveAttemptFinishedBlks);
+
+    for (StorageTypeNodePair dn : blkLocs) {
+      boolean foundDn = dn.getDatanodeInfo().compareTo(reportedDn) == 0 ? true
+          : false;
+      boolean foundType = dn.getStorageType().equals(type);
+      if (foundDn && foundType) {
+        blkLocs.remove(dn);
+        // listener if it is plugged-in
+        if (blkMovementListener != null) {
+          blkMovementListener
+              .notifyMovementTriedBlocks(new Block[] {reportedBlock});
+        }
+        // All the block locations has reported.
+        if (blkLocs.size() <= 0) {
+          movementFinishedBlocks.add(reportedBlock);
+          scheduledBlkLocs.remove(reportedBlock); // clean-up reported block
+        }
+        return; // found
+      }
+    }
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Reported block:{} not found in attempted blocks. Datanode:{}"
+          + ", StorageType:{}", reportedBlock, reportedDn, type);
     }
   }
 
@@ -203,14 +252,12 @@ public class BlockStorageMovementAttemptedItems<T> {
         if (now > itemInfo.getLastAttemptedOrReportedTime()
             + selfRetryTimeout) {
           T file = itemInfo.getFile();
-          synchronized (movementFinishedBlocks) {
-            ItemInfo<T> candidate = new ItemInfo<T>(itemInfo.getStartPath(),
-                file, itemInfo.getRetryCount() + 1);
-            blockStorageMovementNeeded.add(candidate);
-            iter.remove();
-            LOG.info("TrackID: {} becomes timed out and moved to needed "
-                + "retries queue for next iteration.", file);
-          }
+          ItemInfo<T> candidate = new ItemInfo<T>(itemInfo.getStartPath(), file,
+              itemInfo.getRetryCount() + 1);
+          blockStorageMovementNeeded.add(candidate);
+          iter.remove();
+          LOG.info("TrackID: {} becomes timed out and moved to needed "
+              + "retries queue for next iteration.", file);
         }
       }
     }
@@ -218,29 +265,25 @@ public class BlockStorageMovementAttemptedItems<T> {
 
   @VisibleForTesting
   void blockStorageMovementReportedItemsCheck() throws IOException {
-    synchronized (movementFinishedBlocks) {
-      Iterator<Block> finishedBlksIter = movementFinishedBlocks.iterator();
-      while (finishedBlksIter.hasNext()) {
-        Block blk = finishedBlksIter.next();
-        synchronized (storageMovementAttemptedItems) {
-          Iterator<AttemptedItemInfo<T>> iterator =
-              storageMovementAttemptedItems.iterator();
-          while (iterator.hasNext()) {
-            AttemptedItemInfo<T> 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.add(new ItemInfo<T>(attemptedItemInfo
-                  .getStartPath(), attemptedItemInfo.getFile(),
-                  attemptedItemInfo.getRetryCount() + 1));
-              iterator.remove();
-            }
+    // Removes all available blocks from this queue and process it.
+    Collection<Block> finishedBlks = new ArrayList<>();
+    movementFinishedBlocks.drainTo(finishedBlks);
+
+    // Update attempted items list
+    for (Block blk : finishedBlks) {
+      synchronized (storageMovementAttemptedItems) {
+        Iterator<AttemptedItemInfo<T>> iterator = storageMovementAttemptedItems
+            .iterator();
+        while (iterator.hasNext()) {
+          AttemptedItemInfo<T> attemptedItemInfo = iterator.next();
+          attemptedItemInfo.getBlocks().remove(blk);
+          if (attemptedItemInfo.getBlocks().isEmpty()) {
+            blockStorageMovementNeeded.add(new ItemInfo<T>(
+                attemptedItemInfo.getStartPath(), attemptedItemInfo.getFile(),
+                attemptedItemInfo.getRetryCount() + 1));
+            iterator.remove();
           }
         }
-        // Remove attempted blocks from movementFinishedBlocks list.
-        finishedBlksIter.remove();
       }
     }
   }
@@ -252,15 +295,29 @@ public class BlockStorageMovementAttemptedItems<T> {
 
   @VisibleForTesting
   public int getAttemptedItemsCount() {
-    return storageMovementAttemptedItems.size();
+    synchronized (storageMovementAttemptedItems) {
+      return storageMovementAttemptedItems.size();
+    }
   }
 
   public void clearQueues() {
-    synchronized (movementFinishedBlocks) {
-      movementFinishedBlocks.clear();
-    }
+    movementFinishedBlocks.clear();
     synchronized (storageMovementAttemptedItems) {
       storageMovementAttemptedItems.clear();
     }
+    synchronized (scheduledBlkLocs) {
+      scheduledBlkLocs.clear();
+    }
+  }
+
+  /**
+   * Sets external listener for testing.
+   *
+   * @param blkMoveListener
+   *          block movement listener callback object
+   */
+  @VisibleForTesting
+  void setBlockMovementListener(BlockMovementListener blkMoveListener) {
+    this.blkMovementListener = blkMoveListener;
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2acc50b8/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/SPSService.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/SPSService.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/SPSService.java
index 71d8fd1..5032377 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/SPSService.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/SPSService.java
@@ -22,8 +22,10 @@ import java.util.List;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
+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.HdfsConstants.StoragePolicySatisfierMode;
-import org.apache.hadoop.hdfs.server.protocol.BlocksStorageMoveAttemptFinished;
 
 /**
  * An interface for SPSService, which exposes life cycle and processing APIs.
@@ -131,11 +133,16 @@ public interface SPSService<T> {
   void markScanCompletedForPath(T spsPath);
 
   /**
-   * Notify the details of storage movement attempt finished blocks.
+   * Given node is reporting that it received a certain movement attempt
+   * finished block.
    *
-   * @param moveAttemptFinishedBlks
-   *          - array contains all the blocks that are attempted to move
+   * @param dnInfo
+   *          - reported datanode
+   * @param storageType
+   *          - storage type
+   * @param block
+   *          - block that is attempted to move
    */
-  void notifyStorageMovementAttemptFinishedBlks(
-      BlocksStorageMoveAttemptFinished moveAttemptFinishedBlks);
+  void notifyStorageMovementAttemptFinishedBlk(DatanodeInfo dnInfo,
+      StorageType storageType, Block block);
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2acc50b8/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/StoragePolicySatisfier.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/StoragePolicySatisfier.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/StoragePolicySatisfier.java
index 1c7a580..cbd6001 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/StoragePolicySatisfier.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/StoragePolicySatisfier.java
@@ -24,9 +24,12 @@ import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
 import java.util.EnumMap;
+import java.util.HashMap;
+import java.util.HashSet;
 import java.util.Iterator;
 import java.util.LinkedList;
 import java.util.List;
+import java.util.Map;
 import java.util.Set;
 
 import org.apache.hadoop.classification.InterfaceAudience;
@@ -50,7 +53,6 @@ import org.apache.hadoop.hdfs.server.balancer.Matcher;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.namenode.ErasureCodingPolicyManager;
 import org.apache.hadoop.hdfs.server.protocol.BlockStorageMovementCommand.BlockMovingInfo;
-import org.apache.hadoop.hdfs.server.protocol.BlocksStorageMoveAttemptFinished;
 import org.apache.hadoop.hdfs.util.StripedBlockUtil;
 import org.apache.hadoop.util.Daemon;
 import org.apache.hadoop.util.StringUtils;
@@ -83,8 +85,6 @@ public class StoragePolicySatisfier<T> implements SPSService<T>, Runnable {
   private BlockStorageMovementNeeded<T> storageMovementNeeded;
   private BlockStorageMovementAttemptedItems<T> storageMovementsMonitor;
   private volatile boolean isRunning = false;
-  private volatile StoragePolicySatisfierMode spsMode =
-      StoragePolicySatisfierMode.NONE;
   private int spsWorkMultiplier;
   private long blockCount = 0L;
   private int blockMovementMaxRetry;
@@ -128,11 +128,12 @@ public class StoragePolicySatisfier<T> implements SPSService<T>, Runnable {
     }
 
     private Status status = null;
-    private List<Block> assignedBlocks = null;
+    private Map<Block, Set<StorageTypeNodePair>> assignedBlocks = null;
 
-    BlocksMovingAnalysis(Status status, List<Block> blockMovingInfo) {
+    BlocksMovingAnalysis(Status status,
+        Map<Block, Set<StorageTypeNodePair>> assignedBlocks) {
       this.status = status;
-      this.assignedBlocks = blockMovingInfo;
+      this.assignedBlocks = assignedBlocks;
     }
   }
 
@@ -164,7 +165,7 @@ public class StoragePolicySatisfier<T> implements SPSService<T>, Runnable {
           serviceMode);
       return;
     }
-    if (spsMode == StoragePolicySatisfierMode.INTERNAL
+    if (serviceMode == StoragePolicySatisfierMode.INTERNAL
         && ctxt.isMoverRunning()) {
       isRunning = false;
       LOG.error(
@@ -175,14 +176,13 @@ public class StoragePolicySatisfier<T> implements SPSService<T>, Runnable {
     }
     if (reconfigStart) {
       LOG.info("Starting {} StoragePolicySatisfier, as admin requested to "
-          + "start it.", StringUtils.toLowerCase(spsMode.toString()));
+          + "start it.", StringUtils.toLowerCase(serviceMode.toString()));
     } else {
       LOG.info("Starting {} StoragePolicySatisfier.",
-          StringUtils.toLowerCase(spsMode.toString()));
+          StringUtils.toLowerCase(serviceMode.toString()));
     }
 
     isRunning = true;
-    this.spsMode = serviceMode;
     // Ensure that all the previously submitted block movements(if any) have to
     // be stopped in all datanodes.
     addDropSPSWorkCommandsToAllDNs();
@@ -297,36 +297,36 @@ public class StoragePolicySatisfier<T> implements SPSService<T>, Runnable {
                 // be removed on storage movement attempt finished report.
               case BLOCKS_TARGETS_PAIRED:
                 if (LOG.isDebugEnabled()) {
-                  LOG.debug("Block analysis status:{} for the file path:{}."
+                  LOG.debug("Block analysis status:{} for the file id:{}."
                       + " Adding to attempt monitor queue for the storage "
                       + "movement attempt finished report",
-                      status.status, fileStatus.getPath());
+                      status.status, fileStatus.getFileId());
                 }
-                this.storageMovementsMonitor.add(new AttemptedItemInfo<T>(
-                    itemInfo.getStartPath(), itemInfo.getFile(), monotonicNow(),
-                    status.assignedBlocks, itemInfo.getRetryCount()));
+                this.storageMovementsMonitor.add(itemInfo.getStartPath(),
+                    itemInfo.getFile(), monotonicNow(), status.assignedBlocks,
+                    itemInfo.getRetryCount());
                 break;
               case NO_BLOCKS_TARGETS_PAIRED:
                 if (LOG.isDebugEnabled()) {
-                  LOG.debug("Adding trackID:{} for the file path:{} back to"
+                  LOG.debug("Adding trackID:{} for the file id:{} back to"
                       + " retry queue as none of the blocks found its eligible"
-                      + " targets.", trackId, fileStatus.getPath());
+                      + " targets.", trackId, fileStatus.getFileId());
                 }
                 retryItem = true;
                 break;
               case FEW_LOW_REDUNDANCY_BLOCKS:
                 if (LOG.isDebugEnabled()) {
-                  LOG.debug("Adding trackID:{} for the file path:{} back to "
+                  LOG.debug("Adding trackID:{} for the file id:{} back to "
                       + "retry queue as some of the blocks are low redundant.",
-                      trackId, fileStatus.getPath());
+                      trackId, fileStatus.getFileId());
                 }
                 retryItem = true;
                 break;
               case BLOCKS_FAILED_TO_MOVE:
                 if (LOG.isDebugEnabled()) {
-                  LOG.debug("Adding trackID:{} for the file path:{} back to "
+                  LOG.debug("Adding trackID:{} for the file id:{} back to "
                       + "retry queue as some of the blocks movement failed.",
-                      trackId, fileStatus.getPath());
+                      trackId, fileStatus.getFileId());
                 }
                 retryItem = true;
                 break;
@@ -334,9 +334,9 @@ public class StoragePolicySatisfier<T> implements SPSService<T>, Runnable {
               case BLOCKS_TARGET_PAIRING_SKIPPED:
               case BLOCKS_ALREADY_SATISFIED:
               default:
-                LOG.info("Block analysis status:{} for the file path:{}."
+                LOG.info("Block analysis status:{} for the file id:{}."
                     + " So, Cleaning up the Xattrs.", status.status,
-                    fileStatus.getPath());
+                    fileStatus.getFileId());
                 storageMovementNeeded.removeItemTrackInfo(itemInfo, true);
                 break;
               }
@@ -389,19 +389,19 @@ public class StoragePolicySatisfier<T> implements SPSService<T>, Runnable {
     if (!lastBlkComplete) {
       // Postpone, currently file is under construction
       LOG.info("File: {} is under construction. So, postpone"
-          + " this to the next retry iteration", fileInfo.getPath());
+          + " this to the next retry iteration", fileInfo.getFileId());
       return new BlocksMovingAnalysis(
           BlocksMovingAnalysis.Status.ANALYSIS_SKIPPED_FOR_RETRY,
-          new ArrayList<>());
+          new HashMap<>());
     }
 
     List<LocatedBlock> blocks = locatedBlocks.getLocatedBlocks();
     if (blocks.size() == 0) {
       LOG.info("File: {} is not having any blocks."
-          + " So, skipping the analysis.", fileInfo.getPath());
+          + " So, skipping the analysis.", fileInfo.getFileId());
       return new BlocksMovingAnalysis(
           BlocksMovingAnalysis.Status.BLOCKS_TARGET_PAIRING_SKIPPED,
-          new ArrayList<>());
+          new HashMap<>());
     }
     List<BlockMovingInfo> blockMovingInfos = new ArrayList<BlockMovingInfo>();
     boolean hasLowRedundancyBlocks = false;
@@ -432,7 +432,7 @@ public class StoragePolicySatisfier<T> implements SPSService<T>, Runnable {
               + "So, ignoring to move the blocks");
           return new BlocksMovingAnalysis(
               BlocksMovingAnalysis.Status.BLOCKS_TARGET_PAIRING_SKIPPED,
-              new ArrayList<>());
+              new HashMap<>());
         }
       } else {
         expectedStorageTypes = existingStoragePolicy
@@ -465,13 +465,21 @@ public class StoragePolicySatisfier<T> implements SPSService<T>, Runnable {
         && status != BlocksMovingAnalysis.Status.BLOCKS_TARGETS_PAIRED) {
       status = BlocksMovingAnalysis.Status.FEW_LOW_REDUNDANCY_BLOCKS;
     }
-    List<Block> assignedBlockIds = new ArrayList<Block>();
+    Map<Block, Set<StorageTypeNodePair>> assignedBlocks = new HashMap<>();
     for (BlockMovingInfo blkMovingInfo : blockMovingInfos) {
       // Check for at least one block storage movement has been chosen
       try {
         blockMoveTaskHandler.submitMoveTask(blkMovingInfo);
         LOG.debug("BlockMovingInfo: {}", blkMovingInfo);
-        assignedBlockIds.add(blkMovingInfo.getBlock());
+        StorageTypeNodePair nodeStorage = new StorageTypeNodePair(
+            blkMovingInfo.getTargetStorageType(), blkMovingInfo.getTarget());
+        Set<StorageTypeNodePair> nodesWithStorage = assignedBlocks
+            .get(blkMovingInfo.getBlock());
+        if (nodesWithStorage == null) {
+          nodesWithStorage = new HashSet<>();
+          assignedBlocks.put(blkMovingInfo.getBlock(), nodesWithStorage);
+        }
+        nodesWithStorage.add(nodeStorage);
         blockCount++;
       } catch (IOException e) {
         LOG.warn("Exception while scheduling movement task", e);
@@ -479,7 +487,7 @@ public class StoragePolicySatisfier<T> implements SPSService<T>, Runnable {
         status = BlocksMovingAnalysis.Status.BLOCKS_FAILED_TO_MOVE;
       }
     }
-    return new BlocksMovingAnalysis(status, assignedBlockIds);
+    return new BlocksMovingAnalysis(status, assignedBlocks);
   }
 
   /**
@@ -545,6 +553,11 @@ public class StoragePolicySatisfier<T> implements SPSService<T>, Runnable {
           new ArrayList<StorageTypeNodePair>();
       List<DatanodeInfo> existingBlockStorages = new ArrayList<DatanodeInfo>(
           Arrays.asList(storages));
+
+      // Add existing storages into exclude nodes to avoid choosing this as
+      // remote target later.
+      List<DatanodeInfo> excludeNodes = new ArrayList<>(existingBlockStorages);
+
       // if expected type exists in source node already, local movement would be
       // possible, so lets find such sources first.
       Iterator<DatanodeInfo> iterator = existingBlockStorages.iterator();
@@ -582,7 +595,7 @@ public class StoragePolicySatisfier<T> implements SPSService<T>, Runnable {
       foundMatchingTargetNodesForBlock |= findSourceAndTargetToMove(
           blockMovingInfos, blockInfo, sourceWithStorageMap,
           expectedStorageTypes, targetDns,
-          ecPolicy);
+          ecPolicy, excludeNodes);
     }
     return foundMatchingTargetNodesForBlock;
   }
@@ -601,6 +614,10 @@ public class StoragePolicySatisfier<T> implements SPSService<T>, Runnable {
    *          - Expecting storages to move
    * @param targetDns
    *          - Available DNs for expected storage types
+   * @param ecPolicy
+   *          - erasure coding policy of sps invoked file
+   * @param excludeNodes
+   *          - existing source nodes, which has replica copy
    * @return false if some of the block locations failed to find target node to
    *         satisfy the storage policy
    */
@@ -609,9 +626,8 @@ public class StoragePolicySatisfier<T> implements SPSService<T>, Runnable {
       List<StorageTypeNodePair> sourceWithStorageList,
       List<StorageType> expectedTypes,
       EnumMap<StorageType, List<DatanodeWithStorage.StorageDetails>> targetDns,
-      ErasureCodingPolicy ecPolicy) {
+      ErasureCodingPolicy ecPolicy, List<DatanodeInfo> excludeNodes) {
     boolean foundMatchingTargetNodesForBlock = true;
-    List<DatanodeInfo> 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
@@ -638,10 +654,12 @@ public class StoragePolicySatisfier<T> implements SPSService<T>, Runnable {
           expectedTypes.remove(chosenTarget.storageType);
         }
       }
-      // To avoid choosing this excludeNodes as targets later
-      excludeNodes.add(existingTypeNodePair.dn);
     }
-
+    // If all the sources and targets are paired within same node, then simply
+    // return.
+    if (expectedTypes.size() <= 0) {
+      return foundMatchingTargetNodesForBlock;
+    }
     // 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++) {
@@ -824,14 +842,29 @@ public class StoragePolicySatisfier<T> implements SPSService<T>, Runnable {
   /**
    * Keeps datanode with its respective storage type.
    */
-  private static final class StorageTypeNodePair {
+  static final class StorageTypeNodePair {
     private final StorageType storageType;
     private final DatanodeInfo dn;
 
-    private StorageTypeNodePair(StorageType storageType, DatanodeInfo dn) {
+    StorageTypeNodePair(StorageType storageType, DatanodeInfo dn) {
       this.storageType = storageType;
       this.dn = dn;
     }
+
+    public DatanodeInfo getDatanodeInfo() {
+      return dn;
+    }
+
+    public StorageType getStorageType() {
+      return storageType;
+    }
+
+    @Override
+    public String toString() {
+      return new StringBuilder().append("StorageTypeNodePair(\n  ")
+          .append("DatanodeInfo: ").append(dn).append(", StorageType: ")
+          .append(storageType).toString();
+    }
   }
 
   private EnumMap<StorageType, List<DatanodeWithStorage.StorageDetails>>
@@ -1043,18 +1076,19 @@ public class StoragePolicySatisfier<T> implements SPSService<T>, Runnable {
   }
 
   /**
-   * Receives set of storage movement attempt finished blocks report.
+   * Receives storage movement attempt finished block report.
    *
-   * @param moveAttemptFinishedBlks
-   *          set of storage movement attempt finished blocks.
+   * @param dnInfo
+   *          reported datanode
+   * @param storageType
+   *          - storage type
+   * @param block
+   *          movement attempt finished block.
    */
-  public void notifyStorageMovementAttemptFinishedBlks(
-      BlocksStorageMoveAttemptFinished moveAttemptFinishedBlks) {
-    if (moveAttemptFinishedBlks.getBlocks().length <= 0) {
-      return;
-    }
-    storageMovementsMonitor
-        .notifyMovementTriedBlocks(moveAttemptFinishedBlks.getBlocks());
+  @Override
+  public void notifyStorageMovementAttemptFinishedBlk(DatanodeInfo dnInfo,
+      StorageType storageType, Block block) {
+    storageMovementsMonitor.notifyReportedBlock(dnInfo, storageType, block);
   }
 
   @VisibleForTesting
@@ -1086,7 +1120,7 @@ public class StoragePolicySatisfier<T> implements SPSService<T>, Runnable {
    */
   final static class AttemptedItemInfo<T> extends ItemInfo<T> {
     private long lastAttemptedOrReportedTime;
-    private final List<Block> blocks;
+    private final Set<Block> blocks;
 
     /**
      * AttemptedItemInfo constructor.
@@ -1097,10 +1131,14 @@ public class StoragePolicySatisfier<T> implements SPSService<T>, Runnable {
      *          trackId for file.
      * @param lastAttemptedOrReportedTime
      *          last attempted or reported time
+     * @param blocks
+     *          scheduled blocks
+     * @param retryCount
+     *          file retry count
      */
     AttemptedItemInfo(T rootId, T trackId,
         long lastAttemptedOrReportedTime,
-        List<Block> blocks, int retryCount) {
+        Set<Block> blocks, int retryCount) {
       super(rootId, trackId, retryCount);
       this.lastAttemptedOrReportedTime = lastAttemptedOrReportedTime;
       this.blocks = blocks;
@@ -1121,10 +1159,9 @@ public class StoragePolicySatisfier<T> implements SPSService<T>, Runnable {
       this.lastAttemptedOrReportedTime = monotonicNow();
     }
 
-    List<Block> getBlocks() {
+    Set<Block> getBlocks() {
       return this.blocks;
     }
-
   }
 
   /**
@@ -1241,4 +1278,15 @@ public class StoragePolicySatisfier<T> implements SPSService<T>, Runnable {
         "It should be a positive, non-zero integer value.");
     return spsWorkMultiplier;
   }
+
+  /**
+   * Sets external listener for testing.
+   *
+   * @param blkMovementListener
+   *          block movement listener callback object
+   */
+  @VisibleForTesting
+  void setBlockMovementListener(BlockMovementListener blkMovementListener) {
+    storageMovementsMonitor.setBlockMovementListener(blkMovementListener);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2acc50b8/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 fcc2df1..311b68f 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,7 +112,6 @@ 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 storageMovFinishedBlks array of movement attempt finished blocks
    * @throws IOException on error
    */
   @Idempotent
@@ -126,8 +125,7 @@ public interface DatanodeProtocol {
                                        VolumeFailureSummary volumeFailureSummary,
                                        boolean requestFullBlockReportLease,
                                        @Nonnull SlowPeerReports slowPeers,
-                                       @Nonnull SlowDiskReports slowDisks,
-                                       BlocksStorageMoveAttemptFinished storageMovFinishedBlks)
+                                       @Nonnull SlowDiskReports slowDisks)
       throws IOException;
 
   /**


---------------------------------------------------------------------
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-13166: [SPS]: Implement caching mechanism to keep LIVE datanodes to minimize costly getLiveDatanodeStorageReport() calls. Contributed by Rakesh R.

Posted by um...@apache.org.
HDFS-13166: [SPS]: Implement caching mechanism to keep LIVE datanodes to minimize costly getLiveDatanodeStorageReport() calls. 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/75ccc139
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/75ccc139
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/75ccc139

Branch: refs/heads/HDFS-10285
Commit: 75ccc1396b677777cdc0d4992a4af3911f9f88c2
Parents: 8467ec2
Author: Surendra Singh Lilhore <su...@apache.org>
Authored: Thu Mar 1 00:08:37 2018 +0530
Committer: Uma Maheswara Rao Gangumalla <um...@apache.org>
Committed: Sun Aug 12 03:06:05 2018 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/hdfs/DFSConfigKeys.java   |   5 +
 .../NamenodeProtocolServerSideTranslatorPB.java |  19 --
 .../NamenodeProtocolTranslatorPB.java           |  17 -
 .../hdfs/server/namenode/NameNodeRpcServer.java |  13 -
 .../hdfs/server/namenode/sps/Context.java       |  24 +-
 .../namenode/sps/DatanodeCacheManager.java      | 121 +++++++
 .../namenode/sps/IntraSPSNameNodeContext.java   |  23 +-
 .../namenode/sps/StoragePolicySatisfier.java    | 340 ++++++++++++++-----
 .../hdfs/server/protocol/NamenodeProtocol.java  |  16 -
 .../hdfs/server/sps/ExternalSPSContext.java     |  32 +-
 .../src/main/proto/NamenodeProtocol.proto       |  25 --
 .../src/main/resources/hdfs-default.xml         |  11 +
 .../src/site/markdown/ArchivalStorage.md        |   2 +-
 .../TestStoragePolicySatisfyWorker.java         |   3 +
 .../TestPersistentStoragePolicySatisfier.java   |   6 +
 .../TestStoragePolicySatisfierWithHA.java       |   3 +
 .../sps/TestStoragePolicySatisfier.java         |   4 +
 ...stStoragePolicySatisfierWithStripedFile.java |  24 +-
 .../TestStoragePolicySatisfyAdminCommands.java  |   3 +
 19 files changed, 431 insertions(+), 260 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/75ccc139/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 b354d64..cf383d0 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
@@ -646,6 +646,11 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
   public static final String DFS_SPS_MAX_OUTSTANDING_PATHS_KEY =
       "dfs.storage.policy.satisfier.max.outstanding.paths";
   public static final int DFS_SPS_MAX_OUTSTANDING_PATHS_DEFAULT = 10000;
+  // SPS datanode cache config, defaulting to 5mins.
+  public static final String DFS_SPS_DATANODE_CACHE_REFRESH_INTERVAL_MS =
+      "dfs.storage.policy.satisfier.datanode.cache.refresh.interval.ms";
+  public static final long DFS_SPS_DATANODE_CACHE_REFRESH_INTERVAL_MS_DEFAULT =
+      300000L;
 
   // SPS keytab configurations, by default it is disabled.
   public static final String  DFS_SPS_ADDRESS_KEY =

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75ccc139/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/NamenodeProtocolServerSideTranslatorPB.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/NamenodeProtocolServerSideTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/NamenodeProtocolServerSideTranslatorPB.java
index ed176cc..e4283c6 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/NamenodeProtocolServerSideTranslatorPB.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/NamenodeProtocolServerSideTranslatorPB.java
@@ -23,8 +23,6 @@ import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo.DatanodeInfoBuilder;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsServerProtos.VersionRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsServerProtos.VersionResponseProto;
-import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.CheckDNSpaceRequestProto;
-import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.CheckDNSpaceResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.EndCheckpointRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.EndCheckpointResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.ErrorReportRequestProto;
@@ -277,21 +275,4 @@ public class NamenodeProtocolServerSideTranslatorPB implements
       throw new ServiceException(e);
     }
   }
-
-  @Override
-  public CheckDNSpaceResponseProto checkDNSpaceForScheduling(
-      RpcController controller, CheckDNSpaceRequestProto request)
-          throws ServiceException {
-    try {
-      CheckDNSpaceResponseProto build = CheckDNSpaceResponseProto.newBuilder()
-          .setIsGoodDatanodeWithSpace(impl.checkDNSpaceForScheduling(
-              PBHelperClient.convert(request.getDnInfo()),
-              PBHelperClient.convertStorageType(request.getStorageType()),
-              request.getEstimatedSize()))
-          .build();
-      return build;
-    } catch (IOException e) {
-      throw new ServiceException(e);
-    }
-  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75ccc139/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/NamenodeProtocolTranslatorPB.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/NamenodeProtocolTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/NamenodeProtocolTranslatorPB.java
index d2e97a2..97dee9b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/NamenodeProtocolTranslatorPB.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/NamenodeProtocolTranslatorPB.java
@@ -22,12 +22,10 @@ import java.io.IOException;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsServerProtos.NamenodeCommandProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsServerProtos.VersionRequestProto;
-import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.CheckDNSpaceRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.EndCheckpointRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.ErrorReportRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetBlockKeysRequestProto;
@@ -280,19 +278,4 @@ public class NamenodeProtocolTranslatorPB implements NamenodeProtocol,
       throw ProtobufHelper.getRemoteException(e);
     }
   }
-
-  @Override
-  public boolean checkDNSpaceForScheduling(DatanodeInfo dn, StorageType type,
-      long estimatedSize) throws IOException {
-    CheckDNSpaceRequestProto req = CheckDNSpaceRequestProto.newBuilder()
-        .setDnInfo(PBHelperClient.convert(dn))
-        .setStorageType(PBHelperClient.convertStorageType(type))
-        .setEstimatedSize(estimatedSize).build();
-    try {
-      return rpcProxy.checkDNSpaceForScheduling(NULL_CONTROLLER, req)
-          .getIsGoodDatanodeWithSpace();
-    } catch (ServiceException e) {
-      throw ProtobufHelper.getRemoteException(e);
-    }
-  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75ccc139/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 6fe38d6..1590423 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
@@ -2583,17 +2583,4 @@ public class NameNodeRpcServer implements NamenodeProtocols {
     }
     return namesystem.getFilePath(pathId);
   }
-
-  @Override
-  public boolean checkDNSpaceForScheduling(DatanodeInfo dn,
-      StorageType type, long estimatedSize) throws IOException {
-    checkNNStartup();
-    String operationName = "checkDNSpaceForScheduling";
-    namesystem.checkSuperuserPrivilege(operationName);
-    if (nn.isStandbyState()) {
-      throw new StandbyException("Not supported by Standby Namenode.");
-    }
-    return namesystem.getBlockManager().getDatanodeManager()
-        .verifyTargetDatanodeHasSpaceForScheduling(dn, type, estimatedSize);
-  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75ccc139/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/Context.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/Context.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/Context.java
index 84a969d..55a1f7a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/Context.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/Context.java
@@ -23,11 +23,10 @@ import java.io.IOException;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.fs.ParentNotDirectoryException;
-import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.fs.UnresolvedLinkException;
 import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
-import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
+import org.apache.hadoop.hdfs.server.namenode.sps.StoragePolicySatisfier.DatanodeMap;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeStorageReport;
 import org.apache.hadoop.net.NetworkTopology;
 import org.apache.hadoop.security.AccessControlException;
@@ -72,9 +71,12 @@ public interface Context<T> {
   /**
    * Gets the network topology.
    *
+   * @param datanodeMap
+   *          target datanodes
+   *
    * @return network topology
    */
-  NetworkTopology getNetworkTopology();
+  NetworkTopology getNetworkTopology(DatanodeMap datanodeMap);
 
   /**
    * Returns true if the give file exists in the Namespace.
@@ -133,22 +135,6 @@ public interface Context<T> {
       throws IOException;
 
   /**
-   * Checks whether the given datanode has sufficient space to occupy the given
-   * blockSize data.
-   *
-   * @param dn
-   *          datanode info
-   * @param type
-   *          storage type
-   * @param blockSize
-   *          blockSize to be scheduled
-   * @return true if the given datanode has sufficient space to occupy blockSize
-   *         data, false otherwise.
-   */
-  boolean checkDNSpaceForScheduling(DatanodeInfo dn, StorageType type,
-      long blockSize);
-
-  /**
    * @return next SPS path info to process.
    */
   T getNextSPSPath();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75ccc139/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/DatanodeCacheManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/DatanodeCacheManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/DatanodeCacheManager.java
new file mode 100644
index 0000000..3531ecd
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/DatanodeCacheManager.java
@@ -0,0 +1,121 @@
+/**
+ * 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.sps;
+
+import java.io.IOException;
+import java.util.ArrayList;
+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.server.namenode.sps.StoragePolicySatisfier.DatanodeMap;
+import org.apache.hadoop.hdfs.server.protocol.DatanodeStorageReport;
+import org.apache.hadoop.hdfs.server.protocol.StorageReport;
+import org.apache.hadoop.net.NetworkTopology;
+import org.apache.hadoop.util.Time;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * The Datanode cache Manager handles caching of {@link DatanodeStorageReport}.
+ *
+ * This class is instantiated by StoragePolicySatisifer. It maintains the array
+ * of datanode storage reports. It has a configurable refresh interval and
+ * periodically refresh the datanode cache by fetching latest
+ * {@link Context#getLiveDatanodeStorageReport()} once it reaches refresh
+ * interval.
+ */
+@InterfaceAudience.Private
+public class DatanodeCacheManager<T> {
+  private static final Logger LOG = LoggerFactory
+      .getLogger(DatanodeCacheManager.class);
+
+  private final DatanodeMap datanodeMap;
+  private NetworkTopology cluster;
+
+  /**
+   * Interval between scans in milliseconds.
+   */
+  private final long refreshIntervalMs;
+
+  private long lastAccessedTime;
+
+  public DatanodeCacheManager(Configuration conf) {
+    refreshIntervalMs = conf.getLong(
+        DFSConfigKeys.DFS_SPS_DATANODE_CACHE_REFRESH_INTERVAL_MS,
+        DFSConfigKeys.DFS_SPS_DATANODE_CACHE_REFRESH_INTERVAL_MS_DEFAULT);
+
+    LOG.info("DatanodeCacheManager refresh interval is {} milliseconds",
+        refreshIntervalMs);
+    datanodeMap = new DatanodeMap();
+  }
+
+  /**
+   * Returns the live datanodes and its storage details, which has available
+   * space (> 0) to schedule block moves. This will return array of datanodes
+   * from its local cache. It has a configurable refresh interval in millis and
+   * periodically refresh the datanode cache by fetching latest
+   * {@link Context#getLiveDatanodeStorageReport()} once it elapsed refresh
+   * interval.
+   *
+   * @throws IOException
+   */
+  public DatanodeMap getLiveDatanodeStorageReport(
+      Context<T> spsContext) throws IOException {
+    long now = Time.monotonicNow();
+    long elapsedTimeMs = now - lastAccessedTime;
+    boolean refreshNeeded = elapsedTimeMs >= refreshIntervalMs;
+    lastAccessedTime = now;
+    if (refreshNeeded) {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("elapsedTimeMs > refreshIntervalMs : {} > {},"
+            + " so refreshing cache", elapsedTimeMs, refreshIntervalMs);
+      }
+      datanodeMap.reset(); // clear all previously cached items.
+
+      // Fetch live datanodes from namenode and prepare DatanodeMap.
+      DatanodeStorageReport[] liveDns = spsContext
+          .getLiveDatanodeStorageReport();
+      for (DatanodeStorageReport storage : liveDns) {
+        StorageReport[] storageReports = storage.getStorageReports();
+        List<StorageType> storageTypes = new ArrayList<>();
+        List<Long> remainingSizeList = new ArrayList<>();
+        for (StorageReport t : storageReports) {
+          if (t.getRemaining() > 0) {
+            storageTypes.add(t.getStorage().getStorageType());
+            remainingSizeList.add(t.getRemaining());
+          }
+        }
+        datanodeMap.addTarget(storage.getDatanodeInfo(), storageTypes,
+            remainingSizeList);
+      }
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("LIVE datanodes: {}", datanodeMap);
+      }
+      // get network topology
+      cluster = spsContext.getNetworkTopology(datanodeMap);
+    }
+    return datanodeMap;
+  }
+
+  NetworkTopology getCluster() {
+    return cluster;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75ccc139/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/IntraSPSNameNodeContext.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/IntraSPSNameNodeContext.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/IntraSPSNameNodeContext.java
index f6b6d95..a77fe85 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/IntraSPSNameNodeContext.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/IntraSPSNameNodeContext.java
@@ -24,18 +24,16 @@ import java.io.IOException;
 import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.fs.ParentNotDirectoryException;
-import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.fs.UnresolvedLinkException;
 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.HdfsFileStatus;
 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.namenode.INode;
 import org.apache.hadoop.hdfs.server.namenode.Namesystem;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeStorageReport;
+import org.apache.hadoop.hdfs.server.namenode.sps.StoragePolicySatisfier.DatanodeMap;
 import org.apache.hadoop.net.NetworkTopology;
 import org.apache.hadoop.security.AccessControlException;
 import org.slf4j.Logger;
@@ -136,7 +134,7 @@ public class IntraSPSNameNodeContext implements Context<Long> {
   }
 
   @Override
-  public NetworkTopology getNetworkTopology() {
+  public NetworkTopology getNetworkTopology(DatanodeMap datanodeMap) {
     return blockManager.getDatanodeManager().getNetworkTopology();
   }
 
@@ -153,23 +151,6 @@ public class IntraSPSNameNodeContext implements Context<Long> {
   }
 
   @Override
-  public boolean checkDNSpaceForScheduling(DatanodeInfo dn, StorageType type,
-      long blockSize) {
-    namesystem.readLock();
-    try {
-      DatanodeDescriptor datanode = blockManager.getDatanodeManager()
-          .getDatanode(dn.getDatanodeUuid());
-      if (datanode == null) {
-        LOG.debug("Target datanode: " + dn + " doesn't exists");
-        return false;
-      }
-      return null != datanode.chooseStorage4Block(type, blockSize);
-    } finally {
-      namesystem.readUnlock();
-    }
-  }
-
-  @Override
   public Long getNextSPSPath() {
     return blockManager.getSPSManager().getNextPathId();
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75ccc139/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/StoragePolicySatisfier.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/StoragePolicySatisfier.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/StoragePolicySatisfier.java
index 08a26e1..1c7a580 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/StoragePolicySatisfier.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/StoragePolicySatisfier.java
@@ -27,6 +27,7 @@ import java.util.EnumMap;
 import java.util.Iterator;
 import java.util.LinkedList;
 import java.util.List;
+import java.util.Set;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
@@ -50,8 +51,6 @@ import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.namenode.ErasureCodingPolicyManager;
 import org.apache.hadoop.hdfs.server.protocol.BlockStorageMovementCommand.BlockMovingInfo;
 import org.apache.hadoop.hdfs.server.protocol.BlocksStorageMoveAttemptFinished;
-import org.apache.hadoop.hdfs.server.protocol.DatanodeStorageReport;
-import org.apache.hadoop.hdfs.server.protocol.StorageReport;
 import org.apache.hadoop.hdfs.util.StripedBlockUtil;
 import org.apache.hadoop.util.Daemon;
 import org.apache.hadoop.util.StringUtils;
@@ -92,10 +91,12 @@ public class StoragePolicySatisfier<T> implements SPSService<T>, Runnable {
   private Context<T> ctxt;
   private BlockMoveTaskHandler blockMoveTaskHandler;
   private final Configuration conf;
+  private DatanodeCacheManager<T> dnCacheMgr;
 
   public StoragePolicySatisfier(Configuration conf) {
     this.conf = conf;
   }
+
   /**
    * Represents the collective analysis status for all blocks.
    */
@@ -190,6 +191,7 @@ public class StoragePolicySatisfier<T> implements SPSService<T>, Runnable {
     storagePolicySatisfierThread.start();
     this.storageMovementsMonitor.start();
     this.storageMovementNeeded.activate();
+    dnCacheMgr = new DatanodeCacheManager<T>(conf);
   }
 
   @Override
@@ -271,7 +273,6 @@ public class StoragePolicySatisfier<T> implements SPSService<T>, Runnable {
             }
             T trackId = itemInfo.getFile();
             BlocksMovingAnalysis status = null;
-            DatanodeStorageReport[] liveDnReports;
             BlockStoragePolicy existingStoragePolicy;
             // TODO: presently, context internally acquire the lock
             // and returns the result. Need to discuss to move the lock outside?
@@ -282,14 +283,13 @@ public class StoragePolicySatisfier<T> implements SPSService<T>, Runnable {
               // just remove trackId from the queue
               storageMovementNeeded.removeItemTrackInfo(itemInfo, true);
             } else {
-              liveDnReports = ctxt.getLiveDatanodeStorageReport();
               byte existingStoragePolicyID = fileStatus.getStoragePolicy();
               existingStoragePolicy = ctxt
                   .getStoragePolicy(existingStoragePolicyID);
 
               HdfsLocatedFileStatus file = (HdfsLocatedFileStatus) fileStatus;
               status = analyseBlocksStorageMovementsAndAssignToDN(file,
-                  existingStoragePolicy, liveDnReports);
+                  existingStoragePolicy);
               switch (status.status) {
               // Just add to monitor, so it will be retried after timeout
               case ANALYSIS_SKIPPED_FOR_RETRY:
@@ -380,8 +380,7 @@ public class StoragePolicySatisfier<T> implements SPSService<T>, Runnable {
 
   private BlocksMovingAnalysis analyseBlocksStorageMovementsAndAssignToDN(
       HdfsLocatedFileStatus fileInfo,
-      BlockStoragePolicy existingStoragePolicy,
-      DatanodeStorageReport[] liveDns) {
+      BlockStoragePolicy existingStoragePolicy) throws IOException {
     BlocksMovingAnalysis.Status status =
         BlocksMovingAnalysis.Status.BLOCKS_ALREADY_SATISFIED;
     final ErasureCodingPolicy ecPolicy = fileInfo.getErasureCodingPolicy();
@@ -407,6 +406,7 @@ public class StoragePolicySatisfier<T> implements SPSService<T>, Runnable {
     List<BlockMovingInfo> blockMovingInfos = new ArrayList<BlockMovingInfo>();
     boolean hasLowRedundancyBlocks = false;
     int replication = fileInfo.getReplication();
+    DatanodeMap liveDns = dnCacheMgr.getLiveDatanodeStorageReport(ctxt);
     for (int i = 0; i < blocks.size(); i++) {
       LocatedBlock blockInfo = blocks.get(i);
 
@@ -462,7 +462,7 @@ public class StoragePolicySatisfier<T> implements SPSService<T>, Runnable {
     // If there is no block paired and few blocks are low redundant, so marking
     // the status as FEW_LOW_REDUNDANCY_BLOCKS.
     if (hasLowRedundancyBlocks
-        && status == BlocksMovingAnalysis.Status.NO_BLOCKS_TARGETS_PAIRED) {
+        && status != BlocksMovingAnalysis.Status.BLOCKS_TARGETS_PAIRED) {
       status = BlocksMovingAnalysis.Status.FEW_LOW_REDUNDANCY_BLOCKS;
     }
     List<Block> assignedBlockIds = new ArrayList<Block>();
@@ -526,13 +526,17 @@ public class StoragePolicySatisfier<T> implements SPSService<T>, Runnable {
    *          - list to get existing storage types
    * @param storages
    *          - available storages
+   * @param liveDns
+   *          - live datanodes which can be used as targets
+   * @param ecPolicy
+   *          - ec policy of sps invoked file
    * @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, LocatedBlock blockInfo,
       List<StorageType> expectedStorageTypes, List<StorageType> existing,
-      DatanodeInfo[] storages, DatanodeStorageReport[] liveDns,
+      DatanodeInfo[] storages, DatanodeMap liveDns,
       ErasureCodingPolicy ecPolicy) {
     boolean foundMatchingTargetNodesForBlock = true;
     if (!removeOverlapBetweenStorageTypes(expectedStorageTypes,
@@ -572,12 +576,12 @@ public class StoragePolicySatisfier<T> implements SPSService<T>, Runnable {
         }
       }
 
-      StorageTypeNodeMap locsForExpectedStorageTypes =
+      EnumMap<StorageType, List<DatanodeWithStorage.StorageDetails>> targetDns =
           findTargetsForExpectedStorageTypes(expectedStorageTypes, liveDns);
 
       foundMatchingTargetNodesForBlock |= findSourceAndTargetToMove(
           blockMovingInfos, blockInfo, sourceWithStorageMap,
-          expectedStorageTypes, locsForExpectedStorageTypes,
+          expectedStorageTypes, targetDns,
           ecPolicy);
     }
     return foundMatchingTargetNodesForBlock;
@@ -593,9 +597,9 @@ public class StoragePolicySatisfier<T> implements SPSService<T>, Runnable {
    *          - Block
    * @param sourceWithStorageList
    *          - Source Datanode with storages list
-   * @param expected
+   * @param expectedTypes
    *          - Expecting storages to move
-   * @param locsForExpectedStorageTypes
+   * @param targetDns
    *          - Available DNs for expected storage types
    * @return false if some of the block locations failed to find target node to
    *         satisfy the storage policy
@@ -603,8 +607,8 @@ public class StoragePolicySatisfier<T> implements SPSService<T>, Runnable {
   private boolean findSourceAndTargetToMove(
       List<BlockMovingInfo> blockMovingInfos, LocatedBlock blockInfo,
       List<StorageTypeNodePair> sourceWithStorageList,
-      List<StorageType> expected,
-      StorageTypeNodeMap locsForExpectedStorageTypes,
+      List<StorageType> expectedTypes,
+      EnumMap<StorageType, List<DatanodeWithStorage.StorageDetails>> targetDns,
       ErasureCodingPolicy ecPolicy) {
     boolean foundMatchingTargetNodesForBlock = true;
     List<DatanodeInfo> excludeNodes = new ArrayList<>();
@@ -617,9 +621,9 @@ public class StoragePolicySatisfier<T> implements SPSService<T>, Runnable {
 
       // Check whether the block replica is already placed in the expected
       // storage type in this source datanode.
-      if (!expected.contains(existingTypeNodePair.storageType)) {
+      if (!expectedTypes.contains(existingTypeNodePair.storageType)) {
         StorageTypeNodePair chosenTarget = chooseTargetTypeInSameNode(blockInfo,
-            existingTypeNodePair.dn, expected);
+            existingTypeNodePair.dn, targetDns, expectedTypes);
         if (chosenTarget != null) {
           if (blockInfo.isStriped()) {
             buildStripedBlockMovingInfos(blockInfo, existingTypeNodePair.dn,
@@ -631,7 +635,7 @@ public class StoragePolicySatisfier<T> implements SPSService<T>, Runnable {
                 existingTypeNodePair.storageType, chosenTarget.dn,
                 chosenTarget.storageType, blockMovingInfos);
           }
-          expected.remove(chosenTarget.storageType);
+          expectedTypes.remove(chosenTarget.storageType);
         }
       }
       // To avoid choosing this excludeNodes as targets later
@@ -648,24 +652,23 @@ public class StoragePolicySatisfier<T> implements SPSService<T>, Runnable {
       if (checkIfAlreadyChosen(blockMovingInfos, existingTypeNodePair.dn)) {
         continue;
       }
-      if (chosenTarget == null && ctxt
-          .getNetworkTopology().isNodeGroupAware()) {
+      if (chosenTarget == null && dnCacheMgr.getCluster().isNodeGroupAware()) {
         chosenTarget = chooseTarget(blockInfo, existingTypeNodePair.dn,
-            expected, Matcher.SAME_NODE_GROUP, locsForExpectedStorageTypes,
+            expectedTypes, Matcher.SAME_NODE_GROUP, targetDns,
             excludeNodes);
       }
 
       // Then, match nodes on the same rack
       if (chosenTarget == null) {
         chosenTarget =
-            chooseTarget(blockInfo, existingTypeNodePair.dn, expected,
-                Matcher.SAME_RACK, locsForExpectedStorageTypes, excludeNodes);
+            chooseTarget(blockInfo, existingTypeNodePair.dn, expectedTypes,
+                Matcher.SAME_RACK, targetDns, excludeNodes);
       }
 
       if (chosenTarget == null) {
         chosenTarget =
-            chooseTarget(blockInfo, existingTypeNodePair.dn, expected,
-                Matcher.ANY_OTHER, locsForExpectedStorageTypes, excludeNodes);
+            chooseTarget(blockInfo, existingTypeNodePair.dn, expectedTypes,
+                Matcher.ANY_OTHER, targetDns, excludeNodes);
       }
       if (null != chosenTarget) {
         if (blockInfo.isStriped()) {
@@ -678,17 +681,17 @@ public class StoragePolicySatisfier<T> implements SPSService<T>, Runnable {
               chosenTarget.storageType, blockMovingInfos);
         }
 
-        expected.remove(chosenTarget.storageType);
+        expectedTypes.remove(chosenTarget.storageType);
         excludeNodes.add(chosenTarget.dn);
       } else {
         LOG.warn(
             "Failed to choose target datanode for the required"
                 + " storage types {}, block:{}, existing storage type:{}",
-            expected, blockInfo, existingTypeNodePair.storageType);
+            expectedTypes, blockInfo, existingTypeNodePair.storageType);
       }
     }
 
-    if (expected.size() > 0) {
+    if (expectedTypes.size() > 0) {
       foundMatchingTargetNodesForBlock = false;
     }
 
@@ -750,20 +753,38 @@ public class StoragePolicySatisfier<T> implements SPSService<T>, Runnable {
   /**
    * Choose the target storage within same datanode if possible.
    *
-   * @param block
+   * @param blockInfo
    *          - block info
    * @param source
    *          - source datanode
+   * @param targetDns
+   *          - set of target datanodes with its respective storage type
    * @param targetTypes
    *          - list of target storage types
    */
   private StorageTypeNodePair chooseTargetTypeInSameNode(LocatedBlock blockInfo,
-      DatanodeInfo source, List<StorageType> targetTypes) {
+      DatanodeInfo source,
+      EnumMap<StorageType, List<DatanodeWithStorage.StorageDetails>> targetDns,
+      List<StorageType> targetTypes) {
     for (StorageType t : targetTypes) {
-      boolean goodTargetDn =
-          ctxt.checkDNSpaceForScheduling(source, t, blockInfo.getBlockSize());
-      if (goodTargetDn) {
-        return new StorageTypeNodePair(t, source);
+      List<DatanodeWithStorage.StorageDetails> targetNodeStorages =
+          targetDns.get(t);
+      if (targetNodeStorages == null) {
+        continue;
+      }
+      for (DatanodeWithStorage.StorageDetails targetNode : targetNodeStorages) {
+        if (targetNode.getDatanodeInfo().equals(source)) {
+          // Good target with enough space to write the given block size.
+          if (targetNode.hasSpaceForScheduling(blockInfo.getBlockSize())) {
+            targetNode.incScheduledSize(blockInfo.getBlockSize());
+            return new StorageTypeNodePair(t, source);
+          }
+          if (LOG.isDebugEnabled()) {
+            LOG.debug("Datanode:{} storage type:{} doesn't have sufficient "
+                + "space:{} to move the target block size:{}",
+                source, t, targetNode, blockInfo.getBlockSize());
+          }
+        }
       }
     }
     return null;
@@ -771,84 +792,105 @@ public class StoragePolicySatisfier<T> implements SPSService<T>, Runnable {
 
   private StorageTypeNodePair chooseTarget(LocatedBlock block,
       DatanodeInfo source, List<StorageType> targetTypes, Matcher matcher,
-      StorageTypeNodeMap locsForExpectedStorageTypes,
-      List<DatanodeInfo> excludeNodes) {
+      EnumMap<StorageType, List<DatanodeWithStorage.StorageDetails>>
+      locsForExpectedStorageTypes, List<DatanodeInfo> excludeNodes) {
     for (StorageType t : targetTypes) {
-      List<DatanodeInfo> nodesWithStorages =
-          locsForExpectedStorageTypes.getNodesWithStorages(t);
+      List<DatanodeWithStorage.StorageDetails> nodesWithStorages =
+          locsForExpectedStorageTypes.get(t);
       if (nodesWithStorages == null || nodesWithStorages.isEmpty()) {
         continue; // no target nodes with the required storage type.
       }
       Collections.shuffle(nodesWithStorages);
-      for (DatanodeInfo target : nodesWithStorages) {
+      for (DatanodeWithStorage.StorageDetails targetNode : nodesWithStorages) {
+        DatanodeInfo target = targetNode.getDatanodeInfo();
         if (!excludeNodes.contains(target)
-            && matcher.match(ctxt.getNetworkTopology(), source, target)) {
-          boolean goodTargetDn =
-              ctxt.checkDNSpaceForScheduling(target, t, block.getBlockSize());
-          if (goodTargetDn) {
+            && matcher.match(dnCacheMgr.getCluster(), source, target)) {
+          // Good target with enough space to write the given block size.
+          if (targetNode.hasSpaceForScheduling(block.getBlockSize())) {
+            targetNode.incScheduledSize(block.getBlockSize());
             return new StorageTypeNodePair(t, target);
           }
+          if (LOG.isDebugEnabled()) {
+            LOG.debug("Datanode:{} storage type:{} doesn't have sufficient "
+                + "space:{} to move the target block size:{}",
+                target, t, targetNode, block.getBlockSize());
+          }
         }
       }
     }
     return null;
   }
 
-  private static class StorageTypeNodePair {
-    private StorageType storageType = null;
-    private DatanodeInfo dn = null;
+  /**
+   * Keeps datanode with its respective storage type.
+   */
+  private static final class StorageTypeNodePair {
+    private final StorageType storageType;
+    private final DatanodeInfo dn;
 
-    StorageTypeNodePair(StorageType storageType, DatanodeInfo dn) {
+    private StorageTypeNodePair(StorageType storageType, DatanodeInfo dn) {
       this.storageType = storageType;
       this.dn = dn;
     }
   }
 
-  private StorageTypeNodeMap findTargetsForExpectedStorageTypes(
-      List<StorageType> expected, DatanodeStorageReport[] liveDns) {
-    StorageTypeNodeMap targetMap = new StorageTypeNodeMap();
-    for (DatanodeStorageReport dn : liveDns) {
-      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.getDatanodeInfo());
-          }
+  private EnumMap<StorageType, List<DatanodeWithStorage.StorageDetails>>
+      findTargetsForExpectedStorageTypes(List<StorageType> expected,
+        DatanodeMap liveDns) {
+    EnumMap<StorageType, List<DatanodeWithStorage.StorageDetails>> targetsMap =
+        new EnumMap<StorageType, List<DatanodeWithStorage.StorageDetails>>(
+        StorageType.class);
+
+    for (StorageType storageType : expected) {
+      List<DatanodeWithStorage> nodes = liveDns.getTarget(storageType);
+      if (nodes == null) {
+        return targetsMap;
+      }
+      List<DatanodeWithStorage.StorageDetails> listNodes = targetsMap
+          .get(storageType);
+      if (listNodes == null) {
+        listNodes = new ArrayList<>();
+        targetsMap.put(storageType, listNodes);
+      }
+
+      for (DatanodeWithStorage n : nodes) {
+        final DatanodeWithStorage.StorageDetails node = getMaxRemaining(n,
+            storageType);
+        if (node != null) {
+          listNodes.add(node);
         }
       }
     }
-    return targetMap;
+    return targetsMap;
   }
 
-  private static long getMaxRemaining(StorageReport[] storageReports,
-      StorageType t) {
+  private static DatanodeWithStorage.StorageDetails getMaxRemaining(
+      DatanodeWithStorage node, StorageType storageType) {
     long max = 0L;
-    for (StorageReport r : storageReports) {
-      if (r.getStorage().getStorageType() == t) {
-        if (r.getRemaining() > max) {
-          max = r.getRemaining();
-        }
+    DatanodeWithStorage.StorageDetails nodeInfo = null;
+    List<DatanodeWithStorage.StorageDetails> storages = node
+        .getNodesWithStorages(storageType);
+    for (DatanodeWithStorage.StorageDetails n : storages) {
+      if (n.availableSizeToMove() > max) {
+        max = n.availableSizeToMove();
+        nodeInfo = n;
       }
     }
-    return max;
+    return nodeInfo;
   }
 
   private boolean checkSourceAndTargetTypeExists(DatanodeInfo dn,
-      List<StorageType> existing, List<StorageType> expectedStorageTypes,
-      DatanodeStorageReport[] liveDns) {
+      List<StorageType> existingStorageTypes,
+      List<StorageType> expectedStorageTypes, DatanodeMap liveDns) {
     boolean isExpectedTypeAvailable = false;
     boolean isExistingTypeAvailable = false;
-    for (DatanodeStorageReport liveDn : liveDns) {
-      if (dn.equals(liveDn.getDatanodeInfo())) {
-        StorageReport[] storageReports = liveDn.getStorageReports();
-        for (StorageReport eachStorage : storageReports) {
-          StorageType storageType = eachStorage.getStorage().getStorageType();
-          if (existing.contains(storageType)) {
+    for (DatanodeWithStorage liveDn : liveDns.getTargets()) {
+      if (dn.equals(liveDn.datanode)) {
+        for (StorageType eachType : liveDn.getStorageTypes()) {
+          if (existingStorageTypes.contains(eachType)) {
             isExistingTypeAvailable = true;
           }
-          if (expectedStorageTypes.contains(storageType)) {
+          if (expectedStorageTypes.contains(eachType)) {
             isExpectedTypeAvailable = true;
           }
           if (isExistingTypeAvailable && isExpectedTypeAvailable) {
@@ -860,29 +902,143 @@ public class StoragePolicySatisfier<T> implements SPSService<T>, Runnable {
     return isExistingTypeAvailable && isExpectedTypeAvailable;
   }
 
-  private static class StorageTypeNodeMap {
-    private final EnumMap<StorageType, List<DatanodeInfo>> typeNodeMap =
-        new EnumMap<StorageType, List<DatanodeInfo>>(StorageType.class);
+  /**
+   * Maintains storage type map with the available datanodes in the cluster.
+   */
+  public static class DatanodeMap {
+    private final EnumMap<StorageType, List<DatanodeWithStorage>> targetsMap =
+        new EnumMap<StorageType, List<DatanodeWithStorage>>(StorageType.class);
+
+    private List<DatanodeWithStorage> targets = new ArrayList<>();
+
+    /**
+     * Build datanode map with the available storage types.
+     *
+     * @param node
+     *          datanode
+     * @param storageTypes
+     *          list of available storage types in the given datanode
+     * @param maxSize2Move
+     *          available space which can be used for scheduling block move
+     */
+    void addTarget(DatanodeInfo node, List<StorageType> storageTypes,
+        List<Long> maxSize2Move) {
+      DatanodeWithStorage nodeStorage = new DatanodeWithStorage(node);
+      targets.add(nodeStorage);
+      for (int i = 0; i < storageTypes.size(); i++) {
+        StorageType type = storageTypes.get(i);
+        List<DatanodeWithStorage> nodeStorages = targetsMap.get(type);
+        nodeStorage.addStorageType(type, maxSize2Move.get(i));
+        if (nodeStorages == null) {
+          nodeStorages = new LinkedList<>();
+          targetsMap.put(type, nodeStorages);
+        }
+        nodeStorages.add(nodeStorage);
+      }
+    }
+
+    List<DatanodeWithStorage> getTarget(StorageType storageType) {
+      return targetsMap.get(storageType);
+    }
+
+    public List<DatanodeWithStorage> getTargets() {
+      return targets;
+    }
+
+    void reset() {
+      targetsMap.clear();
+    }
+  }
+
+  /**
+   * Keeps datanode with its respective set of supported storage types. It holds
+   * the available space in each volumes and will be used while pairing the
+   * target datanodes.
+   */
+  public static final class DatanodeWithStorage {
+    private final EnumMap<StorageType, List<StorageDetails>> storageMap =
+        new EnumMap<StorageType, List<StorageDetails>>(StorageType.class);
+    private final DatanodeInfo datanode;
+
+    private DatanodeWithStorage(DatanodeInfo datanode) {
+      this.datanode = datanode;
+    }
+
+    public DatanodeInfo getDatanodeInfo() {
+      return datanode;
+    }
+
+    Set<StorageType> getStorageTypes() {
+      return storageMap.keySet();
+    }
 
-    private void add(StorageType t, DatanodeInfo dn) {
-      List<DatanodeInfo> nodesWithStorages = getNodesWithStorages(t);
-      LinkedList<DatanodeInfo> value = null;
+    private void addStorageType(StorageType t, long maxSize2Move) {
+      List<StorageDetails> nodesWithStorages = getNodesWithStorages(t);
       if (nodesWithStorages == null) {
-        value = new LinkedList<DatanodeInfo>();
-        value.add(dn);
-        typeNodeMap.put(t, value);
-      } else {
-        nodesWithStorages.add(dn);
+        nodesWithStorages = new LinkedList<StorageDetails>();
+        storageMap.put(t, nodesWithStorages);
       }
+      nodesWithStorages.add(new StorageDetails(maxSize2Move));
     }
 
     /**
+     * Returns datanode storages which has the given storage type.
+     *
      * @param type
-     *          - Storage type
-     * @return datanodes which has the given storage type
+     *          - storage type
+     * @return datanodes for the given storage type
      */
-    private List<DatanodeInfo> getNodesWithStorages(StorageType type) {
-      return typeNodeMap.get(type);
+    private List<StorageDetails> getNodesWithStorages(StorageType type) {
+      return storageMap.get(type);
+    }
+
+    @Override
+    public String toString() {
+      return new StringBuilder().append("DatanodeWithStorageInfo(\n  ")
+          .append("Datanode: ").append(datanode).append(" StorageTypeNodeMap: ")
+          .append(storageMap).append(")").toString();
+    }
+
+    /** Storage details in a datanode storage type. */
+    final class StorageDetails {
+      private final long maxSize2Move;
+      private long scheduledSize = 0L;
+
+      private StorageDetails(long maxSize2Move) {
+        this.maxSize2Move = maxSize2Move;
+      }
+
+      private DatanodeInfo getDatanodeInfo() {
+        return DatanodeWithStorage.this.datanode;
+      }
+
+      /**
+       * Checks whether this datanode storage has sufficient space to occupy the
+       * given block size.
+       */
+      private synchronized boolean hasSpaceForScheduling(long size) {
+        return availableSizeToMove() > size;
+      }
+
+      /**
+       * @return the total number of bytes that need to be moved.
+       */
+      private synchronized long availableSizeToMove() {
+        return maxSize2Move - scheduledSize;
+      }
+
+      /** Increment scheduled size. */
+      private synchronized void incScheduledSize(long size) {
+        scheduledSize += size;
+      }
+
+      @Override
+      public String toString() {
+        return new StringBuilder().append("StorageDetails(\n  ")
+            .append("maxSize2Move: ").append(maxSize2Move)
+            .append(" scheduledSize: ").append(scheduledSize).append(")")
+            .toString();
+      }
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75ccc139/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/NamenodeProtocol.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/NamenodeProtocol.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/NamenodeProtocol.java
index 615e297..5ff6ffd 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/NamenodeProtocol.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/NamenodeProtocol.java
@@ -21,7 +21,6 @@ package org.apache.hadoop.hdfs.server.protocol;
 import java.io.IOException;
 
 import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.security.token.block.ExportedBlockKeys;
@@ -211,20 +210,5 @@ public interface NamenodeProtocol {
    */
   @AtMostOnce
   String getNextSPSPath() throws IOException;
-
-  /**
-   * Verifies whether the given Datanode has the enough estimated size with
-   * given storage type for scheduling the block. This API used by External SPS.
-   *
-   * @param dn
-   *          - datanode
-   * @param type
-   *          - storage type
-   * @param estimatedSize
-   *          - size
-   */
-  @Idempotent
-  boolean checkDNSpaceForScheduling(DatanodeInfo dn, StorageType type,
-      long estimatedSize) throws IOException;
 }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75ccc139/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/sps/ExternalSPSContext.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/sps/ExternalSPSContext.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/sps/ExternalSPSContext.java
index 5d0aee6..1cd4664 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/sps/ExternalSPSContext.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/sps/ExternalSPSContext.java
@@ -20,15 +20,14 @@ package org.apache.hadoop.hdfs.server.sps;
 
 import java.io.FileNotFoundException;
 import java.io.IOException;
+import java.util.List;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.fs.FSDataOutputStream;
 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.hdfs.protocol.BlockStoragePolicy;
-import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.HdfsLocatedFileStatus;
@@ -37,6 +36,8 @@ import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.namenode.sps.Context;
 import org.apache.hadoop.hdfs.server.namenode.sps.SPSService;
+import org.apache.hadoop.hdfs.server.namenode.sps.StoragePolicySatisfier.DatanodeMap;
+import org.apache.hadoop.hdfs.server.namenode.sps.StoragePolicySatisfier.DatanodeWithStorage;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeStorageReport;
 import org.apache.hadoop.net.NetworkTopology;
 import org.apache.hadoop.security.AccessControlException;
@@ -107,8 +108,14 @@ public class ExternalSPSContext implements Context<String> {
   }
 
   @Override
-  public NetworkTopology getNetworkTopology() {
-    return NetworkTopology.getInstance(service.getConf());
+  public NetworkTopology getNetworkTopology(DatanodeMap datanodeMap) {
+    // create network topology.
+    NetworkTopology cluster = NetworkTopology.getInstance(service.getConf());
+    List<DatanodeWithStorage> targets = datanodeMap.getTargets();
+    for (DatanodeWithStorage node : targets) {
+      cluster.add(node.getDatanodeInfo());
+    }
+    return cluster;
   }
 
   @Override
@@ -168,23 +175,6 @@ public class ExternalSPSContext implements Context<String> {
   }
 
   @Override
-  public boolean checkDNSpaceForScheduling(DatanodeInfo dn, StorageType type,
-      long estimatedSize) {
-    // TODO: Instead of calling namenode for checking the available space, it
-    // can be optimized by maintaining local cache of datanode storage report
-    // and do the computations. This local cache can be refreshed per file or
-    // periodic fashion.
-    try {
-      return nnc.getNNProtocolConnection().checkDNSpaceForScheduling(dn, type,
-          estimatedSize);
-    } catch (IOException e) {
-      LOG.warn("Verify the given datanode:{} is good and has "
-          + "estimated space in it.", dn, e);
-      return false;
-    }
-  }
-
-  @Override
   public String getNextSPSPath() {
     try {
       return nnc.getNNProtocolConnection().getNextSPSPath();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75ccc139/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/NamenodeProtocol.proto
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/NamenodeProtocol.proto b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/NamenodeProtocol.proto
index b137f2f..2acc5a8 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/NamenodeProtocol.proto
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/NamenodeProtocol.proto
@@ -221,24 +221,6 @@ message GetNextSPSPathResponseProto {
   optional string spsPath = 1;
 }
 
-message CheckDNSpaceRequestProto {
-  required DatanodeInfoProto dnInfo = 1;
-  required StorageTypeProto storageType = 2;
-  required uint64 estimatedSize = 3;
-}
-
-message CheckDNSpaceResponseProto {
-  required bool isGoodDatanodeWithSpace = 1;
-}
-
-message HasLowRedundancyBlocksRequestProto {
-  required uint64 inodeId = 1;
-}
-
-message HasLowRedundancyBlocksResponseProto {
-  required bool hasLowRedundancyBlocks = 1;
-}
-
 /**
  * Protocol used by the sub-ordinate namenode to send requests
  * the active/primary namenode.
@@ -326,11 +308,4 @@ service NamenodeProtocolService {
    */
   rpc getNextSPSPath(GetNextSPSPathRequestProto)
       returns (GetNextSPSPathResponseProto);
-
-  /**
-   * Verifies whether the given Datanode has the enough estimated size with
-   * given storage type for scheduling the block movement.
-   */
-  rpc checkDNSpaceForScheduling(CheckDNSpaceRequestProto)
-      returns (CheckDNSpaceResponseProto);
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75ccc139/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 a32d7c3..0f80f97 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
@@ -4581,6 +4581,17 @@
 </property>
 
 <property>
+  <name>dfs.storage.policy.satisfier.datanode.cache.refresh.interval.ms</name>
+  <value>300000</value>
+  <description>
+    How often to refresh the datanode storages cache in milliseconds. This cache
+    keeps live datanode storage reports fetched from namenode. After elapsed time,
+    it will again fetch latest datanodes from namenode.
+    By default, this parameter is set to 5 minutes.
+  </description>
+</property>
+
+<property>
   <name>dfs.pipeline.ecn</name>
   <value>false</value>
   <description>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75ccc139/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 25a6cd9..5872ef8 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/ArchivalStorage.md
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/ArchivalStorage.md
@@ -243,7 +243,7 @@ If administrator wants to switch modes of SPS feature while Namenode is running,
        hdfs dfsadmin -reconfig namenode <host:ipc_port> start
 
 ### Start External SPS Service.
-If administrator wants to start external sps, first he/she needs to configure property `dfs.storage.policy.satisfier.mode` with `external` value in configuration file (`hdfs-site.xml`) and then run Namenode reconfig command. After this start external sps service using following command
+If administrator wants to start external sps, first he/she needs to configure property `dfs.storage.policy.satisfier.mode` with `external` value in configuration file (`hdfs-site.xml`) and then run Namenode reconfig command. Please ensure that network topology configurations in the configuration file are same as namenode, this cluster will be used for matching target nodes. After this, start external sps service using following command
 
 * Command:
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75ccc139/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 5f41a86..06a66f7 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,6 +69,9 @@ public class TestStoragePolicySatisfyWorker {
     conf.setLong(DFSConfigKeys.DFS_BALANCER_MOVEDWINWIDTH_KEY, 2000L);
     conf.set(DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_MODE_KEY,
         StoragePolicySatisfierMode.INTERNAL.toString());
+    // Reduced refresh cycle to update latest datanodes.
+    conf.setLong(DFSConfigKeys.DFS_SPS_DATANODE_CACHE_REFRESH_INTERVAL_MS,
+        1000);
   }
 
   @Before

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75ccc139/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 9f98777..e079471 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
@@ -105,6 +105,9 @@ public class TestPersistentStoragePolicySatisfier {
         "3000");
     conf.set(DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_MODE_KEY,
         StoragePolicySatisfierMode.INTERNAL.toString());
+    // Reduced refresh cycle to update latest datanodes.
+    conf.setLong(DFSConfigKeys.DFS_SPS_DATANODE_CACHE_REFRESH_INTERVAL_MS,
+        1000);
     final int dnNumber = storageTypes.length;
     final short replication = 3;
     MiniDFSCluster.Builder clusterBuilder = new MiniDFSCluster.Builder(conf)
@@ -295,6 +298,9 @@ public class TestPersistentStoragePolicySatisfier {
       conf = new HdfsConfiguration();
       conf.set(DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_MODE_KEY,
           StoragePolicySatisfierMode.INTERNAL.toString());
+      // Reduced refresh cycle to update latest datanodes.
+      conf.setLong(DFSConfigKeys.DFS_SPS_DATANODE_CACHE_REFRESH_INTERVAL_MS,
+          1000);
       haCluster = new MiniDFSCluster
           .Builder(conf)
           .nnTopology(MiniDFSNNTopology.simpleHAFederatedTopology(2))

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75ccc139/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 b3734d1..0cadc83 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
@@ -68,6 +68,9 @@ public class TestStoragePolicySatisfierWithHA {
     config.setLong("dfs.block.size", DEFAULT_BLOCK_SIZE);
     config.set(DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_MODE_KEY,
         StoragePolicySatisfierMode.INTERNAL.toString());
+    // Reduced refresh cycle to update latest datanodes.
+    config.setLong(DFSConfigKeys.DFS_SPS_DATANODE_CACHE_REFRESH_INTERVAL_MS,
+        1000);
     startCluster(config, allDiskTypes, numOfDatanodes, storagesPerDatanode,
         capacity);
     dfs = cluster.getFileSystem(nnIndex);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75ccc139/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/sps/TestStoragePolicySatisfier.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/sps/TestStoragePolicySatisfier.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/sps/TestStoragePolicySatisfier.java
index 2a3d0c8..75aeb86 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/sps/TestStoragePolicySatisfier.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/sps/TestStoragePolicySatisfier.java
@@ -159,6 +159,10 @@ public class TestStoragePolicySatisfier {
     config = new HdfsConfiguration();
     config.set(DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_MODE_KEY,
         StoragePolicySatisfierMode.INTERNAL.toString());
+    // Most of the tests are restarting DNs and NN. So, reduced refresh cycle to
+    // update latest datanodes.
+    config.setLong(DFSConfigKeys.DFS_SPS_DATANODE_CACHE_REFRESH_INTERVAL_MS,
+        3000);
   }
 
   @Test(timeout = 300000)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75ccc139/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/sps/TestStoragePolicySatisfierWithStripedFile.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/sps/TestStoragePolicySatisfierWithStripedFile.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/sps/TestStoragePolicySatisfierWithStripedFile.java
index a39fb92..e69a833 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/sps/TestStoragePolicySatisfierWithStripedFile.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/sps/TestStoragePolicySatisfierWithStripedFile.java
@@ -69,6 +69,7 @@ public class TestStoragePolicySatisfierWithStripedFile {
   private int parityBlocks;
   private int cellSize;
   private int defaultStripeBlockSize;
+  private Configuration conf;
 
   private ErasureCodingPolicy getEcPolicy() {
     return StripedFileTestUtil.getDefaultECPolicy();
@@ -84,6 +85,13 @@ public class TestStoragePolicySatisfierWithStripedFile {
     parityBlocks = ecPolicy.getNumParityUnits();
     cellSize = ecPolicy.getCellSize();
     defaultStripeBlockSize = cellSize * stripesPerBlock;
+    conf = new HdfsConfiguration();
+    conf.set(DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_MODE_KEY,
+        StoragePolicySatisfierMode.INTERNAL.toString());
+    // Reduced refresh cycle to update latest datanodes.
+    conf.setLong(DFSConfigKeys.DFS_SPS_DATANODE_CACHE_REFRESH_INTERVAL_MS,
+        1000);
+    initConfWithStripe(conf, defaultStripeBlockSize);
   }
 
   /**
@@ -103,10 +111,6 @@ public class TestStoragePolicySatisfierWithStripedFile {
       }
     }
 
-    final Configuration conf = new HdfsConfiguration();
-    conf.set(DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_MODE_KEY,
-        StoragePolicySatisfierMode.INTERNAL.toString());
-    initConfWithStripe(conf, defaultStripeBlockSize);
     final MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
         .numDataNodes(numOfDatanodes)
         .storagesPerDatanode(storagesPerDatanode)
@@ -216,10 +220,6 @@ public class TestStoragePolicySatisfierWithStripedFile {
       }
     }
 
-    final Configuration conf = new HdfsConfiguration();
-    conf.set(DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_MODE_KEY,
-        StoragePolicySatisfierMode.INTERNAL.toString());
-    initConfWithStripe(conf, defaultStripeBlockSize);
     final MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
         .numDataNodes(numOfDatanodes)
         .storagesPerDatanode(storagesPerDatanode)
@@ -325,13 +325,9 @@ public class TestStoragePolicySatisfierWithStripedFile {
       }
     }
 
-    final Configuration conf = new HdfsConfiguration();
     conf.set(DFSConfigKeys
         .DFS_STORAGE_POLICY_SATISFIER_RECHECK_TIMEOUT_MILLIS_KEY,
         "3000");
-    conf.set(DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_MODE_KEY,
-        StoragePolicySatisfierMode.INTERNAL.toString());
-    initConfWithStripe(conf, defaultStripeBlockSize);
     final MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
         .numDataNodes(numOfDatanodes)
         .storagesPerDatanode(storagesPerDatanode)
@@ -420,10 +416,6 @@ public class TestStoragePolicySatisfierWithStripedFile {
       }
     }
 
-    final Configuration conf = new HdfsConfiguration();
-    conf.set(DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_MODE_KEY,
-        StoragePolicySatisfierMode.INTERNAL.toString());
-    initConfWithStripe(conf, defaultStripeBlockSize);
     final MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
         .numDataNodes(numOfDatanodes)
         .storagesPerDatanode(storagesPerDatanode)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75ccc139/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
index 6a30c03..8a62e0e 100644
--- 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
@@ -49,6 +49,9 @@ public class TestStoragePolicySatisfyAdminCommands {
     conf = new HdfsConfiguration();
     conf.set(DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_MODE_KEY,
         StoragePolicySatisfierMode.INTERNAL.toString());
+    // Reduced refresh cycle to update latest datanodes.
+    conf.setLong(DFSConfigKeys.DFS_SPS_DATANODE_CACHE_REFRESH_INTERVAL_MS,
+        1000);
     StorageType[][] newtypes = new StorageType[][] {
         {StorageType.ARCHIVE, StorageType.DISK}};
     cluster = new MiniDFSCluster.Builder(conf).numDataNodes(REPL)


---------------------------------------------------------------------
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-13808: [SPS]: Remove unwanted FSNamesystem #isFileOpenedForWrite() and #getFileInfo() function. Contributed by Rakesh R.

Posted by um...@apache.org.
HDFS-13808: [SPS]: Remove unwanted FSNamesystem #isFileOpenedForWrite() and #getFileInfo() function. 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/3ac07b72
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/3ac07b72
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/3ac07b72

Branch: refs/heads/HDFS-10285
Commit: 3ac07b720b7839a7fe6c83f4ccfe319b6a892501
Parents: 39ed3a6
Author: Uma Maheswara Rao Gangumalla <um...@apache.org>
Authored: Sat Aug 11 23:22:59 2018 -0700
Committer: Uma Maheswara Rao Gangumalla <um...@apache.org>
Committed: Sun Aug 12 03:06:07 2018 -0700

----------------------------------------------------------------------
 .../router/RouterNamenodeProtocol.java          |  1 +
 .../server/blockmanagement/BlockManager.java    | 34 ------------
 .../blockmanagement/DatanodeDescriptor.java     |  2 +-
 .../server/blockmanagement/DatanodeManager.java | 17 ------
 .../hdfs/server/datanode/BPServiceActor.java    | 16 ------
 .../hdfs/server/namenode/FSNamesystem.java      | 38 -------------
 .../hadoop/hdfs/server/namenode/Namesystem.java | 22 --------
 .../sps/BlockStorageMovementNeeded.java         | 18 ++-----
 .../hdfs/server/namenode/sps/Context.java       | 28 ----------
 .../hdfs/server/namenode/sps/SPSService.java    |  5 +-
 .../namenode/sps/StoragePolicySatisfier.java    | 19 +++----
 .../hdfs/server/sps/ExternalSPSContext.java     | 57 +++++---------------
 .../sps/ExternalStoragePolicySatisfier.java     |  2 +-
 .../src/site/markdown/ArchivalStorage.md        |  2 +-
 .../TestPersistentStoragePolicySatisfier.java   | 10 +++-
 ...stStoragePolicySatisfierWithStripedFile.java |  2 +-
 .../sps/TestExternalStoragePolicySatisfier.java |  4 +-
 .../TestStoragePolicySatisfyAdminCommands.java  |  2 +-
 18 files changed, 39 insertions(+), 240 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/3ac07b72/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterNamenodeProtocol.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterNamenodeProtocol.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterNamenodeProtocol.java
index edfb391..bf0db6e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterNamenodeProtocol.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterNamenodeProtocol.java
@@ -187,6 +187,7 @@ public class RouterNamenodeProtocol implements NamenodeProtocol {
 
   @Override
   public Long getNextSPSPath() throws IOException {
+    rpcServer.checkOperation(OperationCategory.READ, false);
     // not supported
     return null;
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3ac07b72/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 87bd155..d8a3aa3 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
@@ -4300,21 +4300,6 @@ 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,
@@ -5011,25 +4996,6 @@ public class BlockManager implements BlockStatsMXBean {
   }
 
   /**
-   * Check whether file id has low redundancy blocks.
-   *
-   * @param inodeID
-   *          - inode id
-   */
-  public boolean hasLowRedundancyBlocks(long inodeID) {
-    namesystem.readLock();
-    try {
-      BlockCollection bc = namesystem.getBlockCollection(inodeID);
-      if (bc == null) {
-        return false;
-      }
-      return hasLowRedundancyBlocks(bc);
-    } finally {
-      namesystem.readUnlock();
-    }
-  }
-
-  /**
    * Create SPS manager instance. It manages the user invoked sps paths and does
    * the movement.
    *

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3ac07b72/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 9c96f16..12b5c33 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
@@ -793,7 +793,7 @@ public class DatanodeDescriptor extends DatanodeInfo {
   }
 
   /** Increment the number of blocks scheduled. */
-  public void incrementBlocksScheduled(StorageType t) {
+  void incrementBlocksScheduled(StorageType t) {
     currApproxBlocksScheduled.add(t, 1);
   }
   

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3ac07b72/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 4173f48..7d5d73c 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
@@ -1988,22 +1988,5 @@ public class DatanodeManager {
     }
     return reports;
   }
-
-  public boolean verifyTargetDatanodeHasSpaceForScheduling(DatanodeInfo dn,
-      StorageType type, long estimatedSize) {
-    namesystem.readLock();
-    try {
-      DatanodeDescriptor datanode =
-          blockManager.getDatanodeManager().getDatanode(dn.getDatanodeUuid());
-      if (datanode == null) {
-        LOG.debug("Target datanode: " + dn + " doesn't exists");
-        return false;
-      }
-      return null != datanode.chooseStorage4Block(type, estimatedSize);
-    } finally {
-      namesystem.readUnlock();
-    }
-  }
-
 }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3ac07b72/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 dab8ae9..f09ff66 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,7 +39,6 @@ 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;
@@ -51,7 +50,6 @@ 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.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;
@@ -534,20 +532,6 @@ class BPServiceActor implements Runnable {
     return response;
   }
 
-  private BlocksStorageMoveAttemptFinished getStorageMoveAttemptFinishedBlocks(
-      List<Block> finishedBlks) {
-
-    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
   void sendLifelineForTests() throws IOException {
     lifelineSender.sendLifeline();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3ac07b72/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 7bc9ecc..ecf7fce 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
@@ -3139,29 +3139,6 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
    * @param src The string representation of the path to the file
    * @param resolveLink whether to throw UnresolvedLinkException
    *        if src refers to a symlink
-   * @param needLocation if blockLocations need to be returned
-   *
-   * @throws AccessControlException
-   *           if access is denied
-   * @throws UnresolvedLinkException
-   *           if a symlink is encountered.
-   *
-   * @return object containing information regarding the file or null if file
-   *         not found
-   * @throws StandbyException
-   */
-  @Override
-  public HdfsFileStatus getFileInfo(final String src, boolean resolveLink,
-      boolean needLocation) throws IOException {
-    return getFileInfo(src, resolveLink, needLocation, false);
-  }
-
-  /**
-   * Get the file info for a specific file.
-   *
-   * @param src The string representation of the path to the file
-   * @param resolveLink whether to throw UnresolvedLinkException
-   *        if src refers to a symlink
    *
    * @param needLocation Include {@link LocatedBlocks} in result.
    * @param needBlockToken Include block tokens in {@link LocatedBlocks}
@@ -3604,21 +3581,6 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
   }
 
   @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();
     final INodeFile bc = getBlockCollection(blockCollectionID);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3ac07b72/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 82af4d2..2a52587 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
@@ -20,7 +20,6 @@ package org.apache.hadoop.hdfs.server.namenode;
 import java.io.IOException;
 
 import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockCollection;
 import org.apache.hadoop.hdfs.server.namenode.ha.HAContext;
 import org.apache.hadoop.hdfs.util.RwLock;
@@ -50,31 +49,10 @@ public interface Namesystem extends RwLock, SafeMode {
   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);
-
-  /**
    * Remove xAttr from the inode.
    * @param id
    * @param xattrName
    * @throws IOException
    */
   void removeXattr(long id, String xattrName) throws IOException;
-
-  /**
-   * Gets the fileInfo of the given file path.
-   *
-   * @param filePath string representation of the path to the file
-   * @param resolveLink whether to throw UnresolvedLinkException
-   *        if src refers to a symlink
-   * @param needLocation if blockLocations need to be returned
-   *
-   * @return hdfs file status details
-   * @throws IOException
-   */
-  HdfsFileStatus getFileInfo(String filePath, boolean resolveLink,
-      boolean needLocation) throws IOException;
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3ac07b72/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/BlockStorageMovementNeeded.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/BlockStorageMovementNeeded.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/BlockStorageMovementNeeded.java
index b990bc5..02b9cff 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/BlockStorageMovementNeeded.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/BlockStorageMovementNeeded.java
@@ -19,7 +19,6 @@ package org.apache.hadoop.hdfs.server.namenode.sps;
 
 import java.io.IOException;
 import java.util.HashMap;
-import java.util.Iterator;
 import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
@@ -79,7 +78,9 @@ public class BlockStorageMovementNeeded {
    *          - track info for satisfy the policy
    */
   public synchronized void add(ItemInfo trackInfo) {
-    storageMovementNeeded.add(trackInfo);
+    if (trackInfo != null) {
+      storageMovementNeeded.add(trackInfo);
+    }
   }
 
   /**
@@ -153,7 +154,6 @@ public class BlockStorageMovementNeeded {
   }
 
   public synchronized void clearAll() {
-    ctxt.removeAllSPSPathIds();
     storageMovementNeeded.clear();
     pendingWorkForDirectory.clear();
   }
@@ -188,18 +188,6 @@ public class BlockStorageMovementNeeded {
     }
   }
 
-  public synchronized void clearQueue(long trackId) {
-    ctxt.removeSPSPathId(trackId);
-    Iterator<ItemInfo> iterator = storageMovementNeeded.iterator();
-    while (iterator.hasNext()) {
-      ItemInfo next = iterator.next();
-      if (next.getFile() == trackId) {
-        iterator.remove();
-      }
-    }
-    pendingWorkForDirectory.remove(trackId);
-  }
-
   /**
    * Clean all the movements in spsDirsToBeTraveresed/storageMovementNeeded
    * and notify to clean up required resources.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3ac07b72/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/Context.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/Context.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/Context.java
index afa5a50..b27294c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/Context.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/Context.java
@@ -22,8 +22,6 @@ import java.io.IOException;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.fs.ParentNotDirectoryException;
-import org.apache.hadoop.fs.UnresolvedLinkException;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
@@ -31,7 +29,6 @@ import org.apache.hadoop.hdfs.server.namenode.sps.StoragePolicySatisfier.Datanod
 import org.apache.hadoop.hdfs.server.protocol.DatanodeStorageReport;
 import org.apache.hadoop.hdfs.server.protocol.BlockStorageMovementCommand.BlockMovingInfo;
 import org.apache.hadoop.net.NetworkTopology;
-import org.apache.hadoop.security.AccessControlException;
 
 /**
  * An interface for the communication between SPS and Namenode module.
@@ -51,21 +48,6 @@ public interface Context {
   boolean isInSafeMode();
 
   /**
-   * Returns true if Mover tool is already running, false otherwise.
-   */
-  boolean isMoverRunning();
-
-  /**
-   * Gets the Inode ID number for the given path.
-   *
-   * @param path
-   *          - file/dir path
-   * @return Inode id number
-   */
-  long getFileID(String path) throws UnresolvedLinkException,
-      AccessControlException, ParentNotDirectoryException;
-
-  /**
    * Gets the network topology.
    *
    * @param datanodeMap
@@ -132,16 +114,6 @@ public interface Context {
   Long getNextSPSPath();
 
   /**
-   * Removes the SPS path id.
-   */
-  void removeSPSPathId(long pathId);
-
-  /**
-   * Removes all SPS path ids.
-   */
-  void removeAllSPSPathIds();
-
-  /**
    * Do scan and collects the files under that directory and adds to the given
    * BlockStorageMovementNeeded.
    *

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3ac07b72/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/SPSService.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/SPSService.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/SPSService.java
index a62dd93..a83d32a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/SPSService.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/SPSService.java
@@ -47,12 +47,9 @@ public interface SPSService {
    * Starts the SPS service. Make sure to initialize the helper services before
    * invoking this method.
    *
-   * @param reconfigStart
-   *          - to indicate whether the SPS startup requested from
-   *          reconfiguration service
    * @param spsMode sps service mode
    */
-  void start(boolean reconfigStart, StoragePolicySatisfierMode spsMode);
+  void start(StoragePolicySatisfierMode spsMode);
 
   /**
    * Stops the SPS service gracefully. Timed wait to stop storage policy

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3ac07b72/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/StoragePolicySatisfier.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/StoragePolicySatisfier.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/StoragePolicySatisfier.java
index 7ebd23d..4c04b46 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/StoragePolicySatisfier.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/StoragePolicySatisfier.java
@@ -150,21 +150,14 @@ public class StoragePolicySatisfier implements SPSService, Runnable {
    * movements monitor for retry the attempts if needed.
    */
   @Override
-  public synchronized void start(boolean reconfigStart,
-      StoragePolicySatisfierMode serviceMode) {
+  public synchronized void start(StoragePolicySatisfierMode serviceMode) {
     if (serviceMode == StoragePolicySatisfierMode.NONE) {
       LOG.error("Can't start StoragePolicySatisfier for the given mode:{}",
           serviceMode);
       return;
     }
-    if (reconfigStart) {
-      LOG.info("Starting {} StoragePolicySatisfier, as admin requested to "
-          + "start it.", StringUtils.toLowerCase(serviceMode.toString()));
-    } else {
-      LOG.info("Starting {} StoragePolicySatisfier.",
-          StringUtils.toLowerCase(serviceMode.toString()));
-    }
-
+    LOG.info("Starting {} StoragePolicySatisfier.",
+        StringUtils.toLowerCase(serviceMode.toString()));
     isRunning = true;
     storagePolicySatisfierThread = new Daemon(this);
     storagePolicySatisfierThread.setName("StoragePolicySatisfier");
@@ -229,8 +222,8 @@ public class StoragePolicySatisfier implements SPSService, Runnable {
         }
         continue;
       }
+      ItemInfo itemInfo = null;
       try {
-        ItemInfo itemInfo = null;
         boolean retryItem = false;
         if (!ctxt.isInSafeMode()) {
           itemInfo = storageMovementNeeded.get();
@@ -324,12 +317,14 @@ public class StoragePolicySatisfier implements SPSService, Runnable {
           blockCount = 0L;
         }
         if (retryItem) {
-          // itemInfo.increRetryCount();
           this.storageMovementNeeded.add(itemInfo);
         }
       } catch (IOException e) {
         LOG.error("Exception during StoragePolicySatisfier execution - "
             + "will continue next cycle", e);
+        // Since it could not finish this item in previous iteration due to IOE,
+        // just try again.
+        this.storageMovementNeeded.add(itemInfo);
       } catch (Throwable t) {
         synchronized (this) {
           if (isRunning) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3ac07b72/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/sps/ExternalSPSContext.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/sps/ExternalSPSContext.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/sps/ExternalSPSContext.java
index 3293035..8427e93 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/sps/ExternalSPSContext.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/sps/ExternalSPSContext.java
@@ -24,10 +24,7 @@ import java.util.ArrayList;
 import java.util.List;
 
 import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.fs.FSDataOutputStream;
-import org.apache.hadoop.fs.ParentNotDirectoryException;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.UnresolvedLinkException;
 import org.apache.hadoop.hdfs.DFSUtilClient;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
@@ -47,7 +44,6 @@ import org.apache.hadoop.hdfs.server.namenode.sps.StoragePolicySatisfier.Datanod
 import org.apache.hadoop.hdfs.server.protocol.BlockStorageMovementCommand.BlockMovingInfo;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeStorageReport;
 import org.apache.hadoop.net.NetworkTopology;
-import org.apache.hadoop.security.AccessControlException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -93,35 +89,6 @@ public class ExternalSPSContext implements Context {
   }
 
   @Override
-  public boolean isMoverRunning() {
-    try {
-      FSDataOutputStream out = nnc.getDistributedFileSystem()
-          .append(HdfsServerConstants.MOVER_ID_PATH);
-      out.close();
-      return false;
-    } catch (IOException ioe) {
-      LOG.warn("Exception while checking mover is running..", ioe);
-      return true;
-    }
-
-  }
-
-  @Override
-  public long getFileID(String path) throws UnresolvedLinkException,
-      AccessControlException, ParentNotDirectoryException {
-    HdfsFileStatus fs = null;
-    try {
-      fs = (HdfsFileStatus) nnc.getDistributedFileSystem().getFileStatus(
-          new Path(path));
-      LOG.info("Fetched the fileID:{} for the path:{}", fs.getFileId(), path);
-    } catch (IllegalArgumentException | IOException e) {
-      LOG.warn("Exception while getting file is for the given path:{}.", path,
-          e);
-    }
-    return fs != null ? fs.getFileId() : 0;
-  }
-
-  @Override
   public NetworkTopology getNetworkTopology(DatanodeMap datanodeMap) {
     // create network topology.
     NetworkTopology cluster = NetworkTopology.getInstance(service.getConf());
@@ -152,8 +119,18 @@ public class ExternalSPSContext implements Context {
   @Override
   public void removeSPSHint(long inodeId) throws IOException {
     Path filePath = DFSUtilClient.makePathFromFileId(inodeId);
-    nnc.getDistributedFileSystem().removeXAttr(filePath,
-        HdfsServerConstants.XATTR_SATISFY_STORAGE_POLICY);
+    try {
+      nnc.getDistributedFileSystem().removeXAttr(filePath,
+          HdfsServerConstants.XATTR_SATISFY_STORAGE_POLICY);
+    } catch (IOException e) {
+      List<String> listXAttrs = nnc.getDistributedFileSystem()
+          .listXAttrs(filePath);
+      if (!listXAttrs
+          .contains(HdfsServerConstants.XATTR_SATISFY_STORAGE_POLICY)) {
+        LOG.info("SPS hint already removed for the inodeId:{}."
+            + " Ignoring exception:{}", inodeId, e.getMessage());
+      }
+    }
   }
 
   @Override
@@ -197,16 +174,6 @@ public class ExternalSPSContext implements Context {
   }
 
   @Override
-  public void removeSPSPathId(long pathId) {
-    // We need not specifically implement for external.
-  }
-
-  @Override
-  public void removeAllSPSPathIds() {
-    // We need not specifically implement for external.
-  }
-
-  @Override
   public void scanAndCollectFiles(long path)
       throws IOException, InterruptedException {
     fileCollector.scanAndCollectFiles(path);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3ac07b72/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/sps/ExternalStoragePolicySatisfier.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/sps/ExternalStoragePolicySatisfier.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/sps/ExternalStoragePolicySatisfier.java
index 8e19a7c..15cdc6e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/sps/ExternalStoragePolicySatisfier.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/sps/ExternalStoragePolicySatisfier.java
@@ -70,7 +70,7 @@ public final class ExternalStoragePolicySatisfier {
 
       ExternalSPSContext context = new ExternalSPSContext(sps, nnc);
       sps.init(context);
-      sps.start(true, StoragePolicySatisfierMode.EXTERNAL);
+      sps.start(StoragePolicySatisfierMode.EXTERNAL);
       if (sps != null) {
         sps.join();
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3ac07b72/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 3789779..5fd6612 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/ArchivalStorage.md
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/ArchivalStorage.md
@@ -154,7 +154,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 disabled while starting. In that case, administrator should make sure, Mover execution finished and then enable(internal service inside NN or external service outside NN) 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(internal service inside NN or external service outside NN) or disable 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 external SPS service 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 external service outside NN or disable SPS dynamically.
 
 Storage Policy Commands
 -----------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3ac07b72/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 2ad8640..1ac9257 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
@@ -113,6 +113,8 @@ public class TestPersistentStoragePolicySatisfier {
     // Reduced refresh cycle to update latest datanodes.
     conf.setLong(DFSConfigKeys.DFS_SPS_DATANODE_CACHE_REFRESH_INTERVAL_MS,
         1000);
+    conf.setInt(
+        DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_MAX_RETRY_ATTEMPTS_KEY, 20);
     final int dnNumber = storageTypes.length;
     final short replication = 3;
     MiniDFSCluster.Builder clusterBuilder = new MiniDFSCluster.Builder(conf)
@@ -136,7 +138,7 @@ public class TestPersistentStoragePolicySatisfier {
     ctxt = new ExternalSPSContext(sps, nnc);
 
     sps.init(ctxt);
-    sps.start(true, StoragePolicySatisfierMode.EXTERNAL);
+    sps.start(StoragePolicySatisfierMode.EXTERNAL);
 
     createTestFiles(fs, replication);
   }
@@ -188,6 +190,7 @@ public class TestPersistentStoragePolicySatisfier {
    */
   @Test(timeout = 300000)
   public void testWithCheckpoint() throws Exception {
+    SecondaryNameNode secondary = null;
     try {
       clusterSetUp();
       fs.setStoragePolicy(testFile, WARM);
@@ -196,7 +199,7 @@ public class TestPersistentStoragePolicySatisfier {
       // Start the checkpoint.
       conf.set(
           DFSConfigKeys.DFS_NAMENODE_SECONDARY_HTTP_ADDRESS_KEY, "0.0.0.0:0");
-      SecondaryNameNode secondary = new SecondaryNameNode(conf);
+      secondary = new SecondaryNameNode(conf);
       secondary.doCheckpoint();
       restartCluster();
 
@@ -214,6 +217,9 @@ public class TestPersistentStoragePolicySatisfier {
           childFileName, StorageType.ARCHIVE, 3, timeout, fs);
 
     } finally {
+      if (secondary != null) {
+        secondary.shutdown();
+      }
       clusterShutdown();
     }
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3ac07b72/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/sps/TestStoragePolicySatisfierWithStripedFile.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/sps/TestStoragePolicySatisfierWithStripedFile.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/sps/TestStoragePolicySatisfierWithStripedFile.java
index 250e54b..018a5dc 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/sps/TestStoragePolicySatisfierWithStripedFile.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/sps/TestStoragePolicySatisfierWithStripedFile.java
@@ -510,7 +510,7 @@ public class TestStoragePolicySatisfierWithStripedFile {
     sps = new StoragePolicySatisfier(conf);
     ctxt = new ExternalSPSContext(sps, nnc);
     sps.init(ctxt);
-    sps.start(true, StoragePolicySatisfierMode.EXTERNAL);
+    sps.start(StoragePolicySatisfierMode.EXTERNAL);
   }
 
   private static void initConfWithStripe(Configuration conf,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3ac07b72/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/sps/TestExternalStoragePolicySatisfier.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/sps/TestExternalStoragePolicySatisfier.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/sps/TestExternalStoragePolicySatisfier.java
index d9a93fd..53878e0 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/sps/TestExternalStoragePolicySatisfier.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/sps/TestExternalStoragePolicySatisfier.java
@@ -219,7 +219,7 @@ public class TestExternalStoragePolicySatisfier {
     externalCtxt = new ExternalSPSContext(externalSps, nnc);
 
     externalSps.init(externalCtxt);
-    externalSps.start(true, StoragePolicySatisfierMode.EXTERNAL);
+    externalSps.start(StoragePolicySatisfierMode.EXTERNAL);
     return cluster;
   }
 
@@ -234,7 +234,7 @@ public class TestExternalStoragePolicySatisfier {
 
     externalCtxt = new ExternalSPSContext(externalSps, nnc);
     externalSps.init(externalCtxt);
-    externalSps.start(true, StoragePolicySatisfierMode.EXTERNAL);
+    externalSps.start(StoragePolicySatisfierMode.EXTERNAL);
   }
 
   private void initSecureConf(Configuration conf) throws Exception {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3ac07b72/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
index 1ab7788..61fccfa 100644
--- 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
@@ -71,7 +71,7 @@ public class TestStoragePolicySatisfyAdminCommands {
     Context externalCtxt = new ExternalSPSContext(externalSps, nnc);
 
     externalSps.init(externalCtxt);
-    externalSps.start(true, StoragePolicySatisfierMode.EXTERNAL);
+    externalSps.start(StoragePolicySatisfierMode.EXTERNAL);
   }
 
   @After


---------------------------------------------------------------------
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-13381 : [SPS]: Use DFSUtilClient#makePathFromFileId() to prepare satisfier file path. Contributed by Rakesh R.

Posted by um...@apache.org.
HDFS-13381 : [SPS]: Use DFSUtilClient#makePathFromFileId() to prepare satisfier file path. 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/66e8f9b3
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/66e8f9b3
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/66e8f9b3

Branch: refs/heads/HDFS-10285
Commit: 66e8f9b31529226309c924226a53dead3e6fcf11
Parents: 2acc50b
Author: Uma Maheswara Rao G <um...@intel.com>
Authored: Mon Jul 2 17:22:00 2018 -0700
Committer: Uma Maheswara Rao Gangumalla <um...@apache.org>
Committed: Sun Aug 12 03:06:06 2018 -0700

----------------------------------------------------------------------
 .../NamenodeProtocolServerSideTranslatorPB.java |  2 +-
 .../NamenodeProtocolTranslatorPB.java           |  2 +-
 .../server/blockmanagement/BlockManager.java    |  2 +-
 .../hdfs/server/namenode/FSNamesystem.java      | 11 ---
 .../hdfs/server/namenode/NameNodeRpcServer.java |  8 +-
 .../hadoop/hdfs/server/namenode/Namesystem.java |  9 ---
 .../sps/BlockStorageMovementAttemptedItems.java | 72 +++++++----------
 .../sps/BlockStorageMovementNeeded.java         | 61 ++++++--------
 .../hdfs/server/namenode/sps/Context.java       | 45 ++++++++---
 .../namenode/sps/DatanodeCacheManager.java      |  4 +-
 .../hdfs/server/namenode/sps/FileCollector.java | 13 +--
 .../namenode/sps/IntraSPSNameNodeContext.java   | 54 +++++++++----
 .../sps/IntraSPSNameNodeFileIdCollector.java    | 14 ++--
 .../hdfs/server/namenode/sps/ItemInfo.java      | 34 ++++----
 .../hdfs/server/namenode/sps/SPSService.java    | 31 +++----
 .../namenode/sps/StoragePolicySatisfier.java    | 61 +++++---------
 .../sps/StoragePolicySatisfyManager.java        | 20 ++---
 .../hdfs/server/protocol/NamenodeProtocol.java  |  2 +-
 .../sps/ExternalSPSBlockMoveTaskHandler.java    |  4 +-
 .../hdfs/server/sps/ExternalSPSContext.java     | 85 ++++++++++++++++----
 .../sps/ExternalSPSFilePathCollector.java       | 36 +++++----
 .../sps/ExternalStoragePolicySatisfier.java     | 30 +------
 .../src/main/proto/NamenodeProtocol.proto       |  2 +-
 .../TestBlockStorageMovementAttemptedItems.java | 16 ++--
 .../sps/TestStoragePolicySatisfier.java         | 66 +++++----------
 ...stStoragePolicySatisfierWithStripedFile.java | 41 ++++------
 .../sps/TestExternalStoragePolicySatisfier.java | 35 +++-----
 27 files changed, 346 insertions(+), 414 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/66e8f9b3/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/NamenodeProtocolServerSideTranslatorPB.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/NamenodeProtocolServerSideTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/NamenodeProtocolServerSideTranslatorPB.java
index e4283c6..d9367fb 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/NamenodeProtocolServerSideTranslatorPB.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/NamenodeProtocolServerSideTranslatorPB.java
@@ -265,7 +265,7 @@ public class NamenodeProtocolServerSideTranslatorPB implements
       RpcController controller, GetNextSPSPathRequestProto request)
           throws ServiceException {
     try {
-      String nextSPSPath = impl.getNextSPSPath();
+      Long nextSPSPath = impl.getNextSPSPath();
       if (nextSPSPath == null) {
         return GetNextSPSPathResponseProto.newBuilder().build();
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/66e8f9b3/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/NamenodeProtocolTranslatorPB.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/NamenodeProtocolTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/NamenodeProtocolTranslatorPB.java
index 97dee9b..3bd5986 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/NamenodeProtocolTranslatorPB.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/NamenodeProtocolTranslatorPB.java
@@ -267,7 +267,7 @@ public class NamenodeProtocolTranslatorPB implements NamenodeProtocol,
   }
 
   @Override
-  public String getNextSPSPath() throws IOException {
+  public Long getNextSPSPath() throws IOException {
     GetNextSPSPathRequestProto req =
         GetNextSPSPathRequestProto.newBuilder().build();
     try {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/66e8f9b3/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 42e246c..bae6b4e 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
@@ -3897,7 +3897,7 @@ public class BlockManager implements BlockStatsMXBean {
   private void notifyStorageMovementAttemptFinishedBlk(
       DatanodeStorageInfo storageInfo, Block block) {
     if (getSPSManager() != null) {
-      SPSService<Long> sps = getSPSManager().getInternalSPSService();
+      SPSService sps = getSPSManager().getInternalSPSService();
       if (sps.isRunning()) {
         sps.notifyStorageMovementAttemptFinishedBlk(
             storageInfo.getDatanodeDescriptor(), storageInfo.getStorageType(),

http://git-wip-us.apache.org/repos/asf/hadoop/blob/66e8f9b3/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 e1ceecd..afe9092 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
@@ -3202,17 +3202,6 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     return stat;
   }
 
-  @Override
-  public String getFilePath(Long inodeId) {
-    readLock();
-    try {
-      INode inode = getFSDirectory().getInode(inodeId);
-      return inode == null ? null : inode.getFullPathName();
-    } finally {
-      readUnlock();
-    }
-  }
-
   /**
    * Returns true if the file is closed
    */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/66e8f9b3/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 57e827d..2f3325f 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
@@ -2569,7 +2569,7 @@ public class NameNodeRpcServer implements NamenodeProtocols {
   }
 
   @Override
-  public String getNextSPSPath() throws IOException {
+  public Long getNextSPSPath() throws IOException {
     checkNNStartup();
     String operationName = "getNextSPSPath";
     namesystem.checkSuperuserPrivilege(operationName);
@@ -2589,10 +2589,6 @@ public class NameNodeRpcServer implements NamenodeProtocols {
       throw new IOException("SPS service mode is " + spsMode + ", so "
           + "external SPS service is not allowed to fetch the path Ids");
     }
-    Long pathId = spsMgr.getNextPathId();
-    if (pathId == null) {
-      return null;
-    }
-    return namesystem.getFilePath(pathId);
+    return namesystem.getBlockManager().getSPSManager().getNextPathId();
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/66e8f9b3/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 fc933b7..82af4d2 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
@@ -77,13 +77,4 @@ public interface Namesystem extends RwLock, SafeMode {
    */
   HdfsFileStatus getFileInfo(String filePath, boolean resolveLink,
       boolean needLocation) throws IOException;
-
-  /**
-   * Gets the file path corresponds to the given file id.
-   *
-   * @param inodeId
-   *          file id
-   * @return string file path
-   */
-  String getFilePath(Long inodeId);
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/66e8f9b3/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/BlockStorageMovementAttemptedItems.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/BlockStorageMovementAttemptedItems.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/BlockStorageMovementAttemptedItems.java
index 5b25491..df4f0dd 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/BlockStorageMovementAttemptedItems.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/BlockStorageMovementAttemptedItems.java
@@ -52,13 +52,8 @@ import com.google.common.annotations.VisibleForTesting;
  * 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.
- *
- * @param <T>
- *          is identifier of inode or full path name of inode. Internal sps will
- *          use the file inodeId for the block movement. External sps will use
- *          file string path representation for the block movement.
  */
-public class BlockStorageMovementAttemptedItems<T> {
+public class BlockStorageMovementAttemptedItems {
   private static final Logger LOG =
       LoggerFactory.getLogger(BlockStorageMovementAttemptedItems.class);
 
@@ -66,14 +61,14 @@ public class BlockStorageMovementAttemptedItems<T> {
    * A map holds the items which are already taken for blocks movements
    * processing and sent to DNs.
    */
-  private final List<AttemptedItemInfo<T>> storageMovementAttemptedItems;
+  private final List<AttemptedItemInfo> storageMovementAttemptedItems;
   private Map<Block, Set<StorageTypeNodePair>> scheduledBlkLocs;
   // Maintains separate Queue to keep the movement finished blocks. This Q
   // is used to update the storageMovementAttemptedItems list asynchronously.
   private final BlockingQueue<Block> movementFinishedBlocks;
   private volatile boolean monitorRunning = true;
   private Daemon timerThread = null;
-  private BlockMovementListener blkMovementListener;
+  private final Context context;
   //
   // It might take anywhere between 5 to 10 minutes before
   // a request is timed out.
@@ -85,12 +80,12 @@ public class BlockStorageMovementAttemptedItems<T> {
   // a request is timed out.
   //
   private long minCheckTimeout = 1 * 60 * 1000; // minimum value
-  private BlockStorageMovementNeeded<T> blockStorageMovementNeeded;
-  private final SPSService<T> service;
+  private BlockStorageMovementNeeded blockStorageMovementNeeded;
+  private final SPSService service;
 
-  public BlockStorageMovementAttemptedItems(SPSService<T> service,
-      BlockStorageMovementNeeded<T> unsatisfiedStorageMovementFiles,
-      BlockMovementListener blockMovementListener) {
+  public BlockStorageMovementAttemptedItems(SPSService service,
+      BlockStorageMovementNeeded unsatisfiedStorageMovementFiles,
+      Context context) {
     this.service = service;
     long recheckTimeout = this.service.getConf().getLong(
         DFS_STORAGE_POLICY_SATISFIER_RECHECK_TIMEOUT_MILLIS_KEY,
@@ -106,19 +101,27 @@ public class BlockStorageMovementAttemptedItems<T> {
     storageMovementAttemptedItems = new ArrayList<>();
     scheduledBlkLocs = new HashMap<>();
     movementFinishedBlocks = new LinkedBlockingQueue<>();
-    this.blkMovementListener = blockMovementListener;
+    this.context = context;
   }
 
   /**
    * Add item to block storage movement attempted items map which holds the
    * tracking/blockCollection id versus time stamp.
    *
-   * @param itemInfo
-   *          - tracking info
+   * @param startPathId
+   *          - start satisfier path identifier
+   * @param fileId
+   *          - file identifier
+   * @param monotonicNow
+   *          - time now
+   * @param assignedBlocks
+   *          - assigned blocks for block movement
+   * @param retryCount
+   *          - retry count
    */
-  public void add(T startPath, T file, long monotonicNow,
+  public void add(long startPathId, long fileId, long monotonicNow,
       Map<Block, Set<StorageTypeNodePair>> assignedBlocks, int retryCount) {
-    AttemptedItemInfo<T> itemInfo = new AttemptedItemInfo<T>(startPath, file,
+    AttemptedItemInfo itemInfo = new AttemptedItemInfo(startPathId, fileId,
         monotonicNow, assignedBlocks.keySet(), retryCount);
     synchronized (storageMovementAttemptedItems) {
       storageMovementAttemptedItems.add(itemInfo);
@@ -161,11 +164,9 @@ public class BlockStorageMovementAttemptedItems<T> {
       boolean foundType = dn.getStorageType().equals(type);
       if (foundDn && foundType) {
         blkLocs.remove(dn);
-        // listener if it is plugged-in
-        if (blkMovementListener != null) {
-          blkMovementListener
-              .notifyMovementTriedBlocks(new Block[] {reportedBlock});
-        }
+        Block[] mFinishedBlocks = new Block[1];
+        mFinishedBlocks[0] = reportedBlock;
+        context.notifyMovementTriedBlocks(mFinishedBlocks);
         // All the block locations has reported.
         if (blkLocs.size() <= 0) {
           movementFinishedBlocks.add(reportedBlock);
@@ -244,15 +245,15 @@ public class BlockStorageMovementAttemptedItems<T> {
   @VisibleForTesting
   void blocksStorageMovementUnReportedItemsCheck() {
     synchronized (storageMovementAttemptedItems) {
-      Iterator<AttemptedItemInfo<T>> iter = storageMovementAttemptedItems
+      Iterator<AttemptedItemInfo> iter = storageMovementAttemptedItems
           .iterator();
       long now = monotonicNow();
       while (iter.hasNext()) {
-        AttemptedItemInfo<T> itemInfo = iter.next();
+        AttemptedItemInfo itemInfo = iter.next();
         if (now > itemInfo.getLastAttemptedOrReportedTime()
             + selfRetryTimeout) {
-          T file = itemInfo.getFile();
-          ItemInfo<T> candidate = new ItemInfo<T>(itemInfo.getStartPath(), file,
+          long file = itemInfo.getFile();
+          ItemInfo candidate = new ItemInfo(itemInfo.getStartPath(), file,
               itemInfo.getRetryCount() + 1);
           blockStorageMovementNeeded.add(candidate);
           iter.remove();
@@ -272,13 +273,13 @@ public class BlockStorageMovementAttemptedItems<T> {
     // Update attempted items list
     for (Block blk : finishedBlks) {
       synchronized (storageMovementAttemptedItems) {
-        Iterator<AttemptedItemInfo<T>> iterator = storageMovementAttemptedItems
+        Iterator<AttemptedItemInfo> iterator = storageMovementAttemptedItems
             .iterator();
         while (iterator.hasNext()) {
-          AttemptedItemInfo<T> attemptedItemInfo = iterator.next();
+          AttemptedItemInfo attemptedItemInfo = iterator.next();
           attemptedItemInfo.getBlocks().remove(blk);
           if (attemptedItemInfo.getBlocks().isEmpty()) {
-            blockStorageMovementNeeded.add(new ItemInfo<T>(
+            blockStorageMovementNeeded.add(new ItemInfo(
                 attemptedItemInfo.getStartPath(), attemptedItemInfo.getFile(),
                 attemptedItemInfo.getRetryCount() + 1));
             iterator.remove();
@@ -309,15 +310,4 @@ public class BlockStorageMovementAttemptedItems<T> {
       scheduledBlkLocs.clear();
     }
   }
-
-  /**
-   * Sets external listener for testing.
-   *
-   * @param blkMoveListener
-   *          block movement listener callback object
-   */
-  @VisibleForTesting
-  void setBlockMovementListener(BlockMovementListener blkMoveListener) {
-    this.blkMovementListener = blkMoveListener;
-  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/66e8f9b3/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/BlockStorageMovementNeeded.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/BlockStorageMovementNeeded.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/BlockStorageMovementNeeded.java
index a194876..c95dcda 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/BlockStorageMovementNeeded.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/BlockStorageMovementNeeded.java
@@ -43,47 +43,38 @@ import com.google.common.annotations.VisibleForTesting;
  * 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.
- *
- * @param <T>
- *          is identifier of inode or full path name of inode. Internal sps will
- *          use the file inodeId for the block movement. External sps will use
- *          file string path representation for the block movement.
  */
 @InterfaceAudience.Private
-public class BlockStorageMovementNeeded<T> {
+public class BlockStorageMovementNeeded {
 
   public static final Logger LOG =
       LoggerFactory.getLogger(BlockStorageMovementNeeded.class);
 
-  private final Queue<ItemInfo<T>> storageMovementNeeded =
-      new LinkedList<ItemInfo<T>>();
+  private final Queue<ItemInfo> storageMovementNeeded =
+      new LinkedList<ItemInfo>();
 
   /**
    * Map of startPath and number of child's. Number of child's indicate the
    * number of files pending to satisfy the policy.
    */
-  private final Map<T, DirPendingWorkInfo> pendingWorkForDirectory =
+  private final Map<Long, DirPendingWorkInfo> pendingWorkForDirectory =
       new HashMap<>();
 
-  private final Map<T, StoragePolicySatisfyPathStatusInfo> spsStatus =
+  private final Map<Long, StoragePolicySatisfyPathStatusInfo> spsStatus =
       new ConcurrentHashMap<>();
 
-  private final Context<T> ctxt;
+  private final Context ctxt;
 
   private Daemon pathIdCollector;
 
-  private FileCollector<T> fileCollector;
-
   private SPSPathIdProcessor pathIDProcessor;
 
   // Amount of time to cache the SUCCESS status of path before turning it to
   // NOT_AVAILABLE.
   private static long statusClearanceElapsedTimeMs = 300000;
 
-  public BlockStorageMovementNeeded(Context<T> context,
-      FileCollector<T> fileCollector) {
+  public BlockStorageMovementNeeded(Context context) {
     this.ctxt = context;
-    this.fileCollector = fileCollector;
     pathIDProcessor = new SPSPathIdProcessor();
   }
 
@@ -94,7 +85,7 @@ public class BlockStorageMovementNeeded<T> {
    * @param trackInfo
    *          - track info for satisfy the policy
    */
-  public synchronized void add(ItemInfo<T> trackInfo) {
+  public synchronized void add(ItemInfo trackInfo) {
     spsStatus.put(trackInfo.getFile(),
         new StoragePolicySatisfyPathStatusInfo(
             StoragePolicySatisfyPathStatus.IN_PROGRESS));
@@ -114,7 +105,7 @@ public class BlockStorageMovementNeeded<T> {
    *          scan.
    */
   @VisibleForTesting
-  public synchronized void addAll(T startPath, List<ItemInfo<T>> itemInfoList,
+  public synchronized void addAll(long startPath, List<ItemInfo> itemInfoList,
       boolean scanCompleted) {
     storageMovementNeeded.addAll(itemInfoList);
     updatePendingDirScanStats(startPath, itemInfoList.size(), scanCompleted);
@@ -131,7 +122,7 @@ public class BlockStorageMovementNeeded<T> {
    *          elements to scan.
    */
   @VisibleForTesting
-  public synchronized void add(ItemInfo<T> itemInfo, boolean scanCompleted) {
+  public synchronized void add(ItemInfo itemInfo, boolean scanCompleted) {
     storageMovementNeeded.add(itemInfo);
     // This represents sps start id is file, so no need to update pending dir
     // stats.
@@ -141,7 +132,7 @@ public class BlockStorageMovementNeeded<T> {
     updatePendingDirScanStats(itemInfo.getFile(), 1, scanCompleted);
   }
 
-  private void updatePendingDirScanStats(T startPath, int numScannedFiles,
+  private void updatePendingDirScanStats(long startPath, int numScannedFiles,
       boolean scanCompleted) {
     DirPendingWorkInfo pendingWork = pendingWorkForDirectory.get(startPath);
     if (pendingWork == null) {
@@ -160,7 +151,7 @@ public class BlockStorageMovementNeeded<T> {
    *
    * @return satisfier files
    */
-  public synchronized ItemInfo<T> get() {
+  public synchronized ItemInfo get() {
     return storageMovementNeeded.poll();
   }
 
@@ -181,12 +172,12 @@ public class BlockStorageMovementNeeded<T> {
    * 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<T> trackInfo,
+  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.
-      T startId = trackInfo.getStartPath();
+      long startId = trackInfo.getStartPath();
       if (!ctxt.isFileExist(startId)) {
         // directory deleted just remove it.
         this.pendingWorkForDirectory.remove(startId);
@@ -212,11 +203,11 @@ public class BlockStorageMovementNeeded<T> {
     }
   }
 
-  public synchronized void clearQueue(T trackId) {
+  public synchronized void clearQueue(long trackId) {
     ctxt.removeSPSPathId(trackId);
-    Iterator<ItemInfo<T>> iterator = storageMovementNeeded.iterator();
+    Iterator<ItemInfo> iterator = storageMovementNeeded.iterator();
     while (iterator.hasNext()) {
-      ItemInfo<T> next = iterator.next();
+      ItemInfo next = iterator.next();
       if (next.getFile() == trackId) {
         iterator.remove();
       }
@@ -227,7 +218,7 @@ public class BlockStorageMovementNeeded<T> {
   /**
    * Mark inode status as SUCCESS in map.
    */
-  private void updateStatus(T startId, boolean isSuccess){
+  private void updateStatus(long startId, boolean isSuccess){
     StoragePolicySatisfyPathStatusInfo spsStatusInfo =
         spsStatus.get(startId);
     if (spsStatusInfo == null) {
@@ -249,7 +240,7 @@ public class BlockStorageMovementNeeded<T> {
    */
   public synchronized void clearQueuesWithNotification() {
     // Remove xAttr from directories
-    T trackId;
+    Long trackId;
     while ((trackId = ctxt.getNextSPSPath()) != null) {
       try {
         // Remove xAttr for file
@@ -261,7 +252,7 @@ public class BlockStorageMovementNeeded<T> {
 
     // File's directly added to storageMovementNeeded, So try to remove
     // xAttr for file
-    ItemInfo<T> itemInfo;
+    ItemInfo itemInfo;
     while ((itemInfo = get()) != null) {
       try {
         // Remove xAttr for file
@@ -287,7 +278,7 @@ public class BlockStorageMovementNeeded<T> {
     public void run() {
       LOG.info("Starting SPSPathIdProcessor!.");
       long lastStatusCleanTime = 0;
-      T startINode = null;
+      Long startINode = null;
       while (ctxt.isRunning()) {
         try {
           if (!ctxt.isInSafeMode()) {
@@ -301,7 +292,7 @@ public class BlockStorageMovementNeeded<T> {
               spsStatus.put(startINode,
                   new StoragePolicySatisfyPathStatusInfo(
                       StoragePolicySatisfyPathStatus.IN_PROGRESS));
-              fileCollector.scanAndCollectFiles(startINode);
+              ctxt.scanAndCollectFiles(startINode);
               // check if directory was empty and no child added to queue
               DirPendingWorkInfo dirPendingWorkInfo =
                   pendingWorkForDirectory.get(startINode);
@@ -339,9 +330,9 @@ public class BlockStorageMovementNeeded<T> {
     }
 
     private synchronized void cleanSPSStatus() {
-      for (Iterator<Entry<T, StoragePolicySatisfyPathStatusInfo>> it = spsStatus
-          .entrySet().iterator(); it.hasNext();) {
-        Entry<T, StoragePolicySatisfyPathStatusInfo> entry = it.next();
+      for (Iterator<Entry<Long, StoragePolicySatisfyPathStatusInfo>> it =
+          spsStatus.entrySet().iterator(); it.hasNext();) {
+        Entry<Long, StoragePolicySatisfyPathStatusInfo> entry = it.next();
         if (entry.getValue().canRemove()) {
           it.remove();
         }
@@ -477,7 +468,7 @@ public class BlockStorageMovementNeeded<T> {
     return statusClearanceElapsedTimeMs;
   }
 
-  public void markScanCompletedForDir(T inode) {
+  public void markScanCompletedForDir(long inode) {
     DirPendingWorkInfo pendingWork = pendingWorkForDirectory.get(inode);
     if (pendingWork != null) {
       pendingWork.markScanCompleted();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/66e8f9b3/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/Context.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/Context.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/Context.java
index 55a1f7a..d538374 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/Context.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/Context.java
@@ -24,24 +24,21 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.fs.ParentNotDirectoryException;
 import org.apache.hadoop.fs.UnresolvedLinkException;
+import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.server.namenode.sps.StoragePolicySatisfier.DatanodeMap;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeStorageReport;
+import org.apache.hadoop.hdfs.server.protocol.BlockStorageMovementCommand.BlockMovingInfo;
 import org.apache.hadoop.net.NetworkTopology;
 import org.apache.hadoop.security.AccessControlException;
 
 /**
  * An interface for the communication between SPS and Namenode module.
- *
- * @param <T>
- *          is identifier of inode or full path name of inode. Internal sps will
- *          use the file inodeId for the block movement. External sps will use
- *          file string path representation for the block movement.
  */
 @InterfaceAudience.Private
 @InterfaceStability.Evolving
-public interface Context<T> {
+public interface Context {
 
   /**
    * Returns true if the SPS is running, false otherwise.
@@ -85,7 +82,7 @@ public interface Context<T> {
    *          - file info
    * @return true if the given file exists, false otherwise.
    */
-  boolean isFileExist(T filePath);
+  boolean isFileExist(long filePath);
 
   /**
    * Gets the storage policy details for the given policy ID.
@@ -108,7 +105,7 @@ public interface Context<T> {
    *          - user invoked satisfier path
    * @throws IOException
    */
-  void removeSPSHint(T spsPath) throws IOException;
+  void removeSPSHint(long spsPath) throws IOException;
 
   /**
    * Gets the number of live datanodes in the cluster.
@@ -124,7 +121,7 @@ public interface Context<T> {
    *          file path
    * @return file status metadata information
    */
-  HdfsFileStatus getFileInfo(T file) throws IOException;
+  HdfsFileStatus getFileInfo(long file) throws IOException;
 
   /**
    * Returns all the live datanodes and its storage details.
@@ -137,15 +134,41 @@ public interface Context<T> {
   /**
    * @return next SPS path info to process.
    */
-  T getNextSPSPath();
+  Long getNextSPSPath();
 
   /**
    * Removes the SPS path id.
    */
-  void removeSPSPathId(T pathId);
+  void removeSPSPathId(long pathId);
 
   /**
    * Removes all SPS path ids.
    */
   void removeAllSPSPathIds();
+
+  /**
+   * Do scan and collects the files under that directory and adds to the given
+   * BlockStorageMovementNeeded.
+   *
+   * @param filePath
+   *          file path
+   */
+  void scanAndCollectFiles(long filePath)
+      throws IOException, InterruptedException;
+
+  /**
+   * Handles the block move tasks. BlockMovingInfo must contain the required
+   * info to move the block, that source location, destination location and
+   * storage types.
+   */
+  void submitMoveTask(BlockMovingInfo blkMovingInfo) throws IOException;
+
+  /**
+   * This can be used to notify to the SPS about block movement attempt
+   * finished. Then SPS will re-check whether it needs retry or not.
+   *
+   * @param moveAttemptFinishedBlks
+   *          list of movement attempt finished blocks
+   */
+  void notifyMovementTriedBlocks(Block[] moveAttemptFinishedBlks);
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/66e8f9b3/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/DatanodeCacheManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/DatanodeCacheManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/DatanodeCacheManager.java
index 3531ecd..d4e514b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/DatanodeCacheManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/DatanodeCacheManager.java
@@ -43,7 +43,7 @@ import org.slf4j.LoggerFactory;
  * interval.
  */
 @InterfaceAudience.Private
-public class DatanodeCacheManager<T> {
+public class DatanodeCacheManager {
   private static final Logger LOG = LoggerFactory
       .getLogger(DatanodeCacheManager.class);
 
@@ -78,7 +78,7 @@ public class DatanodeCacheManager<T> {
    * @throws IOException
    */
   public DatanodeMap getLiveDatanodeStorageReport(
-      Context<T> spsContext) throws IOException {
+      Context spsContext) throws IOException {
     long now = Time.monotonicNow();
     long elapsedTimeMs = now - lastAccessedTime;
     boolean refreshNeeded = elapsedTimeMs >= refreshIntervalMs;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/66e8f9b3/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/FileCollector.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/FileCollector.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/FileCollector.java
index dceb5fa..fa8b31b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/FileCollector.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/FileCollector.java
@@ -26,23 +26,18 @@ import org.apache.hadoop.classification.InterfaceStability;
 /**
  * An interface for scanning the directory recursively and collect files
  * under the given directory.
- *
- * @param <T>
- *          is identifier of inode or full path name of inode. Internal sps will
- *          use the file inodeId for the block movement. External sps will use
- *          file string path representation for the block movement.
  */
 @InterfaceAudience.Private
 @InterfaceStability.Evolving
-public interface FileCollector<T> {
+public interface FileCollector {
 
   /**
    * This method can be used to scan and collects the files under that
    * directory and adds to the given BlockStorageMovementNeeded.
    *
-   * @param filePath
-   *          - file path
+   * @param path
+   *          - file path id
    */
-  void scanAndCollectFiles(T filePath)
+  void scanAndCollectFiles(long path)
       throws IOException, InterruptedException;
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/66e8f9b3/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/IntraSPSNameNodeContext.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/IntraSPSNameNodeContext.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/IntraSPSNameNodeContext.java
index a77fe85..2bf4810 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/IntraSPSNameNodeContext.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/IntraSPSNameNodeContext.java
@@ -20,11 +20,14 @@ package org.apache.hadoop.hdfs.server.namenode.sps;
 import static org.apache.hadoop.hdfs.server.common.HdfsServerConstants.XATTR_SATISFY_STORAGE_POLICY;
 
 import java.io.IOException;
+import java.util.Arrays;
 
-import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.fs.ParentNotDirectoryException;
+import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.UnresolvedLinkException;
+import org.apache.hadoop.hdfs.DFSUtilClient;
+import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
@@ -32,6 +35,7 @@ import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.namenode.INode;
 import org.apache.hadoop.hdfs.server.namenode.Namesystem;
+import org.apache.hadoop.hdfs.server.protocol.BlockStorageMovementCommand.BlockMovingInfo;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeStorageReport;
 import org.apache.hadoop.hdfs.server.namenode.sps.StoragePolicySatisfier.DatanodeMap;
 import org.apache.hadoop.net.NetworkTopology;
@@ -45,20 +49,26 @@ import org.slf4j.LoggerFactory;
  * movements to satisfy the storage policy.
  */
 @InterfaceAudience.Private
-public class IntraSPSNameNodeContext implements Context<Long> {
+public class IntraSPSNameNodeContext implements Context {
   private static final Logger LOG = LoggerFactory
       .getLogger(IntraSPSNameNodeContext.class);
 
   private final Namesystem namesystem;
   private final BlockManager blockManager;
 
-  private SPSService<Long> service;
+  private SPSService service;
+  private final FileCollector fileCollector;
+  private final BlockMoveTaskHandler blockMoveTaskHandler;
 
   public IntraSPSNameNodeContext(Namesystem namesystem,
-      BlockManager blockManager, SPSService<Long> service) {
+      BlockManager blockManager, SPSService service) {
     this.namesystem = namesystem;
     this.blockManager = blockManager;
     this.service = service;
+    fileCollector = new IntraSPSNameNodeFileIdCollector(
+        namesystem.getFSDirectory(), service);
+    blockMoveTaskHandler = new IntraSPSNameNodeBlockMoveTaskHandler(
+        blockManager, namesystem);
   }
 
   @Override
@@ -67,17 +77,12 @@ public class IntraSPSNameNodeContext implements Context<Long> {
   }
 
   /**
-   * @return object containing information regarding the file or null if file
-   *         not found.
+   * @return object containing information regarding the file.
    */
   @Override
-  public HdfsFileStatus getFileInfo(Long inodeID) throws IOException {
-    String filePath = namesystem.getFilePath(inodeID);
-    if (StringUtils.isBlank(filePath)) {
-      LOG.debug("File with inodeID:{} doesn't exists!", inodeID);
-      return null;
-    }
-    return namesystem.getFileInfo(filePath, true, true);
+  public HdfsFileStatus getFileInfo(long inodeID) throws IOException {
+    Path filePath = DFSUtilClient.makePathFromFileId(inodeID);
+    return namesystem.getFileInfo(filePath.toString(), true, true);
   }
 
   @Override
@@ -93,12 +98,12 @@ public class IntraSPSNameNodeContext implements Context<Long> {
   }
 
   @Override
-  public boolean isFileExist(Long inodeId) {
+  public boolean isFileExist(long inodeId) {
     return namesystem.getFSDirectory().getInode(inodeId) != null;
   }
 
   @Override
-  public void removeSPSHint(Long inodeId) throws IOException {
+  public void removeSPSHint(long inodeId) throws IOException {
     this.namesystem.removeXattr(inodeId, XATTR_SATISFY_STORAGE_POLICY);
   }
 
@@ -156,7 +161,7 @@ public class IntraSPSNameNodeContext implements Context<Long> {
   }
 
   @Override
-  public void removeSPSPathId(Long trackId) {
+  public void removeSPSPathId(long trackId) {
     blockManager.getSPSManager().removePathId(trackId);
   }
 
@@ -164,4 +169,21 @@ public class IntraSPSNameNodeContext implements Context<Long> {
   public void removeAllSPSPathIds() {
     blockManager.getSPSManager().removeAllPathIds();
   }
+
+  @Override
+  public void scanAndCollectFiles(long filePath)
+      throws IOException, InterruptedException {
+    fileCollector.scanAndCollectFiles(filePath);
+  }
+
+  @Override
+  public void submitMoveTask(BlockMovingInfo blkMovingInfo) throws IOException {
+    blockMoveTaskHandler.submitMoveTask(blkMovingInfo);
+  }
+
+  @Override
+  public void notifyMovementTriedBlocks(Block[] moveAttemptFinishedBlks) {
+    LOG.info("Movement attempted blocks: {}",
+        Arrays.asList(moveAttemptFinishedBlks));
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/66e8f9b3/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/IntraSPSNameNodeFileIdCollector.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/IntraSPSNameNodeFileIdCollector.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/IntraSPSNameNodeFileIdCollector.java
index 27d9e7d..ea3b96f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/IntraSPSNameNodeFileIdCollector.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/IntraSPSNameNodeFileIdCollector.java
@@ -35,16 +35,16 @@ import org.apache.hadoop.hdfs.server.namenode.INode;
  */
 @InterfaceAudience.Private
 public class IntraSPSNameNodeFileIdCollector extends FSTreeTraverser
-    implements FileCollector<Long> {
+    implements FileCollector {
   private int maxQueueLimitToScan;
-  private final SPSService <Long> service;
+  private final SPSService service;
 
   private int remainingCapacity = 0;
 
-  private List<ItemInfo<Long>> currentBatch;
+  private List<ItemInfo> currentBatch;
 
   public IntraSPSNameNodeFileIdCollector(FSDirectory dir,
-      SPSService<Long> service) {
+      SPSService service) {
     super(dir);
     this.service = service;
     this.maxQueueLimitToScan = service.getConf().getInt(
@@ -64,7 +64,7 @@ public class IntraSPSNameNodeFileIdCollector extends FSTreeTraverser
       return false;
     }
     if (inode.isFile() && inode.asFile().numBlocks() != 0) {
-      currentBatch.add(new ItemInfo<Long>(
+      currentBatch.add(new ItemInfo(
           ((SPSTraverseInfo) traverseInfo).getStartId(), inode.getId()));
       remainingCapacity--;
     }
@@ -120,7 +120,7 @@ public class IntraSPSNameNodeFileIdCollector extends FSTreeTraverser
   }
 
   @Override
-  public void scanAndCollectFiles(final Long startINodeId)
+  public void scanAndCollectFiles(final long startINodeId)
       throws IOException, InterruptedException {
     FSDirectory fsd = getFSDirectory();
     INode startInode = fsd.getInode(startINodeId);
@@ -131,7 +131,7 @@ public class IntraSPSNameNodeFileIdCollector extends FSTreeTraverser
       }
       if (startInode.isFile()) {
         currentBatch
-            .add(new ItemInfo<Long>(startInode.getId(), startInode.getId()));
+            .add(new ItemInfo(startInode.getId(), startInode.getId()));
       } else {
         readLock();
         // NOTE: this lock will not be held for full directory scanning. It is

http://git-wip-us.apache.org/repos/asf/hadoop/blob/66e8f9b3/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/ItemInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/ItemInfo.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/ItemInfo.java
index bd8ab92..949e3fc 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/ItemInfo.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/ItemInfo.java
@@ -21,28 +21,26 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 
 /**
- * ItemInfo is a file info object for which need to satisfy the policy. For
- * internal satisfier service, it uses inode id which is Long datatype. For the
- * external satisfier service, it uses the full string representation of the
- * path.
+ * ItemInfo is a file info object for which need to satisfy the policy.
  */
 @InterfaceAudience.Private
 @InterfaceStability.Evolving
-public class ItemInfo<T> {
-  private T startPath;
-  private T file;
+public class ItemInfo {
+  private long startPathId;
+  private long fileId;
   private int retryCount;
 
-  public ItemInfo(T startPath, T file) {
-    this.startPath = startPath;
-    this.file = file;
+  public ItemInfo(long startPathId, long fileId) {
+    this.startPathId = startPathId;
+    this.fileId = fileId;
     // set 0 when item is getting added first time in queue.
     this.retryCount = 0;
   }
 
-  public ItemInfo(final T startPath, final T file, final int retryCount) {
-    this.startPath = startPath;
-    this.file = file;
+  public ItemInfo(final long startPathId, final long fileId,
+      final int retryCount) {
+    this.startPathId = startPathId;
+    this.fileId = fileId;
     this.retryCount = retryCount;
   }
 
@@ -50,22 +48,22 @@ public class ItemInfo<T> {
    * Returns the start path of the current file. This indicates that SPS
    * was invoked on this path.
    */
-  public T getStartPath() {
-    return startPath;
+  public long getStartPath() {
+    return startPathId;
   }
 
   /**
    * Returns the file for which needs to satisfy the policy.
    */
-  public T getFile() {
-    return file;
+  public long getFile() {
+    return fileId;
   }
 
   /**
    * Returns true if the tracking path is a directory, false otherwise.
    */
   public boolean isDir() {
-    return !startPath.equals(file);
+    return !(startPathId == fileId);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/66e8f9b3/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/SPSService.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/SPSService.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/SPSService.java
index 5032377..86634d8 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/SPSService.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/SPSService.java
@@ -29,15 +29,10 @@ import org.apache.hadoop.hdfs.protocol.HdfsConstants.StoragePolicySatisfierMode;
 
 /**
  * An interface for SPSService, which exposes life cycle and processing APIs.
- *
- * @param <T>
- *          is identifier of inode or full path name of inode. Internal sps will
- *          use the file inodeId for the block movement. External sps will use
- *          file string path representation for the block movement.
  */
 @InterfaceAudience.Private
 @InterfaceStability.Evolving
-public interface SPSService<T> {
+public interface SPSService {
 
   /**
    * Initializes the helper services.
@@ -45,16 +40,8 @@ public interface SPSService<T> {
    * @param ctxt
    *          - context is an helper service to provide communication channel
    *          between NN and SPS
-   * @param fileCollector
-   *          - a helper service for scanning the files under a given directory
-   *          id
-   * @param handler
-   *          - a helper service for moving the blocks
-   * @param blkMovementListener
-   *          - listener to know about block movement attempt completion
    */
-  void init(Context<T> ctxt, FileCollector<T> fileCollector,
-      BlockMoveTaskHandler handler, BlockMovementListener blkMovementListener);
+  void init(Context ctxt);
 
   /**
    * Starts the SPS service. Make sure to initialize the helper services before
@@ -94,19 +81,19 @@ public interface SPSService<T> {
    * @param itemInfo
    *          file info object for which need to satisfy the policy
    */
-  void addFileToProcess(ItemInfo<T> itemInfo, boolean scanCompleted);
+  void addFileToProcess(ItemInfo itemInfo, boolean scanCompleted);
 
   /**
    * Adds all the Item information(file etc) to processing queue.
    *
-   * @param startPath
-   *          - directory/file, on which SPS was called.
+   * @param startPathId
+   *          - directoryId/fileId, on which SPS was called.
    * @param itemInfoList
    *          - list of item infos
    * @param scanCompleted
    *          - whether the scanning of directory fully done with itemInfoList
    */
-  void addAllFilesToProcess(T startPath, List<ItemInfo<T>> itemInfoList,
+  void addAllFilesToProcess(long startPathId, List<ItemInfo> itemInfoList,
       boolean scanCompleted);
 
   /**
@@ -117,7 +104,7 @@ public interface SPSService<T> {
   /**
    * Clear inodeId present in the processing queue.
    */
-  void clearQueue(T spsPath);
+  void clearQueue(long spsPath);
 
   /**
    * @return the configuration.
@@ -128,9 +115,9 @@ public interface SPSService<T> {
    * Marks the scanning of directory if finished.
    *
    * @param spsPath
-   *          - satisfier path
+   *          - satisfier path id
    */
-  void markScanCompletedForPath(T spsPath);
+  void markScanCompletedForPath(long spsPath);
 
   /**
    * Given node is reporting that it received a certain movement attempt

http://git-wip-us.apache.org/repos/asf/hadoop/blob/66e8f9b3/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/StoragePolicySatisfier.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/StoragePolicySatisfier.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/StoragePolicySatisfier.java
index cbd6001..4af6c8f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/StoragePolicySatisfier.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/StoragePolicySatisfier.java
@@ -78,20 +78,19 @@ import com.google.common.base.Preconditions;
  * physical block movements.
  */
 @InterfaceAudience.Private
-public class StoragePolicySatisfier<T> implements SPSService<T>, Runnable {
+public class StoragePolicySatisfier implements SPSService, Runnable {
   public static final Logger LOG =
       LoggerFactory.getLogger(StoragePolicySatisfier.class);
   private Daemon storagePolicySatisfierThread;
-  private BlockStorageMovementNeeded<T> storageMovementNeeded;
-  private BlockStorageMovementAttemptedItems<T> storageMovementsMonitor;
+  private BlockStorageMovementNeeded storageMovementNeeded;
+  private BlockStorageMovementAttemptedItems storageMovementsMonitor;
   private volatile boolean isRunning = false;
   private int spsWorkMultiplier;
   private long blockCount = 0L;
   private int blockMovementMaxRetry;
-  private Context<T> ctxt;
-  private BlockMoveTaskHandler blockMoveTaskHandler;
+  private Context ctxt;
   private final Configuration conf;
-  private DatanodeCacheManager<T> dnCacheMgr;
+  private DatanodeCacheManager dnCacheMgr;
 
   public StoragePolicySatisfier(Configuration conf) {
     this.conf = conf;
@@ -137,16 +136,11 @@ public class StoragePolicySatisfier<T> implements SPSService<T>, Runnable {
     }
   }
 
-  public void init(final Context<T> context,
-      final FileCollector<T> fileIDCollector,
-      final BlockMoveTaskHandler blockMovementTaskHandler,
-      final BlockMovementListener blockMovementListener) {
+  public void init(final Context context) {
     this.ctxt = context;
-    this.storageMovementNeeded = new BlockStorageMovementNeeded<T>(context,
-        fileIDCollector);
-    this.storageMovementsMonitor = new BlockStorageMovementAttemptedItems<T>(
-        this, storageMovementNeeded, blockMovementListener);
-    this.blockMoveTaskHandler = blockMovementTaskHandler;
+    this.storageMovementNeeded = new BlockStorageMovementNeeded(context);
+    this.storageMovementsMonitor = new BlockStorageMovementAttemptedItems(
+        this, storageMovementNeeded, context);
     this.spsWorkMultiplier = getSPSWorkMultiplier(getConf());
     this.blockMovementMaxRetry = getConf().getInt(
         DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_MAX_RETRY_ATTEMPTS_KEY,
@@ -191,7 +185,7 @@ public class StoragePolicySatisfier<T> implements SPSService<T>, Runnable {
     storagePolicySatisfierThread.start();
     this.storageMovementsMonitor.start();
     this.storageMovementNeeded.activate();
-    dnCacheMgr = new DatanodeCacheManager<T>(conf);
+    dnCacheMgr = new DatanodeCacheManager(conf);
   }
 
   @Override
@@ -259,7 +253,7 @@ public class StoragePolicySatisfier<T> implements SPSService<T>, Runnable {
         continue;
       }
       try {
-        ItemInfo<T> itemInfo = null;
+        ItemInfo itemInfo = null;
         boolean retryItem = false;
         if (!ctxt.isInSafeMode()) {
           itemInfo = storageMovementNeeded.get();
@@ -271,7 +265,7 @@ public class StoragePolicySatisfier<T> implements SPSService<T>, Runnable {
               storageMovementNeeded.removeItemTrackInfo(itemInfo, false);
               continue;
             }
-            T trackId = itemInfo.getFile();
+            long trackId = itemInfo.getFile();
             BlocksMovingAnalysis status = null;
             BlockStoragePolicy existingStoragePolicy;
             // TODO: presently, context internally acquire the lock
@@ -353,7 +347,7 @@ public class StoragePolicySatisfier<T> implements SPSService<T>, Runnable {
           blockCount = 0L;
         }
         if (retryItem) {
-          itemInfo.increRetryCount();
+          // itemInfo.increRetryCount();
           this.storageMovementNeeded.add(itemInfo);
         }
       } catch (IOException e) {
@@ -469,7 +463,7 @@ public class StoragePolicySatisfier<T> implements SPSService<T>, Runnable {
     for (BlockMovingInfo blkMovingInfo : blockMovingInfos) {
       // Check for at least one block storage movement has been chosen
       try {
-        blockMoveTaskHandler.submitMoveTask(blkMovingInfo);
+        ctxt.submitMoveTask(blkMovingInfo);
         LOG.debug("BlockMovingInfo: {}", blkMovingInfo);
         StorageTypeNodePair nodeStorage = new StorageTypeNodePair(
             blkMovingInfo.getTargetStorageType(), blkMovingInfo.getTarget());
@@ -1092,7 +1086,7 @@ public class StoragePolicySatisfier<T> implements SPSService<T>, Runnable {
   }
 
   @VisibleForTesting
-  public BlockStorageMovementAttemptedItems<T> getAttemptedItemsMonitor() {
+  public BlockStorageMovementAttemptedItems getAttemptedItemsMonitor() {
     return storageMovementsMonitor;
   }
 
@@ -1109,7 +1103,7 @@ public class StoragePolicySatisfier<T> implements SPSService<T>, Runnable {
   /**
    * Clear queues for given track id.
    */
-  public void clearQueue(T trackId) {
+  public void clearQueue(long trackId) {
     storageMovementNeeded.clearQueue(trackId);
   }
 
@@ -1118,7 +1112,7 @@ public class StoragePolicySatisfier<T> implements SPSService<T>, Runnable {
    * attempted or reported time stamp. This is used by
    * {@link BlockStorageMovementAttemptedItems#storageMovementAttemptedItems}.
    */
-  final static class AttemptedItemInfo<T> extends ItemInfo<T> {
+  final static class AttemptedItemInfo extends ItemInfo {
     private long lastAttemptedOrReportedTime;
     private final Set<Block> blocks;
 
@@ -1136,7 +1130,7 @@ public class StoragePolicySatisfier<T> implements SPSService<T>, Runnable {
      * @param retryCount
      *          file retry count
      */
-    AttemptedItemInfo(T rootId, T trackId,
+    AttemptedItemInfo(long rootId, long trackId,
         long lastAttemptedOrReportedTime,
         Set<Block> blocks, int retryCount) {
       super(rootId, trackId, retryCount);
@@ -1179,7 +1173,7 @@ public class StoragePolicySatisfier<T> implements SPSService<T>, Runnable {
   }
 
   @Override
-  public void addFileToProcess(ItemInfo<T> trackInfo, boolean scanCompleted) {
+  public void addFileToProcess(ItemInfo trackInfo, boolean scanCompleted) {
     storageMovementNeeded.add(trackInfo, scanCompleted);
     if (LOG.isDebugEnabled()) {
       LOG.debug("Added track info for inode {} to block "
@@ -1188,7 +1182,7 @@ public class StoragePolicySatisfier<T> implements SPSService<T>, Runnable {
   }
 
   @Override
-  public void addAllFilesToProcess(T startPath, List<ItemInfo<T>> itemInfoList,
+  public void addAllFilesToProcess(long startPath, List<ItemInfo> itemInfoList,
       boolean scanCompleted) {
     getStorageMovementQueue().addAll(startPath, itemInfoList, scanCompleted);
   }
@@ -1204,12 +1198,12 @@ public class StoragePolicySatisfier<T> implements SPSService<T>, Runnable {
   }
 
   @VisibleForTesting
-  public BlockStorageMovementNeeded<T> getStorageMovementQueue() {
+  public BlockStorageMovementNeeded getStorageMovementQueue() {
     return storageMovementNeeded;
   }
 
   @Override
-  public void markScanCompletedForPath(T inodeId) {
+  public void markScanCompletedForPath(long inodeId) {
     getStorageMovementQueue().markScanCompletedForDir(inodeId);
   }
 
@@ -1278,15 +1272,4 @@ public class StoragePolicySatisfier<T> implements SPSService<T>, Runnable {
         "It should be a positive, non-zero integer value.");
     return spsWorkMultiplier;
   }
-
-  /**
-   * Sets external listener for testing.
-   *
-   * @param blkMovementListener
-   *          block movement listener callback object
-   */
-  @VisibleForTesting
-  void setBlockMovementListener(BlockMovementListener blkMovementListener) {
-    storageMovementsMonitor.setBlockMovementListener(blkMovementListener);
-  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/66e8f9b3/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/StoragePolicySatisfyManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/StoragePolicySatisfyManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/StoragePolicySatisfyManager.java
index 5ec0372..0507d6b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/StoragePolicySatisfyManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/StoragePolicySatisfyManager.java
@@ -60,7 +60,7 @@ import org.slf4j.LoggerFactory;
 public class StoragePolicySatisfyManager {
   private static final Logger LOG = LoggerFactory
       .getLogger(StoragePolicySatisfyManager.class);
-  private final StoragePolicySatisfier<Long> spsService;
+  private final StoragePolicySatisfier spsService;
   private final boolean storagePolicyEnabled;
   private volatile StoragePolicySatisfierMode mode;
   private final Queue<Long> pathsToBeTraveresed;
@@ -84,7 +84,7 @@ public class StoragePolicySatisfyManager {
     pathsToBeTraveresed = new LinkedList<Long>();
     // instantiate SPS service by just keeps config reference and not starting
     // any supporting threads.
-    spsService = new StoragePolicySatisfier<Long>(conf);
+    spsService = new StoragePolicySatisfier(conf);
     this.namesystem = namesystem;
     this.blkMgr = blkMgr;
   }
@@ -121,10 +121,7 @@ public class StoragePolicySatisfyManager {
       }
       // starts internal daemon service inside namenode
       spsService.init(
-          new IntraSPSNameNodeContext(namesystem, blkMgr, spsService),
-          new IntraSPSNameNodeFileIdCollector(namesystem.getFSDirectory(),
-              spsService),
-          new IntraSPSNameNodeBlockMoveTaskHandler(blkMgr, namesystem), null);
+          new IntraSPSNameNodeContext(namesystem, blkMgr, spsService));
       spsService.start(false, mode);
       break;
     case EXTERNAL:
@@ -221,13 +218,8 @@ public class StoragePolicySatisfyManager {
             mode);
         return;
       }
-      spsService.init(
-          new IntraSPSNameNodeContext(this.namesystem, this.blkMgr, spsService),
-          new IntraSPSNameNodeFileIdCollector(this.namesystem.getFSDirectory(),
-              spsService),
-          new IntraSPSNameNodeBlockMoveTaskHandler(this.blkMgr,
-              this.namesystem),
-          null);
+      spsService.init(new IntraSPSNameNodeContext(this.namesystem, this.blkMgr,
+          spsService));
       spsService.start(true, newMode);
       break;
     case EXTERNAL:
@@ -309,7 +301,7 @@ public class StoragePolicySatisfyManager {
   /**
    * @return internal SPS service instance.
    */
-  public SPSService<Long> getInternalSPSService() {
+  public SPSService getInternalSPSService() {
     return this.spsService;
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/66e8f9b3/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/NamenodeProtocol.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/NamenodeProtocol.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/NamenodeProtocol.java
index 5ff6ffd..f80477b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/NamenodeProtocol.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/NamenodeProtocol.java
@@ -209,6 +209,6 @@ public interface NamenodeProtocol {
    *         by External SPS.
    */
   @AtMostOnce
-  String getNextSPSPath() throws IOException;
+  Long getNextSPSPath() throws IOException;
 }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/66e8f9b3/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/sps/ExternalSPSBlockMoveTaskHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/sps/ExternalSPSBlockMoveTaskHandler.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/sps/ExternalSPSBlockMoveTaskHandler.java
index f5225d2..3ea0294 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/sps/ExternalSPSBlockMoveTaskHandler.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/sps/ExternalSPSBlockMoveTaskHandler.java
@@ -76,11 +76,11 @@ public class ExternalSPSBlockMoveTaskHandler implements BlockMoveTaskHandler {
   private final SaslDataTransferClient saslClient;
   private final BlockStorageMovementTracker blkMovementTracker;
   private Daemon movementTrackerThread;
-  private final SPSService<String> service;
+  private final SPSService service;
   private final BlockDispatcher blkDispatcher;
 
   public ExternalSPSBlockMoveTaskHandler(Configuration conf,
-      NameNodeConnector nnc, SPSService<String> spsService) {
+      NameNodeConnector nnc, SPSService spsService) {
     int moverThreads = conf.getInt(DFSConfigKeys.DFS_MOVER_MOVERTHREADS_KEY,
         DFSConfigKeys.DFS_MOVER_MOVERTHREADS_DEFAULT);
     moveExecutor = initializeBlockMoverThreadPool(moverThreads);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/66e8f9b3/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/sps/ExternalSPSContext.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/sps/ExternalSPSContext.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/sps/ExternalSPSContext.java
index 1cd4664..189bc2b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/sps/ExternalSPSContext.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/sps/ExternalSPSContext.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.hdfs.server.sps;
 
 import java.io.FileNotFoundException;
 import java.io.IOException;
+import java.util.ArrayList;
 import java.util.List;
 
 import org.apache.hadoop.classification.InterfaceAudience;
@@ -27,6 +28,8 @@ import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.ParentNotDirectoryException;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.UnresolvedLinkException;
+import org.apache.hadoop.hdfs.DFSUtilClient;
+import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
@@ -34,10 +37,14 @@ import org.apache.hadoop.hdfs.protocol.HdfsLocatedFileStatus;
 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.sps.BlockMoveTaskHandler;
+import org.apache.hadoop.hdfs.server.namenode.sps.BlockMovementListener;
 import org.apache.hadoop.hdfs.server.namenode.sps.Context;
+import org.apache.hadoop.hdfs.server.namenode.sps.FileCollector;
 import org.apache.hadoop.hdfs.server.namenode.sps.SPSService;
 import org.apache.hadoop.hdfs.server.namenode.sps.StoragePolicySatisfier.DatanodeMap;
 import org.apache.hadoop.hdfs.server.namenode.sps.StoragePolicySatisfier.DatanodeWithStorage;
+import org.apache.hadoop.hdfs.server.protocol.BlockStorageMovementCommand.BlockMovingInfo;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeStorageReport;
 import org.apache.hadoop.net.NetworkTopology;
 import org.apache.hadoop.security.AccessControlException;
@@ -49,17 +56,24 @@ import org.slf4j.LoggerFactory;
  * SPS from Namenode state.
  */
 @InterfaceAudience.Private
-public class ExternalSPSContext implements Context<String> {
-  public static final Logger LOG =
-      LoggerFactory.getLogger(ExternalSPSContext.class);
-  private SPSService<String> service;
-  private NameNodeConnector nnc = null;
-  private BlockStoragePolicySuite createDefaultSuite =
+public class ExternalSPSContext implements Context {
+  public static final Logger LOG = LoggerFactory
+      .getLogger(ExternalSPSContext.class);
+  private final SPSService service;
+  private final NameNodeConnector nnc;
+  private final BlockStoragePolicySuite createDefaultSuite =
       BlockStoragePolicySuite.createDefaultSuite();
+  private final FileCollector fileCollector;
+  private final BlockMoveTaskHandler externalHandler;
+  private final BlockMovementListener blkMovementListener;
 
-  public ExternalSPSContext(SPSService<String> service, NameNodeConnector nnc) {
+  public ExternalSPSContext(SPSService service, NameNodeConnector nnc) {
     this.service = service;
     this.nnc = nnc;
+    this.fileCollector = new ExternalSPSFilePathCollector(service);
+    this.externalHandler = new ExternalSPSBlockMoveTaskHandler(
+        service.getConf(), nnc, service);
+    this.blkMovementListener = new ExternalBlockMovementListener();
   }
 
   @Override
@@ -119,9 +133,10 @@ public class ExternalSPSContext implements Context<String> {
   }
 
   @Override
-  public boolean isFileExist(String filePath) {
+  public boolean isFileExist(long path) {
+    Path filePath = DFSUtilClient.makePathFromFileId(path);
     try {
-      return nnc.getDistributedFileSystem().exists(new Path(filePath));
+      return nnc.getDistributedFileSystem().exists(filePath);
     } catch (IllegalArgumentException | IOException e) {
       LOG.warn("Exception while getting file is for the given path:{}",
           filePath, e);
@@ -140,8 +155,9 @@ public class ExternalSPSContext implements Context<String> {
   }
 
   @Override
-  public void removeSPSHint(String inodeId) throws IOException {
-    nnc.getDistributedFileSystem().removeXAttr(new Path(inodeId),
+  public void removeSPSHint(long inodeId) throws IOException {
+    Path filePath = DFSUtilClient.makePathFromFileId(inodeId);
+    nnc.getDistributedFileSystem().removeXAttr(filePath,
         HdfsServerConstants.XATTR_SATISFY_STORAGE_POLICY);
   }
 
@@ -157,11 +173,12 @@ public class ExternalSPSContext implements Context<String> {
   }
 
   @Override
-  public HdfsFileStatus getFileInfo(String path) throws IOException {
+  public HdfsFileStatus getFileInfo(long path) throws IOException {
     HdfsLocatedFileStatus fileInfo = null;
     try {
+      Path filePath = DFSUtilClient.makePathFromFileId(path);
       fileInfo = nnc.getDistributedFileSystem().getClient()
-          .getLocatedFileInfo(path, false);
+          .getLocatedFileInfo(filePath.toString(), false);
     } catch (FileNotFoundException e) {
       LOG.debug("Path:{} doesn't exists!", path, e);
     }
@@ -175,7 +192,7 @@ public class ExternalSPSContext implements Context<String> {
   }
 
   @Override
-  public String getNextSPSPath() {
+  public Long getNextSPSPath() {
     try {
       return nnc.getNNProtocolConnection().getNextSPSPath();
     } catch (IOException e) {
@@ -185,7 +202,7 @@ public class ExternalSPSContext implements Context<String> {
   }
 
   @Override
-  public void removeSPSPathId(String pathId) {
+  public void removeSPSPathId(long pathId) {
     // We need not specifically implement for external.
   }
 
@@ -193,4 +210,40 @@ public class ExternalSPSContext implements Context<String> {
   public void removeAllSPSPathIds() {
     // We need not specifically implement for external.
   }
-}
+
+  @Override
+  public void scanAndCollectFiles(long path)
+      throws IOException, InterruptedException {
+    fileCollector.scanAndCollectFiles(path);
+  }
+
+  @Override
+  public void submitMoveTask(BlockMovingInfo blkMovingInfo) throws IOException {
+    externalHandler.submitMoveTask(blkMovingInfo);
+  }
+
+  @Override
+  public void notifyMovementTriedBlocks(Block[] moveAttemptFinishedBlks) {
+    // External listener if it is plugged-in
+    if (blkMovementListener != null) {
+      blkMovementListener.notifyMovementTriedBlocks(moveAttemptFinishedBlks);
+    }
+  }
+
+  /**
+   * Its an implementation of BlockMovementListener.
+   */
+  private static class ExternalBlockMovementListener
+      implements BlockMovementListener {
+
+    private List<Block> actualBlockMovements = new ArrayList<>();
+
+    @Override
+    public void notifyMovementTriedBlocks(Block[] moveAttemptFinishedBlks) {
+      for (Block block : moveAttemptFinishedBlks) {
+        actualBlockMovements.add(block);
+      }
+      LOG.info("Movement attempted blocks", actualBlockMovements);
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/66e8f9b3/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/sps/ExternalSPSFilePathCollector.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/sps/ExternalSPSFilePathCollector.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/sps/ExternalSPSFilePathCollector.java
index 9435475..611ff65 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/sps/ExternalSPSFilePathCollector.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/sps/ExternalSPSFilePathCollector.java
@@ -25,6 +25,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.DFSUtilClient;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.protocol.DirectoryListing;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
@@ -41,14 +42,14 @@ import org.slf4j.LoggerFactory;
  * representation.
  */
 @InterfaceAudience.Private
-public class ExternalSPSFilePathCollector implements FileCollector <String>{
+public class ExternalSPSFilePathCollector implements FileCollector {
   public static final Logger LOG =
       LoggerFactory.getLogger(ExternalSPSFilePathCollector.class);
   private DistributedFileSystem dfs;
-  private SPSService<String> service;
+  private SPSService service;
   private int maxQueueLimitToScan;
 
-  public ExternalSPSFilePathCollector(SPSService<String> service) {
+  public ExternalSPSFilePathCollector(SPSService service) {
     this.service = service;
     this.maxQueueLimitToScan = service.getConf().getInt(
         DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_QUEUE_LIMIT_KEY,
@@ -72,13 +73,13 @@ public class ExternalSPSFilePathCollector implements FileCollector <String>{
    * Recursively scan the given path and add the file info to SPS service for
    * processing.
    */
-  private long processPath(String startID, String childPath) {
+  private long processPath(Long startID, String childPath) {
     long pendingWorkCount = 0; // to be satisfied file counter
     for (byte[] lastReturnedName = HdfsFileStatus.EMPTY_NAME;;) {
       final DirectoryListing children;
       try {
-        children = dfs.getClient().listPaths(childPath, lastReturnedName,
-            false);
+        children = dfs.getClient().listPaths(childPath,
+            lastReturnedName, false);
       } catch (IOException e) {
         LOG.warn("Failed to list directory " + childPath
             + ". Ignore the directory and continue.", e);
@@ -93,18 +94,18 @@ public class ExternalSPSFilePathCollector implements FileCollector <String>{
       }
 
       for (HdfsFileStatus child : children.getPartialListing()) {
-        String childFullPath = child.getFullName(childPath);
         if (child.isFile()) {
-          service.addFileToProcess(
-              new ItemInfo<String>(startID, childFullPath), false);
+          service.addFileToProcess(new ItemInfo(startID, child.getFileId()),
+              false);
           checkProcessingQueuesFree();
           pendingWorkCount++; // increment to be satisfied file count
         } else {
+          String childFullPathName = child.getFullName(childPath);
           if (child.isDirectory()) {
-            if (!childFullPath.endsWith(Path.SEPARATOR)) {
-              childFullPath = childFullPath + Path.SEPARATOR;
+            if (!childFullPathName.endsWith(Path.SEPARATOR)) {
+              childFullPathName = childFullPathName + Path.SEPARATOR;
             }
-            pendingWorkCount += processPath(startID, childFullPath);
+            pendingWorkCount += processPath(startID, childFullPathName);
           }
         }
       }
@@ -150,11 +151,12 @@ public class ExternalSPSFilePathCollector implements FileCollector <String>{
   }
 
   @Override
-  public void scanAndCollectFiles(String path) throws IOException {
+  public void scanAndCollectFiles(long pathId) throws IOException {
     if (dfs == null) {
       dfs = getFS(service.getConf());
     }
-    long pendingSatisfyItemsCount = processPath(path, path);
+    Path filePath = DFSUtilClient.makePathFromFileId(pathId);
+    long pendingSatisfyItemsCount = processPath(pathId, filePath.toString());
     // Check whether the given path contains any item to be tracked
     // or the no to be satisfied paths. In case of empty list, add the given
     // inodeId to the 'pendingWorkForDirectory' with empty list so that later
@@ -162,10 +164,10 @@ public class ExternalSPSFilePathCollector implements FileCollector <String>{
     // this path is already satisfied the storage policy.
     if (pendingSatisfyItemsCount <= 0) {
       LOG.debug("There is no pending items to satisfy the given path "
-          + "inodeId:{}", path);
-      service.addAllFilesToProcess(path, new ArrayList<>(), true);
+          + "inodeId:{}", pathId);
+      service.addAllFilesToProcess(pathId, new ArrayList<>(), true);
     } else {
-      service.markScanCompletedForPath(path);
+      service.markScanCompletedForPath(pathId);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/66e8f9b3/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/sps/ExternalStoragePolicySatisfier.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/sps/ExternalStoragePolicySatisfier.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/sps/ExternalStoragePolicySatisfier.java
index 236b887..af90f0d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/sps/ExternalStoragePolicySatisfier.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/sps/ExternalStoragePolicySatisfier.java
@@ -22,7 +22,6 @@ import static org.apache.hadoop.util.ExitUtil.terminate;
 import java.io.IOException;
 import java.net.InetSocketAddress;
 import java.net.URI;
-import java.util.ArrayList;
 import java.util.Collection;
 import java.util.List;
 
@@ -32,11 +31,9 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
-import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.StoragePolicySatisfierMode;
 import org.apache.hadoop.hdfs.server.balancer.NameNodeConnector;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
-import org.apache.hadoop.hdfs.server.namenode.sps.BlockMovementListener;
 import org.apache.hadoop.hdfs.server.namenode.sps.StoragePolicySatisfier;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.SecurityUtil;
@@ -68,8 +65,7 @@ public final class ExternalStoragePolicySatisfier {
       HdfsConfiguration spsConf = new HdfsConfiguration();
       // login with SPS keytab
       secureLogin(spsConf);
-      StoragePolicySatisfier<String> sps = new StoragePolicySatisfier<String>(
-          spsConf);
+      StoragePolicySatisfier sps = new StoragePolicySatisfier(spsConf);
       nnc = getNameNodeConnector(spsConf);
 
       boolean spsRunning;
@@ -82,12 +78,7 @@ public final class ExternalStoragePolicySatisfier {
       }
 
       ExternalSPSContext context = new ExternalSPSContext(sps, nnc);
-      ExternalBlockMovementListener blkMoveListener =
-          new ExternalBlockMovementListener();
-      ExternalSPSBlockMoveTaskHandler externalHandler =
-          new ExternalSPSBlockMoveTaskHandler(spsConf, nnc, sps);
-      sps.init(context, new ExternalSPSFilePathCollector(sps), externalHandler,
-          blkMoveListener);
+      sps.init(context);
       sps.start(true, StoragePolicySatisfierMode.EXTERNAL);
       if (sps != null) {
         sps.join();
@@ -132,21 +123,4 @@ public final class ExternalStoragePolicySatisfier {
       }
     }
   }
-
-  /**
-   * It is implementation of BlockMovementListener.
-   */
-  private static class ExternalBlockMovementListener
-      implements BlockMovementListener {
-
-    private List<Block> actualBlockMovements = new ArrayList<>();
-
-    @Override
-    public void notifyMovementTriedBlocks(Block[] moveAttemptFinishedBlks) {
-      for (Block block : moveAttemptFinishedBlks) {
-        actualBlockMovements.add(block);
-      }
-      LOG.info("Movement attempted blocks:{}", actualBlockMovements);
-    }
-  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/66e8f9b3/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/NamenodeProtocol.proto
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/NamenodeProtocol.proto b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/NamenodeProtocol.proto
index 2acc5a8..89edfbf 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/NamenodeProtocol.proto
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/NamenodeProtocol.proto
@@ -218,7 +218,7 @@ message GetNextSPSPathRequestProto {
 }
 
 message GetNextSPSPathResponseProto {
-  optional string spsPath = 1;
+  optional uint64 spsPath = 1;
 }
 
 /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/66e8f9b3/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/sps/TestBlockStorageMovementAttemptedItems.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/sps/TestBlockStorageMovementAttemptedItems.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/sps/TestBlockStorageMovementAttemptedItems.java
index ed1fe92..f85769f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/sps/TestBlockStorageMovementAttemptedItems.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/sps/TestBlockStorageMovementAttemptedItems.java
@@ -45,22 +45,22 @@ import org.mockito.Mockito;
  */
 public class TestBlockStorageMovementAttemptedItems {
 
-  private BlockStorageMovementAttemptedItems<Long> bsmAttemptedItems;
-  private BlockStorageMovementNeeded<Long> unsatisfiedStorageMovementFiles;
+  private BlockStorageMovementAttemptedItems bsmAttemptedItems;
+  private BlockStorageMovementNeeded unsatisfiedStorageMovementFiles;
   private final int selfRetryTimeout = 500;
 
   @Before
   public void setup() throws Exception {
     Configuration config = new HdfsConfiguration();
-    Context<Long> ctxt = Mockito.mock(IntraSPSNameNodeContext.class);
-    SPSService<Long> sps = new StoragePolicySatisfier<Long>(config);
+    Context ctxt = Mockito.mock(IntraSPSNameNodeContext.class);
+    SPSService sps = new StoragePolicySatisfier(config);
     Mockito.when(ctxt.isRunning()).thenReturn(true);
     Mockito.when(ctxt.isInSafeMode()).thenReturn(false);
     Mockito.when(ctxt.isFileExist(Mockito.anyLong())).thenReturn(true);
     unsatisfiedStorageMovementFiles =
-        new BlockStorageMovementNeeded<Long>(ctxt, null);
-    bsmAttemptedItems = new BlockStorageMovementAttemptedItems<Long>(sps,
-        unsatisfiedStorageMovementFiles, null);
+        new BlockStorageMovementNeeded(ctxt);
+    bsmAttemptedItems = new BlockStorageMovementAttemptedItems(sps,
+        unsatisfiedStorageMovementFiles, ctxt);
   }
 
   @After
@@ -76,7 +76,7 @@ public class TestBlockStorageMovementAttemptedItems {
     long stopTime = monotonicNow() + (retryTimeout * 2);
     boolean isItemFound = false;
     while (monotonicNow() < (stopTime)) {
-      ItemInfo<Long> ele = null;
+      ItemInfo ele = null;
       while ((ele = unsatisfiedStorageMovementFiles.get()) != null) {
         if (item == ele.getFile()) {
           isItemFound = true;


---------------------------------------------------------------------
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-13165: [SPS]: Collects successfully moved block details via IBR. Contributed by Rakesh R.

Posted by um...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/2acc50b8/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/sps/ExternalSPSBlockMoveTaskHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/sps/ExternalSPSBlockMoveTaskHandler.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/sps/ExternalSPSBlockMoveTaskHandler.java
index 7580ba9..f5225d2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/sps/ExternalSPSBlockMoveTaskHandler.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/sps/ExternalSPSBlockMoveTaskHandler.java
@@ -20,13 +20,10 @@ package org.apache.hadoop.hdfs.server.sps;
 
 import java.io.IOException;
 import java.net.Socket;
-import java.util.ArrayList;
-import java.util.List;
 import java.util.concurrent.Callable;
 import java.util.concurrent.CompletionService;
 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;
@@ -39,7 +36,6 @@ import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSUtilClient;
 import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
-import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.datatransfer.TrustedChannelResolver;
@@ -48,15 +44,14 @@ import org.apache.hadoop.hdfs.protocol.datatransfer.sasl.SaslDataTransferClient;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
 import org.apache.hadoop.hdfs.server.balancer.KeyManager;
 import org.apache.hadoop.hdfs.server.balancer.NameNodeConnector;
+import org.apache.hadoop.hdfs.server.common.sps.BlockDispatcher;
 import org.apache.hadoop.hdfs.server.common.sps.BlockMovementAttemptFinished;
 import org.apache.hadoop.hdfs.server.common.sps.BlockMovementStatus;
 import org.apache.hadoop.hdfs.server.common.sps.BlockStorageMovementTracker;
 import org.apache.hadoop.hdfs.server.common.sps.BlocksMovementsStatusHandler;
-import org.apache.hadoop.hdfs.server.common.sps.BlockDispatcher;
 import org.apache.hadoop.hdfs.server.namenode.sps.BlockMoveTaskHandler;
 import org.apache.hadoop.hdfs.server.namenode.sps.SPSService;
 import org.apache.hadoop.hdfs.server.protocol.BlockStorageMovementCommand.BlockMovingInfo;
-import org.apache.hadoop.hdfs.server.protocol.BlocksStorageMoveAttemptFinished;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.util.Daemon;
 import org.slf4j.Logger;
@@ -105,12 +100,14 @@ public class ExternalSPSBlockMoveTaskHandler implements BlockMoveTaskHandler {
     int ioFileBufferSize = DFSUtilClient.getIoFileBufferSize(conf);
     blkDispatcher = new BlockDispatcher(HdfsConstants.READ_TIMEOUT,
         ioFileBufferSize, connectToDnViaHostname);
+
+    startMovementTracker();
   }
 
   /**
    * Initializes block movement tracker daemon and starts the thread.
    */
-  public void init() {
+  private void startMovementTracker() {
     movementTrackerThread = new Daemon(this.blkMovementTracker);
     movementTrackerThread.setName("BlockStorageMovementTracker");
     movementTrackerThread.start();
@@ -156,24 +153,16 @@ public class ExternalSPSBlockMoveTaskHandler implements BlockMoveTaskHandler {
     // dn.incrementBlocksScheduled(blkMovingInfo.getTargetStorageType());
     LOG.debug("Received BlockMovingTask {}", blkMovingInfo);
     BlockMovingTask blockMovingTask = new BlockMovingTask(blkMovingInfo);
-    Future<BlockMovementAttemptFinished> moveCallable = mCompletionServ
-        .submit(blockMovingTask);
-    blkMovementTracker.addBlock(blkMovingInfo.getBlock(), moveCallable);
+    mCompletionServ.submit(blockMovingTask);
   }
 
   private class ExternalBlocksMovementsStatusHandler
-      extends BlocksMovementsStatusHandler {
+      implements BlocksMovementsStatusHandler {
     @Override
-    public void handle(
-        List<BlockMovementAttemptFinished> moveAttemptFinishedBlks) {
-      List<Block> blocks = new ArrayList<>();
-      for (BlockMovementAttemptFinished item : moveAttemptFinishedBlks) {
-        blocks.add(item.getBlock());
-      }
-      BlocksStorageMoveAttemptFinished blkAttempted =
-          new BlocksStorageMoveAttemptFinished(
-          blocks.toArray(new Block[blocks.size()]));
-      service.notifyStorageMovementAttemptFinishedBlks(blkAttempted);
+    public void handle(BlockMovementAttemptFinished attemptedMove) {
+      service.notifyStorageMovementAttemptFinishedBlk(
+          attemptedMove.getTargetDatanode(), attemptedMove.getTargetType(),
+          attemptedMove.getBlock());
     }
   }
 
@@ -194,6 +183,7 @@ public class ExternalSPSBlockMoveTaskHandler implements BlockMoveTaskHandler {
       BlockMovementStatus blkMovementStatus = moveBlock();
       return new BlockMovementAttemptFinished(blkMovingInfo.getBlock(),
           blkMovingInfo.getSource(), blkMovingInfo.getTarget(),
+          blkMovingInfo.getTargetStorageType(),
           blkMovementStatus);
     }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2acc50b8/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/sps/ExternalStoragePolicySatisfier.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/sps/ExternalStoragePolicySatisfier.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/sps/ExternalStoragePolicySatisfier.java
index 6fc35ea..236b887 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/sps/ExternalStoragePolicySatisfier.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/sps/ExternalStoragePolicySatisfier.java
@@ -86,7 +86,6 @@ public final class ExternalStoragePolicySatisfier {
           new ExternalBlockMovementListener();
       ExternalSPSBlockMoveTaskHandler externalHandler =
           new ExternalSPSBlockMoveTaskHandler(spsConf, nnc, sps);
-      externalHandler.init();
       sps.init(context, new ExternalSPSFilePathCollector(sps), externalHandler,
           blkMoveListener);
       sps.start(true, StoragePolicySatisfierMode.EXTERNAL);
@@ -147,7 +146,7 @@ public final class ExternalStoragePolicySatisfier {
       for (Block block : moveAttemptFinishedBlks) {
         actualBlockMovements.add(block);
       }
-      LOG.info("Movement attempted blocks", actualBlockMovements);
+      LOG.info("Movement attempted blocks:{}", actualBlockMovements);
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2acc50b8/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 7c35494..baf7ec7 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/DatanodeProtocol.proto
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/DatanodeProtocol.proto
@@ -185,14 +185,6 @@ message BlockMovingInfoProto {
 }
 
 /**
- * Blocks for which storage movements has been attempted and finished
- * with either success or failure.
- */
-message BlocksStorageMoveAttemptFinishedProto {
-  repeated BlockProto blocks = 1;
-}
-
-/**
  * registration - Information of the datanode registering with the namenode
  */
 message RegisterDatanodeRequestProto {
@@ -249,7 +241,6 @@ message HeartbeatRequestProto {
   optional bool requestFullBlockReportLease = 9 [ default = false ];
   repeated SlowPeerReportProto slowPeers = 10;
   repeated SlowDiskReportProto slowDisks = 11;
-  optional BlocksStorageMoveAttemptFinishedProto storageMoveAttemptFinishedBlks = 12;
 }
 
 /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2acc50b8/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 0f80f97..0b533c2 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
@@ -4592,6 +4592,47 @@
 </property>
 
 <property>
+  <name>dfs.storage.policy.satisfier.max.outstanding.paths</name>
+  <value>10000</value>
+  <description>
+    Defines the maximum number of paths to satisfy that can be queued up in the
+    Satisfier call queue in a period of time. Default value is 10000.
+  </description>
+</property>
+
+<property>
+  <name>dfs.storage.policy.satisfier.address</name>
+  <value>0.0.0.0:0</value>
+  <description>
+    The hostname used for a keytab based Kerberos login. Keytab based login
+    is required when dfs.storage.policy.satisfier.mode is external.
+  </description>
+</property>
+
+<property>
+  <name>dfs.storage.policy.satisfier.keytab.file</name>
+  <value></value>
+  <description>
+    The keytab file used by external StoragePolicySatisfier to login as its
+    service principal. The principal name is configured with
+    dfs.storage.policy.satisfier.kerberos.principal. Keytab based login
+    is required when dfs.storage.policy.satisfier.mode is external.
+  </description>
+</property>
+
+<property>
+  <name>dfs.storage.policy.satisfier.kerberos.principal</name>
+  <value></value>
+  <description>
+    The StoragePolicySatisfier principal. This is typically set to
+    satisfier/_HOST@REALM.TLD. The StoragePolicySatisfier will substitute
+    _HOST with its own fully qualified hostname at startup. The _HOST placeholder
+    allows using the same configuration setting on different servers. Keytab
+    based login is required when dfs.storage.policy.satisfier.mode is external.
+  </description>
+</property>
+
+<property>
   <name>dfs.pipeline.ecn</name>
   <value>false</value>
   <description>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2acc50b8/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 f247370..05b6d30 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,6 @@ 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.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,8 +116,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,
-          new BlocksStorageMoveAttemptFinished(null));
+          SlowDiskReports.EMPTY_REPORT);
 
       // Check that the missing storage was pruned.
       assertThat(dnDescriptor.getStorageInfos().length, is(expectedStoragesAfterTest));

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2acc50b8/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 d13d717..b453991 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,6 @@ 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.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;
@@ -168,8 +167,7 @@ public class InternalDataNodeTestUtils {
             Mockito.anyInt(), Mockito.any(VolumeFailureSummary.class),
             Mockito.anyBoolean(),
             Mockito.any(SlowPeerReports.class),
-            Mockito.any(SlowDiskReports.class),
-            Mockito.any(BlocksStorageMoveAttemptFinished.class))).thenReturn(
+            Mockito.any(SlowDiskReports.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/2acc50b8/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/SimpleBlocksMovementsStatusHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/SimpleBlocksMovementsStatusHandler.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/SimpleBlocksMovementsStatusHandler.java
new file mode 100644
index 0000000..b361ce5
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/SimpleBlocksMovementsStatusHandler.java
@@ -0,0 +1,88 @@
+/**
+ * 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.Collections;
+import java.util.List;
+
+import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.server.common.sps.BlockMovementAttemptFinished;
+import org.apache.hadoop.hdfs.server.common.sps.BlocksMovementsStatusHandler;
+
+/**
+ * Blocks movements status handler, which is used to collect details of the
+ * completed block movements and later these attempted finished(with success or
+ * failure) blocks can be accessed to notify respective listeners, if any.
+ */
+public class SimpleBlocksMovementsStatusHandler
+    implements BlocksMovementsStatusHandler {
+  private final List<Block> blockIdVsMovementStatus = new ArrayList<>();
+
+  /**
+   * Collect all the storage movement attempt finished blocks. Later this will
+   * be send to namenode via heart beat.
+   *
+   * @param moveAttemptFinishedBlk
+   *          storage movement attempt finished block
+   */
+  public void handle(BlockMovementAttemptFinished moveAttemptFinishedBlk) {
+    // Adding to the tracking report list. Later this can be accessed to know
+    // the attempted block movements.
+    synchronized (blockIdVsMovementStatus) {
+      blockIdVsMovementStatus.add(moveAttemptFinishedBlk.getBlock());
+    }
+  }
+
+  /**
+   * @return unmodifiable list of storage movement attempt finished blocks.
+   */
+  public 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);
+      }
+    }
+    return moveAttemptFinishedBlks;
+  }
+
+  /**
+   * Remove the storage movement attempt finished blocks from the tracking list.
+   *
+   * @param moveAttemptFinishedBlks
+   *          set of storage movement attempt finished blocks
+   */
+  public void remove(List<Block> moveAttemptFinishedBlks) {
+    if (moveAttemptFinishedBlks != null) {
+      blockIdVsMovementStatus.removeAll(moveAttemptFinishedBlks);
+    }
+  }
+
+  /**
+   * Clear the blockID vs movement status tracking map.
+   */
+  public void removeAll() {
+    synchronized (blockIdVsMovementStatus) {
+      blockIdVsMovementStatus.clear();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2acc50b8/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 0fa1696..d0c3a83 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
@@ -49,7 +49,6 @@ 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.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;
@@ -124,8 +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();
+    Mockito.doReturn(new StoragePolicySatisfyWorker(conf, mockDn, null))
+        .when(mockDn).getStoragePolicySatisfyWorker();
 
     // Set up a simulated dataset with our fake BP
     mockFSDataset = Mockito.spy(new SimulatedFSDataset(null, conf));
@@ -160,8 +159,7 @@ public class TestBPOfferService {
           Mockito.any(VolumeFailureSummary.class),
           Mockito.anyBoolean(),
           Mockito.any(SlowPeerReports.class),
-          Mockito.any(SlowDiskReports.class),
-          Mockito.any(BlocksStorageMoveAttemptFinished.class));
+          Mockito.any(SlowDiskReports.class));
     mockHaStatuses[nnIdx] = new NNHAStatusHeartbeat(HAServiceState.STANDBY, 0);
     datanodeCommands[nnIdx] = new DatanodeCommand[0];
     return mock;
@@ -380,8 +378,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();
+    Mockito.doReturn(new StoragePolicySatisfyWorker(conf, mockDn, null))
+        .when(mockDn).getStoragePolicySatisfyWorker();
     final AtomicInteger count = new AtomicInteger();
     Mockito.doAnswer(new Answer<Void>() {
       @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2acc50b8/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 052eb87..07fd4ae 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
@@ -93,7 +93,6 @@ 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.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;
@@ -233,8 +232,7 @@ public class TestBlockRecovery {
             Mockito.any(VolumeFailureSummary.class),
             Mockito.anyBoolean(),
             Mockito.any(SlowPeerReports.class),
-            Mockito.any(SlowDiskReports.class),
-            Mockito.any(BlocksStorageMoveAttemptFinished.class)))
+            Mockito.any(SlowDiskReports.class)))
         .thenReturn(new HeartbeatResponse(
             new DatanodeCommand[0],
             new NNHAStatusHeartbeat(HAServiceState.ACTIVE, 1),

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2acc50b8/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 0dd15c3..28427bc 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,6 @@ 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.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;
@@ -173,8 +172,7 @@ public class TestDataNodeLifeline {
             any(VolumeFailureSummary.class),
             anyBoolean(),
             any(SlowPeerReports.class),
-            any(SlowDiskReports.class),
-            any(BlocksStorageMoveAttemptFinished.class));
+            any(SlowDiskReports.class));
 
     // Intercept lifeline to trigger latch count-down on each call.
     doAnswer(new LatchCountingAnswer<Void>(lifelinesSent))
@@ -239,8 +237,7 @@ public class TestDataNodeLifeline {
             any(VolumeFailureSummary.class),
             anyBoolean(),
             any(SlowPeerReports.class),
-            any(SlowDiskReports.class),
-            any(BlocksStorageMoveAttemptFinished.class));
+            any(SlowDiskReports.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/2acc50b8/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 d47da69..bb1d9ef 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,6 @@ 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.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;
@@ -223,8 +222,7 @@ public class TestDatanodeProtocolRetryPolicy {
            Mockito.any(VolumeFailureSummary.class),
            Mockito.anyBoolean(),
            Mockito.any(SlowPeerReports.class),
-           Mockito.any(SlowDiskReports.class),
-           Mockito.any(BlocksStorageMoveAttemptFinished.class));
+           Mockito.any(SlowDiskReports.class));
 
     dn = new DataNode(conf, locations, null, null) {
       @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2acc50b8/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 3732b2e..2dbd5b9 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,6 @@ 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.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;
@@ -209,8 +208,7 @@ public class TestFsDatasetCache {
           (StorageReport[]) any(), anyLong(), anyLong(),
           anyInt(), anyInt(), anyInt(), (VolumeFailureSummary) any(),
           anyBoolean(), any(SlowPeerReports.class),
-          any(SlowDiskReports.class),
-          any(BlocksStorageMoveAttemptFinished.class));
+          any(SlowDiskReports.class));
     } finally {
       lock.writeLock().unlock();
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2acc50b8/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 06a66f7..51d3254 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,8 +17,6 @@
  */
 package org.apache.hadoop.hdfs.server.datanode;
 
-import static org.junit.Assert.assertTrue;
-
 import java.io.IOException;
 import java.net.InetSocketAddress;
 import java.util.ArrayList;
@@ -35,8 +33,8 @@ 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.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.StoragePolicySatisfierMode;
+import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.server.protocol.BlockStorageMovementCommand.BlockMovingInfo;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.junit.After;
@@ -173,8 +171,10 @@ public class TestStoragePolicySatisfyWorker {
     DatanodeInfo targetDnInfo = DFSTestUtil
         .getLocalDatanodeInfo(src.getXferPort());
 
+    SimpleBlocksMovementsStatusHandler handler =
+        new SimpleBlocksMovementsStatusHandler();
     StoragePolicySatisfyWorker worker = new StoragePolicySatisfyWorker(conf,
-        src);
+        src, handler);
     try {
       worker.start();
       List<BlockMovingInfo> blockMovingInfos = new ArrayList<>();
@@ -184,81 +184,19 @@ public class TestStoragePolicySatisfyWorker {
       blockMovingInfos.add(blockMovingInfo);
       worker.processBlockMovingTasks(cluster.getNamesystem().getBlockPoolId(),
           blockMovingInfos);
-
-      waitForBlockMovementCompletion(worker, 1, 30000);
+      waitForBlockMovementCompletion(handler, 1, 30000);
     } finally {
       worker.stop();
     }
   }
 
-  /**
-   * 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);
-    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);
-      }
-      worker.processBlockMovingTasks(cluster.getNamesystem().getBlockPoolId(),
-          blockMovingInfos);
-      // Wait till results queue build up
-      waitForBlockMovementResult(worker, 30000);
-      worker.dropSPSWork();
-      assertTrue(worker.getBlocksMovementsStatusHandler()
-          .getMoveAttemptFinishedBlocks().size() == 0);
-    } finally {
-      worker.stop();
-    }
-  }
-
-  private void waitForBlockMovementResult(
-      final StoragePolicySatisfyWorker worker, int timeout) throws Exception {
-    GenericTestUtils.waitFor(new Supplier<Boolean>() {
-      @Override
-      public Boolean get() {
-        List<Block> completedBlocks = worker.getBlocksMovementsStatusHandler()
-            .getMoveAttemptFinishedBlocks();
-        return completedBlocks.size() > 0;
-      }
-    }, 100, timeout);
-  }
-
   private void waitForBlockMovementCompletion(
-      final StoragePolicySatisfyWorker worker,
+      final SimpleBlocksMovementsStatusHandler handler,
       int expectedFinishedItemsCount, int timeout) throws Exception {
     GenericTestUtils.waitFor(new Supplier<Boolean>() {
       @Override
       public Boolean get() {
-        List<Block> completedBlocks = worker.getBlocksMovementsStatusHandler()
-            .getMoveAttemptFinishedBlocks();
+        List<Block> completedBlocks = handler.getMoveAttemptFinishedBlocks();
         int finishedCount = completedBlocks.size();
         LOG.info("Block movement completed count={}, expected={} and actual={}",
             completedBlocks.size(), expectedFinishedItemsCount, finishedCount);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2acc50b8/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 20402f2..5f62ddb 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,6 @@ 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.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;
@@ -111,8 +110,7 @@ public class TestStorageReport {
         anyLong(), anyLong(), anyInt(), anyInt(), anyInt(),
         Mockito.any(VolumeFailureSummary.class), Mockito.anyBoolean(),
         Mockito.any(SlowPeerReports.class),
-        Mockito.any(SlowDiskReports.class),
-        Mockito.any(BlocksStorageMoveAttemptFinished.class));
+        Mockito.any(SlowDiskReports.class));
 
     StorageReport[] reports = captor.getValue();
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2acc50b8/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 ec00ae7..3a3c471 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,6 @@ 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.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;
@@ -957,8 +956,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,
-          new BlocksStorageMoveAttemptFinished(null)).getCommands();
+          SlowPeerReports.EMPTY_REPORT, SlowDiskReports.EMPTY_REPORT)
+          .getCommands();
       if(cmds != null) {
         for (DatanodeCommand cmd : cmds ) {
           if(LOG.isDebugEnabled()) {
@@ -1008,8 +1007,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,
-          new BlocksStorageMoveAttemptFinished(null)).getCommands();
+          SlowPeerReports.EMPTY_REPORT, SlowDiskReports.EMPTY_REPORT)
+          .getCommands();
       if (cmds != null) {
         for (DatanodeCommand cmd : cmds) {
           if (cmd.getAction() == DatanodeProtocol.DNA_TRANSFER) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2acc50b8/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 899bb82..b85527a 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,6 @@ 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.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;
@@ -131,8 +130,7 @@ 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,
-        new BlocksStorageMoveAttemptFinished(null));
+        SlowPeerReports.EMPTY_REPORT, SlowDiskReports.EMPTY_REPORT);
   }
 
   public static boolean setReplication(final FSNamesystem ns,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2acc50b8/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 65628b9..df74107 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,6 @@ 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.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,8 +139,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,
-            new BlocksStorageMoveAttemptFinished(null)).getCommands();
+            SlowPeerReports.EMPTY_REPORT, SlowDiskReports.EMPTY_REPORT)
+        .getCommands();
     assertEquals(1, cmd.length);
     assertEquals(cmd[0].getAction(), RegisterCommand.REGISTER
         .getAction());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2acc50b8/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 47ea39f..ee0b2e6 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
@@ -37,6 +37,7 @@ 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;
+import org.apache.hadoop.hdfs.server.namenode.sps.StoragePolicySatisfyManager;
 import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.test.GenericTestUtils;
 
@@ -250,10 +251,9 @@ public class TestNameNodeReconfigure {
         StoragePolicySatisfierMode.INTERNAL.toString());
 
     // 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().getSPSManager()
-            .isInternalSatisfierRunning());
+    assertNull("SPS shouldn't start as "
+        + DFSConfigKeys.DFS_STORAGE_POLICY_ENABLED_KEY + " is disabled",
+            nameNode.getNamesystem().getBlockManager().getSPSManager());
     verifySPSEnabled(nameNode, DFS_STORAGE_POLICY_SATISFIER_MODE_KEY,
         StoragePolicySatisfierMode.INTERNAL, false);
 
@@ -352,9 +352,12 @@ public class TestNameNodeReconfigure {
 
   void verifySPSEnabled(final NameNode nameNode, String property,
       StoragePolicySatisfierMode expected, boolean isSatisfierRunning) {
-    assertEquals(property + " has wrong value", isSatisfierRunning, nameNode
-        .getNamesystem().getBlockManager().getSPSManager()
-        .isInternalSatisfierRunning());
+    StoragePolicySatisfyManager spsMgr = nameNode
+            .getNamesystem().getBlockManager().getSPSManager();
+    boolean isInternalSatisfierRunning = spsMgr != null
+        ? spsMgr.isInternalSatisfierRunning() : false;
+    assertEquals(property + " has wrong value", isSatisfierRunning,
+        isInternalSatisfierRunning);
     String actual = nameNode.getConf().get(property,
         DFS_STORAGE_POLICY_SATISFIER_MODE_DEFAULT);
     assertEquals(property + " has wrong value", expected,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2acc50b8/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/sps/TestBlockStorageMovementAttemptedItems.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/sps/TestBlockStorageMovementAttemptedItems.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/sps/TestBlockStorageMovementAttemptedItems.java
index 29af885..ed1fe92 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/sps/TestBlockStorageMovementAttemptedItems.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/sps/TestBlockStorageMovementAttemptedItems.java
@@ -22,13 +22,18 @@ 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 java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.StorageType;
+import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.protocol.Block;
-import org.apache.hadoop.hdfs.server.namenode.sps.StoragePolicySatisfier.AttemptedItemInfo;
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+import org.apache.hadoop.hdfs.server.namenode.sps.StoragePolicySatisfier.StorageTypeNodePair;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
@@ -92,14 +97,16 @@ public class TestBlockStorageMovementAttemptedItems {
    */
   @Test(timeout = 30000)
   public void testAddReportedMoveAttemptFinishedBlocks() throws Exception {
-    bsmAttemptedItems.start(); // start block movement result monitor thread
     Long item = new Long(1234);
-    List<Block> blocks = new ArrayList<Block>();
-    blocks.add(new Block(item));
-    bsmAttemptedItems.add(new AttemptedItemInfo<Long>(0L, 0L, 0L, blocks, 0));
-    Block[] blockArray = new Block[blocks.size()];
-    blocks.toArray(blockArray);
-    bsmAttemptedItems.notifyMovementTriedBlocks(blockArray);
+    Block block = new Block(item);
+    DatanodeInfo dnInfo = DFSTestUtil.getLocalDatanodeInfo(9867);
+    Set<StorageTypeNodePair> locs = new HashSet<>();
+    locs.add(new StorageTypeNodePair(StorageType.ARCHIVE, dnInfo));
+    Map<Block, Set<StorageTypeNodePair>> blocksMap = new HashMap<>();
+    blocksMap.put(block, locs);
+    bsmAttemptedItems.add(0L, 0L, 0L, blocksMap, 0);
+    bsmAttemptedItems.notifyReportedBlock(dnInfo, StorageType.ARCHIVE,
+        block);
     assertEquals("Failed to receive result!", 1,
         bsmAttemptedItems.getMovementFinishedBlocksCount());
   }
@@ -111,9 +118,13 @@ public class TestBlockStorageMovementAttemptedItems {
   public void testNoBlockMovementAttemptFinishedReportAdded() throws Exception {
     bsmAttemptedItems.start(); // start block movement report monitor thread
     Long item = new Long(1234);
-    List<Block> blocks = new ArrayList<>();
-    blocks.add(new Block(item));
-    bsmAttemptedItems.add(new AttemptedItemInfo<Long>(0L, 0L, 0L, blocks, 0));
+    Block block = new Block(item);
+    DatanodeInfo dnInfo = DFSTestUtil.getLocalDatanodeInfo(9867);
+    Set<StorageTypeNodePair> locs = new HashSet<>();
+    locs.add(new StorageTypeNodePair(StorageType.ARCHIVE, dnInfo));
+    Map<Block, Set<StorageTypeNodePair>> blocksMap = new HashMap<>();
+    blocksMap.put(block, locs);
+    bsmAttemptedItems.add(0L, 0L, 0L, blocksMap, 0);
     assertEquals("Shouldn't receive result", 0,
         bsmAttemptedItems.getMovementFinishedBlocksCount());
     assertEquals("Item doesn't exist in the attempted list", 1,
@@ -129,15 +140,18 @@ public class TestBlockStorageMovementAttemptedItems {
   @Test(timeout = 30000)
   public void testPartialBlockMovementShouldBeRetried1() throws Exception {
     Long item = new Long(1234);
-    List<Block> blocks = new ArrayList<>();
-    blocks.add(new Block(item));
-    blocks.add(new Block(5678L));
+    Block block1 = new Block(item);
+    Block block2 = new Block(5678L);
     Long trackID = 0L;
-    bsmAttemptedItems
-        .add(new AttemptedItemInfo<Long>(trackID, trackID, 0L, blocks, 0));
-    Block[] blksMovementReport = new Block[1];
-    blksMovementReport[0] = new Block(item);
-    bsmAttemptedItems.notifyMovementTriedBlocks(blksMovementReport);
+    DatanodeInfo dnInfo = DFSTestUtil.getLocalDatanodeInfo(9867);
+    Set<StorageTypeNodePair> locs = new HashSet<>();
+    locs.add(new StorageTypeNodePair(StorageType.ARCHIVE, dnInfo));
+    Map<Block, Set<StorageTypeNodePair>> blocksMap = new HashMap<>();
+    blocksMap.put(block1, locs);
+    blocksMap.put(block2, locs);
+    bsmAttemptedItems.add(trackID, trackID, 0L, blocksMap, 0);
+    bsmAttemptedItems.notifyReportedBlock(dnInfo, StorageType.ARCHIVE,
+        block1);
 
     // start block movement report monitor thread
     bsmAttemptedItems.start();
@@ -155,14 +169,16 @@ public class TestBlockStorageMovementAttemptedItems {
   @Test(timeout = 30000)
   public void testPartialBlockMovementShouldBeRetried2() throws Exception {
     Long item = new Long(1234);
+    Block block = new Block(item);
     Long trackID = 0L;
-    List<Block> blocks = new ArrayList<>();
-    blocks.add(new Block(item));
-    bsmAttemptedItems
-        .add(new AttemptedItemInfo<Long>(trackID, trackID, 0L, blocks, 0));
-    Block[] blksMovementReport = new Block[1];
-    blksMovementReport[0] = new Block(item);
-    bsmAttemptedItems.notifyMovementTriedBlocks(blksMovementReport);
+    DatanodeInfo dnInfo = DFSTestUtil.getLocalDatanodeInfo(9867);
+    Set<StorageTypeNodePair> locs = new HashSet<>();
+    locs.add(new StorageTypeNodePair(StorageType.ARCHIVE, dnInfo));
+    Map<Block, Set<StorageTypeNodePair>> blocksMap = new HashMap<>();
+    blocksMap.put(block, locs);
+    bsmAttemptedItems.add(trackID, trackID, 0L, blocksMap, 0);
+    bsmAttemptedItems.notifyReportedBlock(dnInfo, StorageType.ARCHIVE,
+        block);
 
     Thread.sleep(selfRetryTimeout * 2); // Waiting to get timed out
 
@@ -183,14 +199,16 @@ public class TestBlockStorageMovementAttemptedItems {
   public void testPartialBlockMovementWithEmptyAttemptedQueue()
       throws Exception {
     Long item = new Long(1234);
+    Block block = new Block(item);
     Long trackID = 0L;
-    List<Block> blocks = new ArrayList<>();
-    blocks.add(new Block(item));
-    bsmAttemptedItems
-        .add(new AttemptedItemInfo<Long>(trackID, trackID, 0L, blocks, 0));
-    Block[] blksMovementReport = new Block[1];
-    blksMovementReport[0] = new Block(item);
-    bsmAttemptedItems.notifyMovementTriedBlocks(blksMovementReport);
+    DatanodeInfo dnInfo = DFSTestUtil.getLocalDatanodeInfo(9867);
+    Set<StorageTypeNodePair> locs = new HashSet<>();
+    locs.add(new StorageTypeNodePair(StorageType.ARCHIVE, dnInfo));
+    Map<Block, Set<StorageTypeNodePair>> blocksMap = new HashMap<>();
+    blocksMap.put(block, locs);
+    bsmAttemptedItems.add(trackID, trackID, 0L, blocksMap, 0);
+    bsmAttemptedItems.notifyReportedBlock(dnInfo, StorageType.ARCHIVE,
+        block);
     assertFalse(
         "Should not add in queue again if it is not there in"
             + " storageMovementAttemptedItems",

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2acc50b8/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/sps/TestStoragePolicySatisfier.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/sps/TestStoragePolicySatisfier.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/sps/TestStoragePolicySatisfier.java
index 75aeb86..b05717a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/sps/TestStoragePolicySatisfier.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/sps/TestStoragePolicySatisfier.java
@@ -51,6 +51,7 @@ 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;
+import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.ClientProtocol;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
@@ -107,6 +108,8 @@ public class TestStoragePolicySatisfier {
   public static final long CAPACITY = 2 * 256 * 1024 * 1024;
   public static final String FILE = "/testMoveToSatisfyStoragePolicy";
   public static final int DEFAULT_BLOCK_SIZE = 1024;
+  private ExternalBlockMovementListener blkMoveListener =
+      new ExternalBlockMovementListener();
 
   /**
    * Sets hdfs cluster.
@@ -1029,6 +1032,9 @@ public class TestStoragePolicySatisfier {
       config.set(DFSConfigKeys
           .DFS_STORAGE_POLICY_SATISFIER_RECHECK_TIMEOUT_MILLIS_KEY,
           "3000");
+      config.set(DFSConfigKeys
+          .DFS_STORAGE_POLICY_SATISFIER_SELF_RETRY_TIMEOUT_MILLIS_KEY,
+          "5000");
       StorageType[][] newtypes = new StorageType[][] {
           {StorageType.ARCHIVE, StorageType.DISK},
           {StorageType.ARCHIVE, StorageType.DISK},
@@ -1072,6 +1078,9 @@ public class TestStoragePolicySatisfier {
       config.set(DFSConfigKeys
           .DFS_STORAGE_POLICY_SATISFIER_RECHECK_TIMEOUT_MILLIS_KEY,
           "3000");
+      config.set(DFSConfigKeys
+          .DFS_STORAGE_POLICY_SATISFIER_SELF_RETRY_TIMEOUT_MILLIS_KEY,
+          "5000");
       StorageType[][] newtypes = new StorageType[][] {
           {StorageType.ARCHIVE, StorageType.DISK},
           {StorageType.ARCHIVE, StorageType.DISK},
@@ -1089,7 +1098,7 @@ public class TestStoragePolicySatisfier {
       fs.setStoragePolicy(filePath, "COLD");
       fs.satisfyStoragePolicy(filePath);
       DFSTestUtil.waitExpectedStorageType(filePath.toString(),
-          StorageType.ARCHIVE, 3, 30000, hdfsCluster.getFileSystem());
+          StorageType.ARCHIVE, 3, 60000, hdfsCluster.getFileSystem());
       assertFalse("Log output does not contain expected log message: ",
           logs.getOutput().contains("some of the blocks are low redundant"));
     } finally {
@@ -1425,6 +1434,9 @@ public class TestStoragePolicySatisfier {
       config.set(DFSConfigKeys
           .DFS_STORAGE_POLICY_SATISFIER_RECHECK_TIMEOUT_MILLIS_KEY,
           "3000");
+      config.set(DFSConfigKeys
+          .DFS_STORAGE_POLICY_SATISFIER_SELF_RETRY_TIMEOUT_MILLIS_KEY,
+          "5000");
       config.setBoolean(DFSConfigKeys
           .DFS_STORAGE_POLICY_SATISFIER_LOW_MAX_STREAMS_PREFERENCE_KEY,
           false);
@@ -1467,7 +1479,7 @@ public class TestStoragePolicySatisfier {
       for (int i = 1; i <= 10; i++) {
         Path filePath = new Path("/file" + i);
         DFSTestUtil.waitExpectedStorageType(filePath.toString(),
-            StorageType.DISK, 4, 30000, hdfsCluster.getFileSystem());
+            StorageType.DISK, 4, 60000, hdfsCluster.getFileSystem());
       }
       for (int i = 11; i <= 20; i++) {
         Path filePath = new Path("/file" + i);
@@ -1725,20 +1737,16 @@ public class TestStoragePolicySatisfier {
   public void waitForBlocksMovementAttemptReport(
       long expectedMovementFinishedBlocksCount, int timeout)
           throws TimeoutException, InterruptedException {
-    BlockManager blockManager = hdfsCluster.getNamesystem().getBlockManager();
-    final StoragePolicySatisfier<Long> sps =
-        (StoragePolicySatisfier<Long>) blockManager
-        .getSPSManager().getInternalSPSService();
+    Assert.assertNotNull("Didn't set external block move listener",
+        blkMoveListener);
     GenericTestUtils.waitFor(new Supplier<Boolean>() {
       @Override
       public Boolean get() {
+        int actualCount = blkMoveListener.getActualBlockMovements().size();
         LOG.info("MovementFinishedBlocks: expectedCount={} actualCount={}",
             expectedMovementFinishedBlocksCount,
-            ((BlockStorageMovementAttemptedItems<Long>) (sps
-                .getAttemptedItemsMonitor())).getMovementFinishedBlocksCount());
-        return ((BlockStorageMovementAttemptedItems<Long>) (sps
-            .getAttemptedItemsMonitor()))
-                .getMovementFinishedBlocksCount()
+            actualCount);
+        return actualCount
             >= expectedMovementFinishedBlocksCount;
       }
     }, 100, timeout);
@@ -1790,11 +1798,54 @@ public class TestStoragePolicySatisfier {
         .numDataNodes(numberOfDatanodes).storagesPerDatanode(storagesPerDn)
         .storageTypes(storageTypes).storageCapacities(capacities).build();
     cluster.waitActive();
+
+    // Sets external listener for assertion.
+    blkMoveListener.clear();
+    BlockManager blockManager = cluster.getNamesystem().getBlockManager();
+    final StoragePolicySatisfier<Long> sps =
+        (StoragePolicySatisfier<Long>) blockManager
+        .getSPSManager().getInternalSPSService();
+    sps.setBlockMovementListener(blkMoveListener);
     return cluster;
   }
 
   public void restartNamenode() throws IOException {
     hdfsCluster.restartNameNodes();
     hdfsCluster.waitActive();
+    BlockManager blockManager = hdfsCluster.getNamesystem().getBlockManager();
+    StoragePolicySatisfyManager spsMgr = blockManager.getSPSManager();
+    if (spsMgr != null && spsMgr.isInternalSatisfierRunning()) {
+      // Sets external listener for assertion.
+      blkMoveListener.clear();
+      final StoragePolicySatisfier<Long> sps =
+          (StoragePolicySatisfier<Long>) spsMgr.getInternalSPSService();
+      sps.setBlockMovementListener(blkMoveListener);
+    }
+  }
+
+  /**
+   * Implementation of listener callback, where it collects all the sps move
+   * attempted blocks for assertion.
+   */
+  public static final class ExternalBlockMovementListener
+      implements BlockMovementListener {
+
+    private List<Block> actualBlockMovements = new ArrayList<>();
+
+    @Override
+    public void notifyMovementTriedBlocks(Block[] moveAttemptFinishedBlks) {
+      for (Block block : moveAttemptFinishedBlks) {
+        actualBlockMovements.add(block);
+      }
+      LOG.info("Movement attempted blocks:{}", actualBlockMovements);
+    }
+
+    public List<Block> getActualBlockMovements() {
+      return actualBlockMovements;
+    }
+
+    public void clear() {
+      actualBlockMovements.clear();
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2acc50b8/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/sps/TestStoragePolicySatisfierWithStripedFile.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/sps/TestStoragePolicySatisfierWithStripedFile.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/sps/TestStoragePolicySatisfierWithStripedFile.java
index e69a833..857bd6c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/sps/TestStoragePolicySatisfierWithStripedFile.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/sps/TestStoragePolicySatisfierWithStripedFile.java
@@ -43,6 +43,7 @@ import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.StoragePolicySatisfierMode;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
+import org.apache.hadoop.hdfs.server.namenode.sps.TestStoragePolicySatisfier.ExternalBlockMovementListener;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.junit.Assert;
 import org.junit.Before;
@@ -70,6 +71,8 @@ public class TestStoragePolicySatisfierWithStripedFile {
   private int cellSize;
   private int defaultStripeBlockSize;
   private Configuration conf;
+  private ExternalBlockMovementListener blkMoveListener =
+      new ExternalBlockMovementListener();
 
   private ErasureCodingPolicy getEcPolicy() {
     return StripedFileTestUtil.getDefaultECPolicy();
@@ -131,6 +134,15 @@ public class TestStoragePolicySatisfierWithStripedFile {
     HdfsAdmin hdfsAdmin = new HdfsAdmin(FileSystem.getDefaultUri(conf), conf);
     try {
       cluster.waitActive();
+
+      // Sets external listener for assertion.
+      blkMoveListener.clear();
+      BlockManager blockManager = cluster.getNamesystem().getBlockManager();
+      final StoragePolicySatisfier<Long> sps =
+          (StoragePolicySatisfier<Long>) blockManager
+          .getSPSManager().getInternalSPSService();
+      sps.setBlockMovementListener(blkMoveListener);
+
       DistributedFileSystem dfs = cluster.getFileSystem();
       dfs.enableErasureCodingPolicy(
           StripedFileTestUtil.getDefaultECPolicy().getName());
@@ -240,6 +252,15 @@ public class TestStoragePolicySatisfierWithStripedFile {
     HdfsAdmin hdfsAdmin = new HdfsAdmin(FileSystem.getDefaultUri(conf), conf);
     try {
       cluster.waitActive();
+
+      // Sets external listener for assertion.
+      blkMoveListener.clear();
+      BlockManager blockManager = cluster.getNamesystem().getBlockManager();
+      final StoragePolicySatisfier<Long> sps =
+          (StoragePolicySatisfier<Long>) blockManager
+          .getSPSManager().getInternalSPSService();
+      sps.setBlockMovementListener(blkMoveListener);
+
       DistributedFileSystem dfs = cluster.getFileSystem();
       dfs.enableErasureCodingPolicy(
           StripedFileTestUtil.getDefaultECPolicy().getName());
@@ -328,6 +349,9 @@ public class TestStoragePolicySatisfierWithStripedFile {
     conf.set(DFSConfigKeys
         .DFS_STORAGE_POLICY_SATISFIER_RECHECK_TIMEOUT_MILLIS_KEY,
         "3000");
+    conf.set(DFSConfigKeys
+        .DFS_STORAGE_POLICY_SATISFIER_SELF_RETRY_TIMEOUT_MILLIS_KEY,
+        "5000");
     final MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
         .numDataNodes(numOfDatanodes)
         .storagesPerDatanode(storagesPerDatanode)
@@ -559,22 +583,16 @@ public class TestStoragePolicySatisfierWithStripedFile {
   private void waitForBlocksMovementAttemptReport(MiniDFSCluster cluster,
       long expectedMoveFinishedBlks, int timeout)
           throws TimeoutException, InterruptedException {
-    BlockManager blockManager = cluster.getNamesystem().getBlockManager();
-    final StoragePolicySatisfier<Long> sps =
-        (StoragePolicySatisfier<Long>) blockManager
-        .getSPSManager().getInternalSPSService();
-    Assert.assertNotNull("Failed to get SPS object reference!", sps);
-
+    Assert.assertNotNull("Didn't set external block move listener",
+        blkMoveListener);
     GenericTestUtils.waitFor(new Supplier<Boolean>() {
       @Override
       public Boolean get() {
+        int actualCount = blkMoveListener.getActualBlockMovements().size();
         LOG.info("MovementFinishedBlocks: expectedCount={} actualCount={}",
             expectedMoveFinishedBlks,
-            ((BlockStorageMovementAttemptedItems<Long>) sps
-                .getAttemptedItemsMonitor()).getMovementFinishedBlocksCount());
-        return ((BlockStorageMovementAttemptedItems<Long>) sps
-            .getAttemptedItemsMonitor())
-                .getMovementFinishedBlocksCount() >= expectedMoveFinishedBlks;
+            actualCount);
+        return actualCount >= expectedMoveFinishedBlks;
       }
     }, 100, timeout);
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2acc50b8/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/sps/TestExternalStoragePolicySatisfier.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/sps/TestExternalStoragePolicySatisfier.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/sps/TestExternalStoragePolicySatisfier.java
index 28e172a..be243cb 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/sps/TestExternalStoragePolicySatisfier.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/sps/TestExternalStoragePolicySatisfier.java
@@ -54,11 +54,9 @@ import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
-import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.StoragePolicySatisfierMode;
 import org.apache.hadoop.hdfs.server.balancer.NameNodeConnector;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
-import org.apache.hadoop.hdfs.server.namenode.sps.BlockMovementListener;
 import org.apache.hadoop.hdfs.server.namenode.sps.BlockStorageMovementAttemptedItems;
 import org.apache.hadoop.hdfs.server.namenode.sps.StoragePolicySatisfier;
 import org.apache.hadoop.hdfs.server.namenode.sps.TestStoragePolicySatisfier;
@@ -92,6 +90,8 @@ public class TestExternalStoragePolicySatisfier
   private File baseDir;
   private StoragePolicySatisfier<String> externalSps;
   private ExternalSPSContext externalCtxt;
+  private ExternalBlockMovementListener blkMoveListener =
+      new ExternalBlockMovementListener();
 
   @After
   public void destroy() throws Exception {
@@ -144,15 +144,12 @@ public class TestExternalStoragePolicySatisfier
     nnc = getNameNodeConnector(getConf());
 
     externalSps = new StoragePolicySatisfier<String>(getConf());
-    externalCtxt = new ExternalSPSContext(externalSps,
-        getNameNodeConnector(conf));
+    externalCtxt = new ExternalSPSContext(externalSps, nnc);
 
-    ExternalBlockMovementListener blkMoveListener =
-        new ExternalBlockMovementListener();
+    blkMoveListener.clear();
     ExternalSPSBlockMoveTaskHandler externalHandler =
         new ExternalSPSBlockMoveTaskHandler(conf, nnc,
             externalSps);
-    externalHandler.init();
     externalSps.init(externalCtxt,
         new ExternalSPSFilePathCollector(externalSps), externalHandler,
         blkMoveListener);
@@ -169,33 +166,17 @@ public class TestExternalStoragePolicySatisfier
     getCluster().waitActive();
     externalSps = new StoragePolicySatisfier<>(getConf());
 
-    externalCtxt = new ExternalSPSContext(externalSps,
-        getNameNodeConnector(getConf()));
-    ExternalBlockMovementListener blkMoveListener =
-        new ExternalBlockMovementListener();
+    externalCtxt = new ExternalSPSContext(externalSps, nnc);
+    blkMoveListener.clear();
     ExternalSPSBlockMoveTaskHandler externalHandler =
         new ExternalSPSBlockMoveTaskHandler(getConf(), nnc,
             externalSps);
-    externalHandler.init();
     externalSps.init(externalCtxt,
         new ExternalSPSFilePathCollector(externalSps), externalHandler,
         blkMoveListener);
     externalSps.start(true, StoragePolicySatisfierMode.EXTERNAL);
   }
 
-  private class ExternalBlockMovementListener implements BlockMovementListener {
-
-    private List<Block> actualBlockMovements = new ArrayList<>();
-
-    @Override
-    public void notifyMovementTriedBlocks(Block[] moveAttemptFinishedBlks) {
-      for (Block block : moveAttemptFinishedBlks) {
-        actualBlockMovements.add(block);
-      }
-      LOG.info("Movement attempted blocks", actualBlockMovements);
-    }
-  }
-
   private NameNodeConnector getNameNodeConnector(Configuration conf)
       throws IOException {
     final Collection<URI> namenodes = DFSUtil.getInternalNsRpcUris(conf);
@@ -237,16 +218,15 @@ public class TestExternalStoragePolicySatisfier
   public void waitForBlocksMovementAttemptReport(
       long expectedMovementFinishedBlocksCount, int timeout)
           throws TimeoutException, InterruptedException {
+    Assert.assertNotNull("Didn't set external block move listener",
+        blkMoveListener);
     GenericTestUtils.waitFor(new Supplier<Boolean>() {
       @Override
       public Boolean get() {
+        int actualCount = blkMoveListener.getActualBlockMovements().size();
         LOG.info("MovementFinishedBlocks: expectedCount={} actualCount={}",
-            expectedMovementFinishedBlocksCount,
-            ((BlockStorageMovementAttemptedItems<String>) (externalSps
-                .getAttemptedItemsMonitor())).getMovementFinishedBlocksCount());
-        return ((BlockStorageMovementAttemptedItems<String>) (externalSps
-            .getAttemptedItemsMonitor()))
-                .getMovementFinishedBlocksCount()
+            expectedMovementFinishedBlocksCount, actualCount);
+        return actualCount
             >= expectedMovementFinishedBlocksCount;
       }
     }, 100, timeout);
@@ -352,6 +332,8 @@ public class TestExternalStoragePolicySatisfier
       files.add(FILE);
       DistributedFileSystem fs = getFS();
 
+      // stops sps to make the SPS Q with many outstanding requests.
+      externalSps.stopGracefully();
       // Creates 4 more files. Send all of them for satisfying the storage
       // policy together.
       for (int i = 0; i < 3; i++) {


---------------------------------------------------------------------
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-13076: [SPS]: Resolve conflicts after rebasing HDFS-10285 branch to trunk. Contributed by Rakesh R.

Posted by um...@apache.org.
HDFS-13076: [SPS]: Resolve conflicts after rebasing HDFS-10285 branch to trunk. 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/db3f227d
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/db3f227d
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/db3f227d

Branch: refs/heads/HDFS-10285
Commit: db3f227d8aeeea8b5bb473fed9ca4f6a17b0fca5
Parents: 66e8f9b
Author: Rakesh Radhakrishnan <ra...@apache.org>
Authored: Thu Jul 5 10:10:13 2018 +0530
Committer: Uma Maheswara Rao Gangumalla <um...@apache.org>
Committed: Sun Aug 12 03:06:06 2018 -0700

----------------------------------------------------------------------
 .../hdfs/server/federation/router/RouterNamenodeProtocol.java | 6 ++++++
 .../hadoop/hdfs/server/federation/router/RouterRpcServer.java | 7 +++++++
 .../hadoop/hdfs/server/blockmanagement/BlockManager.java      | 2 +-
 .../server/namenode/sps/IntraSPSNameNodeFileIdCollector.java  | 4 ++--
 4 files changed, 16 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/db3f227d/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterNamenodeProtocol.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterNamenodeProtocol.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterNamenodeProtocol.java
index 0433650..edfb391 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterNamenodeProtocol.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterNamenodeProtocol.java
@@ -184,4 +184,10 @@ public class RouterNamenodeProtocol implements NamenodeProtocol {
     rpcServer.checkOperation(OperationCategory.READ, false);
     return false;
   }
+
+  @Override
+  public Long getNextSPSPath() throws IOException {
+    // not supported
+    return null;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/db3f227d/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcServer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcServer.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcServer.java
index d93f99d..36645c9 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcServer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcServer.java
@@ -2509,4 +2509,11 @@ public class RouterRpcServer extends AbstractService
     checkOperation(OperationCategory.READ, false);
     return StoragePolicySatisfyPathStatus.NOT_AVAILABLE;
   }
+
+  @Override
+  public Long getNextSPSPath() throws IOException {
+    checkOperation(OperationCategory.READ, false);
+    // not supported
+    return null;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/db3f227d/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 bae6b4e..bb63f2a 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
@@ -5078,7 +5078,7 @@ public class BlockManager implements BlockStatsMXBean {
         DFSConfigKeys.DFS_STORAGE_POLICY_ENABLED_KEY,
         DFSConfigKeys.DFS_STORAGE_POLICY_ENABLED_DEFAULT);
     String modeVal = spsMode;
-    if (org.apache.commons.lang.StringUtils.isBlank(modeVal)) {
+    if (org.apache.commons.lang3.StringUtils.isBlank(modeVal)) {
       modeVal = conf.get(DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_MODE_KEY,
           DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_MODE_DEFAULT);
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/db3f227d/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/IntraSPSNameNodeFileIdCollector.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/IntraSPSNameNodeFileIdCollector.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/IntraSPSNameNodeFileIdCollector.java
index ea3b96f..0473b9d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/IntraSPSNameNodeFileIdCollector.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/IntraSPSNameNodeFileIdCollector.java
@@ -45,7 +45,7 @@ public class IntraSPSNameNodeFileIdCollector extends FSTreeTraverser
 
   public IntraSPSNameNodeFileIdCollector(FSDirectory dir,
       SPSService service) {
-    super(dir);
+    super(dir, service.getConf());
     this.service = service;
     this.maxQueueLimitToScan = service.getConf().getInt(
         DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_QUEUE_LIMIT_KEY,
@@ -72,7 +72,7 @@ public class IntraSPSNameNodeFileIdCollector extends FSTreeTraverser
   }
 
   @Override
-  protected boolean canSubmitCurrentBatch() {
+  protected boolean shouldSubmitCurrentBatch() {
     return remainingCapacity <= 0;
   }
 


---------------------------------------------------------------------
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-13076: [SPS]: Addendum. Resolve conflicts after rebasing branch to trunk. Contributed by Rakesh R.

Posted by um...@apache.org.
HDFS-13076: [SPS]: Addendum. Resolve conflicts after rebasing branch to trunk. 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/dfcb331b
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/dfcb331b
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/dfcb331b

Branch: refs/heads/HDFS-10285
Commit: dfcb331ba3516264398121c9f23af3a79c0509cc
Parents: db3f227
Author: Rakesh Radhakrishnan <ra...@apache.org>
Authored: Fri Jul 20 10:59:16 2018 +0530
Committer: Uma Maheswara Rao Gangumalla <um...@apache.org>
Committed: Sun Aug 12 03:06:06 2018 -0700

----------------------------------------------------------------------
 .../java/org/apache/hadoop/hdfs/server/datanode/DataNode.java    | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/dfcb331b/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 a714602..21af33f 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
@@ -3624,8 +3624,8 @@ public class DataNode extends ReconfigurableBase
     }
     return this.diskBalancer;
   }
-}
 
   StoragePolicySatisfyWorker getStoragePolicySatisfyWorker() {
     return storagePolicySatisfyWorker;
-  }}
+  }
+}


---------------------------------------------------------------------
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-12955: [SPS]: Move SPS classes to a separate package. Contributed by Rakesh R.

Posted by um...@apache.org.
HDFS-12955: [SPS]: Move SPS classes to a separate package. 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/78420719
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/78420719
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/78420719

Branch: refs/heads/HDFS-10285
Commit: 78420719eb1f138c6f10558befb7bc8ebcc28a54
Parents: c561cb3
Author: Uma Maheswara Rao G <um...@intel.com>
Authored: Fri Dec 22 09:10:12 2017 -0800
Committer: Uma Maheswara Rao Gangumalla <um...@apache.org>
Committed: Sun Aug 12 03:06:02 2018 -0700

----------------------------------------------------------------------
 .../server/blockmanagement/BlockManager.java    |    6 +-
 .../BlockStorageMovementAttemptedItems.java     |  241 ---
 .../namenode/BlockStorageMovementNeeded.java    |  574 ------
 .../hdfs/server/namenode/FSNamesystem.java      |    1 +
 .../hdfs/server/namenode/IntraNNSPSContext.java |   41 +
 .../server/namenode/StoragePolicySatisfier.java |  973 ----------
 .../sps/BlockStorageMovementAttemptedItems.java |  241 +++
 .../sps/BlockStorageMovementNeeded.java         |  572 ++++++
 .../namenode/sps/StoragePolicySatisfier.java    |  988 ++++++++++
 .../hdfs/server/namenode/sps/package-info.java  |   28 +
 .../TestBlockStorageMovementAttemptedItems.java |  196 --
 .../namenode/TestStoragePolicySatisfier.java    | 1775 -----------------
 ...stStoragePolicySatisfierWithStripedFile.java |  580 ------
 .../TestBlockStorageMovementAttemptedItems.java |  196 ++
 .../sps/TestStoragePolicySatisfier.java         | 1779 ++++++++++++++++++
 ...stStoragePolicySatisfierWithStripedFile.java |  580 ++++++
 16 files changed, 4430 insertions(+), 4341 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/78420719/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 0957fe2..ec99a9f 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,11 +89,12 @@ import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
 import org.apache.hadoop.hdfs.server.namenode.CachedBlock;
 import org.apache.hadoop.hdfs.server.namenode.INode.BlocksMapUpdateInfo;
 import org.apache.hadoop.hdfs.server.namenode.INodesInPath;
+import org.apache.hadoop.hdfs.server.namenode.IntraNNSPSContext;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.namenode.Namesystem;
-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.namenode.sps.StoragePolicySatisfier;
 import org.apache.hadoop.hdfs.server.protocol.BlockCommand;
 import org.apache.hadoop.hdfs.server.protocol.BlockReportContext;
 import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations;
@@ -478,7 +479,8 @@ 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, this, conf);
+    StoragePolicySatisfier.Context spsctxt = new IntraNNSPSContext(namesystem);
+    sps = new StoragePolicySatisfier(namesystem, this, conf, spsctxt);
     blockTokenSecretManager = createBlockTokenSecretManager(conf);
 
     providedStorageMap = new ProvidedStorageMap(namesystem, this, conf);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/78420719/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
deleted file mode 100644
index 643255f..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BlockStorageMovementAttemptedItems.java
+++ /dev/null
@@ -1,241 +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 static org.apache.hadoop.util.Time.monotonicNow;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Iterator;
-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.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 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 =
-      LoggerFactory.getLogger(BlockStorageMovementAttemptedItems.class);
-
-  /**
-   * A map holds the items which are already taken for blocks movements
-   * processing and sent to DNs.
-   */
-  private final List<AttemptedItemInfo> storageMovementAttemptedItems;
-  private final List<Block> movementFinishedBlocks;
-  private volatile boolean monitorRunning = true;
-  private Daemon timerThread = null;
-  //
-  // It might take anywhere between 5 to 10 minutes before
-  // a request is timed out.
-  //
-  private long selfRetryTimeout = 5 * 60 * 1000;
-
-  //
-  // It might take anywhere between 1 to 2 minutes before
-  // a request is timed out.
-  //
-  private long minCheckTimeout = 1 * 60 * 1000; // minimum value
-  private BlockStorageMovementNeeded blockStorageMovementNeeded;
-
-  public BlockStorageMovementAttemptedItems(long recheckTimeout,
-      long selfRetryTimeout,
-      BlockStorageMovementNeeded unsatisfiedStorageMovementFiles) {
-    if (recheckTimeout > 0) {
-      this.minCheckTimeout = Math.min(minCheckTimeout, recheckTimeout);
-    }
-
-    this.selfRetryTimeout = selfRetryTimeout;
-    this.blockStorageMovementNeeded = unsatisfiedStorageMovementFiles;
-    storageMovementAttemptedItems = new ArrayList<>();
-    movementFinishedBlocks = new ArrayList<>();
-  }
-
-  /**
-   * Add item to block storage movement attempted items map which holds the
-   * tracking/blockCollection id versus time stamp.
-   *
-   * @param itemInfo
-   *          - tracking info
-   */
-  public void add(AttemptedItemInfo itemInfo) {
-    synchronized (storageMovementAttemptedItems) {
-      storageMovementAttemptedItems.add(itemInfo);
-    }
-  }
-
-  /**
-   * Add the storage movement attempt finished blocks to
-   * storageMovementFinishedBlocks.
-   *
-   * @param moveAttemptFinishedBlks
-   *          storage movement attempt finished blocks
-   */
-  public void addReportedMovedBlocks(Block[] moveAttemptFinishedBlks) {
-    if (moveAttemptFinishedBlks.length == 0) {
-      return;
-    }
-    synchronized (movementFinishedBlocks) {
-      movementFinishedBlocks.addAll(Arrays.asList(moveAttemptFinishedBlks));
-    }
-  }
-
-  /**
-   * Starts the monitor thread.
-   */
-  public synchronized void start() {
-    monitorRunning = true;
-    timerThread = new Daemon(new BlocksStorageMovementAttemptMonitor());
-    timerThread.setName("BlocksStorageMovementAttemptMonitor");
-    timerThread.start();
-  }
-
-  /**
-   * Sets running flag to false. Also, this will interrupt monitor thread and
-   * clear all the queued up tasks.
-   */
-  public synchronized void stop() {
-    monitorRunning = false;
-    if (timerThread != null) {
-      timerThread.interrupt();
-    }
-    this.clearQueues();
-  }
-
-  /**
-   * Timed wait to stop monitor thread.
-   */
-  synchronized void stopGracefully() {
-    if (timerThread == null) {
-      return;
-    }
-    if (monitorRunning) {
-      stop();
-    }
-    try {
-      timerThread.join(3000);
-    } catch (InterruptedException ie) {
-    }
-  }
-
-  /**
-   * A monitor class for checking block storage movement attempt status and long
-   * waiting items periodically.
-   */
-  private class BlocksStorageMovementAttemptMonitor implements Runnable {
-    @Override
-    public void run() {
-      while (monitorRunning) {
-        try {
-          blockStorageMovementReportedItemsCheck();
-          blocksStorageMovementUnReportedItemsCheck();
-          Thread.sleep(minCheckTimeout);
-        } catch (InterruptedException ie) {
-          LOG.info("BlocksStorageMovementAttemptMonitor thread "
-              + "is interrupted.", ie);
-        } catch (IOException ie) {
-          LOG.warn("BlocksStorageMovementAttemptMonitor thread "
-              + "received exception and exiting.", ie);
-        }
-      }
-    }
-  }
-
-  @VisibleForTesting
-  void blocksStorageMovementUnReportedItemsCheck() {
-    synchronized (storageMovementAttemptedItems) {
-      Iterator<AttemptedItemInfo> iter = storageMovementAttemptedItems
-          .iterator();
-      long now = monotonicNow();
-      while (iter.hasNext()) {
-        AttemptedItemInfo itemInfo = iter.next();
-        if (now > itemInfo.getLastAttemptedOrReportedTime()
-            + selfRetryTimeout) {
-          Long blockCollectionID = itemInfo.getTrackId();
-          synchronized (movementFinishedBlocks) {
-            ItemInfo candidate = new ItemInfo(itemInfo.getStartId(),
-                blockCollectionID, itemInfo.getRetryCount() + 1);
-            blockStorageMovementNeeded.add(candidate);
-            iter.remove();
-            LOG.info("TrackID: {} becomes timed out and moved to needed "
-                + "retries queue for next iteration.", blockCollectionID);
-          }
-        }
-      }
-
-    }
-  }
-
-  @VisibleForTesting
-  void blockStorageMovementReportedItemsCheck() throws IOException {
-    synchronized (movementFinishedBlocks) {
-      Iterator<Block> finishedBlksIter = movementFinishedBlocks.iterator();
-      while (finishedBlksIter.hasNext()) {
-        Block blk = finishedBlksIter.next();
-        synchronized (storageMovementAttemptedItems) {
-          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.add(new ItemInfo(attemptedItemInfo
-                  .getStartId(), attemptedItemInfo.getTrackId(),
-                  attemptedItemInfo.getRetryCount() + 1));
-              iterator.remove();
-            }
-          }
-        }
-        // Remove attempted blocks from movementFinishedBlocks list.
-        finishedBlksIter.remove();
-      }
-    }
-  }
-
-  @VisibleForTesting
-  public int getMovementFinishedBlocksCount() {
-    return movementFinishedBlocks.size();
-  }
-
-  @VisibleForTesting
-  public int getAttemptedItemsCount() {
-    return storageMovementAttemptedItems.size();
-  }
-
-  public void clearQueues() {
-    movementFinishedBlocks.clear();
-    storageMovementAttemptedItems.clear();
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/78420719/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
deleted file mode 100644
index 89bcbff..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BlockStorageMovementNeeded.java
+++ /dev/null
@@ -1,574 +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 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.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;
-
-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.
- * 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 {
-
-  public static final Logger LOG =
-      LoggerFactory.getLogger(BlockStorageMovementNeeded.class);
-
-  private final Queue<ItemInfo> storageMovementNeeded =
-      new LinkedList<ItemInfo>();
-
-  /**
-   * 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, 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
-  private final Queue<Long> spsDirsToBeTraveresed = new LinkedList<Long>();
-
-  private final StoragePolicySatisfier sps;
-
-  private Daemon inodeIdCollector;
-
-  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;
-    this.sps = sps;
-    this.maxQueuedItem = queueLimit;
-  }
-
-  /**
-   * Add the candidate to tracking list for which storage movement
-   * expected if necessary.
-   *
-   * @param trackInfo
-   *          - track info for satisfy the policy
-   */
-  public synchronized void add(ItemInfo trackInfo) {
-    spsStatus.put(trackInfo.getStartId(),
-        new StoragePolicySatisfyPathStatusInfo(
-            StoragePolicySatisfyPathStatus.IN_PROGRESS));
-    storageMovementNeeded.add(trackInfo);
-  }
-
-  /**
-   * Add the itemInfo to tracking list for which storage movement
-   * expected if necessary.
-   * @param startId
-   *            - start id
-   * @param itemInfoList
-   *            - List of child in the directory
-   */
-  @VisibleForTesting
-  public synchronized void addAll(long startId,
-      List<ItemInfo> itemInfoList, boolean scanCompleted) {
-    storageMovementNeeded.addAll(itemInfoList);
-    DirPendingWorkInfo pendingWork = pendingWorkForDirectory.get(startId);
-    if (pendingWork == null) {
-      pendingWork = new DirPendingWorkInfo();
-      pendingWorkForDirectory.put(startId, pendingWork);
-    }
-    pendingWork.addPendingWorkCount(itemInfoList.size());
-    if (scanCompleted) {
-      pendingWork.markScanCompleted();
-    }
-  }
-
-  /**
-   * Gets the block collection id for which storage movements check necessary
-   * and make the movement if required.
-   *
-   * @return block collection ID
-   */
-  public synchronized ItemInfo get() {
-    return storageMovementNeeded.poll();
-  }
-
-  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.
-    synchronized (spsDirsToBeTraveresed) {
-      spsDirsToBeTraveresed.notify();
-    }
-  }
-
-  /**
-   * 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();
-    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,
-      boolean isSuccess) throws IOException {
-    if (trackInfo.isDir()) {
-      // 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(startId);
-        updateStatus(startId, isSuccess);
-      } else {
-        DirPendingWorkInfo pendingWork = pendingWorkForDirectory.get(startId);
-        if (pendingWork != null) {
-          pendingWork.decrementPendingWorkCount();
-          if (pendingWork.isDirWorkDone()) {
-            namesystem.removeXattr(startId, XATTR_SATISFY_STORAGE_POLICY);
-            pendingWorkForDirectory.remove(startId);
-            pendingWork.setFailure(!isSuccess);
-            updateStatus(startId, pendingWork.isPolicySatisfied());
-          }
-          pendingWork.setFailure(isSuccess);
-        }
-      }
-    } else {
-      // Remove xAttr if trackID doesn't exist in
-      // storageMovementAttemptedItems or file policy satisfied.
-      namesystem.removeXattr(trackInfo.getTrackId(),
-          XATTR_SATISFY_STORAGE_POLICY);
-      updateStatus(trackInfo.getStartId(), isSuccess);
-    }
-  }
-
-  public synchronized void clearQueue(long trackId) {
-    spsDirsToBeTraveresed.remove(trackId);
-    Iterator<ItemInfo> iterator = storageMovementNeeded.iterator();
-    while (iterator.hasNext()) {
-      ItemInfo next = iterator.next();
-      if (next.getStartId() == trackId) {
-        iterator.remove();
-      }
-    }
-    pendingWorkForDirectory.remove(trackId);
-  }
-
-  /**
-   * Mark inode status as SUCCESS in map.
-   */
-  private void updateStatus(long startId, boolean isSuccess){
-    StoragePolicySatisfyPathStatusInfo spsStatusInfo =
-        spsStatus.get(startId);
-    if (spsStatusInfo == null) {
-      spsStatusInfo = new StoragePolicySatisfyPathStatusInfo();
-      spsStatus.put(startId, spsStatusInfo);
-    }
-
-    if (isSuccess) {
-      spsStatusInfo.setSuccess();
-    } else {
-      spsStatusInfo.setFailure();
-    }
-  }
-
-  /**
-   * 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 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!.");
-      long lastStatusCleanTime = 0;
-      while (namesystem.isRunning() && sps.isRunning()) {
-        try {
-          if (!namesystem.isInSafeMode()) {
-            FSDirectory fsd = namesystem.getFSDirectory();
-            Long startINodeId = spsDirsToBeTraveresed.poll();
-            if (startINodeId == null) {
-              // Waiting for SPS path
-              synchronized (spsDirsToBeTraveresed) {
-                spsDirsToBeTraveresed.wait(5000);
-              }
-            } else {
-              INode startInode = fsd.getInode(startINodeId);
-              if (startInode != null) {
-                try {
-                  remainingCapacity = remainingCapacity();
-                  spsStatus.put(startINodeId,
-                      new StoragePolicySatisfyPathStatusInfo(
-                          StoragePolicySatisfyPathStatus.IN_PROGRESS));
-                  readLock();
-                  traverseDir(startInode.asDirectory(), startINodeId,
-                      HdfsFileStatus.EMPTY_NAME,
-                      new SPSTraverseInfo(startINodeId));
-                } finally {
-                  readUnlock();
-                }
-                // 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());
-                  updateStatus(startInode.getId(), true);
-                }
-              }
-            }
-            //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);
-        }
-      }
-    }
-
-    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
-    }
-
-    @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;
-    }
-  }
-
-  /**
-   * Info for directory recursive scan.
-   */
-  public static class DirPendingWorkInfo {
-
-    private int pendingWorkCount = 0;
-    private boolean fullyScanned = false;
-    private boolean success = true;
-
-    /**
-     * 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;
-    }
-
-    /**
-     * 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() {
-    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;
-    }
-  }
-
-  /**
-   * 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 void setFailure() {
-      this.status = StoragePolicySatisfyPathStatus.FAILURE;
-      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
-          || StoragePolicySatisfyPathStatus.FAILURE == 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/78420719/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 5990a5f..2f625ac 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
@@ -258,6 +258,7 @@ import org.apache.hadoop.hdfs.server.namenode.metrics.NameNodeMetrics;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.DirectorySnapshottableFeature;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotManager;
+import org.apache.hadoop.hdfs.server.namenode.sps.StoragePolicySatisfier;
 import org.apache.hadoop.hdfs.server.namenode.startupprogress.Phase;
 import org.apache.hadoop.hdfs.server.namenode.startupprogress.StartupProgress;
 import org.apache.hadoop.hdfs.server.namenode.startupprogress.Step;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/78420719/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/IntraNNSPSContext.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/IntraNNSPSContext.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/IntraNNSPSContext.java
new file mode 100644
index 0000000..111cabb
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/IntraNNSPSContext.java
@@ -0,0 +1,41 @@
+/**
+ * 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.hdfs.server.namenode.sps.StoragePolicySatisfier;
+
+/**
+ * This class is the Namenode implementation for analyzing the file blocks which
+ * are expecting to change its storages and assigning the block storage
+ * movements to satisfy the storage policy.
+ */
+// TODO: Now, added one API which is required for sps package. Will refine
+// this interface via HDFS-12911.
+public class IntraNNSPSContext implements StoragePolicySatisfier.Context {
+  private final Namesystem namesystem;
+
+  public IntraNNSPSContext(Namesystem namesystem) {
+    this.namesystem = namesystem;
+  }
+
+  @Override
+  public int getNumLiveDataNodes() {
+    return namesystem.getFSDirectory().getBlockManager().getDatanodeManager()
+        .getNumLiveDataNodes();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/78420719/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
deleted file mode 100644
index 972e744..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/StoragePolicySatisfier.java
+++ /dev/null
@@ -1,973 +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 static org.apache.hadoop.util.Time.monotonicNow;
-
-import java.io.IOException;
-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.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;
-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;
-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;
-import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
-import org.apache.hadoop.hdfs.server.protocol.BlockStorageMovementCommand.BlockMovingInfo;
-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;
-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
- * 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;
-  private final BlockStorageMovementAttemptedItems storageMovementsMonitor;
-  private volatile boolean isRunning = false;
-  private int spsWorkMultiplier;
-  private long blockCount = 0L;
-  private int blockMovementMaxRetry;
-  /**
-   * Represents the collective analysis status for all 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,
-      final BlockManager blkManager, Configuration conf) {
-    this.namesystem = namesystem;
-    this.storageMovementNeeded = new BlockStorageMovementNeeded(namesystem,
-        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(
-            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);
-    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);
-  }
-
-  /**
-   * Start storage policy satisfier demon thread. Also start block storage
-   * movements monitor for retry the attempts if needed.
-   */
-  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 "
-          + "start it.");
-    } else {
-      LOG.info("Starting StoragePolicySatisfier.");
-    }
-
-    // Ensure that all the previously submitted block movements(if any) have to
-    // be stopped in all datanodes.
-    addDropSPSWorkCommandsToAllDNs();
-    storageMovementNeeded.init();
-    storagePolicySatisfierThread = new Daemon(this);
-    storagePolicySatisfierThread.setName("StoragePolicySatisfier");
-    storagePolicySatisfierThread.start();
-    this.storageMovementsMonitor.start();
-  }
-
-  /**
-   * Disables storage policy satisfier by stopping its services.
-   *
-   * @param forceStop
-   *          true represents that it should stop SPS service by clearing all
-   *          pending SPS work
-   */
-  public synchronized void disable(boolean forceStop) {
-    isRunning = false;
-
-    if (storagePolicySatisfierThread == null) {
-      return;
-    }
-
-    storageMovementNeeded.close();
-
-    storagePolicySatisfierThread.interrupt();
-    this.storageMovementsMonitor.stop();
-    if (forceStop) {
-      storageMovementNeeded.clearQueuesWithNotification();
-      addDropSPSWorkCommandsToAllDNs();
-    } else {
-      LOG.info("Stopping StoragePolicySatisfier.");
-    }
-  }
-
-  /**
-   * Timed wait to stop storage policy satisfier daemon threads.
-   */
-  public synchronized void stopGracefully() {
-    if (isRunning) {
-      disable(true);
-    }
-    this.storageMovementsMonitor.stopGracefully();
-
-    if (storagePolicySatisfierThread == null) {
-      return;
-    }
-    try {
-      storagePolicySatisfierThread.join(3000);
-    } catch (InterruptedException ie) {
-    }
-  }
-
-  /**
-   * 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() {
-    String moverId = HdfsServerConstants.MOVER_ID_PATH.toString();
-    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() {
-    while (namesystem.isRunning() && isRunning) {
-      try {
-        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;
-            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, true);
-              } else {
-                status =
-                    analyseBlocksStorageMovementsAndAssignToDN(
-                        blockCollection);
-              }
-            } finally {
-              namesystem.readUnlock();
-            }
-            if (blockCollection != null) {
-              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 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, itemInfo.getRetryCount()));
-                break;
-              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.");
-                }
-                itemInfo.retryCount++;
-                this.storageMovementNeeded.add(itemInfo);
-                break;
-              case FEW_LOW_REDUNDANCY_BLOCKS:
-                if (LOG.isDebugEnabled()) {
-                  LOG.debug("Adding trackID " + trackId
-                      + " back to retry queue as some of the blocks"
-                      + " are low redundant.");
-                }
-                this.storageMovementNeeded.add(itemInfo);
-                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.");
-                storageMovementNeeded.removeItemTrackInfo(itemInfo, true);
-                break;
-              }
-            }
-          }
-        }
-        int numLiveDn = namesystem.getFSDirectory().getBlockManager()
-            .getDatanodeManager().getNumLiveDataNodes();
-        if (storageMovementNeeded.size() == 0
-            || blockCount > (numLiveDn * spsWorkMultiplier)) {
-          Thread.sleep(3000);
-          blockCount = 0L;
-        }
-      } catch (Throwable t) {
-        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);
-            }
-            return;
-          }
-        }
-      }
-    }
-    LOG.error("StoragePolicySatisfier thread received runtime exception. "
-        + "Stopping Storage policy satisfier work", t);
-    return;
-  }
-
-  private BlocksMovingAnalysis analyseBlocksStorageMovementsAndAssignToDN(
-      BlockCollection blockCollection) {
-    BlocksMovingAnalysis.Status status =
-        BlocksMovingAnalysis.Status.BLOCKS_ALREADY_SATISFIED;
-    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
-      LOG.info("BlockCollectionID: {} file is under construction. So, postpone"
-          + " this to the next retry iteration", blockCollection.getId());
-      return new BlocksMovingAnalysis(
-          BlocksMovingAnalysis.Status.ANALYSIS_SKIPPED_FOR_RETRY,
-          new ArrayList<>());
-    }
-
-    BlockInfo[] blocks = blockCollection.getBlocks();
-    if (blocks.length == 0) {
-      LOG.info("BlockCollectionID: {} file is not having any blocks."
-          + " So, skipping the analysis.", blockCollection.getId());
-      return new BlocksMovingAnalysis(
-          BlocksMovingAnalysis.Status.BLOCKS_TARGET_PAIRING_SKIPPED,
-          new ArrayList<>());
-    }
-    List<BlockMovingInfo> blockMovingInfos = new ArrayList<BlockMovingInfo>();
-
-    for (int i = 0; i < blocks.length; i++) {
-      BlockInfo blockInfo = blocks[i];
-      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 new BlocksMovingAnalysis(
-              BlocksMovingAnalysis.Status.BLOCKS_TARGET_PAIRING_SKIPPED,
-              new ArrayList<>());
-        }
-      } else {
-        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)) {
-        boolean blocksPaired = computeBlockMovingInfos(blockMovingInfos,
-            blockInfo, expectedStorageTypes, existing, storages);
-        if (blocksPaired) {
-          status = BlocksMovingAnalysis.Status.BLOCKS_TARGETS_PAIRED;
-        } else {
-          // 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 = BlocksMovingAnalysis.Status.FEW_LOW_REDUNDANCY_BLOCKS;
-        }
-      }
-    }
-
-    List<Block> assignedBlockIds = new ArrayList<Block>();
-    for (BlockMovingInfo blkMovingInfo : blockMovingInfos) {
-      // 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++;
-      }
-    }
-    return new BlocksMovingAnalysis(status, assignedBlockIds);
-  }
-
-  /**
-   * 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
-   * @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> existing,
-      DatanodeStorageInfo[] storages) {
-    boolean foundMatchingTargetNodesForBlock = true;
-    if (!DFSUtil.removeOverlapBetweenStorageTypes(expectedStorageTypes,
-        existing, true)) {
-      List<StorageTypeNodePair> sourceWithStorageMap =
-          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 = 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);
-
-      foundMatchingTargetNodesForBlock |= findSourceAndTargetToMove(
-          blockMovingInfos, blockInfo, sourceWithStorageMap,
-          expectedStorageTypes, locsForExpectedStorageTypes);
-    }
-    return foundMatchingTargetNodesForBlock;
-  }
-
-  /**
-   * 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 sourceWithStorageList
-   *          - Source Datanode with storages list
-   * @param expected
-   *          - Expecting storages to move
-   * @param locsForExpectedStorageTypes
-   *          - Available DNs for expected storage types
-   * @return false if some of the block locations failed to find target node to
-   *         satisfy the storage policy
-   */
-  private boolean findSourceAndTargetToMove(
-      List<BlockMovingInfo> blockMovingInfos, BlockInfo blockInfo,
-      List<StorageTypeNodePair> sourceWithStorageList,
-      List<StorageType> expected,
-      StorageTypeNodeMap locsForExpectedStorageTypes) {
-    boolean foundMatchingTargetNodesForBlock = true;
-    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);
-
-      // 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) {
-          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?
-        }
-      }
-      // To avoid choosing this excludeNodes as targets later
-      excludeNodes.add(existingTypeNodePair.dn);
-    }
-
-    // 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 (checkIfAlreadyChosen(blockMovingInfos, existingTypeNodePair.dn)) {
-        continue;
-      }
-      if (chosenTarget == null && blockManager.getDatanodeManager()
-          .getNetworkTopology().isNodeGroupAware()) {
-        chosenTarget = chooseTarget(blockInfo, existingTypeNodePair.dn,
-            expected, Matcher.SAME_NODE_GROUP, locsForExpectedStorageTypes,
-            excludeNodes);
-      }
-
-      // Then, match nodes on the same rack
-      if (chosenTarget == null) {
-        chosenTarget =
-            chooseTarget(blockInfo, existingTypeNodePair.dn, expected,
-                Matcher.SAME_RACK, locsForExpectedStorageTypes, excludeNodes);
-      }
-
-      if (chosenTarget == null) {
-        chosenTarget =
-            chooseTarget(blockInfo, existingTypeNodePair.dn, expected,
-                Matcher.ANY_OTHER, locsForExpectedStorageTypes, excludeNodes);
-      }
-      if (null != chosenTarget) {
-        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?
-      } else {
-        LOG.warn(
-            "Failed to choose target datanode for the required"
-                + " storage types {}, block:{}, existing storage type:{}",
-            expected, blockInfo, existingTypeNodePair.storageType);
-      }
-    }
-
-    if (expected.size() > 0) {
-      foundMatchingTargetNodesForBlock = false;
-    }
-
-    return foundMatchingTargetNodesForBlock;
-  }
-
-  private boolean checkIfAlreadyChosen(List<BlockMovingInfo> blockMovingInfos,
-      DatanodeDescriptor dn) {
-    for (BlockMovingInfo blockMovingInfo : blockMovingInfos) {
-      if (blockMovingInfo.getSource().equals(dn)) {
-        return true;
-      }
-    }
-    return false;
-  }
-
-  private void buildContinuousBlockMovingInfos(BlockInfo blockInfo,
-      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, sourceNode,
-        targetNode, sourceStorageType, targetStorageType);
-    blkMovingInfos.add(blkMovingInfo);
-  }
-
-  private void buildStripedBlockMovingInfos(BlockInfo blockInfo,
-      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
-    // 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();
-        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);
-        }
-      }
-    }
-  }
-
-  /**
-   * Choose the target storage within same datanode if possible.
-   *
-   * @param block
-   *          - block info
-   * @param source
-   *          - source datanode
-   * @param targetTypes
-   *          - list of target storage types
-   */
-  private StorageTypeNodePair chooseTargetTypeInSameNode(Block block,
-      DatanodeDescriptor source, List<StorageType> targetTypes) {
-    for (StorageType t : targetTypes) {
-      DatanodeStorageInfo chooseStorage4Block =
-          source.chooseStorage4Block(t, block.getNumBytes());
-      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> excludeNodes) {
-    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 (!excludeNodes.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 {
-    private StorageType storageType = null;
-    private DatanodeDescriptor dn = null;
-
-    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 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);
-
-    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);
-    }
-  }
-
-  /**
-   * Receives set of storage movement attempt finished blocks report.
-   *
-   * @param moveAttemptFinishedBlks
-   *          set of storage movement attempt finished blocks.
-   */
-  void handleStorageMovementAttemptFinishedBlks(
-      BlocksStorageMoveAttemptFinished moveAttemptFinishedBlks) {
-    if (moveAttemptFinishedBlks.getBlocks().length <= 0) {
-      return;
-    }
-    storageMovementsMonitor
-        .addReportedMovedBlocks(moveAttemptFinishedBlks.getBlocks());
-  }
-
-  @VisibleForTesting
-  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();
-  }
-
-  /**
-   * Set file inode in queue for which storage movement needed for its blocks.
-   *
-   * @param inodeId
-   *          - file inode/blockcollection id.
-   */
-  public void satisfyStoragePolicy(Long inodeId) {
-    //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 "
-          + "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);
-  }
-
-  /**
-   * ItemInfo is a file info object for which need to satisfy the
-   * policy.
-   */
-  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;
-    }
-
-    /**
-     * Return the start inode id of the current track Id.
-     */
-    public long getStartId() {
-      return startId;
-    }
-
-    /**
-     * 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 (startId != trackId);
-    }
-
-    /**
-     * Get the attempted retry count of the block for satisfy the policy.
-     */
-    public int getRetryCount() {
-      return retryCount;
-    }
-  }
-
-  /**
-   * 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, int retryCount) {
-      super(rootId, trackId, retryCount);
-      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;
-    }
-
-  }
-
-  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/78420719/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/BlockStorageMovementAttemptedItems.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/BlockStorageMovementAttemptedItems.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/BlockStorageMovementAttemptedItems.java
new file mode 100644
index 0000000..b044f30
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/BlockStorageMovementAttemptedItems.java
@@ -0,0 +1,241 @@
+/**
+ * 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.sps;
+
+import static org.apache.hadoop.util.Time.monotonicNow;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Iterator;
+import java.util.List;
+
+import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.server.namenode.sps.StoragePolicySatisfier.AttemptedItemInfo;
+import org.apache.hadoop.hdfs.server.namenode.sps.StoragePolicySatisfier.ItemInfo;
+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 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 =
+      LoggerFactory.getLogger(BlockStorageMovementAttemptedItems.class);
+
+  /**
+   * A map holds the items which are already taken for blocks movements
+   * processing and sent to DNs.
+   */
+  private final List<AttemptedItemInfo> storageMovementAttemptedItems;
+  private final List<Block> movementFinishedBlocks;
+  private volatile boolean monitorRunning = true;
+  private Daemon timerThread = null;
+  //
+  // It might take anywhere between 5 to 10 minutes before
+  // a request is timed out.
+  //
+  private long selfRetryTimeout = 5 * 60 * 1000;
+
+  //
+  // It might take anywhere between 1 to 2 minutes before
+  // a request is timed out.
+  //
+  private long minCheckTimeout = 1 * 60 * 1000; // minimum value
+  private BlockStorageMovementNeeded blockStorageMovementNeeded;
+
+  public BlockStorageMovementAttemptedItems(long recheckTimeout,
+      long selfRetryTimeout,
+      BlockStorageMovementNeeded unsatisfiedStorageMovementFiles) {
+    if (recheckTimeout > 0) {
+      this.minCheckTimeout = Math.min(minCheckTimeout, recheckTimeout);
+    }
+
+    this.selfRetryTimeout = selfRetryTimeout;
+    this.blockStorageMovementNeeded = unsatisfiedStorageMovementFiles;
+    storageMovementAttemptedItems = new ArrayList<>();
+    movementFinishedBlocks = new ArrayList<>();
+  }
+
+  /**
+   * Add item to block storage movement attempted items map which holds the
+   * tracking/blockCollection id versus time stamp.
+   *
+   * @param itemInfo
+   *          - tracking info
+   */
+  public void add(AttemptedItemInfo itemInfo) {
+    synchronized (storageMovementAttemptedItems) {
+      storageMovementAttemptedItems.add(itemInfo);
+    }
+  }
+
+  /**
+   * Add the storage movement attempt finished blocks to
+   * storageMovementFinishedBlocks.
+   *
+   * @param moveAttemptFinishedBlks
+   *          storage movement attempt finished blocks
+   */
+  public void addReportedMovedBlocks(Block[] moveAttemptFinishedBlks) {
+    if (moveAttemptFinishedBlks.length == 0) {
+      return;
+    }
+    synchronized (movementFinishedBlocks) {
+      movementFinishedBlocks.addAll(Arrays.asList(moveAttemptFinishedBlks));
+    }
+  }
+
+  /**
+   * Starts the monitor thread.
+   */
+  public synchronized void start() {
+    monitorRunning = true;
+    timerThread = new Daemon(new BlocksStorageMovementAttemptMonitor());
+    timerThread.setName("BlocksStorageMovementAttemptMonitor");
+    timerThread.start();
+  }
+
+  /**
+   * Sets running flag to false. Also, this will interrupt monitor thread and
+   * clear all the queued up tasks.
+   */
+  public synchronized void stop() {
+    monitorRunning = false;
+    if (timerThread != null) {
+      timerThread.interrupt();
+    }
+    this.clearQueues();
+  }
+
+  /**
+   * Timed wait to stop monitor thread.
+   */
+  synchronized void stopGracefully() {
+    if (timerThread == null) {
+      return;
+    }
+    if (monitorRunning) {
+      stop();
+    }
+    try {
+      timerThread.join(3000);
+    } catch (InterruptedException ie) {
+    }
+  }
+
+  /**
+   * A monitor class for checking block storage movement attempt status and long
+   * waiting items periodically.
+   */
+  private class BlocksStorageMovementAttemptMonitor implements Runnable {
+    @Override
+    public void run() {
+      while (monitorRunning) {
+        try {
+          blockStorageMovementReportedItemsCheck();
+          blocksStorageMovementUnReportedItemsCheck();
+          Thread.sleep(minCheckTimeout);
+        } catch (InterruptedException ie) {
+          LOG.info("BlocksStorageMovementAttemptMonitor thread "
+              + "is interrupted.", ie);
+        } catch (IOException ie) {
+          LOG.warn("BlocksStorageMovementAttemptMonitor thread "
+              + "received exception and exiting.", ie);
+        }
+      }
+    }
+  }
+
+  @VisibleForTesting
+  void blocksStorageMovementUnReportedItemsCheck() {
+    synchronized (storageMovementAttemptedItems) {
+      Iterator<AttemptedItemInfo> iter = storageMovementAttemptedItems
+          .iterator();
+      long now = monotonicNow();
+      while (iter.hasNext()) {
+        AttemptedItemInfo itemInfo = iter.next();
+        if (now > itemInfo.getLastAttemptedOrReportedTime()
+            + selfRetryTimeout) {
+          Long blockCollectionID = itemInfo.getTrackId();
+          synchronized (movementFinishedBlocks) {
+            ItemInfo candidate = new ItemInfo(itemInfo.getStartId(),
+                blockCollectionID, itemInfo.getRetryCount() + 1);
+            blockStorageMovementNeeded.add(candidate);
+            iter.remove();
+            LOG.info("TrackID: {} becomes timed out and moved to needed "
+                + "retries queue for next iteration.", blockCollectionID);
+          }
+        }
+      }
+
+    }
+  }
+
+  @VisibleForTesting
+  void blockStorageMovementReportedItemsCheck() throws IOException {
+    synchronized (movementFinishedBlocks) {
+      Iterator<Block> finishedBlksIter = movementFinishedBlocks.iterator();
+      while (finishedBlksIter.hasNext()) {
+        Block blk = finishedBlksIter.next();
+        synchronized (storageMovementAttemptedItems) {
+          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.add(new ItemInfo(attemptedItemInfo
+                  .getStartId(), attemptedItemInfo.getTrackId(),
+                  attemptedItemInfo.getRetryCount() + 1));
+              iterator.remove();
+            }
+          }
+        }
+        // Remove attempted blocks from movementFinishedBlocks list.
+        finishedBlksIter.remove();
+      }
+    }
+  }
+
+  @VisibleForTesting
+  public int getMovementFinishedBlocksCount() {
+    return movementFinishedBlocks.size();
+  }
+
+  @VisibleForTesting
+  public int getAttemptedItemsCount() {
+    return storageMovementAttemptedItems.size();
+  }
+
+  public void clearQueues() {
+    movementFinishedBlocks.clear();
+    storageMovementAttemptedItems.clear();
+  }
+}


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