|
@@ -21,6 +21,7 @@ package org.apache.hadoop.fs.aliyun.oss;
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
|
import org.apache.hadoop.fs.FileAlreadyExistsException;
|
|
|
import org.apache.hadoop.fs.FileStatus;
|
|
|
+import org.apache.hadoop.fs.FileSystem;
|
|
|
import org.apache.hadoop.fs.FileSystemContractBaseTest;
|
|
|
import org.apache.hadoop.fs.Path;
|
|
|
|
|
@@ -359,4 +360,79 @@ public class TestAliyunOSSFileSystemContract
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+ @Test
|
|
|
+ public void testRenameChangingDirShouldFail() throws Exception {
|
|
|
+ testRenameDir(true, false, false);
|
|
|
+ testRenameDir(true, true, true);
|
|
|
+ }
|
|
|
+
|
|
|
+ @Test
|
|
|
+ public void testRenameDir() throws Exception {
|
|
|
+ testRenameDir(false, true, false);
|
|
|
+ testRenameDir(false, true, true);
|
|
|
+ }
|
|
|
+
|
|
|
+ private void testRenameDir(boolean changing, boolean result, boolean empty)
|
|
|
+ throws Exception {
|
|
|
+ fs.getConf().setLong(Constants.FS_OSS_BLOCK_SIZE_KEY, 1024);
|
|
|
+ String key = "a/b/test.file";
|
|
|
+ for (int i = 0; i < 100; i++) {
|
|
|
+ if (empty) {
|
|
|
+ fs.createNewFile(this.path(key + "." + i));
|
|
|
+ } else {
|
|
|
+ createFile(this.path(key + "." + i));
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ Path srcPath = this.path("a");
|
|
|
+ Path dstPath = this.path("b");
|
|
|
+ TestRenameTask task = new TestRenameTask(fs, srcPath, dstPath);
|
|
|
+ Thread thread = new Thread(task);
|
|
|
+ thread.start();
|
|
|
+ while (!task.isRunning()) {
|
|
|
+ Thread.sleep(1000);
|
|
|
+ }
|
|
|
+
|
|
|
+ if (changing) {
|
|
|
+ fs.delete(this.path("a/b"), true);
|
|
|
+ }
|
|
|
+
|
|
|
+ thread.join();
|
|
|
+ assertEquals(result, task.isSucceed());
|
|
|
+ }
|
|
|
+
|
|
|
+ class TestRenameTask implements Runnable {
|
|
|
+ private FileSystem fs;
|
|
|
+ private Path srcPath;
|
|
|
+ private Path dstPath;
|
|
|
+ private boolean result;
|
|
|
+ private boolean running;
|
|
|
+ TestRenameTask(FileSystem fs, Path srcPath, Path dstPath) {
|
|
|
+ this.fs = fs;
|
|
|
+ this.srcPath = srcPath;
|
|
|
+ this.dstPath = dstPath;
|
|
|
+ this.result = false;
|
|
|
+ this.running = false;
|
|
|
+ }
|
|
|
+
|
|
|
+ boolean isSucceed() {
|
|
|
+ return this.result;
|
|
|
+ }
|
|
|
+
|
|
|
+ boolean isRunning() {
|
|
|
+ return this.running;
|
|
|
+ }
|
|
|
+ @Override
|
|
|
+ public void run() {
|
|
|
+ try {
|
|
|
+ running = true;
|
|
|
+ result = fs.rename(srcPath, dstPath);
|
|
|
+ } catch (Exception e) {
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ protected int getGlobalTimeout() {
|
|
|
+ return 120 * 1000;
|
|
|
+ }
|
|
|
}
|