HDFS-14184. [SPS] Add support for URI based path in satisfystoragepolicy command. Contributed by Ayush Saxena.

This commit is contained in:
Surendra Singh Lilhore 2019-01-03 18:36:53 +05:30
parent cb26f15428
commit 21fe77e3ab
2 changed files with 25 additions and 4 deletions

View File

@ -23,7 +23,6 @@
import org.apache.hadoop.fs.BlockStoragePolicySpi;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hdfs.DistributedFileSystem;
import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
@ -278,10 +277,10 @@ public int run(Configuration conf, List<String> args) throws IOException {
"policy.\nUsage: " + getLongUsage());
return 1;
}
final DistributedFileSystem dfs = AdminHelper.getDFS(conf);
Path p = new Path(path);
final FileSystem fs = FileSystem.get(p.toUri(), conf);
try {
dfs.satisfyStoragePolicy(new Path(path));
fs.satisfyStoragePolicy(p);
System.out.println("Scheduled blocks to move based on the current"
+ " storage policy on " + path);
} catch (Exception e) {

View File

@ -109,4 +109,26 @@ public void testStoragePolicySatisfierCommand() throws Exception {
DFSTestUtil.waitExpectedStorageType(file, StorageType.ARCHIVE, 1, 30000,
dfs);
}
@Test(timeout = 30000)
public void testStoragePolicySatisfierCommandWithURI() throws Exception {
final String file = "/testStoragePolicySatisfierCommandURI";
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 " + dfs.getUri() + file, 0,
"Scheduled blocks to move based on the current storage policy on "
+ dfs.getUri() + file.toString());
DFSTestUtil.waitExpectedStorageType(file, StorageType.ARCHIVE, 1, 30000,
dfs);
}
}