Kaynağa Gözat

HDFS-14153. [SPS] : Add Support for Storage Policy Satisfier in WEBHDFS. Contributed by Ayush Saxena.

Surendra Singh Lilhore 6 yıl önce
ebeveyn
işleme
221e308cb5

+ 6 - 0
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java

@@ -1311,6 +1311,12 @@ public class WebHdfsFileSystem extends FileSystem
     new FsPathRunner(op, p).run();
   }
 
+  @Override
+  public void satisfyStoragePolicy(final Path p) throws IOException {
+    final HttpOpParam.Op op = PutOpParam.Op.SATISFYSTORAGEPOLICY;
+    new FsPathRunner(op, p).run();
+  }
+
   public void enableECPolicy(String policyName) throws IOException {
     statistics.incrementWriteOps(1);
     storageStatistics.incrementOpCounter(OpType.ENABLE_EC_POLICY);

+ 1 - 0
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/resources/PutOpParam.java

@@ -41,6 +41,7 @@ public class PutOpParam extends HttpOpParam<PutOpParam.Op> {
     REMOVEACLENTRIES(false, HttpURLConnection.HTTP_OK),
     REMOVEDEFAULTACL(false, HttpURLConnection.HTTP_OK),
     REMOVEACL(false, HttpURLConnection.HTTP_OK),
+    SATISFYSTORAGEPOLICY(false, HttpURLConnection.HTTP_OK),
     SETACL(false, HttpURLConnection.HTTP_OK),
 
     SETXATTR(false, HttpURLConnection.HTTP_OK),

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

@@ -256,6 +256,7 @@ public class RouterWebHdfsMethods extends NamenodeWebHdfsMethods {
     case SETSTORAGEPOLICY:
     case ENABLEECPOLICY:
     case DISABLEECPOLICY:
+    case SATISFYSTORAGEPOLICY:
     {
       // Whitelist operations that can handled by NamenodeWebHdfsMethods
       return super.put(ugi, delegation, username, doAsUser, fullpath, op,

+ 4 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/web/resources/NamenodeWebHdfsMethods.java

@@ -801,6 +801,10 @@ public class NamenodeWebHdfsMethods {
       cp.setStoragePolicy(fullpath, policyName.getValue());
       return Response.ok().type(MediaType.APPLICATION_OCTET_STREAM).build();
     }
+    case SATISFYSTORAGEPOLICY:
+      cp.satisfyStoragePolicy(fullpath);
+      return Response.ok().type(MediaType.APPLICATION_OCTET_STREAM).build();
+
     case ENABLEECPOLICY:
       validateOpParams(op, ecpolicy);
       cp.enableErasureCodingPolicy(ecpolicy.getValue());

+ 14 - 0
hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/WebHDFS.md

@@ -72,6 +72,7 @@ The HTTP REST API supports the complete [FileSystem](../../api/org/apache/hadoop
     * [`SETXATTR`](#Set_XAttr) (see [FileSystem](../../api/org/apache/hadoop/fs/FileSystem.html).setXAttr)
     * [`REMOVEXATTR`](#Remove_XAttr) (see [FileSystem](../../api/org/apache/hadoop/fs/FileSystem.html).removeXAttr)
     * [`SETSTORAGEPOLICY`](#Set_Storage_Policy) (see [FileSystem](../../api/org/apache/hadoop/fs/FileSystem.html).setStoragePolicy)
+    * [`SATISFYSTORAGEPOLICY`](#Satisfy_Storage_Policy) (see [ArchivalStorage](./ArchivalStorage.html#Satisfy_Storage_Policy).satisfyStoragePolicy)
     * [`ENABLEECPOLICY`](#Enable_EC_Policy) (see [HDFSErasureCoding](./HDFSErasureCoding.html#Administrative_commands).enablePolicy)
     * [`DISABLEECPOLICY`](#Disable_EC_Policy) (see [HDFSErasureCoding](./HDFSErasureCoding.html#Administrative_commands).disablePolicy)
     * [`SETECPOLICY`](#Set_EC_Policy) (see [HDFSErasureCoding](./HDFSErasureCoding.html#Administrative_commands).setErasureCodingPolicy)
@@ -1095,6 +1096,19 @@ See also: [FileSystem](../../api/org/apache/hadoop/fs/FileSystem.html).unsetStor
 
 See also: [FileSystem](../../api/org/apache/hadoop/fs/FileSystem.html).getStoragePolicy
 
+### Satisfy Storage Policy
+
+* Submit a HTTP PUT request.
+
+        curl -i -X PUT "http://<HOST>:<PORT>/webhdfs/v1/<PATH>?op=SATISFYSTORAGEPOLICY"
+
+    The client receives a response with zero content length:
+
+        HTTP/1.1 200 OK
+        Content-Length: 0
+
+See also: [ArchivalStorage](./ArchivalStorage.html#Satisfy_Storage_Policy).satisfyStoragePolicy
+
 ### Get File Block Locations
 
 * Submit a HTTP GET request.

+ 33 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHDFS.java

@@ -88,16 +88,20 @@ import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicyInfo;
 import org.apache.hadoop.hdfs.protocol.SystemErasureCodingPolicies;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants.StoragePolicySatisfierMode;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
 import static org.apache.hadoop.hdfs.protocol.SnapshotDiffReport.DiffType;
 import static org.apache.hadoop.hdfs.protocol.SnapshotDiffReport.DiffReportEntry;
 import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotTestHelper;
+import org.apache.hadoop.hdfs.server.namenode.sps.StoragePolicySatisfier;
 import org.apache.hadoop.hdfs.server.namenode.web.resources.NamenodeWebHdfsMethods;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
+import org.apache.hadoop.hdfs.server.sps.ExternalSPSContext;
 import org.apache.hadoop.hdfs.web.WebHdfsFileSystem.WebHdfsInputStream;
 import org.apache.hadoop.hdfs.web.resources.LengthParam;
 import org.apache.hadoop.hdfs.web.resources.NoRedirectParam;
@@ -1657,6 +1661,35 @@ public class TestWebHDFS {
     }
   }
 
+  // Test for Storage Policy Satisfier in DFS.
+
+  @Test
+  public void testWebHdfsSps() throws Exception {
+    final Configuration conf = new HdfsConfiguration();
+    conf.set(DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_MODE_KEY,
+        StoragePolicySatisfierMode.EXTERNAL.toString());
+    StoragePolicySatisfier sps = new StoragePolicySatisfier(conf);
+    try (MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
+        .storageTypes(
+            new StorageType[][] {{StorageType.DISK, StorageType.ARCHIVE}})
+        .storagesPerDatanode(2).numDataNodes(1).build()) {
+      cluster.waitActive();
+      sps.init(new ExternalSPSContext(sps, DFSTestUtil.getNameNodeConnector(
+          conf, HdfsServerConstants.MOVER_ID_PATH, 1, false)));
+      sps.start(StoragePolicySatisfierMode.EXTERNAL);
+      sps.start(StoragePolicySatisfierMode.EXTERNAL);
+      DistributedFileSystem dfs = cluster.getFileSystem();
+      DFSTestUtil.createFile(dfs, new Path("/file"), 1024L, (short) 1, 0L);
+      DFSTestUtil.waitForReplication(dfs, new Path("/file"), (short) 1, 5000);
+      dfs.setStoragePolicy(new Path("/file"), "COLD");
+      dfs.satisfyStoragePolicy(new Path("/file"));
+      DFSTestUtil.waitExpectedStorageType("/file", StorageType.ARCHIVE, 1,
+          30000, dfs);
+    } finally {
+      sps.stopGracefully();
+    }
+  }
+
   @Test
   public void testWebHdfsAppend() throws Exception {
     MiniDFSCluster cluster = null;