|
@@ -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;
|