Skip to content
Merged
Original file line number Diff line number Diff line change
Expand Up @@ -26,11 +26,11 @@
import java.util.concurrent.Callable;

import org.junit.After;
import org.junit.Assert;
import org.junit.Before;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

import org.apache.commons.lang3.StringUtils;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
import org.apache.hadoop.fs.FileSystem;
Expand Down Expand Up @@ -84,6 +84,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 @@ -270,7 +271,8 @@ protected void createFilesystemForSASTests() throws Exception {
// so first create temporary instance of the filesystem using SharedKey
// then re-use the filesystem it creates with SAS auth instead of SharedKey.
AzureBlobFileSystem tempFs = (AzureBlobFileSystem) FileSystem.newInstance(rawConfig);
Assert.assertTrue(tempFs.exists(new Path("/")));
ContractTestUtils.assertPathExists(tempFs, "This path should exist",
new Path("/"));
abfsConfig.set(FS_AZURE_ACCOUNT_AUTH_TYPE_PROPERTY_NAME, AuthType.SAS.name());
usingFilesystemForSASTests = true;
}
Expand Down Expand Up @@ -440,7 +442,20 @@ public Path makeQualified(Path path) throws java.io.IOException {
*/
protected Path path(String filepath) throws IOException {
return getFileSystem().makeQualified(
new Path(getTestPath(), filepath));
new Path(getTestPath(), getUniquePath(filepath)));
}

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

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -93,7 +93,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 @@ -102,7 +102,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 @@ -114,9 +114,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 @@ -152,7 +153,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 @@ -237,8 +237,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 = path("testFile");
String testFileName = testFile.toString();
getFileSystem().create(testFile);
setPageSize(10);
RemoteIterator<FileStatus> fsItr = fs.listStatusIterator(testFile);
Expand Down Expand Up @@ -304,7 +304,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 = path(testDirectoryName);
getFileSystem().mkdirs(testDirectory);
return testDirectory;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -45,7 +45,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 @@ -75,13 +75,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 = path(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)) {
((AbfsInputStream) inputStream.getWrappedStream()).registerListener(
new TracingHeaderValidator(abfsConfiguration.getClientCorrelationId(),
fs.getFileSystemId(), FSOperationType.READ, true, 0,
Expand Down Expand Up @@ -112,7 +113,8 @@ public void testReadAheadRequestID() throws java.io.IOException {

final byte[] b = new byte[bufferSize * 10];
new Random().nextBytes(b);
try (FSDataOutputStream stream = fs.create(TEST_PATH)) {
Path testPath = path(TEST_PATH);
try (FSDataOutputStream stream = fs.create(testPath)) {
((AbfsOutputStream) stream.getWrappedStream()).registerListener(
new TracingHeaderValidator(abfsConfiguration.getClientCorrelationId(),
fs.getFileSystemId(), FSOperationType.WRITE, false, 0,
Expand All @@ -126,7 +128,7 @@ public void testReadAheadRequestID() throws java.io.IOException {
fs.registerListener(
new TracingHeaderValidator(abfsConfiguration.getClientCorrelationId(),
fs.getFileSystemId(), FSOperationType.OPEN, false, 0));
try (FSDataInputStream inputStream = fs.open(TEST_PATH)) {
try (FSDataInputStream inputStream = fs.open(testPath)) {
((AbfsInputStream) inputStream.getWrappedStream()).registerListener(
new TracingHeaderValidator(abfsConfiguration.getClientCorrelationId(),
fs.getFileSystemId(), FSOperationType.READ, false, 0,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -31,6 +31,8 @@

import static org.apache.hadoop.fs.CommonConfigurationKeys.IOSTATISTICS_LOGGING_LEVEL;
import static org.apache.hadoop.fs.CommonConfigurationKeys.IOSTATISTICS_LOGGING_LEVEL_INFO;
import static org.apache.hadoop.fs.contract.ContractTestUtils.assertPathExists;
import static org.apache.hadoop.fs.contract.ContractTestUtils.assertPathDoesNotExist;

/**
* Tests AzureBlobFileSystem Statistics.
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 = path("testOneReadWriteOps");
Path largeOperationsFile = path("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 @@ -35,8 +35,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 @@ -45,15 +45,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 = path(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(path(TEST_FILE_PATH))) {
final byte[] b = new byte[1024];
new Random().nextBytes(b);
stream.write(b, 1000, 0);
Expand All @@ -65,7 +65,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 = path(TEST_FILE_PATH);
ContractTestUtils.touch(fs, filePath);
fs.delete(filePath, false);

Expand All @@ -75,18 +75,19 @@ 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 = path(TEST_FOLDER_PATH);
fs.mkdirs(folderPath);
fs.append(folderPath);
}

@Test
public void testTracingForAppend() throws IOException {
AzureBlobFileSystem fs = getFileSystem();
fs.create(TEST_FILE_PATH);
Path testPath = path(TEST_FILE_PATH);
fs.create(testPath);
fs.registerListener(new TracingHeaderValidator(
fs.getAbfsStore().getAbfsConfiguration().getClientCorrelationId(),
fs.getFileSystemId(), FSOperationType.APPEND, false, 0));
fs.append(TEST_FILE_PATH, 10);
fs.append(testPath, 10);
}
}
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 @@ -352,7 +352,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 = path(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 = path("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 @@ -69,7 +69,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 @@ -92,7 +92,8 @@ 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 = path(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");
Expand All @@ -101,7 +102,7 @@ public void testCreateNonRecursive() throws Exception {
fs.registerListener(new TracingHeaderValidator(
fs.getAbfsStore().getAbfsConfiguration().getClientCorrelationId(),
fs.getFileSystemId(), FSOperationType.MKDIR, false, 0));
fs.mkdirs(TEST_FOLDER_PATH);
fs.mkdirs(testFolderPath);
fs.registerListener(null);

fs.createNonRecursive(testFile, true, 1024, (short) 1, 1024, null)
Expand All @@ -113,13 +114,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 = path(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 @@ -131,13 +133,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 = path(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 @@ -149,7 +152,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 = path(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 @@ -169,7 +173,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 = path(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 @@ -202,7 +207,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 = path(TEST_FOLDER_PATH);
Path testPath = new Path(testFolderPath, TEST_CHILD_FILE);
FSDataOutputStream out = fs.create(testPath);
intercept(FileNotFoundException.class,
() -> {
Expand Down
Loading