Skip to content
Merged
Original file line number Diff line number Diff line change
Expand Up @@ -25,6 +25,8 @@
import java.util.UUID;
import java.util.concurrent.Callable;

import org.apache.commons.lang3.StringUtils;

import org.junit.After;
import org.junit.Assert;
import org.junit.Before;
Expand Down Expand Up @@ -81,6 +83,7 @@ public abstract class AbstractAbfsIntegrationTest extends
private AuthType authType;
private boolean useConfiguredFileSystem = false;
private boolean usingFilesystemForSASTests = false;
private static final int SHORTENED_GUID_LEN = 12;

protected AbstractAbfsIntegrationTest() throws Exception {
fileSystemName = TEST_CONTAINER_PREFIX + UUID.randomUUID().toString();
Expand Down Expand Up @@ -414,6 +417,16 @@ protected Path path(String filepath) throws IOException {
new Path(getTestPath(), filepath));
}

/**
* Generate a unique path using the given filepath
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: add .

* @param filepath path string
* @return unique path created from filepath and a GUID
*/
protected Path getUniquePath(String filepath) {
return new Path(filepath + StringUtils
.right(UUID.randomUUID().toString(), SHORTENED_GUID_LEN));
}

/**
* Get any Delegation Token manager created by the filesystem.
* @return the DT manager or null.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -91,7 +91,7 @@ public void testUnknownHost() throws Exception {
public void testListPathWithValidListMaxResultsValues()
throws IOException, ExecutionException, InterruptedException {
final int fileCount = 10;
final String directory = "testWithValidListMaxResultsValues";
final Path directory = getUniquePath("testWithValidListMaxResultsValues");
createDirectoryWithNFiles(directory, fileCount);
final int[] testData = {fileCount + 100, fileCount + 1, fileCount,
fileCount - 1, 1};
Expand All @@ -100,7 +100,7 @@ public void testListPathWithValidListMaxResultsValues()
setListMaxResults(listMaxResults);
int expectedListResultsSize =
listMaxResults > fileCount ? fileCount : listMaxResults;
Assertions.assertThat(listPath(directory)).describedAs(
Assertions.assertThat(listPath(directory.toString())).describedAs(
"AbfsClient.listPath result should contain %d items when "
+ "listMaxResults is %d and directory contains %d items",
expectedListResultsSize, listMaxResults, fileCount)
Expand All @@ -112,9 +112,10 @@ public void testListPathWithValidListMaxResultsValues()
public void testListPathWithValueGreaterThanServerMaximum()
throws IOException, ExecutionException, InterruptedException {
setListMaxResults(LIST_MAX_RESULTS_SERVER + 100);
final String directory = "testWithValueGreaterThanServerMaximum";
final Path directory = getUniquePath(
"testWithValueGreaterThanServerMaximum");
createDirectoryWithNFiles(directory, LIST_MAX_RESULTS_SERVER + 200);
Assertions.assertThat(listPath(directory)).describedAs(
Assertions.assertThat(listPath(directory.toString())).describedAs(
"AbfsClient.listPath result will contain a maximum of %d items "
+ "even if listMaxResults >= %d or directory "
+ "contains more than %d items", LIST_MAX_RESULTS_SERVER,
Expand Down Expand Up @@ -149,7 +150,7 @@ private void setListMaxResults(int listMaxResults) throws IOException {
.setListMaxResults(listMaxResults);
}

private void createDirectoryWithNFiles(String directory, int n)
private void createDirectoryWithNFiles(Path directory, int n)
throws ExecutionException, InterruptedException {
final List<Future<Void>> tasks = new ArrayList<>();
ExecutorService es = Executors.newFixedThreadPool(10);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -231,8 +231,8 @@ public void testHasNextForEmptyDir() throws Exception {
@Test
public void testHasNextForFile() throws Exception {
final AzureBlobFileSystem fs = getFileSystem();
String testFileName = "testFile";
Path testFile = new Path(testFileName);
Path testFile = getUniquePath("testFile");
String testFileName = testFile.toString();
getFileSystem().create(testFile);
setPageSize(10);
RemoteIterator<FileStatus> fsItr = fs.listStatusIterator(testFile);
Expand Down Expand Up @@ -297,7 +297,7 @@ public String listStatus(Path path, String startFrom,

private Path createTestDirectory() throws IOException {
String testDirectoryName = "testDirectory" + System.currentTimeMillis();
Path testDirectory = new Path(testDirectoryName);
Path testDirectory = getUniquePath(testDirectoryName);
getFileSystem().mkdirs(testDirectory);
return testDirectory;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -41,7 +41,7 @@
*/
@RunWith(Parameterized.class)
public class ITestAbfsReadWriteAndSeek extends AbstractAbfsScaleTest {
private static final Path TEST_PATH = new Path("/testfile");
private static final String TEST_PATH = "/testfile";

@Parameterized.Parameters(name = "Size={0}")
public static Iterable<Object[]> sizes() {
Expand Down Expand Up @@ -73,13 +73,14 @@ private void testReadWriteAndSeek(int bufferSize) throws Exception {
final byte[] b = new byte[2 * bufferSize];
new Random().nextBytes(b);

try (FSDataOutputStream stream = fs.create(TEST_PATH)) {
Path testPath = getUniquePath(TEST_PATH);
try (FSDataOutputStream stream = fs.create(testPath)) {
stream.write(b);
}

final byte[] readBuffer = new byte[2 * bufferSize];
int result;
try (FSDataInputStream inputStream = fs.open(TEST_PATH)) {
try (FSDataInputStream inputStream = fs.open(testPath)) {
inputStream.seek(bufferSize);
result = inputStream.read(readBuffer, bufferSize, bufferSize);
assertNotEquals(-1, result);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -52,8 +52,8 @@ public void testAbfsStreamOps() throws Exception {
+ "Abfs");

final AzureBlobFileSystem fs = getFileSystem();
Path smallOperationsFile = new Path("testOneReadWriteOps");
Path largeOperationsFile = new Path("testLargeReadWriteOps");
Path smallOperationsFile = getUniquePath("testOneReadWriteOps");
Path largeOperationsFile = getUniquePath("testLargeReadWriteOps");
FileSystem.Statistics statistics = fs.getFsStatistics();
String testReadWriteOps = "test this";
statistics.reset();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -32,8 +32,8 @@
*/
public class ITestAzureBlobFileSystemAppend extends
AbstractAbfsIntegrationTest {
private static final Path TEST_FILE_PATH = new Path("testfile");
private static final Path TEST_FOLDER_PATH = new Path("testFolder");
private static final String TEST_FILE_PATH = "testfile";
private static final String TEST_FOLDER_PATH = "testFolder";

public ITestAzureBlobFileSystemAppend() throws Exception {
super();
Expand All @@ -42,15 +42,15 @@ public ITestAzureBlobFileSystemAppend() throws Exception {
@Test(expected = FileNotFoundException.class)
public void testAppendDirShouldFail() throws Exception {
final AzureBlobFileSystem fs = getFileSystem();
final Path filePath = TEST_FILE_PATH;
final Path filePath = getUniquePath(TEST_FILE_PATH);
fs.mkdirs(filePath);
fs.append(filePath, 0);
}

@Test
public void testAppendWithLength0() throws Exception {
final AzureBlobFileSystem fs = getFileSystem();
try(FSDataOutputStream stream = fs.create(TEST_FILE_PATH)) {
try(FSDataOutputStream stream = fs.create(getUniquePath(TEST_FILE_PATH))) {
final byte[] b = new byte[1024];
new Random().nextBytes(b);
stream.write(b, 1000, 0);
Expand All @@ -62,7 +62,7 @@ public void testAppendWithLength0() throws Exception {
@Test(expected = FileNotFoundException.class)
public void testAppendFileAfterDelete() throws Exception {
final AzureBlobFileSystem fs = getFileSystem();
final Path filePath = TEST_FILE_PATH;
final Path filePath = getUniquePath(TEST_FILE_PATH);
ContractTestUtils.touch(fs, filePath);
fs.delete(filePath, false);

Expand All @@ -72,7 +72,7 @@ public void testAppendFileAfterDelete() throws Exception {
@Test(expected = FileNotFoundException.class)
public void testAppendDirectory() throws Exception {
final AzureBlobFileSystem fs = getFileSystem();
final Path folderPath = TEST_FOLDER_PATH;
final Path folderPath = getUniquePath(TEST_FOLDER_PATH);
fs.mkdirs(folderPath);
fs.append(folderPath);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -50,13 +50,16 @@ public void testBlobBackCompat() throws Exception {
CloudBlobContainer container = blobClient.getContainerReference(this.getFileSystemName());
container.createIfNotExists();

CloudBlockBlob blockBlob = container.getBlockBlobReference("test/10/10/10");
Path testPath = getUniquePath("test");
CloudBlockBlob blockBlob = container
.getBlockBlobReference(testPath + "/10/10/10");
blockBlob.uploadText("");

blockBlob = container.getBlockBlobReference("test/10/123/3/2/1/3");
blockBlob = container.getBlockBlobReference(testPath + "/10/123/3/2/1/3");
blockBlob.uploadText("");

FileStatus[] fileStatuses = fs.listStatus(new Path("/test/10/"));
FileStatus[] fileStatuses = fs
.listStatus(new Path(String.format("/%s/10/", testPath)));
assertEquals(2, fileStatuses.length);
assertEquals("10", fileStatuses[0].getPath().getName());
assertTrue(fileStatuses[0].isDirectory());
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -350,7 +350,8 @@ private void modifyAcl(Path file, String uid, FsAction fsAction)

private Path setupTestDirectoryAndUserAccess(String testFileName,
FsAction fsAction) throws Exception {
Path file = new Path(TEST_FOLDER_PATH + testFileName);
Path testPath = getUniquePath(TEST_FOLDER_PATH);
Path file = new Path(testPath + testFileName);
file = this.superUserFs.makeQualified(file);
this.superUserFs.delete(file, true);
this.superUserFs.create(file);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -53,7 +53,7 @@ public void testCopyFromLocalFileSystem() throws Exception {
localFs.delete(localFilePath, true);
try {
writeString(localFs, localFilePath, "Testing");
Path dstPath = new Path("copiedFromLocal");
Path dstPath = getUniquePath("copiedFromLocal");
assertTrue(FileUtil.copy(localFs, localFilePath, fs, dstPath, false,
fs.getConf()));
assertIsFile(fs, dstPath);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -66,7 +66,7 @@
public class ITestAzureBlobFileSystemCreate extends
AbstractAbfsIntegrationTest {
private static final Path TEST_FILE_PATH = new Path("testfile");
private static final Path TEST_FOLDER_PATH = new Path("testFolder");
private static final String TEST_FOLDER_PATH = "testFolder";
private static final String TEST_CHILD_FILE = "childFile";

public ITestAzureBlobFileSystemCreate() throws Exception {
Expand All @@ -89,13 +89,14 @@ public void testEnsureFileCreatedImmediately() throws Exception {
@SuppressWarnings("deprecation")
public void testCreateNonRecursive() throws Exception {
final AzureBlobFileSystem fs = getFileSystem();
Path testFile = new Path(TEST_FOLDER_PATH, TEST_CHILD_FILE);
Path testFolderPath = getUniquePath(TEST_FOLDER_PATH);
Path testFile = new Path(testFolderPath, TEST_CHILD_FILE);
try {
fs.createNonRecursive(testFile, true, 1024, (short) 1, 1024, null);
fail("Should've thrown");
} catch (FileNotFoundException expected) {
}
fs.mkdirs(TEST_FOLDER_PATH);
fs.mkdirs(testFolderPath);
fs.createNonRecursive(testFile, true, 1024, (short) 1, 1024, null)
.close();
assertIsFile(fs, testFile);
Expand All @@ -105,13 +106,14 @@ public void testCreateNonRecursive() throws Exception {
@SuppressWarnings("deprecation")
public void testCreateNonRecursive1() throws Exception {
final AzureBlobFileSystem fs = getFileSystem();
Path testFile = new Path(TEST_FOLDER_PATH, TEST_CHILD_FILE);
Path testFolderPath = getUniquePath(TEST_FOLDER_PATH);
Path testFile = new Path(testFolderPath, TEST_CHILD_FILE);
try {
fs.createNonRecursive(testFile, FsPermission.getDefault(), EnumSet.of(CreateFlag.CREATE, CreateFlag.OVERWRITE), 1024, (short) 1, 1024, null);
fail("Should've thrown");
} catch (FileNotFoundException expected) {
}
fs.mkdirs(TEST_FOLDER_PATH);
fs.mkdirs(testFolderPath);
fs.createNonRecursive(testFile, true, 1024, (short) 1, 1024, null)
.close();
assertIsFile(fs, testFile);
Expand All @@ -123,13 +125,14 @@ public void testCreateNonRecursive1() throws Exception {
public void testCreateNonRecursive2() throws Exception {
final AzureBlobFileSystem fs = getFileSystem();

Path testFile = new Path(TEST_FOLDER_PATH, TEST_CHILD_FILE);
Path testFolderPath = getUniquePath(TEST_FOLDER_PATH);
Path testFile = new Path(testFolderPath, TEST_CHILD_FILE);
try {
fs.createNonRecursive(testFile, FsPermission.getDefault(), false, 1024, (short) 1, 1024, null);
fail("Should've thrown");
} catch (FileNotFoundException e) {
}
fs.mkdirs(TEST_FOLDER_PATH);
fs.mkdirs(testFolderPath);
fs.createNonRecursive(testFile, true, 1024, (short) 1, 1024, null)
.close();
assertIsFile(fs, testFile);
Expand All @@ -141,7 +144,8 @@ public void testCreateNonRecursive2() throws Exception {
@Test
public void testWriteAfterClose() throws Throwable {
final AzureBlobFileSystem fs = getFileSystem();
Path testPath = new Path(TEST_FOLDER_PATH, TEST_CHILD_FILE);
Path testFolderPath = getUniquePath(TEST_FOLDER_PATH);
Path testPath = new Path(testFolderPath, TEST_CHILD_FILE);
FSDataOutputStream out = fs.create(testPath);
out.close();
intercept(IOException.class, () -> out.write('a'));
Expand All @@ -161,7 +165,8 @@ public void testWriteAfterClose() throws Throwable {
@Test
public void testTryWithResources() throws Throwable {
final AzureBlobFileSystem fs = getFileSystem();
Path testPath = new Path(TEST_FOLDER_PATH, TEST_CHILD_FILE);
Path testFolderPath = getUniquePath(TEST_FOLDER_PATH);
Path testPath = new Path(testFolderPath, TEST_CHILD_FILE);
try (FSDataOutputStream out = fs.create(testPath)) {
out.write('1');
out.hsync();
Expand Down Expand Up @@ -194,7 +199,8 @@ public void testTryWithResources() throws Throwable {
@Test
public void testFilterFSWriteAfterClose() throws Throwable {
final AzureBlobFileSystem fs = getFileSystem();
Path testPath = new Path(TEST_FOLDER_PATH, TEST_CHILD_FILE);
Path testFolderPath = getUniquePath(TEST_FOLDER_PATH);
Path testPath = new Path(testFolderPath, TEST_CHILD_FILE);
FSDataOutputStream out = fs.create(testPath);
intercept(FileNotFoundException.class,
() -> {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -395,7 +395,7 @@ public void testProperties() throws Exception {
@Test
public void testSignatureMask() throws Exception {
final AzureBlobFileSystem fs = getFileSystem();
String src = "/testABC/test.xt";
String src = String.format("/testABC/test%s.xt", UUID.randomUUID());
fs.create(new Path(src));
AbfsRestOperation abfsHttpRestOperation = fs.getAbfsClient()
.renamePath(src, "/testABC" + "/abc.txt", null);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -76,12 +76,13 @@ public ITestAzureBlobFileSystemDelete() throws Exception {
public void testDeleteRoot() throws Exception {
final AzureBlobFileSystem fs = getFileSystem();

fs.mkdirs(new Path("/testFolder0"));
fs.mkdirs(new Path("/testFolder1"));
fs.mkdirs(new Path("/testFolder2"));
touch(new Path("/testFolder1/testfile"));
touch(new Path("/testFolder1/testfile2"));
touch(new Path("/testFolder1/testfile3"));
Path testPath = getUniquePath("/testFolder");
fs.mkdirs(new Path(testPath + "_0"));
fs.mkdirs(new Path(testPath + "_1"));
fs.mkdirs(new Path(testPath + "_2"));
touch(new Path(testPath + "_1/testfile"));
touch(new Path(testPath + "_1/testfile2"));
touch(new Path(testPath + "_1/testfile3"));

Path root = new Path("/");
FileStatus[] ls = fs.listStatus(root);
Expand All @@ -95,7 +96,7 @@ public void testDeleteRoot() throws Exception {
@Test()
public void testOpenFileAfterDelete() throws Exception {
final AzureBlobFileSystem fs = getFileSystem();
Path testfile = new Path("/testFile");
Path testfile = getUniquePath("/testFile");
touch(testfile);
assertDeleted(fs, testfile, false);

Expand All @@ -106,7 +107,7 @@ public void testOpenFileAfterDelete() throws Exception {
@Test
public void testEnsureFileIsDeleted() throws Exception {
final AzureBlobFileSystem fs = getFileSystem();
Path testfile = new Path("testfile");
Path testfile = getUniquePath("testfile");
touch(testfile);
assertDeleted(fs, testfile, false);
assertPathDoesNotExist(fs, "deleted", testfile);
Expand All @@ -115,10 +116,10 @@ public void testEnsureFileIsDeleted() throws Exception {
@Test
public void testDeleteDirectory() throws Exception {
final AzureBlobFileSystem fs = getFileSystem();
Path dir = new Path("testfile");
Path dir = getUniquePath("testfile");
fs.mkdirs(dir);
fs.mkdirs(new Path("testfile/test1"));
fs.mkdirs(new Path("testfile/test1/test2"));
fs.mkdirs(new Path(dir + "/test1"));
fs.mkdirs(new Path(dir + "/test1/test2"));

assertDeleted(fs, dir, true);
assertPathDoesNotExist(fs, "deleted", dir);
Expand All @@ -130,8 +131,9 @@ public void testDeleteFirstLevelDirectory() throws Exception {
final List<Future<Void>> tasks = new ArrayList<>();

ExecutorService es = Executors.newFixedThreadPool(10);
Path dir = getUniquePath("/test");
for (int i = 0; i < 1000; i++) {
final Path fileName = new Path("/test/" + i);
final Path fileName = new Path(dir + "/" + i);
Callable<Void> callable = new Callable<Void>() {
@Override
public Void call() throws Exception {
Expand All @@ -148,7 +150,6 @@ public Void call() throws Exception {
}

es.shutdownNow();
Path dir = new Path("/test");
// first try a non-recursive delete, expect failure
intercept(FileAlreadyExistsException.class,
() -> fs.delete(dir, false));
Expand Down
Loading