|
@@ -19,101 +19,166 @@
|
|
|
package org.apache.hadoop.fs.azure;
|
|
|
|
|
|
|
|
|
+import org.apache.hadoop.fs.FSDataOutputStream;
|
|
|
import org.apache.hadoop.fs.FileSystem;
|
|
|
import org.apache.hadoop.fs.Path;
|
|
|
import org.junit.Assert;
|
|
|
import org.junit.Test;
|
|
|
|
|
|
+import java.util.ArrayList;
|
|
|
+import java.util.List;
|
|
|
+import java.util.concurrent.Callable;
|
|
|
+import java.util.concurrent.ExecutorService;
|
|
|
+import java.util.concurrent.Executors;
|
|
|
+import java.util.concurrent.Future;
|
|
|
+
|
|
|
/***
|
|
|
* Test class to hold all Live Azure storage concurrency tests.
|
|
|
*/
|
|
|
public class TestNativeAzureFileSystemConcurrencyLive
|
|
|
extends AbstractWasbTestBase {
|
|
|
|
|
|
- private static final int TEST_COUNT = 102;
|
|
|
+ private static final int THREAD_COUNT = 102;
|
|
|
+ private static final int TEST_EXECUTION_TIMEOUT = 5000;
|
|
|
@Override
|
|
|
protected AzureBlobStorageTestAccount createTestAccount() throws Exception {
|
|
|
return AzureBlobStorageTestAccount.create();
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
- * Test multi-threaded deletes in WASB. Expected behavior is one of the thread
|
|
|
- * should be to successfully delete the file and return true and all other
|
|
|
- * threads need to return false.
|
|
|
+ * Validate contract for FileSystem.create when overwrite is true and there
|
|
|
+ * are concurrent callers of FileSystem.delete. An existing file should be
|
|
|
+ * overwritten, even if the original destination exists but is deleted by an
|
|
|
+ * external agent during the create operation.
|
|
|
*/
|
|
|
- @Test
|
|
|
- public void testMultiThreadedDeletes() throws Exception {
|
|
|
+ @Test(timeout = TEST_EXECUTION_TIMEOUT)
|
|
|
+ public void testConcurrentCreateDeleteFile() throws Exception {
|
|
|
Path testFile = new Path("test.dat");
|
|
|
- fs.create(testFile).close();
|
|
|
|
|
|
- int threadCount = TEST_COUNT;
|
|
|
- DeleteHelperThread[] helperThreads = new DeleteHelperThread[threadCount];
|
|
|
+ List<CreateFileTask> tasks = new ArrayList<>(THREAD_COUNT);
|
|
|
|
|
|
- for (int i = 0; i < threadCount; i++) {
|
|
|
- helperThreads[i] = new DeleteHelperThread(fs, testFile);
|
|
|
+ for (int i = 0; i < THREAD_COUNT; i++) {
|
|
|
+ tasks.add(new CreateFileTask(fs, testFile));
|
|
|
}
|
|
|
|
|
|
- Thread[] threads = new Thread[threadCount];
|
|
|
+ ExecutorService es = null;
|
|
|
+
|
|
|
+ try {
|
|
|
+ es = Executors.newFixedThreadPool(THREAD_COUNT);
|
|
|
+
|
|
|
+ List<Future<Void>> futures = es.invokeAll(tasks);
|
|
|
+
|
|
|
+ for (Future<Void> future : futures) {
|
|
|
+ Assert.assertTrue(future.isDone());
|
|
|
|
|
|
- for (int i = 0; i < threadCount; i++) {
|
|
|
- threads[i] = new Thread(helperThreads[i]);
|
|
|
- threads[i].start();
|
|
|
+ // we are using Callable<V>, so if an exception
|
|
|
+ // occurred during the operation, it will be thrown
|
|
|
+ // when we call get
|
|
|
+ Assert.assertEquals(null, future.get());
|
|
|
+ }
|
|
|
+ } finally {
|
|
|
+ if (es != null) {
|
|
|
+ es.shutdownNow();
|
|
|
+ }
|
|
|
}
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Validate contract for FileSystem.delete when invoked concurrently.
|
|
|
+ * One of the threads should successfully delete the file and return true;
|
|
|
+ * all other threads should return false.
|
|
|
+ */
|
|
|
+ @Test(timeout = TEST_EXECUTION_TIMEOUT)
|
|
|
+ public void testConcurrentDeleteFile() throws Exception {
|
|
|
+ Path testFile = new Path("test.dat");
|
|
|
+ fs.create(testFile).close();
|
|
|
|
|
|
- for (int i = 0; i < threadCount; i++) {
|
|
|
- threads[i].join();
|
|
|
+ List<DeleteFileTask> tasks = new ArrayList<>(THREAD_COUNT);
|
|
|
+
|
|
|
+ for (int i = 0; i < THREAD_COUNT; i++) {
|
|
|
+ tasks.add(new DeleteFileTask(fs, testFile));
|
|
|
}
|
|
|
|
|
|
- boolean deleteSuccess = false;
|
|
|
+ ExecutorService es = null;
|
|
|
+ try {
|
|
|
+ es = Executors.newFixedThreadPool(THREAD_COUNT);
|
|
|
+
|
|
|
+ List<Future<Boolean>> futures = es.invokeAll(tasks);
|
|
|
|
|
|
- for (int i = 0; i < threadCount; i++) {
|
|
|
+ int successCount = 0;
|
|
|
+ for (Future<Boolean> future : futures) {
|
|
|
+ Assert.assertTrue(future.isDone());
|
|
|
|
|
|
- Assert.assertFalse("child thread has exception : " + helperThreads[i].getException(),
|
|
|
- helperThreads[i].getExceptionEncounteredFlag());
|
|
|
+ // we are using Callable<V>, so if an exception
|
|
|
+ // occurred during the operation, it will be thrown
|
|
|
+ // when we call get
|
|
|
+ Boolean success = future.get();
|
|
|
+ if (success) {
|
|
|
+ successCount++;
|
|
|
+ }
|
|
|
+ }
|
|
|
|
|
|
- if (deleteSuccess) {
|
|
|
- Assert.assertFalse("More than one thread delete() retuhelperThreads[i].getDeleteSuccess()",
|
|
|
- helperThreads[i].getExceptionEncounteredFlag());
|
|
|
- } else {
|
|
|
- deleteSuccess = helperThreads[i].getDeleteSuccess();
|
|
|
+ Assert.assertEquals(
|
|
|
+ "Exactly one delete operation should return true.",
|
|
|
+ 1,
|
|
|
+ successCount);
|
|
|
+ } finally {
|
|
|
+ if (es != null) {
|
|
|
+ es.shutdownNow();
|
|
|
}
|
|
|
}
|
|
|
-
|
|
|
- Assert.assertTrue("No successfull delete found", deleteSuccess);
|
|
|
}
|
|
|
}
|
|
|
|
|
|
-class DeleteHelperThread implements Runnable {
|
|
|
+abstract class FileSystemTask<V> implements Callable<V> {
|
|
|
+ private final FileSystem fileSystem;
|
|
|
+ private final Path path;
|
|
|
+
|
|
|
+ protected FileSystem getFileSystem() {
|
|
|
+ return this.fileSystem;
|
|
|
+ }
|
|
|
|
|
|
- private FileSystem fs;
|
|
|
- private Path p;
|
|
|
- private boolean deleteSuccess;
|
|
|
- private boolean exceptionEncountered;
|
|
|
- private Exception ex;
|
|
|
+ protected Path getFilePath() {
|
|
|
+ return this.path;
|
|
|
+ }
|
|
|
|
|
|
- public DeleteHelperThread(FileSystem fs, Path p) {
|
|
|
- this.fs = fs;
|
|
|
- this.p = p;
|
|
|
+ FileSystemTask(FileSystem fs, Path p) {
|
|
|
+ this.fileSystem = fs;
|
|
|
+ this.path = p;
|
|
|
}
|
|
|
|
|
|
- public void run() {
|
|
|
- try {
|
|
|
- deleteSuccess = fs.delete(p, false);
|
|
|
- } catch (Exception ioEx) {
|
|
|
- exceptionEncountered = true;
|
|
|
- this.ex = ioEx;
|
|
|
- }
|
|
|
+ public abstract V call() throws Exception;
|
|
|
+}
|
|
|
+
|
|
|
+class DeleteFileTask extends FileSystemTask<Boolean> {
|
|
|
+
|
|
|
+ DeleteFileTask(FileSystem fs, Path p) {
|
|
|
+ super(fs, p);
|
|
|
}
|
|
|
|
|
|
- public boolean getDeleteSuccess() {
|
|
|
- return deleteSuccess;
|
|
|
+ @Override
|
|
|
+ public Boolean call() throws Exception {
|
|
|
+ return this.getFileSystem().delete(this.getFilePath(), false);
|
|
|
}
|
|
|
+}
|
|
|
|
|
|
- public boolean getExceptionEncounteredFlag() {
|
|
|
- return exceptionEncountered;
|
|
|
+class CreateFileTask extends FileSystemTask<Void> {
|
|
|
+ CreateFileTask(FileSystem fs, Path p) {
|
|
|
+ super(fs, p);
|
|
|
}
|
|
|
|
|
|
- public Exception getException() {
|
|
|
- return ex;
|
|
|
+ public Void call() throws Exception {
|
|
|
+ FileSystem fs = getFileSystem();
|
|
|
+ Path p = getFilePath();
|
|
|
+
|
|
|
+ // Create an empty file and close the stream.
|
|
|
+ FSDataOutputStream stream = fs.create(p, true);
|
|
|
+ stream.close();
|
|
|
+
|
|
|
+ // Delete the file. We don't care if delete returns true or false.
|
|
|
+ // We just want to ensure the file does not exist.
|
|
|
+ this.getFileSystem().delete(this.getFilePath(), false);
|
|
|
+
|
|
|
+ return null;
|
|
|
}
|
|
|
}
|