HADOOP-15446. ABFS: tune imports & javadocs; stabilise tests.

Contributed by Steve Loughran and Da Zhou.
This commit is contained in:
Thomas Marquardt 2018-08-08 18:52:12 +00:00
parent a271fd0eca
commit ce03a93f78
88 changed files with 1984 additions and 1485 deletions

View File

@ -291,7 +291,7 @@ public class RawLocalFileSystem extends FileSystem {
Progressable progress) throws IOException { Progressable progress) throws IOException {
FileStatus status = getFileStatus(f); FileStatus status = getFileStatus(f);
if (status.isDirectory()) { if (status.isDirectory()) {
throw new IOException("Cannot append to a diretory (=" + f + " )"); throw new FileAlreadyExistsException("Cannot append to a directory: " + f);
} }
return new FSDataOutputStream(new BufferedOutputStream( return new FSDataOutputStream(new BufferedOutputStream(
createOutputStreamWithMode(f, true, null), bufferSize), statistics, createOutputStreamWithMode(f, true, null), bufferSize), statistics,

View File

@ -1639,16 +1639,31 @@
SAS keys to communicate with Azure storage. SAS keys to communicate with Azure storage.
</description> </description>
</property> </property>
<property> <property>
<name>fs.abfs.impl</name> <name>fs.abfs.impl</name>
<value>org.apache.hadoop.fs.azurebfs.AzureBlobFileSystem</value> <value>org.apache.hadoop.fs.azurebfs.AzureBlobFileSystem</value>
<description>The implementation class of the Azure Blob Filesystem</description> <description>The implementation class of the Azure Blob Filesystem</description>
</property> </property>
<property> <property>
<name>fs.abfss.impl</name> <name>fs.abfss.impl</name>
<value>org.apache.hadoop.fs.azurebfs.SecureAzureBlobFileSystem</value> <value>org.apache.hadoop.fs.azurebfs.SecureAzureBlobFileSystem</value>
<description>The implementation class of the Secure Azure Blob Filesystem</description> <description>The implementation class of the Secure Azure Blob Filesystem</description>
</property> </property>
<property>
<name>fs.AbstractFileSystem.abfs.impl</name>
<value>org.apache.hadoop.fs.azurebfs.Abfs</value>
<description>AbstractFileSystem implementation class of abfs://</description>
</property>
<property>
<name>fs.AbstractFileSystem.abfss.impl</name>
<value>org.apache.hadoop.fs.azurebfs.Abfss</value>
<description>AbstractFileSystem implementation class of abfss://</description>
</property>
<property> <property>
<name>fs.azure.local.sas.key.mode</name> <name>fs.azure.local.sas.key.mode</name>
<value>false</value> <value>false</value>

View File

@ -544,15 +544,6 @@ atomic. The combined operation, including `mkdirs(parent(F))` MAY be atomic.
The return value is always true&mdash;even if a new directory is not created The return value is always true&mdash;even if a new directory is not created
(this is defined in HDFS). (this is defined in HDFS).
#### Implementation Notes: Local FileSystem
The local FileSystem does not raise an exception if `mkdirs(p)` is invoked
on a path that exists and is a file. Instead the operation returns false.
if isFile(FS, p):
FS' = FS
result = False
### <a name='FileSystem.create'></a> `FSDataOutputStream create(Path, ...)` ### <a name='FileSystem.create'></a> `FSDataOutputStream create(Path, ...)`
@ -641,7 +632,7 @@ Implementations without a compliant call SHOULD throw `UnsupportedOperationExcep
if not exists(FS, p) : raise FileNotFoundException if not exists(FS, p) : raise FileNotFoundException
if not isFile(FS, p) : raise [FileNotFoundException, IOException] if not isFile(FS, p) : raise [FileAlreadyExistsException, FileNotFoundException, IOException]
#### Postconditions #### Postconditions

View File

@ -18,7 +18,12 @@
package org.apache.hadoop.fs.contract; package org.apache.hadoop.fs.contract;
import java.io.FileNotFoundException;
import java.io.IOException;
import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileAlreadyExistsException;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.junit.Test; import org.junit.Test;
import org.slf4j.Logger; import org.slf4j.Logger;
@ -27,6 +32,7 @@ import org.slf4j.LoggerFactory;
import static org.apache.hadoop.fs.contract.ContractTestUtils.createFile; import static org.apache.hadoop.fs.contract.ContractTestUtils.createFile;
import static org.apache.hadoop.fs.contract.ContractTestUtils.dataset; import static org.apache.hadoop.fs.contract.ContractTestUtils.dataset;
import static org.apache.hadoop.fs.contract.ContractTestUtils.touch; import static org.apache.hadoop.fs.contract.ContractTestUtils.touch;
import static org.apache.hadoop.test.LambdaTestUtils.intercept;
/** /**
* Test append -if supported * Test append -if supported
@ -75,15 +81,10 @@ public abstract class AbstractContractAppendTest extends AbstractFSContractTestB
@Test @Test
public void testAppendNonexistentFile() throws Throwable { public void testAppendNonexistentFile() throws Throwable {
try {
FSDataOutputStream out = getFileSystem().append(target);
//got here: trouble
out.close();
fail("expected a failure");
} catch (Exception e) {
//expected //expected
handleExpectedException(e); handleExpectedException(
} intercept(Exception.class,
() -> getFileSystem().append(target).close()));
} }
@Test @Test
@ -116,15 +117,9 @@ public abstract class AbstractContractAppendTest extends AbstractFSContractTestB
@Test @Test
public void testAppendMissingTarget() throws Throwable { public void testAppendMissingTarget() throws Throwable {
try { handleExpectedException(
FSDataOutputStream out = getFileSystem().append(target); intercept(Exception.class,
//got here: trouble () -> getFileSystem().append(target).close()));
out.close();
fail("expected a failure");
} catch (Exception e) {
//expected
handleExpectedException(e);
}
} }
@Test @Test
@ -149,4 +144,30 @@ public abstract class AbstractContractAppendTest extends AbstractFSContractTestB
dataset.length); dataset.length);
ContractTestUtils.compareByteArrays(dataset, bytes, dataset.length); ContractTestUtils.compareByteArrays(dataset, bytes, dataset.length);
} }
@Test
public void testAppendFileAfterDelete() throws Exception {
final FileSystem fs = getFileSystem();
final Path filePath = target;
fs.create(filePath);
fs.delete(filePath, false);
intercept(FileNotFoundException.class,
() -> fs.append(filePath));
}
@Test
public void testAppendDirectory() throws Exception {
final FileSystem fs = getFileSystem();
final Path folderPath = target;
fs.mkdirs(folderPath);
IOException ex = intercept(IOException.class,
() -> fs.append(folderPath));
if (ex instanceof FileAlreadyExistsException) {
handleExpectedException(ex);
} else {
handleRelaxedException("Append to a directory",
"FileAlreadyExistsException", ex);
}
}
} }

View File

@ -19,15 +19,16 @@
package org.apache.hadoop.fs.contract; package org.apache.hadoop.fs.contract;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.junit.Test; import org.junit.Test;
import org.slf4j.Logger; import org.slf4j.Logger;
import org.slf4j.LoggerFactory; import org.slf4j.LoggerFactory;
import static org.apache.hadoop.fs.contract.ContractTestUtils.assertFileHasLength; import static org.apache.hadoop.fs.contract.ContractTestUtils.assertFileHasLength;
import static org.apache.hadoop.fs.contract.ContractTestUtils.cleanup;
import static org.apache.hadoop.fs.contract.ContractTestUtils.createFile; import static org.apache.hadoop.fs.contract.ContractTestUtils.createFile;
import static org.apache.hadoop.fs.contract.ContractTestUtils.dataset; import static org.apache.hadoop.fs.contract.ContractTestUtils.dataset;
import static org.apache.hadoop.fs.contract.ContractTestUtils.touch; import static org.apache.hadoop.fs.contract.ContractTestUtils.touch;
import static org.apache.hadoop.test.LambdaTestUtils.intercept;
/** /**
* Test concat -if supported * Test concat -if supported
@ -60,25 +61,15 @@ public abstract class AbstractContractConcatTest extends AbstractFSContractTestB
@Test @Test
public void testConcatEmptyFiles() throws Throwable { public void testConcatEmptyFiles() throws Throwable {
touch(getFileSystem(), target); touch(getFileSystem(), target);
try { handleExpectedException(intercept(Exception.class,
getFileSystem().concat(target, new Path[0]); () -> getFileSystem().concat(target, new Path[0])));
fail("expected a failure");
} catch (Exception e) {
//expected
handleExpectedException(e);
}
} }
@Test @Test
public void testConcatMissingTarget() throws Throwable { public void testConcatMissingTarget() throws Throwable {
try { handleExpectedException(
getFileSystem().concat(target, intercept(Exception.class,
new Path[] { zeroByteFile}); () -> getFileSystem().concat(target, new Path[]{zeroByteFile})));
fail("expected a failure");
} catch (Exception e) {
//expected
handleExpectedException(e);
}
} }
@Test @Test
@ -98,15 +89,8 @@ public abstract class AbstractContractConcatTest extends AbstractFSContractTestB
public void testConcatOnSelf() throws Throwable { public void testConcatOnSelf() throws Throwable {
byte[] block = dataset(TEST_FILE_LEN, 0, 255); byte[] block = dataset(TEST_FILE_LEN, 0, 255);
createFile(getFileSystem(), target, false, block); createFile(getFileSystem(), target, false, block);
try { handleExpectedException(intercept(Exception.class,
getFileSystem().concat(target, () -> getFileSystem().concat(target, new Path[]{target})));
new Path[]{target});
} catch (Exception e) {
//expected
handleExpectedException(e);
} }
}
} }

View File

@ -32,6 +32,7 @@ import org.apache.hadoop.fs.RemoteIterator;
import org.junit.Test; import org.junit.Test;
import static org.apache.hadoop.fs.contract.ContractTestUtils.*; import static org.apache.hadoop.fs.contract.ContractTestUtils.*;
import static org.apache.hadoop.test.LambdaTestUtils.intercept;
/** /**
* Test getFileStatus and related listing operations. * Test getFileStatus and related listing operations.
@ -275,35 +276,22 @@ public abstract class AbstractContractGetFileStatusTest extends
@Test @Test
public void testLocatedStatusNoDir() throws Throwable { public void testLocatedStatusNoDir() throws Throwable {
describe("test the LocatedStatus call on a path which is not present"); describe("test the LocatedStatus call on a path which is not present");
try { intercept(FileNotFoundException.class,
RemoteIterator<LocatedFileStatus> iterator () -> getFileSystem().listLocatedStatus(path("missing")));
= getFileSystem().listLocatedStatus(path("missing"));
fail("Expected an exception, got an iterator: " + iterator);
} catch (FileNotFoundException expected) {
// expected
}
} }
@Test @Test
public void testListStatusNoDir() throws Throwable { public void testListStatusNoDir() throws Throwable {
describe("test the listStatus(path) call on a path which is not present"); describe("test the listStatus(path) call on a path which is not present");
try { intercept(FileNotFoundException.class,
getFileSystem().listStatus(path("missing")); () -> getFileSystem().listStatus(path("missing")));
fail("Expected an exception");
} catch (FileNotFoundException expected) {
// expected
}
} }
@Test @Test
public void testListStatusFilteredNoDir() throws Throwable { public void testListStatusFilteredNoDir() throws Throwable {
describe("test the listStatus(path, filter) call on a missing path"); describe("test the listStatus(path, filter) call on a missing path");
try { intercept(FileNotFoundException.class,
getFileSystem().listStatus(path("missing"), ALL_PATHS); () -> getFileSystem().listStatus(path("missing"), ALL_PATHS));
fail("Expected an exception");
} catch (FileNotFoundException expected) {
// expected
}
} }
@Test @Test

View File

@ -26,6 +26,7 @@ import org.junit.Test;
import java.io.IOException; import java.io.IOException;
import static org.apache.hadoop.fs.contract.ContractTestUtils.assertMkdirs;
import static org.apache.hadoop.fs.contract.ContractTestUtils.createFile; import static org.apache.hadoop.fs.contract.ContractTestUtils.createFile;
import static org.apache.hadoop.fs.contract.ContractTestUtils.dataset; import static org.apache.hadoop.fs.contract.ContractTestUtils.dataset;
@ -175,4 +176,11 @@ public abstract class AbstractContractMkdirTest extends AbstractFSContractTestBa
} }
} }
@Test
public void testCreateDirWithExistingDir() throws Exception {
Path path = path("testCreateDirWithExistingDir");
final FileSystem fs = getFileSystem();
assertMkdirs(fs, path);
assertMkdirs(fs, path);
}
} }

View File

@ -148,7 +148,6 @@ public abstract class AbstractFSContract extends Configured {
* @param feature feature to query * @param feature feature to query
* @param defval default value * @param defval default value
* @return true if the feature is supported * @return true if the feature is supported
* @throws IOException IO problems
*/ */
public boolean isSupported(String feature, boolean defval) { public boolean isSupported(String feature, boolean defval) {
return getConf().getBoolean(getConfKey(feature), defval); return getConf().getBoolean(getConfKey(feature), defval);
@ -160,7 +159,6 @@ public abstract class AbstractFSContract extends Configured {
* @param feature feature to query * @param feature feature to query
* @param defval default value * @param defval default value
* @return true if the feature is supported * @return true if the feature is supported
* @throws IOException IO problems
*/ */
public int getLimit(String feature, int defval) { public int getLimit(String feature, int defval) {
return getConf().getInt(getConfKey(feature), defval); return getConf().getInt(getConfKey(feature), defval);

View File

@ -187,8 +187,11 @@ public class ContractTestUtils extends Assert {
(short) 1, (short) 1,
buffersize); buffersize);
} }
try {
out.write(src, 0, len); out.write(src, 0, len);
} finally {
out.close(); out.close();
}
assertFileHasLength(fs, path, len); assertFileHasLength(fs, path, len);
} }
@ -1021,6 +1024,18 @@ public class ContractTestUtils extends Assert {
found); found);
} }
/**
* Execute {@link FileSystem#mkdirs(Path)}; expect {@code true} back.
* (Note: does not work for localFS if the directory already exists)
* Does not perform any validation of the created directory.
* @param fs filesystem
* @param dir directory to create
* @throws IOException IO Problem
*/
public static void assertMkdirs(FileSystem fs, Path dir) throws IOException {
assertTrue("mkdirs(" + dir + ") returned false", fs.mkdirs(dir));
}
/** /**
* Test for the host being an OSX machine. * Test for the host being an OSX machine.
* @return true if the JVM thinks that is running on OSX * @return true if the JVM thinks that is running on OSX

View File

@ -22,7 +22,6 @@ import java.io.IOException;
import java.net.URI; import java.net.URI;
import java.net.URISyntaxException; import java.net.URISyntaxException;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.DelegateToFileSystem; import org.apache.hadoop.fs.DelegateToFileSystem;
@ -32,7 +31,6 @@ import org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes;
* Azure Blob File System implementation of AbstractFileSystem. * Azure Blob File System implementation of AbstractFileSystem.
* This impl delegates to the old FileSystem * This impl delegates to the old FileSystem
*/ */
@InterfaceAudience.Public
@InterfaceStability.Evolving @InterfaceStability.Evolving
public class Abfs extends DelegateToFileSystem { public class Abfs extends DelegateToFileSystem {

View File

@ -22,7 +22,6 @@ import java.io.IOException;
import java.net.URI; import java.net.URI;
import java.net.URISyntaxException; import java.net.URISyntaxException;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.DelegateToFileSystem; import org.apache.hadoop.fs.DelegateToFileSystem;
@ -32,7 +31,6 @@ import org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes;
* Azure Blob File System implementation of AbstractFileSystem. * Azure Blob File System implementation of AbstractFileSystem.
* This impl delegates to the old FileSystem * This impl delegates to the old FileSystem
*/ */
@InterfaceAudience.Public
@InterfaceStability.Evolving @InterfaceStability.Evolving
public class Abfss extends DelegateToFileSystem { public class Abfss extends DelegateToFileSystem {

View File

@ -36,14 +36,10 @@ import java.util.concurrent.Future;
import com.google.common.annotations.VisibleForTesting; import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Preconditions; import com.google.common.base.Preconditions;
import org.slf4j.Logger; import org.slf4j.Logger;
import org.slf4j.LoggerFactory; import org.slf4j.LoggerFactory;
import org.apache.hadoop.fs.PathIOException;
import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException;
import org.apache.commons.lang.ArrayUtils; import org.apache.commons.lang.ArrayUtils;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.BlockLocation; import org.apache.hadoop.fs.BlockLocation;
@ -54,13 +50,15 @@ import org.apache.hadoop.fs.FileAlreadyExistsException;
import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.PathIOException;
import org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations; import org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations;
import org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes; import org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes;
import org.apache.hadoop.fs.azurebfs.contracts.services.AzureServiceErrorCode; import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException;
import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException; import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException;
import org.apache.hadoop.fs.azurebfs.contracts.exceptions.FileSystemOperationUnhandledException; import org.apache.hadoop.fs.azurebfs.contracts.exceptions.FileSystemOperationUnhandledException;
import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidUriAuthorityException; import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidUriAuthorityException;
import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidUriException; import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidUriException;
import org.apache.hadoop.fs.azurebfs.contracts.services.AzureServiceErrorCode;
import org.apache.hadoop.fs.permission.FsPermission; import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.util.Progressable; import org.apache.hadoop.util.Progressable;
@ -69,8 +67,7 @@ import org.apache.hadoop.util.Progressable;
* A {@link org.apache.hadoop.fs.FileSystem} for reading and writing files stored on <a * A {@link org.apache.hadoop.fs.FileSystem} for reading and writing files stored on <a
* href="http://store.azure.com/">Windows Azure</a> * href="http://store.azure.com/">Windows Azure</a>
*/ */
@InterfaceAudience.Public @InterfaceStability.Evolving
@InterfaceStability.Stable
public class AzureBlobFileSystem extends FileSystem { public class AzureBlobFileSystem extends FileSystem {
public static final Logger LOG = LoggerFactory.getLogger(AzureBlobFileSystem.class); public static final Logger LOG = LoggerFactory.getLogger(AzureBlobFileSystem.class);
private URI uri; private URI uri;
@ -88,8 +85,7 @@ public class AzureBlobFileSystem extends FileSystem {
super.initialize(uri, configuration); super.initialize(uri, configuration);
setConf(configuration); setConf(configuration);
this.LOG.debug( LOG.debug("Initializing AzureBlobFileSystem for {}", uri);
"Initializing AzureBlobFileSystem for {}", uri);
this.uri = URI.create(uri.getScheme() + "://" + uri.getAuthority()); this.uri = URI.create(uri.getScheme() + "://" + uri.getAuthority());
this.userGroupInformation = UserGroupInformation.getCurrentUser(); this.userGroupInformation = UserGroupInformation.getCurrentUser();
@ -97,16 +93,24 @@ public class AzureBlobFileSystem extends FileSystem {
this.primaryUserGroup = userGroupInformation.getPrimaryGroupName(); this.primaryUserGroup = userGroupInformation.getPrimaryGroupName();
this.abfsStore = new AzureBlobFileSystemStore(uri, this.isSecure(), configuration, userGroupInformation); this.abfsStore = new AzureBlobFileSystemStore(uri, this.isSecure(), configuration, userGroupInformation);
this.LOG.debug( LOG.debug("Initializing NativeAzureFileSystem for {}", uri);
"Initializing NativeAzureFileSystem for {}", uri);
this.setWorkingDirectory(this.getHomeDirectory()); this.setWorkingDirectory(this.getHomeDirectory());
if (abfsStore.getAbfsConfiguration().getCreateRemoteFileSystemDuringInitialization()) { if (abfsStore.getAbfsConfiguration().getCreateRemoteFileSystemDuringInitialization()) {
this.createFileSystem(); this.createFileSystem();
} }
}
this.mkdirs(this.workingDir); @Override
public String toString() {
final StringBuilder sb = new StringBuilder(
"AzureBlobFileSystem{");
sb.append("uri=").append(uri);
sb.append(", user='").append(user).append('\'');
sb.append(", primaryUserGroup='").append(primaryUserGroup).append('\'');
sb.append('}');
return sb.toString();
} }
public boolean isSecure() { public boolean isSecure() {
@ -120,8 +124,7 @@ public class AzureBlobFileSystem extends FileSystem {
@Override @Override
public FSDataInputStream open(final Path path, final int bufferSize) throws IOException { public FSDataInputStream open(final Path path, final int bufferSize) throws IOException {
this.LOG.debug( LOG.debug("AzureBlobFileSystem.open path: {} bufferSize: {}", path, bufferSize);
"AzureBlobFileSystem.open path: {} bufferSize: {}", path.toString(), bufferSize);
try { try {
InputStream inputStream = abfsStore.openFileForRead(makeQualified(path), statistics); InputStream inputStream = abfsStore.openFileForRead(makeQualified(path), statistics);
@ -135,9 +138,8 @@ public class AzureBlobFileSystem extends FileSystem {
@Override @Override
public FSDataOutputStream create(final Path f, final FsPermission permission, final boolean overwrite, final int bufferSize, public FSDataOutputStream create(final Path f, final FsPermission permission, final boolean overwrite, final int bufferSize,
final short replication, final long blockSize, final Progressable progress) throws IOException { final short replication, final long blockSize, final Progressable progress) throws IOException {
this.LOG.debug( LOG.debug("AzureBlobFileSystem.create path: {} permission: {} overwrite: {} bufferSize: {}",
"AzureBlobFileSystem.create path: {} permission: {} overwrite: {} bufferSize: {}", f,
f.toString(),
permission, permission,
overwrite, overwrite,
blockSize); blockSize);
@ -196,7 +198,7 @@ public class AzureBlobFileSystem extends FileSystem {
@Override @Override
public FSDataOutputStream append(final Path f, final int bufferSize, final Progressable progress) throws IOException { public FSDataOutputStream append(final Path f, final int bufferSize, final Progressable progress) throws IOException {
this.LOG.debug( LOG.debug(
"AzureBlobFileSystem.append path: {} bufferSize: {}", "AzureBlobFileSystem.append path: {} bufferSize: {}",
f.toString(), f.toString(),
bufferSize); bufferSize);
@ -211,7 +213,7 @@ public class AzureBlobFileSystem extends FileSystem {
} }
public boolean rename(final Path src, final Path dst) throws IOException { public boolean rename(final Path src, final Path dst) throws IOException {
this.LOG.debug( LOG.debug(
"AzureBlobFileSystem.rename src: {} dst: {}", src.toString(), dst.toString()); "AzureBlobFileSystem.rename src: {} dst: {}", src.toString(), dst.toString());
Path parentFolder = src.getParent(); Path parentFolder = src.getParent();
@ -250,7 +252,7 @@ public class AzureBlobFileSystem extends FileSystem {
@Override @Override
public boolean delete(final Path f, final boolean recursive) throws IOException { public boolean delete(final Path f, final boolean recursive) throws IOException {
this.LOG.debug( LOG.debug(
"AzureBlobFileSystem.delete path: {} recursive: {}", f.toString(), recursive); "AzureBlobFileSystem.delete path: {} recursive: {}", f.toString(), recursive);
if (f.isRoot()) { if (f.isRoot()) {
@ -273,7 +275,7 @@ public class AzureBlobFileSystem extends FileSystem {
@Override @Override
public FileStatus[] listStatus(final Path f) throws IOException { public FileStatus[] listStatus(final Path f) throws IOException {
this.LOG.debug( LOG.debug(
"AzureBlobFileSystem.listStatus path: {}", f.toString()); "AzureBlobFileSystem.listStatus path: {}", f.toString());
try { try {
@ -287,8 +289,8 @@ public class AzureBlobFileSystem extends FileSystem {
@Override @Override
public boolean mkdirs(final Path f, final FsPermission permission) throws IOException { public boolean mkdirs(final Path f, final FsPermission permission) throws IOException {
this.LOG.debug( LOG.debug(
"AzureBlobFileSystem.mkdirs path: {} permissions: {}", f.toString(), permission); "AzureBlobFileSystem.mkdirs path: {} permissions: {}", f, permission);
final Path parentFolder = f.getParent(); final Path parentFolder = f.getParent();
if (parentFolder == null) { if (parentFolder == null) {
@ -312,13 +314,13 @@ public class AzureBlobFileSystem extends FileSystem {
} }
super.close(); super.close();
this.LOG.debug("AzureBlobFileSystem.close"); LOG.debug("AzureBlobFileSystem.close");
this.isClosed = true; this.isClosed = true;
} }
@Override @Override
public FileStatus getFileStatus(final Path f) throws IOException { public FileStatus getFileStatus(final Path f) throws IOException {
this.LOG.debug("AzureBlobFileSystem.getFileStatus path: {}", f.toString()); LOG.debug("AzureBlobFileSystem.getFileStatus path: {}", f);
try { try {
return abfsStore.getFileStatus(makeQualified(f)); return abfsStore.getFileStatus(makeQualified(f));
@ -350,7 +352,8 @@ public class AzureBlobFileSystem extends FileSystem {
@Override @Override
public Path getHomeDirectory() { public Path getHomeDirectory() {
return makeQualified(new Path( return makeQualified(new Path(
FileSystemConfigurations.USER_HOME_DIRECTORY_PREFIX + "/" + this.userGroupInformation.getShortUserName())); FileSystemConfigurations.USER_HOME_DIRECTORY_PREFIX
+ "/" + this.userGroupInformation.getShortUserName()));
} }
/** /**
@ -360,7 +363,7 @@ public class AzureBlobFileSystem extends FileSystem {
*/ */
@Override @Override
public BlockLocation[] getFileBlockLocations(FileStatus file, public BlockLocation[] getFileBlockLocations(FileStatus file,
long start, long len) throws IOException { long start, long len) {
if (file == null) { if (file == null) {
return null; return null;
} }
@ -403,7 +406,7 @@ public class AzureBlobFileSystem extends FileSystem {
} }
private boolean deleteRoot() throws IOException { private boolean deleteRoot() throws IOException {
this.LOG.debug("Deleting root content"); LOG.debug("Deleting root content");
final ExecutorService executorService = Executors.newFixedThreadPool(10); final ExecutorService executorService = Executors.newFixedThreadPool(10);
@ -441,15 +444,14 @@ public class AzureBlobFileSystem extends FileSystem {
private FileStatus tryGetFileStatus(final Path f) { private FileStatus tryGetFileStatus(final Path f) {
try { try {
return getFileStatus(f); return getFileStatus(f);
} } catch (IOException ex) {
catch (IOException ex) { LOG.debug("File not found {}", f);
this.LOG.debug("File not found {}", f.toString());
return null; return null;
} }
} }
private void createFileSystem() throws IOException { private void createFileSystem() throws IOException {
this.LOG.debug( LOG.debug(
"AzureBlobFileSystem.createFileSystem uri: {}", uri); "AzureBlobFileSystem.createFileSystem uri: {}", uri);
try { try {
this.abfsStore.createFilesystem(); this.abfsStore.createFilesystem();
@ -493,7 +495,8 @@ public class AzureBlobFileSystem extends FileSystem {
return false; return false;
} }
if (scheme.equals(FileSystemUriSchemes.ABFS_SCHEME) || scheme.equals(FileSystemUriSchemes.ABFS_SECURE_SCHEME)) { if (scheme.equals(FileSystemUriSchemes.ABFS_SCHEME)
|| scheme.equals(FileSystemUriSchemes.ABFS_SECURE_SCHEME)) {
return true; return true;
} }
@ -501,34 +504,45 @@ public class AzureBlobFileSystem extends FileSystem {
} }
@VisibleForTesting @VisibleForTesting
<T> FileSystemOperation execute( <T> FileSystemOperation<T> execute(
final String scopeDescription, final String scopeDescription,
final Callable<T> callableFileOperation) throws IOException { final Callable<T> callableFileOperation) throws IOException {
return execute(scopeDescription, callableFileOperation, null); return execute(scopeDescription, callableFileOperation, null);
} }
@VisibleForTesting @VisibleForTesting
<T> FileSystemOperation execute( <T> FileSystemOperation<T> execute(
final String scopeDescription, final String scopeDescription,
final Callable<T> callableFileOperation, final Callable<T> callableFileOperation,
T defaultResultValue) throws IOException { T defaultResultValue) throws IOException {
try { try {
final T executionResult = callableFileOperation.call(); final T executionResult = callableFileOperation.call();
return new FileSystemOperation(executionResult, null); return new FileSystemOperation<>(executionResult, null);
} catch (AbfsRestOperationException abfsRestOperationException) { } catch (AbfsRestOperationException abfsRestOperationException) {
return new FileSystemOperation(defaultResultValue, abfsRestOperationException); return new FileSystemOperation<>(defaultResultValue, abfsRestOperationException);
} catch (AzureBlobFileSystemException azureBlobFileSystemException) { } catch (AzureBlobFileSystemException azureBlobFileSystemException) {
throw new IOException(azureBlobFileSystemException); throw new IOException(azureBlobFileSystemException);
} catch (Exception exception) { } catch (Exception exception) {
if (exception instanceof ExecutionException) { if (exception instanceof ExecutionException) {
exception = (Exception) getRootCause(exception); exception = (Exception) getRootCause(exception);
} }
final FileSystemOperationUnhandledException fileSystemOperationUnhandledException = new FileSystemOperationUnhandledException(exception); final FileSystemOperationUnhandledException fileSystemOperationUnhandledException
= new FileSystemOperationUnhandledException(exception);
throw new IOException(fileSystemOperationUnhandledException); throw new IOException(fileSystemOperationUnhandledException);
} }
} }
/**
* Given a path and exception, choose which IOException subclass
* to create.
* Will return if and only iff the error code is in the list of allowed
* error codes.
* @param path path of operation triggering exception; may be null
* @param exception the exception caught
* @param allowedErrorCodesList varargs list of error codes.
* @throws IOException if the exception error code is not on the allowed list.
*/
private void checkException(final Path path, private void checkException(final Path path,
final AzureBlobFileSystemException exception, final AzureBlobFileSystemException exception,
final AzureServiceErrorCode... allowedErrorCodesList) throws IOException { final AzureServiceErrorCode... allowedErrorCodesList) throws IOException {
@ -542,9 +556,11 @@ public class AzureBlobFileSystem extends FileSystem {
//AbfsRestOperationException.getMessage() contains full error info including path/uri. //AbfsRestOperationException.getMessage() contains full error info including path/uri.
if (statusCode == HttpURLConnection.HTTP_NOT_FOUND) { if (statusCode == HttpURLConnection.HTTP_NOT_FOUND) {
throw new FileNotFoundException(ere.getMessage()); throw (IOException)new FileNotFoundException(ere.getMessage())
.initCause(exception);
} else if (statusCode == HttpURLConnection.HTTP_CONFLICT) { } else if (statusCode == HttpURLConnection.HTTP_CONFLICT) {
throw new FileAlreadyExistsException(ere.getMessage()); throw (IOException)new FileAlreadyExistsException(ere.getMessage())
.initCause(exception);
} else { } else {
throw ere; throw ere;
} }

View File

@ -103,7 +103,7 @@ public class AzureBlobFileSystemStore {
private final Set<String> azureAtomicRenameDirSet; private final Set<String> azureAtomicRenameDirSet;
public AzureBlobFileSystemStore(URI uri, boolean isSeure, Configuration configuration, UserGroupInformation userGroupInformation) public AzureBlobFileSystemStore(URI uri, boolean isSecure, Configuration configuration, UserGroupInformation userGroupInformation)
throws AzureBlobFileSystemException { throws AzureBlobFileSystemException {
this.uri = uri; this.uri = uri;
try { try {
@ -113,9 +113,10 @@ public class AzureBlobFileSystemStore {
} }
this.userGroupInformation = userGroupInformation; this.userGroupInformation = userGroupInformation;
this.azureAtomicRenameDirSet = new HashSet<>(Arrays.asList(abfsConfiguration.getAzureAtomicRenameDirs().split(AbfsHttpConstants.COMMA))); this.azureAtomicRenameDirSet = new HashSet<>(Arrays.asList(
abfsConfiguration.getAzureAtomicRenameDirs().split(AbfsHttpConstants.COMMA)));
initializeClient(uri, isSeure); initializeClient(uri, isSecure);
} }
@VisibleForTesting @VisibleForTesting
@ -134,8 +135,7 @@ public class AzureBlobFileSystemStore {
} }
public Hashtable<String, String> getFilesystemProperties() throws AzureBlobFileSystemException { public Hashtable<String, String> getFilesystemProperties() throws AzureBlobFileSystemException {
this.LOG.debug( LOG.debug("getFilesystemProperties for filesystem: {}",
"getFilesystemProperties for filesystem: {}",
client.getFileSystem()); client.getFileSystem());
final Hashtable<String, String> parsedXmsProperties; final Hashtable<String, String> parsedXmsProperties;
@ -148,13 +148,13 @@ public class AzureBlobFileSystemStore {
return parsedXmsProperties; return parsedXmsProperties;
} }
public void setFilesystemProperties(final Hashtable<String, String> properties) throws AzureBlobFileSystemException { public void setFilesystemProperties(final Hashtable<String, String> properties)
if (properties == null || properties.size() == 0) { throws AzureBlobFileSystemException {
if (properties == null || properties.isEmpty()) {
return; return;
} }
this.LOG.debug( LOG.debug("setFilesystemProperties for filesystem: {} with properties: {}",
"setFilesystemProperties for filesystem: {} with properties: {}",
client.getFileSystem(), client.getFileSystem(),
properties); properties);
@ -169,10 +169,9 @@ public class AzureBlobFileSystemStore {
} }
public Hashtable<String, String> getPathProperties(final Path path) throws AzureBlobFileSystemException { public Hashtable<String, String> getPathProperties(final Path path) throws AzureBlobFileSystemException {
this.LOG.debug( LOG.debug("getPathProperties for filesystem: {} path: {}",
"getPathProperties for filesystem: {} path: {}",
client.getFileSystem(), client.getFileSystem(),
path.toString()); path);
final Hashtable<String, String> parsedXmsProperties; final Hashtable<String, String> parsedXmsProperties;
final AbfsRestOperation op = client.getPathProperties(AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path)); final AbfsRestOperation op = client.getPathProperties(AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path));
@ -185,10 +184,9 @@ public class AzureBlobFileSystemStore {
} }
public void setPathProperties(final Path path, final Hashtable<String, String> properties) throws AzureBlobFileSystemException { public void setPathProperties(final Path path, final Hashtable<String, String> properties) throws AzureBlobFileSystemException {
this.LOG.debug( LOG.debug("setFilesystemProperties for filesystem: {} path: {} with properties: {}",
"setFilesystemProperties for filesystem: {} path: {} with properties: {}",
client.getFileSystem(), client.getFileSystem(),
path.toString(), path,
properties); properties);
final String commaSeparatedProperties; final String commaSeparatedProperties;
@ -201,26 +199,23 @@ public class AzureBlobFileSystemStore {
} }
public void createFilesystem() throws AzureBlobFileSystemException { public void createFilesystem() throws AzureBlobFileSystemException {
this.LOG.debug( LOG.debug("createFilesystem for filesystem: {}",
"createFilesystem for filesystem: {}",
client.getFileSystem()); client.getFileSystem());
client.createFilesystem(); client.createFilesystem();
} }
public void deleteFilesystem() throws AzureBlobFileSystemException { public void deleteFilesystem() throws AzureBlobFileSystemException {
this.LOG.debug( LOG.debug("deleteFilesystem for filesystem: {}",
"deleteFilesystem for filesystem: {}",
client.getFileSystem()); client.getFileSystem());
client.deleteFilesystem(); client.deleteFilesystem();
} }
public OutputStream createFile(final Path path, final boolean overwrite) throws AzureBlobFileSystemException { public OutputStream createFile(final Path path, final boolean overwrite) throws AzureBlobFileSystemException {
this.LOG.debug( LOG.debug("createFile filesystem: {} path: {} overwrite: {}",
"createFile filesystem: {} path: {} overwrite: {}",
client.getFileSystem(), client.getFileSystem(),
path.toString(), path,
overwrite); overwrite);
client.createPath(AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path), true, overwrite); client.createPath(AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path), true, overwrite);
@ -232,23 +227,19 @@ public class AzureBlobFileSystemStore {
return outputStream; return outputStream;
} }
public Void createDirectory(final Path path) throws AzureBlobFileSystemException { public void createDirectory(final Path path) throws AzureBlobFileSystemException {
this.LOG.debug( LOG.debug("createDirectory filesystem: {} path: {}",
"createDirectory filesystem: {} path: {} overwrite: {}",
client.getFileSystem(), client.getFileSystem(),
path.toString()); path);
client.createPath("/" + getRelativePath(path), false, true); client.createPath("/" + getRelativePath(path), false, true);
return null;
} }
public InputStream openFileForRead(final Path path, final FileSystem.Statistics statistics) throws AzureBlobFileSystemException { public InputStream openFileForRead(final Path path, final FileSystem.Statistics statistics) throws AzureBlobFileSystemException {
this.LOG.debug( LOG.debug("openFileForRead filesystem: {} path: {}",
"openFileForRead filesystem: {} path: {}",
client.getFileSystem(), client.getFileSystem(),
path.toString()); path);
final AbfsRestOperation op = client.getPathProperties(AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path)); final AbfsRestOperation op = client.getPathProperties(AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path));
@ -266,16 +257,16 @@ public class AzureBlobFileSystemStore {
// Add statistics for InputStream // Add statistics for InputStream
return new FSDataInputStream( return new FSDataInputStream(
new AbfsInputStream(client, statistics, AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path), contentLength, new AbfsInputStream(client, statistics,
AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path), contentLength,
abfsConfiguration.getReadBufferSize(), abfsConfiguration.getReadAheadQueueDepth(), eTag)); abfsConfiguration.getReadBufferSize(), abfsConfiguration.getReadAheadQueueDepth(), eTag));
} }
public OutputStream openFileForWrite(final Path path, final boolean overwrite) throws public OutputStream openFileForWrite(final Path path, final boolean overwrite) throws
AzureBlobFileSystemException { AzureBlobFileSystemException {
this.LOG.debug( LOG.debug("openFileForWrite filesystem: {} path: {} overwrite: {}",
"openFileForWrite filesystem: {} path: {} overwrite: {}",
client.getFileSystem(), client.getFileSystem(),
path.toString(), path,
overwrite); overwrite);
final AbfsRestOperation op = client.getPathProperties(AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path)); final AbfsRestOperation op = client.getPathProperties(AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path));
@ -304,23 +295,21 @@ public class AzureBlobFileSystemStore {
AzureBlobFileSystemException { AzureBlobFileSystemException {
if (isAtomicRenameKey(source.getName())) { if (isAtomicRenameKey(source.getName())) {
this.LOG.warn("The atomic rename feature is not supported by the ABFS scheme; however rename," LOG.warn("The atomic rename feature is not supported by the ABFS scheme; however rename,"
+" create and delete operations are atomic if Namespace is enabled for your Azure Storage account."); +" create and delete operations are atomic if Namespace is enabled for your Azure Storage account.");
} }
this.LOG.debug( LOG.debug("renameAsync filesystem: {} source: {} destination: {}",
"renameAsync filesystem: {} source: {} destination: {}",
client.getFileSystem(), client.getFileSystem(),
source.toString(), source,
destination.toString()); destination);
String continuation = null; String continuation = null;
long deadline = now() + RENAME_TIMEOUT_MILISECONDS; long deadline = now() + RENAME_TIMEOUT_MILISECONDS;
do { do {
if (now() > deadline) { if (now() > deadline) {
LOG.debug( LOG.debug("Rename {} to {} timed out.",
"Rename {} to {} timed out.",
source, source,
destination); destination);
@ -334,13 +323,12 @@ public class AzureBlobFileSystemStore {
} while (continuation != null && !continuation.isEmpty()); } while (continuation != null && !continuation.isEmpty());
} }
public void delete(final Path path, final boolean recursive) throws public void delete(final Path path, final boolean recursive)
AzureBlobFileSystemException { throws AzureBlobFileSystemException {
this.LOG.debug( LOG.debug("delete filesystem: {} path: {} recursive: {}",
"delete filesystem: {} path: {} recursive: {}",
client.getFileSystem(), client.getFileSystem(),
path.toString(), path,
String.valueOf(recursive)); String.valueOf(recursive));
String continuation = null; String continuation = null;
@ -348,13 +336,13 @@ public class AzureBlobFileSystemStore {
do { do {
if (now() > deadline) { if (now() > deadline) {
this.LOG.debug( LOG.debug("Delete directory {} timed out.", path);
"Delete directory {} timed out.", path);
throw new TimeoutException("Delete directory timed out."); throw new TimeoutException("Delete directory timed out.");
} }
AbfsRestOperation op = client.deletePath(AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path), recursive, continuation); AbfsRestOperation op = client.deletePath(
AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path), recursive, continuation);
continuation = op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_CONTINUATION); continuation = op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_CONTINUATION);
} while (continuation != null && !continuation.isEmpty()); } while (continuation != null && !continuation.isEmpty());
@ -362,10 +350,9 @@ public class AzureBlobFileSystemStore {
public FileStatus getFileStatus(final Path path) throws IOException { public FileStatus getFileStatus(final Path path) throws IOException {
this.LOG.debug( LOG.debug("getFileStatus filesystem: {} path: {}",
"getFileStatus filesystem: {} path: {}",
client.getFileSystem(), client.getFileSystem(),
path.toString()); path);
if (path.isRoot()) { if (path.isRoot()) {
AbfsRestOperation op = client.getFilesystemProperties(); AbfsRestOperation op = client.getFilesystemProperties();
@ -405,10 +392,9 @@ public class AzureBlobFileSystemStore {
} }
public FileStatus[] listStatus(final Path path) throws IOException { public FileStatus[] listStatus(final Path path) throws IOException {
this.LOG.debug( LOG.debug("listStatus filesystem: {} path: {}",
"listStatus filesystem: {} path: {}",
client.getFileSystem(), client.getFileSystem(),
path.toString()); path);
String relativePath = path.isRoot() ? AbfsHttpConstants.EMPTY_STRING : getRelativePath(path); String relativePath = path.isRoot() ? AbfsHttpConstants.EMPTY_STRING : getRelativePath(path);
String continuation = null; String continuation = null;
@ -480,10 +466,12 @@ public class AzureBlobFileSystemStore {
final String[] authorityParts = authority.split(AbfsHttpConstants.AZURE_DISTRIBUTED_FILE_SYSTEM_AUTHORITY_DELIMITER, 2); final String[] authorityParts = authority.split(AbfsHttpConstants.AZURE_DISTRIBUTED_FILE_SYSTEM_AUTHORITY_DELIMITER, 2);
if (authorityParts.length < 2 || "".equals(authorityParts[0])) { if (authorityParts.length < 2 || authorityParts[0] != null
&& authorityParts[0].isEmpty()) {
final String errMsg = String final String errMsg = String
.format("URI '%s' has a malformed authority, expected container name. " .format("'%s' has a malformed authority, expected container name. "
+ "Authority takes the form "+ FileSystemUriSchemes.ABFS_SCHEME + "://[<container name>@]<account name>", + "Authority takes the form "
+ FileSystemUriSchemes.ABFS_SCHEME + "://[<container name>@]<account name>",
uri.toString()); uri.toString());
throw new InvalidUriException(errMsg); throw new InvalidUriException(errMsg);
} }
@ -499,11 +487,16 @@ public class AzureBlobFileSystemStore {
try { try {
baseUrl = new URL(url); baseUrl = new URL(url);
} catch (MalformedURLException e) { } catch (MalformedURLException e) {
throw new InvalidUriException(String.format("URI '%s' is malformed", uri.toString())); throw new InvalidUriException(uri.toString());
} }
int dotIndex = accountName.indexOf(AbfsHttpConstants.DOT);
if (dotIndex <= 0) {
throw new InvalidUriException(
uri.toString() + " - account name is not fully qualified.");
}
SharedKeyCredentials creds = SharedKeyCredentials creds =
new SharedKeyCredentials(accountName.substring(0, accountName.indexOf(AbfsHttpConstants.DOT)), new SharedKeyCredentials(accountName.substring(0, dotIndex),
this.abfsConfiguration.getStorageAccountKey(accountName)); this.abfsConfiguration.getStorageAccountKey(accountName));
this.client = new AbfsClient(baseUrl, creds, abfsConfiguration, new ExponentialRetryPolicy()); this.client = new AbfsClient(baseUrl, creds, abfsConfiguration, new ExponentialRetryPolicy());
@ -513,7 +506,7 @@ public class AzureBlobFileSystemStore {
Preconditions.checkNotNull(path, "path"); Preconditions.checkNotNull(path, "path");
final String relativePath = path.toUri().getPath(); final String relativePath = path.toUri().getPath();
if (relativePath.length() == 0) { if (relativePath.isEmpty()) {
return relativePath; return relativePath;
} }
@ -537,7 +530,8 @@ public class AzureBlobFileSystemStore {
} }
private boolean parseIsDirectory(final String resourceType) { private boolean parseIsDirectory(final String resourceType) {
return resourceType == null ? false : resourceType.equalsIgnoreCase(AbfsHttpConstants.DIRECTORY); return resourceType != null
&& resourceType.equalsIgnoreCase(AbfsHttpConstants.DIRECTORY);
} }
private DateTime parseLastModifiedTime(final String lastModifiedTime) { private DateTime parseLastModifiedTime(final String lastModifiedTime) {
@ -628,7 +622,7 @@ public class AzureBlobFileSystemStore {
} }
} }
} catch (URISyntaxException e) { } catch (URISyntaxException e) {
this.LOG.info("URI syntax error creating URI for {}", dir); LOG.info("URI syntax error creating URI for {}", dir);
} }
} }
@ -658,22 +652,23 @@ public class AzureBlobFileSystemStore {
*/ */
@Override @Override
public boolean equals(Object obj) { public boolean equals(Object obj) {
if (obj == this) { if (!(obj instanceof FileStatus)) {
return false;
}
FileStatus other = (FileStatus) obj;
if (!other.equals(this)) {// compare the path
return false;
}
if (other instanceof VersionedFileStatus) {
return this.version.equals(((VersionedFileStatus)other).version);
}
return true; return true;
} }
if (obj == null) {
return false;
}
if (this.getClass() == obj.getClass()) {
VersionedFileStatus other = (VersionedFileStatus) obj;
return this.getPath().equals(other.getPath()) && this.version.equals(other.version);
}
return false;
}
/** /**
* Returns a hash code value for the object, which is defined as * Returns a hash code value for the object, which is defined as
* the hash code of the path name. * the hash code of the path name.
@ -695,6 +690,16 @@ public class AzureBlobFileSystemStore {
public String getVersion() { public String getVersion() {
return this.version; return this.version;
} }
@Override
public String toString() {
final StringBuilder sb = new StringBuilder(
"VersionedFileStatus{");
sb.append(super.toString());
sb.append("; version='").append(version).append('\'');
sb.append('}');
return sb.toString();
}
} }

View File

@ -18,7 +18,6 @@
package org.apache.hadoop.fs.azurebfs; package org.apache.hadoop.fs.azurebfs;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes; import org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes;
@ -26,7 +25,6 @@ import org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes;
* A secure {@link org.apache.hadoop.fs.FileSystem} for reading and writing files stored on <a * A secure {@link org.apache.hadoop.fs.FileSystem} for reading and writing files stored on <a
* href="http://store.azure.com/">Windows Azure</a> * href="http://store.azure.com/">Windows Azure</a>
*/ */
@InterfaceAudience.Public
@InterfaceStability.Evolving @InterfaceStability.Evolving
public class SecureAzureBlobFileSystem extends AzureBlobFileSystem { public class SecureAzureBlobFileSystem extends AzureBlobFileSystem {
@Override @Override

View File

@ -22,7 +22,7 @@ import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.classification.InterfaceStability;
/** /**
* Responsible to keep all constant keys used in abfs rest client here * Responsible to keep all constant keys used in abfs rest client here.
*/ */
@InterfaceAudience.Public @InterfaceAudience.Public
@InterfaceStability.Evolving @InterfaceStability.Evolving

View File

@ -21,7 +21,7 @@ import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.classification.InterfaceStability;
/** /**
* Responsible to keep all abfs http headers here * Responsible to keep all abfs http headers here.
*/ */
@InterfaceAudience.Public @InterfaceAudience.Public
@InterfaceStability.Evolving @InterfaceStability.Evolving

View File

@ -21,7 +21,7 @@ import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.classification.InterfaceStability;
/** /**
* Responsible to keep all Http Query params here * Responsible to keep all Http Query params here.
*/ */
@InterfaceAudience.Public @InterfaceAudience.Public
@InterfaceStability.Evolving @InterfaceStability.Evolving

View File

@ -25,12 +25,12 @@ import java.lang.annotation.Target;
import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.classification.InterfaceStability;
/** /**
* Definitions of Annotations for all types of the validators * Definitions of Annotations for all types of the validators.
*/ */
@InterfaceStability.Evolving @InterfaceStability.Evolving
public class ConfigurationValidationAnnotations { public class ConfigurationValidationAnnotations {
/** /**
* Describes the requirements when validating the annotated int field * Describes the requirements when validating the annotated int field.
*/ */
@Target({ ElementType.FIELD }) @Target({ ElementType.FIELD })
@Retention(RetentionPolicy.RUNTIME) @Retention(RetentionPolicy.RUNTIME)
@ -47,7 +47,7 @@ public class ConfigurationValidationAnnotations {
} }
/** /**
* Describes the requirements when validating the annotated long field * Describes the requirements when validating the annotated long field.
*/ */
@Target({ ElementType.FIELD }) @Target({ ElementType.FIELD })
@Retention(RetentionPolicy.RUNTIME) @Retention(RetentionPolicy.RUNTIME)
@ -64,7 +64,7 @@ public class ConfigurationValidationAnnotations {
} }
/** /**
* Describes the requirements when validating the annotated String field * Describes the requirements when validating the annotated String field.
*/ */
@Target({ ElementType.FIELD }) @Target({ ElementType.FIELD })
@Retention(RetentionPolicy.RUNTIME) @Retention(RetentionPolicy.RUNTIME)
@ -77,7 +77,7 @@ public class ConfigurationValidationAnnotations {
} }
/** /**
* Describes the requirements when validating the annotated String field * Describes the requirements when validating the annotated String field.
*/ */
@Target({ ElementType.FIELD }) @Target({ ElementType.FIELD })
@Retention(RetentionPolicy.RUNTIME) @Retention(RetentionPolicy.RUNTIME)
@ -90,7 +90,7 @@ public class ConfigurationValidationAnnotations {
} }
/** /**
* Describes the requirements when validating the annotated boolean field * Describes the requirements when validating the annotated boolean field.
*/ */
@Target({ ElementType.FIELD }) @Target({ ElementType.FIELD })
@Retention(RetentionPolicy.RUNTIME) @Retention(RetentionPolicy.RUNTIME)

View File

@ -18,19 +18,17 @@
package org.apache.hadoop.fs.azurebfs.contracts.diagnostics; package org.apache.hadoop.fs.azurebfs.contracts.diagnostics;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidConfigurationValueException; import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidConfigurationValueException;
/** /**
* ConfigurationValidator to validate the value of a configuration key * ConfigurationValidator to validate the value of a configuration key
* @param <T> the type of the validator and the validated value * @param <T> the type of the validator and the validated value.
*/ */
@InterfaceAudience.Public
@InterfaceStability.Evolving @InterfaceStability.Evolving
public interface ConfigurationValidator<T> { public interface ConfigurationValidator<T> {
/** /**
* Validates the configValue * Validates the configValue.
* @return validated value of type T * @return validated value of type T
*/ */
T validate(String configValue) throws InvalidConfigurationValueException; T validate(String configValue) throws InvalidConfigurationValueException;

View File

@ -40,7 +40,7 @@ public abstract class AzureBlobFileSystemException extends IOException {
@Override @Override
public String toString() { public String toString() {
if (this.getMessage() == null && this.getCause() == null) { if (this.getMessage() == null && this.getCause() == null) {
return ""; return "AzureBlobFileSystemException";
} }
if (this.getCause() == null) { if (this.getCause() == null) {

View File

@ -28,6 +28,6 @@ import org.apache.hadoop.classification.InterfaceStability;
@InterfaceStability.Evolving @InterfaceStability.Evolving
public final class InvalidUriException extends AzureBlobFileSystemException { public final class InvalidUriException extends AzureBlobFileSystemException {
public InvalidUriException(String url) { public InvalidUriException(String url) {
super(String.format("%s is invalid.", url)); super(String.format("Invalid URI %s", url));
} }
} }

View File

@ -26,7 +26,7 @@ import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidConfigurationVa
import org.apache.commons.codec.binary.Base64; import org.apache.commons.codec.binary.Base64;
/** /**
* String Base64 configuration value Validator * String Base64 configuration value Validator.
*/ */
@InterfaceAudience.Public @InterfaceAudience.Public
@InterfaceStability.Evolving @InterfaceStability.Evolving

View File

@ -23,7 +23,7 @@ import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidConfigurationValueException; import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidConfigurationValueException;
/** /**
* Boolean configuration value validator * Boolean configuration value validator.
*/ */
@InterfaceAudience.Public @InterfaceAudience.Public
@InterfaceStability.Evolving @InterfaceStability.Evolving

View File

@ -38,7 +38,7 @@ abstract class ConfigurationBasicValidator<T> implements ConfigurationValidator
/** /**
* This method handles the base case where the configValue is null, based on the throwIfInvalid it either throws or returns the defaultVal, * This method handles the base case where the configValue is null, based on the throwIfInvalid it either throws or returns the defaultVal,
* otherwise it returns null indicating that the configValue needs to be validated further * otherwise it returns null indicating that the configValue needs to be validated further.
* @param configValue the configuration value set by the user * @param configValue the configuration value set by the user
* @return the defaultVal in case the configValue is null and not required to be set, null in case the configValue not null * @return the defaultVal in case the configValue is null and not required to be set, null in case the configValue not null
* @throws InvalidConfigurationValueException in case the configValue is null and required to be set * @throws InvalidConfigurationValueException in case the configValue is null and required to be set

View File

@ -24,7 +24,7 @@ import org.apache.hadoop.fs.azurebfs.contracts.diagnostics.ConfigurationValidato
import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidConfigurationValueException; import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidConfigurationValueException;
/** /**
* Integer configuration value Validator * Integer configuration value Validator.
*/ */
@InterfaceAudience.Public @InterfaceAudience.Public
@InterfaceStability.Evolving @InterfaceStability.Evolving

View File

@ -17,15 +17,13 @@
*/ */
package org.apache.hadoop.fs.azurebfs.diagnostics; package org.apache.hadoop.fs.azurebfs.diagnostics;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.fs.azurebfs.contracts.diagnostics.ConfigurationValidator; import org.apache.hadoop.fs.azurebfs.contracts.diagnostics.ConfigurationValidator;
import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidConfigurationValueException; import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidConfigurationValueException;
/** /**
* Long configuration value Validator * Long configuration value Validator.
*/ */
@InterfaceAudience.Public
@InterfaceStability.Evolving @InterfaceStability.Evolving
public class LongConfigurationBasicValidator extends ConfigurationBasicValidator<Long> implements ConfigurationValidator { public class LongConfigurationBasicValidator extends ConfigurationBasicValidator<Long> implements ConfigurationValidator {
private final long min; private final long min;

View File

@ -18,15 +18,13 @@
package org.apache.hadoop.fs.azurebfs.diagnostics; package org.apache.hadoop.fs.azurebfs.diagnostics;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.fs.azurebfs.contracts.diagnostics.ConfigurationValidator; import org.apache.hadoop.fs.azurebfs.contracts.diagnostics.ConfigurationValidator;
import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidConfigurationValueException; import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidConfigurationValueException;
/** /**
* String configuration value Validator * String configuration value Validator.
*/ */
@InterfaceAudience.Public
@InterfaceStability.Evolving @InterfaceStability.Evolving
public class StringConfigurationBasicValidator extends ConfigurationBasicValidator<String> implements ConfigurationValidator{ public class StringConfigurationBasicValidator extends ConfigurationBasicValidator<String> implements ConfigurationValidator{

View File

@ -26,14 +26,17 @@ import java.util.ArrayList;
import java.util.List; import java.util.List;
import java.util.Locale; import java.util.Locale;
import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException;
import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidUriException;
import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants;
import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations;
import org.apache.hadoop.fs.azurebfs.constants.HttpQueryParams;
import org.slf4j.Logger; import org.slf4j.Logger;
import org.slf4j.LoggerFactory; import org.slf4j.LoggerFactory;
import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException;
import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidUriException;
import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.*;
import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.*;
import static org.apache.hadoop.fs.azurebfs.constants.HttpQueryParams.*;
/** /**
* AbfsClient * AbfsClient
*/ */
@ -53,7 +56,7 @@ public class AbfsClient {
this.baseUrl = baseUrl; this.baseUrl = baseUrl;
this.sharedKeyCredentials = sharedKeyCredentials; this.sharedKeyCredentials = sharedKeyCredentials;
String baseUrlString = baseUrl.toString(); String baseUrlString = baseUrl.toString();
this.filesystem = baseUrlString.substring(baseUrlString.lastIndexOf(AbfsHttpConstants.FORWARD_SLASH) + 1); this.filesystem = baseUrlString.substring(baseUrlString.lastIndexOf(FORWARD_SLASH) + 1);
this.abfsConfiguration = abfsConfiguration; this.abfsConfiguration = abfsConfiguration;
this.retryPolicy = exponentialRetryPolicy; this.retryPolicy = exponentialRetryPolicy;
this.userAgent = initializeUserAgent(); this.userAgent = initializeUserAgent();
@ -73,19 +76,19 @@ public class AbfsClient {
List<AbfsHttpHeader> createDefaultHeaders() { List<AbfsHttpHeader> createDefaultHeaders() {
final List<AbfsHttpHeader> requestHeaders = new ArrayList<AbfsHttpHeader>(); final List<AbfsHttpHeader> requestHeaders = new ArrayList<AbfsHttpHeader>();
requestHeaders.add(new AbfsHttpHeader(HttpHeaderConfigurations.X_MS_VERSION, xMsVersion)); requestHeaders.add(new AbfsHttpHeader(X_MS_VERSION, xMsVersion));
requestHeaders.add(new AbfsHttpHeader(HttpHeaderConfigurations.ACCEPT, AbfsHttpConstants.APPLICATION_JSON requestHeaders.add(new AbfsHttpHeader(ACCEPT, APPLICATION_JSON
+ AbfsHttpConstants.COMMA + AbfsHttpConstants.SINGLE_WHITE_SPACE + AbfsHttpConstants.APPLICATION_OCTET_STREAM)); + COMMA + SINGLE_WHITE_SPACE + APPLICATION_OCTET_STREAM));
requestHeaders.add(new AbfsHttpHeader(HttpHeaderConfigurations.ACCEPT_CHARSET, requestHeaders.add(new AbfsHttpHeader(ACCEPT_CHARSET,
AbfsHttpConstants.UTF_8)); UTF_8));
requestHeaders.add(new AbfsHttpHeader(HttpHeaderConfigurations.CONTENT_TYPE, AbfsHttpConstants.EMPTY_STRING)); requestHeaders.add(new AbfsHttpHeader(CONTENT_TYPE, EMPTY_STRING));
requestHeaders.add(new AbfsHttpHeader(HttpHeaderConfigurations.USER_AGENT, userAgent)); requestHeaders.add(new AbfsHttpHeader(USER_AGENT, userAgent));
return requestHeaders; return requestHeaders;
} }
AbfsUriQueryBuilder createDefaultUriQueryBuilder() { AbfsUriQueryBuilder createDefaultUriQueryBuilder() {
final AbfsUriQueryBuilder abfsUriQueryBuilder = new AbfsUriQueryBuilder(); final AbfsUriQueryBuilder abfsUriQueryBuilder = new AbfsUriQueryBuilder();
abfsUriQueryBuilder.addQuery(HttpQueryParams.QUERY_PARAM_TIMEOUT, AbfsHttpConstants.DEFAULT_TIMEOUT); abfsUriQueryBuilder.addQuery(QUERY_PARAM_TIMEOUT, DEFAULT_TIMEOUT);
return abfsUriQueryBuilder; return abfsUriQueryBuilder;
} }
@ -93,12 +96,12 @@ public class AbfsClient {
final List<AbfsHttpHeader> requestHeaders = createDefaultHeaders(); final List<AbfsHttpHeader> requestHeaders = createDefaultHeaders();
final AbfsUriQueryBuilder abfsUriQueryBuilder = new AbfsUriQueryBuilder(); final AbfsUriQueryBuilder abfsUriQueryBuilder = new AbfsUriQueryBuilder();
abfsUriQueryBuilder.addQuery(HttpQueryParams.QUERY_PARAM_RESOURCE, AbfsHttpConstants.FILESYSTEM); abfsUriQueryBuilder.addQuery(QUERY_PARAM_RESOURCE, FILESYSTEM);
final URL url = createRequestUrl(abfsUriQueryBuilder.toString()); final URL url = createRequestUrl(abfsUriQueryBuilder.toString());
final AbfsRestOperation op = new AbfsRestOperation( final AbfsRestOperation op = new AbfsRestOperation(
this, this,
AbfsHttpConstants.HTTP_METHOD_PUT, HTTP_METHOD_PUT,
url, url,
requestHeaders); requestHeaders);
op.execute(); op.execute();
@ -109,19 +112,19 @@ public class AbfsClient {
final List<AbfsHttpHeader> requestHeaders = createDefaultHeaders(); final List<AbfsHttpHeader> requestHeaders = createDefaultHeaders();
// JDK7 does not support PATCH, so to workaround the issue we will use // JDK7 does not support PATCH, so to workaround the issue we will use
// PUT and specify the real method in the X-Http-Method-Override header. // PUT and specify the real method in the X-Http-Method-Override header.
requestHeaders.add(new AbfsHttpHeader(HttpHeaderConfigurations.X_HTTP_METHOD_OVERRIDE, requestHeaders.add(new AbfsHttpHeader(X_HTTP_METHOD_OVERRIDE,
AbfsHttpConstants.HTTP_METHOD_PATCH)); HTTP_METHOD_PATCH));
requestHeaders.add(new AbfsHttpHeader(HttpHeaderConfigurations.X_MS_PROPERTIES, requestHeaders.add(new AbfsHttpHeader(X_MS_PROPERTIES,
properties)); properties));
final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder(); final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder();
abfsUriQueryBuilder.addQuery(HttpQueryParams.QUERY_PARAM_RESOURCE, AbfsHttpConstants.FILESYSTEM); abfsUriQueryBuilder.addQuery(QUERY_PARAM_RESOURCE, FILESYSTEM);
final URL url = createRequestUrl(abfsUriQueryBuilder.toString()); final URL url = createRequestUrl(abfsUriQueryBuilder.toString());
final AbfsRestOperation op = new AbfsRestOperation( final AbfsRestOperation op = new AbfsRestOperation(
this, this,
AbfsHttpConstants.HTTP_METHOD_PUT, HTTP_METHOD_PUT,
url, url,
requestHeaders); requestHeaders);
op.execute(); op.execute();
@ -133,16 +136,16 @@ public class AbfsClient {
final List<AbfsHttpHeader> requestHeaders = createDefaultHeaders(); final List<AbfsHttpHeader> requestHeaders = createDefaultHeaders();
final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder(); final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder();
abfsUriQueryBuilder.addQuery(HttpQueryParams.QUERY_PARAM_RESOURCE, AbfsHttpConstants.FILESYSTEM); abfsUriQueryBuilder.addQuery(QUERY_PARAM_RESOURCE, FILESYSTEM);
abfsUriQueryBuilder.addQuery(HttpQueryParams.QUERY_PARAM_DIRECTORY, relativePath == null ? "" : urlEncode(relativePath)); abfsUriQueryBuilder.addQuery(QUERY_PARAM_DIRECTORY, relativePath == null ? "" : urlEncode(relativePath));
abfsUriQueryBuilder.addQuery(HttpQueryParams.QUERY_PARAM_RECURSIVE, String.valueOf(recursive)); abfsUriQueryBuilder.addQuery(QUERY_PARAM_RECURSIVE, String.valueOf(recursive));
abfsUriQueryBuilder.addQuery(HttpQueryParams.QUERY_PARAM_CONTINUATION, continuation); abfsUriQueryBuilder.addQuery(QUERY_PARAM_CONTINUATION, continuation);
abfsUriQueryBuilder.addQuery(HttpQueryParams.QUERY_PARAM_MAXRESULTS, String.valueOf(listMaxResults)); abfsUriQueryBuilder.addQuery(QUERY_PARAM_MAXRESULTS, String.valueOf(listMaxResults));
final URL url = createRequestUrl(abfsUriQueryBuilder.toString()); final URL url = createRequestUrl(abfsUriQueryBuilder.toString());
final AbfsRestOperation op = new AbfsRestOperation( final AbfsRestOperation op = new AbfsRestOperation(
this, this,
AbfsHttpConstants.HTTP_METHOD_GET, HTTP_METHOD_GET,
url, url,
requestHeaders); requestHeaders);
op.execute(); op.execute();
@ -153,12 +156,12 @@ public class AbfsClient {
final List<AbfsHttpHeader> requestHeaders = createDefaultHeaders(); final List<AbfsHttpHeader> requestHeaders = createDefaultHeaders();
final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder(); final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder();
abfsUriQueryBuilder.addQuery(HttpQueryParams.QUERY_PARAM_RESOURCE, AbfsHttpConstants.FILESYSTEM); abfsUriQueryBuilder.addQuery(QUERY_PARAM_RESOURCE, FILESYSTEM);
final URL url = createRequestUrl(abfsUriQueryBuilder.toString()); final URL url = createRequestUrl(abfsUriQueryBuilder.toString());
final AbfsRestOperation op = new AbfsRestOperation( final AbfsRestOperation op = new AbfsRestOperation(
this, this,
AbfsHttpConstants.HTTP_METHOD_HEAD, HTTP_METHOD_HEAD,
url, url,
requestHeaders); requestHeaders);
op.execute(); op.execute();
@ -169,12 +172,12 @@ public class AbfsClient {
final List<AbfsHttpHeader> requestHeaders = createDefaultHeaders(); final List<AbfsHttpHeader> requestHeaders = createDefaultHeaders();
final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder(); final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder();
abfsUriQueryBuilder.addQuery(HttpQueryParams.QUERY_PARAM_RESOURCE, AbfsHttpConstants.FILESYSTEM); abfsUriQueryBuilder.addQuery(QUERY_PARAM_RESOURCE, FILESYSTEM);
final URL url = createRequestUrl(abfsUriQueryBuilder.toString()); final URL url = createRequestUrl(abfsUriQueryBuilder.toString());
final AbfsRestOperation op = new AbfsRestOperation( final AbfsRestOperation op = new AbfsRestOperation(
this, this,
AbfsHttpConstants.HTTP_METHOD_DELETE, HTTP_METHOD_DELETE,
url, url,
requestHeaders); requestHeaders);
op.execute(); op.execute();
@ -185,16 +188,16 @@ public class AbfsClient {
throws AzureBlobFileSystemException { throws AzureBlobFileSystemException {
final List<AbfsHttpHeader> requestHeaders = createDefaultHeaders(); final List<AbfsHttpHeader> requestHeaders = createDefaultHeaders();
if (!overwrite) { if (!overwrite) {
requestHeaders.add(new AbfsHttpHeader(HttpHeaderConfigurations.IF_NONE_MATCH, "*")); requestHeaders.add(new AbfsHttpHeader(IF_NONE_MATCH, "*"));
} }
final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder(); final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder();
abfsUriQueryBuilder.addQuery(HttpQueryParams.QUERY_PARAM_RESOURCE, isFile ? AbfsHttpConstants.FILE : AbfsHttpConstants.DIRECTORY); abfsUriQueryBuilder.addQuery(QUERY_PARAM_RESOURCE, isFile ? FILE : DIRECTORY);
final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString()); final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString());
final AbfsRestOperation op = new AbfsRestOperation( final AbfsRestOperation op = new AbfsRestOperation(
this, this,
AbfsHttpConstants.HTTP_METHOD_PUT, HTTP_METHOD_PUT,
url, url,
requestHeaders); requestHeaders);
op.execute(); op.execute();
@ -205,17 +208,17 @@ public class AbfsClient {
throws AzureBlobFileSystemException { throws AzureBlobFileSystemException {
final List<AbfsHttpHeader> requestHeaders = createDefaultHeaders(); final List<AbfsHttpHeader> requestHeaders = createDefaultHeaders();
final String encodedRenameSource = urlEncode(AbfsHttpConstants.FORWARD_SLASH + this.getFileSystem() + source); final String encodedRenameSource = urlEncode(FORWARD_SLASH + this.getFileSystem() + source);
requestHeaders.add(new AbfsHttpHeader(HttpHeaderConfigurations.X_MS_RENAME_SOURCE, encodedRenameSource)); requestHeaders.add(new AbfsHttpHeader(X_MS_RENAME_SOURCE, encodedRenameSource));
requestHeaders.add(new AbfsHttpHeader(HttpHeaderConfigurations.IF_NONE_MATCH, AbfsHttpConstants.STAR)); requestHeaders.add(new AbfsHttpHeader(IF_NONE_MATCH, STAR));
final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder(); final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder();
abfsUriQueryBuilder.addQuery(HttpQueryParams.QUERY_PARAM_CONTINUATION, continuation); abfsUriQueryBuilder.addQuery(QUERY_PARAM_CONTINUATION, continuation);
final URL url = createRequestUrl(destination, abfsUriQueryBuilder.toString()); final URL url = createRequestUrl(destination, abfsUriQueryBuilder.toString());
final AbfsRestOperation op = new AbfsRestOperation( final AbfsRestOperation op = new AbfsRestOperation(
this, this,
AbfsHttpConstants.HTTP_METHOD_PUT, HTTP_METHOD_PUT,
url, url,
requestHeaders); requestHeaders);
op.execute(); op.execute();
@ -227,17 +230,17 @@ public class AbfsClient {
final List<AbfsHttpHeader> requestHeaders = createDefaultHeaders(); final List<AbfsHttpHeader> requestHeaders = createDefaultHeaders();
// JDK7 does not support PATCH, so to workaround the issue we will use // JDK7 does not support PATCH, so to workaround the issue we will use
// PUT and specify the real method in the X-Http-Method-Override header. // PUT and specify the real method in the X-Http-Method-Override header.
requestHeaders.add(new AbfsHttpHeader(HttpHeaderConfigurations.X_HTTP_METHOD_OVERRIDE, requestHeaders.add(new AbfsHttpHeader(X_HTTP_METHOD_OVERRIDE,
AbfsHttpConstants.HTTP_METHOD_PATCH)); HTTP_METHOD_PATCH));
final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder(); final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder();
abfsUriQueryBuilder.addQuery(HttpQueryParams.QUERY_PARAM_ACTION, AbfsHttpConstants.APPEND_ACTION); abfsUriQueryBuilder.addQuery(QUERY_PARAM_ACTION, APPEND_ACTION);
abfsUriQueryBuilder.addQuery(HttpQueryParams.QUERY_PARAM_POSITION, Long.toString(position)); abfsUriQueryBuilder.addQuery(QUERY_PARAM_POSITION, Long.toString(position));
final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString()); final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString());
final AbfsRestOperation op = new AbfsRestOperation( final AbfsRestOperation op = new AbfsRestOperation(
this, this,
AbfsHttpConstants.HTTP_METHOD_PUT, HTTP_METHOD_PUT,
url, url,
requestHeaders, buffer, offset, length); requestHeaders, buffer, offset, length);
op.execute(); op.execute();
@ -245,44 +248,46 @@ public class AbfsClient {
} }
public AbfsRestOperation flush(final String path, final long position, boolean retainUncommittedData) throws AzureBlobFileSystemException { public AbfsRestOperation flush(final String path, final long position, boolean retainUncommittedData)
throws AzureBlobFileSystemException {
final List<AbfsHttpHeader> requestHeaders = createDefaultHeaders(); final List<AbfsHttpHeader> requestHeaders = createDefaultHeaders();
// JDK7 does not support PATCH, so to workaround the issue we will use // JDK7 does not support PATCH, so to workaround the issue we will use
// PUT and specify the real method in the X-Http-Method-Override header. // PUT and specify the real method in the X-Http-Method-Override header.
requestHeaders.add(new AbfsHttpHeader(HttpHeaderConfigurations.X_HTTP_METHOD_OVERRIDE, requestHeaders.add(new AbfsHttpHeader(X_HTTP_METHOD_OVERRIDE,
AbfsHttpConstants.HTTP_METHOD_PATCH)); HTTP_METHOD_PATCH));
final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder(); final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder();
abfsUriQueryBuilder.addQuery(HttpQueryParams.QUERY_PARAM_ACTION, AbfsHttpConstants.FLUSH_ACTION); abfsUriQueryBuilder.addQuery(QUERY_PARAM_ACTION, FLUSH_ACTION);
abfsUriQueryBuilder.addQuery(HttpQueryParams.QUERY_PARAM_POSITION, Long.toString(position)); abfsUriQueryBuilder.addQuery(QUERY_PARAM_POSITION, Long.toString(position));
abfsUriQueryBuilder.addQuery(HttpQueryParams.QUERY_PARAM_RETAIN_UNCOMMITTED_DATA, String.valueOf(retainUncommittedData)); abfsUriQueryBuilder.addQuery(QUERY_PARAM_RETAIN_UNCOMMITTED_DATA, String.valueOf(retainUncommittedData));
final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString()); final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString());
final AbfsRestOperation op = new AbfsRestOperation( final AbfsRestOperation op = new AbfsRestOperation(
this, this,
AbfsHttpConstants.HTTP_METHOD_PUT, HTTP_METHOD_PUT,
url, url,
requestHeaders); requestHeaders);
op.execute(); op.execute();
return op; return op;
} }
public AbfsRestOperation setPathProperties(final String path, final String properties) throws AzureBlobFileSystemException { public AbfsRestOperation setPathProperties(final String path, final String properties)
throws AzureBlobFileSystemException {
final List<AbfsHttpHeader> requestHeaders = createDefaultHeaders(); final List<AbfsHttpHeader> requestHeaders = createDefaultHeaders();
// JDK7 does not support PATCH, so to workaround the issue we will use // JDK7 does not support PATCH, so to workaround the issue we will use
// PUT and specify the real method in the X-Http-Method-Override header. // PUT and specify the real method in the X-Http-Method-Override header.
requestHeaders.add(new AbfsHttpHeader(HttpHeaderConfigurations.X_HTTP_METHOD_OVERRIDE, requestHeaders.add(new AbfsHttpHeader(X_HTTP_METHOD_OVERRIDE,
AbfsHttpConstants.HTTP_METHOD_PATCH)); HTTP_METHOD_PATCH));
requestHeaders.add(new AbfsHttpHeader(HttpHeaderConfigurations.X_MS_PROPERTIES, properties)); requestHeaders.add(new AbfsHttpHeader(X_MS_PROPERTIES, properties));
final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder(); final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder();
abfsUriQueryBuilder.addQuery(HttpQueryParams.QUERY_PARAM_ACTION, AbfsHttpConstants.SET_PROPERTIES_ACTION); abfsUriQueryBuilder.addQuery(QUERY_PARAM_ACTION, SET_PROPERTIES_ACTION);
final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString()); final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString());
final AbfsRestOperation op = new AbfsRestOperation( final AbfsRestOperation op = new AbfsRestOperation(
this, this,
AbfsHttpConstants.HTTP_METHOD_PUT, HTTP_METHOD_PUT,
url, url,
requestHeaders); requestHeaders);
op.execute(); op.execute();
@ -297,7 +302,7 @@ public class AbfsClient {
final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString()); final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString());
final AbfsRestOperation op = new AbfsRestOperation( final AbfsRestOperation op = new AbfsRestOperation(
this, this,
AbfsHttpConstants.HTTP_METHOD_HEAD, HTTP_METHOD_HEAD,
url, url,
requestHeaders); requestHeaders);
op.execute(); op.execute();
@ -307,9 +312,9 @@ public class AbfsClient {
public AbfsRestOperation read(final String path, final long position, final byte[] buffer, final int bufferOffset, public AbfsRestOperation read(final String path, final long position, final byte[] buffer, final int bufferOffset,
final int bufferLength, final String eTag) throws AzureBlobFileSystemException { final int bufferLength, final String eTag) throws AzureBlobFileSystemException {
final List<AbfsHttpHeader> requestHeaders = createDefaultHeaders(); final List<AbfsHttpHeader> requestHeaders = createDefaultHeaders();
requestHeaders.add(new AbfsHttpHeader(HttpHeaderConfigurations.RANGE, requestHeaders.add(new AbfsHttpHeader(RANGE,
String.format("bytes=%d-%d", position, position + bufferLength - 1))); String.format("bytes=%d-%d", position, position + bufferLength - 1)));
requestHeaders.add(new AbfsHttpHeader(HttpHeaderConfigurations.IF_MATCH, eTag)); requestHeaders.add(new AbfsHttpHeader(IF_MATCH, eTag));
final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder(); final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder();
@ -317,7 +322,7 @@ public class AbfsClient {
final AbfsRestOperation op = new AbfsRestOperation( final AbfsRestOperation op = new AbfsRestOperation(
this, this,
AbfsHttpConstants.HTTP_METHOD_GET, HTTP_METHOD_GET,
url, url,
requestHeaders, requestHeaders,
buffer, buffer,
@ -333,13 +338,13 @@ public class AbfsClient {
final List<AbfsHttpHeader> requestHeaders = createDefaultHeaders(); final List<AbfsHttpHeader> requestHeaders = createDefaultHeaders();
final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder(); final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder();
abfsUriQueryBuilder.addQuery(HttpQueryParams.QUERY_PARAM_RECURSIVE, String.valueOf(recursive)); abfsUriQueryBuilder.addQuery(QUERY_PARAM_RECURSIVE, String.valueOf(recursive));
abfsUriQueryBuilder.addQuery(HttpQueryParams.QUERY_PARAM_CONTINUATION, continuation); abfsUriQueryBuilder.addQuery(QUERY_PARAM_CONTINUATION, continuation);
final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString()); final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString());
final AbfsRestOperation op = new AbfsRestOperation( final AbfsRestOperation op = new AbfsRestOperation(
this, this,
AbfsHttpConstants.HTTP_METHOD_DELETE, HTTP_METHOD_DELETE,
url, url,
requestHeaders); requestHeaders);
op.execute(); op.execute();
@ -347,7 +352,7 @@ public class AbfsClient {
} }
private URL createRequestUrl(final String query) throws AzureBlobFileSystemException { private URL createRequestUrl(final String query) throws AzureBlobFileSystemException {
return createRequestUrl(AbfsHttpConstants.EMPTY_STRING, query); return createRequestUrl(EMPTY_STRING, query);
} }
private URL createRequestUrl(final String path, final String query) private URL createRequestUrl(final String path, final String query)
@ -357,8 +362,8 @@ public class AbfsClient {
try { try {
encodedPath = urlEncode(path); encodedPath = urlEncode(path);
} catch (AzureBlobFileSystemException ex) { } catch (AzureBlobFileSystemException ex) {
this.LOG.debug( LOG.debug("Unexpected error.", ex);
"Unexpected error.", ex); throw new InvalidUriException(path);
} }
final StringBuilder sb = new StringBuilder(); final StringBuilder sb = new StringBuilder();
@ -378,9 +383,9 @@ public class AbfsClient {
private static String urlEncode(final String value) throws AzureBlobFileSystemException { private static String urlEncode(final String value) throws AzureBlobFileSystemException {
String encodedString = null; String encodedString = null;
try { try {
encodedString = URLEncoder.encode(value, AbfsHttpConstants.UTF_8) encodedString = URLEncoder.encode(value, UTF_8)
.replace(AbfsHttpConstants.PLUS, AbfsHttpConstants.PLUS_ENCODE) .replace(PLUS, PLUS_ENCODE)
.replace(AbfsHttpConstants.FORWARD_SLASH_ENCODE, AbfsHttpConstants.FORWARD_SLASH); .replace(FORWARD_SLASH_ENCODE, FORWARD_SLASH);
} catch (UnsupportedEncodingException ex) { } catch (UnsupportedEncodingException ex) {
throw new InvalidUriException(value); throw new InvalidUriException(value);
} }
@ -391,11 +396,11 @@ public class AbfsClient {
private String initializeUserAgent() { private String initializeUserAgent() {
final String userAgentComment = String.format(Locale.ROOT, final String userAgentComment = String.format(Locale.ROOT,
"(JavaJRE %s; %s %s)", "(JavaJRE %s; %s %s)",
System.getProperty(AbfsHttpConstants.JAVA_VERSION), System.getProperty(JAVA_VERSION),
System.getProperty(AbfsHttpConstants.OS_NAME) System.getProperty(OS_NAME)
.replaceAll(AbfsHttpConstants.SINGLE_WHITE_SPACE, AbfsHttpConstants.EMPTY_STRING), .replaceAll(SINGLE_WHITE_SPACE, EMPTY_STRING),
System.getProperty(AbfsHttpConstants.OS_VERSION)); System.getProperty(OS_VERSION));
return String.format(AbfsHttpConstants.CLIENT_VERSION + " %s", userAgentComment); return String.format(CLIENT_VERSION + " %s", userAgentComment);
} }
} }

View File

@ -30,12 +30,12 @@ import com.fasterxml.jackson.core.JsonFactory;
import com.fasterxml.jackson.core.JsonParser; import com.fasterxml.jackson.core.JsonParser;
import com.fasterxml.jackson.core.JsonToken; import com.fasterxml.jackson.core.JsonToken;
import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.ObjectMapper;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants; import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants;
import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations; import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations;
import org.apache.hadoop.fs.azurebfs.contracts.services.ListResultSchema; import org.apache.hadoop.fs.azurebfs.contracts.services.ListResultSchema;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/** /**
* Represents an HTTP operation. * Represents an HTTP operation.

View File

@ -29,7 +29,7 @@ import org.apache.hadoop.fs.FileSystem.Statistics;
import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException; import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException;
/** /**
* The AbfsInputStream for AbfsClient * The AbfsInputStream for AbfsClient.
*/ */
public class AbfsInputStream extends FSInputStream { public class AbfsInputStream extends FSInputStream {
private final AbfsClient client; private final AbfsClient client;
@ -59,7 +59,6 @@ public class AbfsInputStream extends FSInputStream {
final int bufferSize, final int bufferSize,
final int readAheadQueueDepth, final int readAheadQueueDepth,
final String eTag) { final String eTag) {
super();
this.client = client; this.client = client;
this.statistics = statistics; this.statistics = statistics;
this.path = path; this.path = path;

View File

@ -19,6 +19,7 @@
package org.apache.hadoop.fs.azurebfs.services; package org.apache.hadoop.fs.azurebfs.services;
import java.io.IOException; import java.io.IOException;
import java.io.InterruptedIOException;
import java.io.OutputStream; import java.io.OutputStream;
import java.util.concurrent.ConcurrentLinkedDeque; import java.util.concurrent.ConcurrentLinkedDeque;
import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.LinkedBlockingQueue;
@ -35,7 +36,7 @@ import org.apache.hadoop.fs.Syncable;
import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException; import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException;
/** /**
* The BlobFsOutputStream for Rest AbfsClient * The BlobFsOutputStream for Rest AbfsClient.
*/ */
public class AbfsOutputStream extends OutputStream implements Syncable { public class AbfsOutputStream extends OutputStream implements Syncable {
private final AbfsClient client; private final AbfsClient client;
@ -79,8 +80,8 @@ public class AbfsOutputStream extends OutputStream implements Syncable {
maxConcurrentRequestCount, maxConcurrentRequestCount,
10L, 10L,
TimeUnit.SECONDS, TimeUnit.SECONDS,
new LinkedBlockingQueue()); new LinkedBlockingQueue<>());
this.completionService = new ExecutorCompletionService(this.threadExecutor); this.completionService = new ExecutorCompletionService<>(this.threadExecutor);
} }
/** /**
@ -111,9 +112,7 @@ public class AbfsOutputStream extends OutputStream implements Syncable {
@Override @Override
public synchronized void write(final byte[] data, final int off, final int length) public synchronized void write(final byte[] data, final int off, final int length)
throws IOException { throws IOException {
if (this.lastError != null) { maybeThrowLastError();
throw this.lastError;
}
Preconditions.checkArgument(data != null, "null data"); Preconditions.checkArgument(data != null, "null data");
@ -142,6 +141,19 @@ public class AbfsOutputStream extends OutputStream implements Syncable {
} }
} }
/**
* Throw the last error recorded if not null.
* After the stream is closed, this is always set to
* an exception, so acts as a guard against method invocation once
* closed.
* @throws IOException if lastError is set
*/
private void maybeThrowLastError() throws IOException {
if (lastError != null) {
throw lastError;
}
}
/** /**
* Flushes this output stream and forces any buffered output bytes to be * Flushes this output stream and forces any buffered output bytes to be
* written out. If any data remains in the payload it is committed to the * written out. If any data remains in the payload it is committed to the
@ -150,7 +162,7 @@ public class AbfsOutputStream extends OutputStream implements Syncable {
*/ */
@Override @Override
public void flush() throws IOException { public void flush() throws IOException {
this.flushInternalAsync(); flushInternalAsync();
} }
/** Similar to posix fsync, flush out the data in client's user buffer /** Similar to posix fsync, flush out the data in client's user buffer
@ -159,7 +171,7 @@ public class AbfsOutputStream extends OutputStream implements Syncable {
*/ */
@Override @Override
public void hsync() throws IOException { public void hsync() throws IOException {
this.flushInternal(); flushInternal();
} }
/** Flush out the data in client's user buffer. After the return of /** Flush out the data in client's user buffer. After the return of
@ -168,7 +180,7 @@ public class AbfsOutputStream extends OutputStream implements Syncable {
*/ */
@Override @Override
public void hflush() throws IOException { public void hflush() throws IOException {
this.flushInternal(); flushInternal();
} }
/** /**
@ -186,34 +198,30 @@ public class AbfsOutputStream extends OutputStream implements Syncable {
} }
try { try {
this.flushInternal(); flushInternal();
this.threadExecutor.shutdown(); threadExecutor.shutdown();
} finally { } finally {
this.lastError = new IOException(FSExceptionMessages.STREAM_IS_CLOSED); lastError = new IOException(FSExceptionMessages.STREAM_IS_CLOSED);
this.buffer = null; buffer = null;
this.bufferIndex = 0; bufferIndex = 0;
this.closed = true; closed = true;
this.writeOperations.clear(); writeOperations.clear();
if (!this.threadExecutor.isShutdown()) { if (!threadExecutor.isShutdown()) {
this.threadExecutor.shutdownNow(); threadExecutor.shutdownNow();
} }
} }
} }
private synchronized void flushInternal() throws IOException { private synchronized void flushInternal() throws IOException {
if (this.lastError != null) { maybeThrowLastError();
throw this.lastError; writeCurrentBufferToService();
} flushWrittenBytesToService();
this.writeCurrentBufferToService();
this.flushWrittenBytesToService();
} }
private synchronized void flushInternalAsync() throws IOException { private synchronized void flushInternalAsync() throws IOException {
if (this.lastError != null) { maybeThrowLastError();
throw this.lastError; writeCurrentBufferToService();
} flushWrittenBytesToServiceAsync();
this.writeCurrentBufferToService();
this.flushWrittenBytesToServiceAsync();
} }
private synchronized void writeCurrentBufferToService() throws IOException { private synchronized void writeCurrentBufferToService() throws IOException {
@ -221,19 +229,19 @@ public class AbfsOutputStream extends OutputStream implements Syncable {
return; return;
} }
final byte[] bytes = this.buffer; final byte[] bytes = buffer;
final int bytesLength = bufferIndex; final int bytesLength = bufferIndex;
this.buffer = new byte[bufferSize]; buffer = new byte[bufferSize];
this.bufferIndex = 0; bufferIndex = 0;
final long offset = this.position; final long offset = position;
this.position += bytesLength; position += bytesLength;
if (this.threadExecutor.getQueue().size() >= maxConcurrentRequestCount * 2) { if (threadExecutor.getQueue().size() >= maxConcurrentRequestCount * 2) {
this.waitForTaskToComplete(); waitForTaskToComplete();
} }
final Future job = this.completionService.submit(new Callable<Void>() { final Future<Void> job = completionService.submit(new Callable<Void>() {
@Override @Override
public Void call() throws Exception { public Void call() throws Exception {
client.append(path, offset, bytes, 0, client.append(path, offset, bytes, 0,
@ -242,25 +250,25 @@ public class AbfsOutputStream extends OutputStream implements Syncable {
} }
}); });
this.writeOperations.add(new WriteOperation(job, offset, bytesLength)); writeOperations.add(new WriteOperation(job, offset, bytesLength));
// Try to shrink the queue // Try to shrink the queue
shrinkWriteOperationQueue(); shrinkWriteOperationQueue();
} }
private synchronized void flushWrittenBytesToService() throws IOException { private synchronized void flushWrittenBytesToService() throws IOException {
for (WriteOperation writeOperation : this.writeOperations) { for (WriteOperation writeOperation : writeOperations) {
try { try {
writeOperation.task.get(); writeOperation.task.get();
} catch (Exception ex) { } catch (Exception ex) {
if (AzureBlobFileSystemException.class.isInstance(ex.getCause())) { if (ex.getCause() instanceof AzureBlobFileSystemException) {
ex = AzureBlobFileSystemException.class.cast(ex.getCause()); ex = (AzureBlobFileSystemException)ex.getCause();
} }
this.lastError = new IOException(ex); lastError = new IOException(ex);
throw this.lastError; throw lastError;
} }
} }
flushWrittenBytesToServiceInternal(this.position, false); flushWrittenBytesToServiceInternal(position, false);
} }
private synchronized void flushWrittenBytesToServiceAsync() throws IOException { private synchronized void flushWrittenBytesToServiceAsync() throws IOException {
@ -273,7 +281,8 @@ public class AbfsOutputStream extends OutputStream implements Syncable {
this.lastTotalAppendOffset = 0; this.lastTotalAppendOffset = 0;
} }
private synchronized void flushWrittenBytesToServiceInternal(final long offset, final boolean retainUncommitedData) throws IOException { private synchronized void flushWrittenBytesToServiceInternal(final long offset,
final boolean retainUncommitedData) throws IOException {
try { try {
client.flush(path, offset, retainUncommitedData); client.flush(path, offset, retainUncommitedData);
} catch (AzureBlobFileSystemException ex) { } catch (AzureBlobFileSystemException ex) {
@ -288,31 +297,33 @@ public class AbfsOutputStream extends OutputStream implements Syncable {
*/ */
private synchronized void shrinkWriteOperationQueue() throws IOException { private synchronized void shrinkWriteOperationQueue() throws IOException {
try { try {
while (this.writeOperations.peek() != null && this.writeOperations.peek().task.isDone()) { while (writeOperations.peek() != null && writeOperations.peek().task.isDone()) {
this.writeOperations.peek().task.get(); writeOperations.peek().task.get();
this.lastTotalAppendOffset += this.writeOperations.peek().length; lastTotalAppendOffset += writeOperations.peek().length;
this.writeOperations.remove(); writeOperations.remove();
} }
} catch (Exception e) { } catch (Exception e) {
if (AzureBlobFileSystemException.class.isInstance(e.getCause())) { if (e.getCause() instanceof AzureBlobFileSystemException) {
this.lastError = IOException.class.cast(e.getCause()); lastError = (AzureBlobFileSystemException)e.getCause();
} else { } else {
this.lastError = new IOException(e); lastError = new IOException(e);
} }
throw this.lastError; throw lastError;
} }
} }
private void waitForTaskToComplete() throws IOException { private void waitForTaskToComplete() throws IOException {
boolean completed; boolean completed;
for (completed = false; this.completionService.poll() != null; completed = true) {} for (completed = false; completionService.poll() != null; completed = true) {
// keep polling until there is no data
}
if (!completed) { if (!completed) {
try { try {
this.completionService.take(); completionService.take();
} catch (InterruptedException e) { } catch (InterruptedException e) {
this.lastError = new IOException(e); lastError = (IOException)new InterruptedIOException(e.toString()).initCause(e);
throw this.lastError; throw lastError;
} }
} }
} }

View File

@ -23,15 +23,16 @@ import java.net.HttpURLConnection;
import java.net.URL; import java.net.URL;
import java.util.List; import java.util.List;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants; import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants;
import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException;
import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException; import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException;
import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException;
import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidAbfsRestOperationException; import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidAbfsRestOperationException;
import org.slf4j.Logger;
/** /**
* The AbfsRestOperation for Rest AbfsClient * The AbfsRestOperation for Rest AbfsClient.
*/ */
public class AbfsRestOperation { public class AbfsRestOperation {
// Blob FS client, which has the credentials, retry policy, and logs. // Blob FS client, which has the credentials, retry policy, and logs.
@ -47,7 +48,7 @@ public class AbfsRestOperation {
// request body and all the download methods have a response body. // request body and all the download methods have a response body.
private final boolean hasRequestBody; private final boolean hasRequestBody;
private final Logger logger; private final Logger LOG = LoggerFactory.getLogger(AbfsClient.class);
// For uploads, this is the request entity body. For downloads, // For uploads, this is the request entity body. For downloads,
// this will hold the response entity body. // this will hold the response entity body.
@ -79,7 +80,6 @@ public class AbfsRestOperation {
this.requestHeaders = requestHeaders; this.requestHeaders = requestHeaders;
this.hasRequestBody = (AbfsHttpConstants.HTTP_METHOD_PUT.equals(method) this.hasRequestBody = (AbfsHttpConstants.HTTP_METHOD_PUT.equals(method)
|| AbfsHttpConstants.HTTP_METHOD_PATCH.equals(method)); || AbfsHttpConstants.HTTP_METHOD_PATCH.equals(method));
this.logger = client.LOG;
} }
/** /**
@ -150,11 +150,11 @@ public class AbfsRestOperation {
httpOperation.processResponse(buffer, bufferOffset, bufferLength); httpOperation.processResponse(buffer, bufferOffset, bufferLength);
} catch (IOException ex) { } catch (IOException ex) {
if (logger.isDebugEnabled()) { if (LOG.isDebugEnabled()) {
if (httpOperation != null) { if (httpOperation != null) {
logger.debug("HttpRequestFailure: " + httpOperation.toString(), ex); LOG.debug("HttpRequestFailure: " + httpOperation.toString(), ex);
} else { } else {
logger.debug("HttpRequestFailure: " + method + "," + url, ex); LOG.debug("HttpRequestFailure: " + method + "," + url, ex);
} }
} }
if (!client.getRetryPolicy().shouldRetry(retryCount, -1)) { if (!client.getRetryPolicy().shouldRetry(retryCount, -1)) {
@ -163,8 +163,8 @@ public class AbfsRestOperation {
return false; return false;
} }
if (logger.isDebugEnabled()) { if (LOG.isDebugEnabled()) {
logger.debug("HttpRequest: " + httpOperation.toString()); LOG.debug("HttpRequest: " + httpOperation.toString());
} }
if (client.getRetryPolicy().shouldRetry(retryCount, httpOperation.getStatusCode())) { if (client.getRetryPolicy().shouldRetry(retryCount, httpOperation.getStatusCode())) {

View File

@ -18,13 +18,13 @@
package org.apache.hadoop.fs.azurebfs.services; package org.apache.hadoop.fs.azurebfs.services;
import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants;
import java.util.HashMap; import java.util.HashMap;
import java.util.Map; import java.util.Map;
import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants;
/** /**
* The UrlQueryBuilder for Rest AbfsClient * The UrlQueryBuilder for Rest AbfsClient.
*/ */
public class AbfsUriQueryBuilder { public class AbfsUriQueryBuilder {
private Map<String, String> parameters; private Map<String, String> parameters;

View File

@ -43,7 +43,7 @@ public class ExponentialRetryPolicy {
private static final int DEFAULT_MAX_BACKOFF = 1000 * 30; private static final int DEFAULT_MAX_BACKOFF = 1000 * 30;
/** /**
*Represents the default minimum amount of time used when calculating the exponential * Represents the default minimum amount of time used when calculating the exponential
* delay between retries. * delay between retries.
*/ */
private static final int DEFAULT_MIN_BACKOFF = 1000 * 3; private static final int DEFAULT_MIN_BACKOFF = 1000 * 3;

View File

@ -18,10 +18,10 @@
package org.apache.hadoop.fs.azurebfs.services; package org.apache.hadoop.fs.azurebfs.services;
import org.apache.hadoop.fs.azurebfs.contracts.services.ReadBufferStatus;
import java.util.concurrent.CountDownLatch; import java.util.concurrent.CountDownLatch;
import org.apache.hadoop.fs.azurebfs.contracts.services.ReadBufferStatus;
class ReadBuffer { class ReadBuffer {
private AbfsInputStream stream; private AbfsInputStream stream;

View File

@ -28,7 +28,7 @@ import java.util.Stack;
import java.util.concurrent.CountDownLatch; import java.util.concurrent.CountDownLatch;
/** /**
* The Read Buffer Manager for Rest AbfsClient * The Read Buffer Manager for Rest AbfsClient.
*/ */
final class ReadBufferManager { final class ReadBufferManager {
private static final Logger LOGGER = LoggerFactory.getLogger(ReadBufferManager.class); private static final Logger LOGGER = LoggerFactory.getLogger(ReadBufferManager.class);
@ -40,11 +40,11 @@ final class ReadBufferManager {
private Thread[] threads = new Thread[NUM_THREADS]; private Thread[] threads = new Thread[NUM_THREADS];
private byte[][] buffers; // array of byte[] buffers, to hold the data that is read private byte[][] buffers; // array of byte[] buffers, to hold the data that is read
private Stack<Integer> freeList = new Stack<Integer>(); // indices in buffers[] array that are available private Stack<Integer> freeList = new Stack<>(); // indices in buffers[] array that are available
private Queue<ReadBuffer> readAheadQueue = new LinkedList<ReadBuffer>(); // queue of requests that are not picked up by any worker thread yet private Queue<ReadBuffer> readAheadQueue = new LinkedList<>(); // queue of requests that are not picked up by any worker thread yet
private LinkedList<ReadBuffer> inProgressList = new LinkedList<ReadBuffer>(); // requests being processed by worker threads private LinkedList<ReadBuffer> inProgressList = new LinkedList<>(); // requests being processed by worker threads
private LinkedList<ReadBuffer> completedReadList = new LinkedList<ReadBuffer>(); // buffers available for reading private LinkedList<ReadBuffer> completedReadList = new LinkedList<>(); // buffers available for reading
private static final ReadBufferManager BUFFER_MANAGER; // singleton, initialized in static initialization block private static final ReadBufferManager BUFFER_MANAGER; // singleton, initialized in static initialization block
static { static {
@ -85,7 +85,7 @@ final class ReadBufferManager {
/** /**
* {@link AbfsInputStream} calls this method to queue read-aheads * {@link AbfsInputStream} calls this method to queue read-aheads.
* *
* @param stream The {@link AbfsInputStream} for which to do the read-ahead * @param stream The {@link AbfsInputStream} for which to do the read-ahead
* @param requestedOffset The offset in the file which shoukd be read * @param requestedOffset The offset in the file which shoukd be read
@ -93,15 +93,15 @@ final class ReadBufferManager {
*/ */
void queueReadAhead(final AbfsInputStream stream, final long requestedOffset, final int requestedLength) { void queueReadAhead(final AbfsInputStream stream, final long requestedOffset, final int requestedLength) {
if (LOGGER.isTraceEnabled()) { if (LOGGER.isTraceEnabled()) {
LOGGER.trace("Start Queueing readAhead for " + stream.getPath() + " offset " + requestedOffset LOGGER.trace("Start Queueing readAhead for {} offset {} length {}",
+ " length " + requestedLength); stream.getPath(), requestedOffset, requestedLength);
} }
ReadBuffer buffer; ReadBuffer buffer;
synchronized (this) { synchronized (this) {
if (isAlreadyQueued(stream, requestedOffset)) { if (isAlreadyQueued(stream, requestedOffset)) {
return; // already queued, do not queue again return; // already queued, do not queue again
} }
if (freeList.size() == 0 && !tryEvict()) { if (freeList.isEmpty() && !tryEvict()) {
return; // no buffers available, cannot queue anything return; // no buffers available, cannot queue anything
} }
@ -121,8 +121,8 @@ final class ReadBufferManager {
notifyAll(); notifyAll();
} }
if (LOGGER.isTraceEnabled()) { if (LOGGER.isTraceEnabled()) {
LOGGER.trace("Done q-ing readAhead for file " + stream.getPath() + " offset " + requestedOffset LOGGER.trace("Done q-ing readAhead for file {} offset {} buffer idx {}",
+ " buffer idx " + buffer.getBufferindex()); stream.getPath(), requestedOffset, buffer.getBufferindex());
} }
} }
@ -144,7 +144,8 @@ final class ReadBufferManager {
int getBlock(final AbfsInputStream stream, final long position, final int length, final byte[] buffer) { int getBlock(final AbfsInputStream stream, final long position, final int length, final byte[] buffer) {
// not synchronized, so have to be careful with locking // not synchronized, so have to be careful with locking
if (LOGGER.isTraceEnabled()) { if (LOGGER.isTraceEnabled()) {
LOGGER.trace("getBlock for file " + stream.getPath() + " position " + position + " thread " + Thread.currentThread().getName()); LOGGER.trace("getBlock for file {} position {} thread {}",
stream.getPath(), position, Thread.currentThread().getName());
} }
waitForProcess(stream, position); waitForProcess(stream, position);
@ -155,12 +156,13 @@ final class ReadBufferManager {
} }
if (bytesRead > 0) { if (bytesRead > 0) {
if (LOGGER.isTraceEnabled()) { if (LOGGER.isTraceEnabled()) {
LOGGER.trace("Done read from Cache for " + stream.getPath() + " position " + position + " length " + bytesRead); LOGGER.trace("Done read from Cache for {} position {} length {}",
stream.getPath(), position, bytesRead);
} }
return bytesRead; return bytesRead;
} }
// otherwise, just say we got nothing - calling thread can do it's own read // otherwise, just say we got nothing - calling thread can do its own read
return 0; return 0;
} }
@ -179,8 +181,8 @@ final class ReadBufferManager {
if (readBuf != null) { // if in in-progress queue, then block for it if (readBuf != null) { // if in in-progress queue, then block for it
try { try {
if (LOGGER.isTraceEnabled()) { if (LOGGER.isTraceEnabled()) {
LOGGER.trace("got a relevant read buffer for file " + stream.getPath() + " offset " + readBuf.getOffset() LOGGER.trace("got a relevant read buffer for file {} offset {} buffer idx {}",
+ " buffer idx " + readBuf.getBufferindex()); stream.getPath(), readBuf.getOffset(), readBuf.getBufferindex());
} }
readBuf.getLatch().await(); // blocking wait on the caller stream's thread readBuf.getLatch().await(); // blocking wait on the caller stream's thread
// Note on correctness: readBuf gets out of inProgressList only in 1 place: after worker thread // Note on correctness: readBuf gets out of inProgressList only in 1 place: after worker thread
@ -193,8 +195,8 @@ final class ReadBufferManager {
Thread.currentThread().interrupt(); Thread.currentThread().interrupt();
} }
if (LOGGER.isTraceEnabled()) { if (LOGGER.isTraceEnabled()) {
LOGGER.trace("latch done for file " + stream.getPath() + " buffer idx " + readBuf.getBufferindex() LOGGER.trace("latch done for file {} buffer idx {} length {}",
+ " length " + readBuf.getLength()); stream.getPath(), readBuf.getBufferindex(), readBuf.getLength());
} }
} }
} }
@ -254,8 +256,8 @@ final class ReadBufferManager {
freeList.push(buf.getBufferindex()); freeList.push(buf.getBufferindex());
completedReadList.remove(buf); completedReadList.remove(buf);
if (LOGGER.isTraceEnabled()) { if (LOGGER.isTraceEnabled()) {
LOGGER.trace("Evicting buffer idx " + buf.getBufferindex() + "; was used for file " + buf.getStream().getPath() LOGGER.trace("Evicting buffer idx {}; was used for file {} offset {} length {}",
+ " offset " + buf.getOffset() + " length " + buf.getLength()); buf.getBufferindex(), buf.getStream().getPath(), buf.getOffset(), buf.getLength());
} }
return true; return true;
} }
@ -344,13 +346,14 @@ final class ReadBufferManager {
inProgressList.add(buffer); inProgressList.add(buffer);
} }
if (LOGGER.isTraceEnabled()) { if (LOGGER.isTraceEnabled()) {
LOGGER.trace("ReadBufferWorker picked file " + buffer.getStream().getPath() + " for offset " + buffer.getOffset()); LOGGER.trace("ReadBufferWorker picked file {} for offset {}",
buffer.getStream().getPath(), buffer.getOffset());
} }
return buffer; return buffer;
} }
/** /**
* ReadBufferWorker thread calls this method to post completion * ReadBufferWorker thread calls this method to post completion.
* *
* @param buffer the buffer whose read was completed * @param buffer the buffer whose read was completed
* @param result the {@link ReadBufferStatus} after the read operation in the worker thread * @param result the {@link ReadBufferStatus} after the read operation in the worker thread
@ -358,8 +361,8 @@ final class ReadBufferManager {
*/ */
void doneReading(final ReadBuffer buffer, final ReadBufferStatus result, final int bytesActuallyRead) { void doneReading(final ReadBuffer buffer, final ReadBufferStatus result, final int bytesActuallyRead) {
if (LOGGER.isTraceEnabled()) { if (LOGGER.isTraceEnabled()) {
LOGGER.trace("ReadBufferWorker completed file " + buffer.getStream().getPath() + " for offset " + buffer.getOffset() LOGGER.trace("ReadBufferWorker completed file {} for offset {} bytes {}",
+ " bytes " + bytesActuallyRead); buffer.getStream().getPath(), buffer.getOffset(), bytesActuallyRead);
} }
synchronized (this) { synchronized (this) {
inProgressList.remove(buffer); inProgressList.remove(buffer);
@ -380,8 +383,9 @@ final class ReadBufferManager {
/** /**
* Similar to System.currentTimeMillis, except implemented with System.nanoTime(). * Similar to System.currentTimeMillis, except implemented with System.nanoTime().
* System.currentTimeMillis can go backwards when system clock is changed (e.g., with NTP time synchronization), * System.currentTimeMillis can go backwards when system clock is changed (e.g., with NTP time synchronization),
* making it unsuitable for measuring time intervals. nanotime is strictly monotonically increasing, * making it unsuitable for measuring time intervals. nanotime is strictly monotonically increasing per CPU core.
* so it is much more suitable to measuring intervals. * Note: it is not monotonic across Sockets, and even within a CPU, its only the
* more recent parts which share a clock across all cores.
* *
* @return current time in milliseconds * @return current time in milliseconds
*/ */

View File

@ -18,10 +18,10 @@
package org.apache.hadoop.fs.azurebfs.services; package org.apache.hadoop.fs.azurebfs.services;
import org.apache.hadoop.fs.azurebfs.contracts.services.ReadBufferStatus;
import java.util.concurrent.CountDownLatch; import java.util.concurrent.CountDownLatch;
import org.apache.hadoop.fs.azurebfs.contracts.services.ReadBufferStatus;
class ReadBufferWorker implements Runnable { class ReadBufferWorker implements Runnable {
protected static final CountDownLatch UNLEASH_WORKERS = new CountDownLatch(1); protected static final CountDownLatch UNLEASH_WORKERS = new CountDownLatch(1);

View File

@ -22,6 +22,7 @@ import javax.crypto.Mac;
import javax.crypto.spec.SecretKeySpec; import javax.crypto.spec.SecretKeySpec;
import java.io.UnsupportedEncodingException; import java.io.UnsupportedEncodingException;
import java.net.HttpURLConnection; import java.net.HttpURLConnection;
import java.net.URL;
import java.net.URLDecoder; import java.net.URLDecoder;
import java.text.DateFormat; import java.text.DateFormat;
import java.text.SimpleDateFormat; import java.text.SimpleDateFormat;
@ -38,11 +39,11 @@ import java.util.TimeZone;
import java.util.regex.Matcher; import java.util.regex.Matcher;
import java.util.regex.Pattern; import java.util.regex.Pattern;
import org.apache.commons.codec.Charsets;
import org.apache.commons.codec.binary.Base64;
import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants; import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants;
import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations; import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations;
import org.apache.commons.codec.binary.Base64;
import org.apache.commons.codec.Charsets;
/** /**
* Represents the shared key credentials used to access an Azure Storage * Represents the shared key credentials used to access an Azure Storage
* account. * account.
@ -89,7 +90,7 @@ public class SharedKeyCredentials {
} }
private String computeHmac256(final String stringToSign) { private String computeHmac256(final String stringToSign) {
byte[] utf8Bytes = null; byte[] utf8Bytes;
try { try {
utf8Bytes = stringToSign.getBytes(AbfsHttpConstants.UTF_8); utf8Bytes = stringToSign.getBytes(AbfsHttpConstants.UTF_8);
} catch (final UnsupportedEncodingException e) { } catch (final UnsupportedEncodingException e) {
@ -158,7 +159,7 @@ public class SharedKeyCredentials {
} }
/** /**
* Initialie the HmacSha256 associated with the account key. * Initialize the HmacSha256 associated with the account key.
*/ */
private void initializeMac() { private void initializeMac() {
// Initializes the HMAC-SHA256 Mac and SecretKey. // Initializes the HMAC-SHA256 Mac and SecretKey.
@ -171,7 +172,7 @@ public class SharedKeyCredentials {
} }
/** /**
* Append a string to a string builder with a newline constant * Append a string to a string builder with a newline constant.
* *
* @param builder the StringBuilder object * @param builder the StringBuilder object
* @param element the string to append. * @param element the string to append.
@ -194,9 +195,10 @@ public class SharedKeyCredentials {
* @param conn the HttpURLConnection for the operation. * @param conn the HttpURLConnection for the operation.
* @return A canonicalized string. * @return A canonicalized string.
*/ */
private static String canonicalizeHttpRequest(final java.net.URL address, final String accountName, private static String canonicalizeHttpRequest(final URL address,
final String method, final String contentType, final long contentLength, final String date, final String accountName, final String method, final String contentType,
final HttpURLConnection conn) throws UnsupportedEncodingException { final long contentLength, final String date, final HttpURLConnection conn)
throws UnsupportedEncodingException {
// The first element should be the Method of the request. // The first element should be the Method of the request.
// I.e. GET, POST, PUT, or HEAD. // I.e. GET, POST, PUT, or HEAD.
@ -246,7 +248,8 @@ public class SharedKeyCredentials {
* @param accountName the account name for the request. * @param accountName the account name for the request.
* @return the canonicalized resource string. * @return the canonicalized resource string.
*/ */
private static String getCanonicalizedResource(final java.net.URL address, final String accountName) throws UnsupportedEncodingException { private static String getCanonicalizedResource(final URL address,
final String accountName) throws UnsupportedEncodingException {
// Resource path // Resource path
final StringBuilder resourcepath = new StringBuilder(AbfsHttpConstants.FORWARD_SLASH); final StringBuilder resourcepath = new StringBuilder(AbfsHttpConstants.FORWARD_SLASH);
resourcepath.append(accountName); resourcepath.append(accountName);
@ -263,7 +266,7 @@ public class SharedKeyCredentials {
final Map<String, String[]> queryVariables = parseQueryString(address.getQuery()); final Map<String, String[]> queryVariables = parseQueryString(address.getQuery());
final Map<String, String> lowercasedKeyNameValue = new HashMap<String, String>(); final Map<String, String> lowercasedKeyNameValue = new HashMap<>();
for (final Entry<String, String[]> entry : queryVariables.entrySet()) { for (final Entry<String, String[]> entry : queryVariables.entrySet()) {
// sort the value and organize it as comma separated values // sort the value and organize it as comma separated values
@ -303,14 +306,17 @@ public class SharedKeyCredentials {
} }
/** /**
* Gets all the values for the given header in the one to many map, performs a trimStart() on each return value * Gets all the values for the given header in the one to many map,
* performs a trimStart() on each return value.
* *
* @param headers a one to many map of key / values representing the header values for the connection. * @param headers a one to many map of key / values representing the header values for the connection.
* @param headerName the name of the header to lookup * @param headerName the name of the header to lookup
* @return an ArrayList<String> of all trimmed values corresponding to the requested headerName. This may be empty * @return an ArrayList<String> of all trimmed values corresponding to the requested headerName. This may be empty
* if the header is not found. * if the header is not found.
*/ */
private static ArrayList<String> getHeaderValues(final Map<String, List<String>> headers, final String headerName) { private static ArrayList<String> getHeaderValues(
final Map<String, List<String>> headers,
final String headerName) {
final ArrayList<String> arrayOfValues = new ArrayList<String>(); final ArrayList<String> arrayOfValues = new ArrayList<String>();
List<String> values = null; List<String> values = null;
@ -338,7 +344,7 @@ public class SharedKeyCredentials {
* @return a HashMap<String, String[]> of the key values. * @return a HashMap<String, String[]> of the key values.
*/ */
private static HashMap<String, String[]> parseQueryString(String parseString) throws UnsupportedEncodingException { private static HashMap<String, String[]> parseQueryString(String parseString) throws UnsupportedEncodingException {
final HashMap<String, String[]> retVals = new HashMap<String, String[]>(); final HashMap<String, String[]> retVals = new HashMap<>();
if (parseString == null || parseString.isEmpty()) { if (parseString == null || parseString.isEmpty()) {
return retVals; return retVals;
} }

View File

@ -0,0 +1,72 @@
<!---
Licensed under the Apache License, Version 2.0 (the "License");
you may not use this file except in compliance with the License.
You may obtain a copy of the License at
http://www.apache.org/licenses/LICENSE-2.0
Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License. See accompanying LICENSE file.
-->
# Hadoop Azure Support: ABFS — Azure Data Lake Storage Gen2
<!-- MACRO{toc|fromDepth=1|toDepth=3} -->
## Introduction
The `hadoop-azure` module provides support for the Azure Data Lake Storage Gen2
storage layer through the "abfs" connector
To make it part of Apache Hadoop's default classpath, simply make sure that
`HADOOP_OPTIONAL_TOOLS` in `hadoop-env.sh` has `hadoop-azure` in the list.
## Features
* Read and write data stored in an Azure Blob Storage account.
* *Fully Consistent* view of the storage across all clients.
* Can read data written through the wasb: connector.
* Present a hierarchical file system view by implementing the standard Hadoop
[`FileSystem`](../api/org/apache/hadoop/fs/FileSystem.html) interface.
* Supports configuration of multiple Azure Blob Storage accounts.
* Can act as a source or destination of data in Hadoop MapReduce, Apache Hive, Apache Spark
* Tested at scale on both Linux and Windows.
* Can be used as a replacement for HDFS on Hadoop clusters deployed in Azure infrastructure.
## Limitations
* File last access time is not tracked.
## Technical notes
### Security
### Consistency and Concurrency
*TODO*: complete/review
The abfs client has a fully consistent view of the store, which has complete Create Read Update and Delete consistency for data and metadata.
(Compare and contrast with S3 which only offers Create consistency; S3Guard adds CRUD to metadata, but not the underlying data).
### Performance
*TODO*: check these.
* File Rename: `O(1)`.
* Directory Rename: `O(files)`.
* Directory Delete: `O(files)`.
## Testing ABFS
See the relevant section in [Testing Azure](testing_azure.html).
## References
* [A closer look at Azure Data Lake Storage Gen2](https://azure.microsoft.com/en-gb/blog/a-closer-look-at-azure-data-lake-storage-gen2/);
MSDN Article from June 28, 2018.

View File

@ -574,3 +574,79 @@ mvn test -Dtest=CleanupTestContainers
This will delete the containers; the output log of the test run will This will delete the containers; the output log of the test run will
provide the details and summary of the operation. provide the details and summary of the operation.
## Testing ABFS
The ABFS Connector tests share the same account as the wasb tests; this is
needed for cross-connector compatibility tests.
This makes for a somewhat complex set of configuration options.
Here are the settings for an account `ACCOUNTNAME`
```xml
<property>
<name>abfs.account.name</name>
<value>ACCOUNTNAME</value>
</property>
<property>
<name>abfs.account.full.name</name>
<value>${abfs.account.name}.dfs.core.windows.net</value>
</property>
<property>
<name>abfs.account.key</name>
<value>SECRETKEY==</value>
</property>
<property>
<name>fs.azure.account.key.ACCOUNTNAME.dfs.core.windows.net</name>
<value>${abfs.account.key}</value>
</property>
<property>
<name>fs.azure.account.key.ACCOUNTNAME.blob.core.windows.net</name>
<value>${abfs.account.key}</value>
</property>
<property>
<name>fs.azure.test.account.key.ACCOUNTNAME.dfs.core.windows.net</name>
<value>${abfs.account.key}</value>
</property>
<property>
<name>fs.azure.test.account.key.ACCOUNTNAME.blob.core.windows.net</name>
<value>${abfs.account.key}</value>
</property>
<property>
<name>fs.azure.account.key.ACCOUNTNAME</name>
<value>${abfs.account.key}</value>
</property>
<property>
<name>fs.azure.test.account.key.ACCOUNTNAME</name>
<value>${abfs.account.key}</value>
</property>
<property>
<name>fs.azure.test.account.name</name>
<value>${abfs.account.full.name}</value>
</property>
<property>
<name>fs.contract.test.fs.abfs</name>
<value>abfs://TESTCONTAINER@ACCOUNTNAME.dfs.core.windows.net</value>
<description>Container for contract tests</description>
</property>
<property>
<name>fs.contract.test.fs.abfss</name>
<value>abfss://TESTCONTAINER@ACCOUNTNAME.dfs.core.windows.net</value>
<description>Container for contract tests</description>
</property>
```

View File

@ -18,10 +18,19 @@
package org.apache.hadoop.fs.azure.contract; package org.apache.hadoop.fs.azure.contract;
import java.io.FileNotFoundException;
import java.io.IOException;
import org.junit.Test;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileAlreadyExistsException;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.contract.AbstractContractAppendTest; import org.apache.hadoop.fs.contract.AbstractContractAppendTest;
import org.apache.hadoop.fs.contract.AbstractFSContract; import org.apache.hadoop.fs.contract.AbstractFSContract;
import static org.apache.hadoop.fs.contract.ContractTestUtils.skip; import static org.apache.hadoop.fs.contract.ContractTestUtils.skip;
import static org.apache.hadoop.test.LambdaTestUtils.intercept;
/** /**
* Append test, skipping one of them. * Append test, skipping one of them.
@ -38,4 +47,18 @@ public class ITestAzureNativeContractAppend extends AbstractContractAppendTest {
public void testRenameFileBeingAppended() throws Throwable { public void testRenameFileBeingAppended() throws Throwable {
skip("Skipping as renaming an opened file is not supported"); skip("Skipping as renaming an opened file is not supported");
} }
/**
* Wasb returns a different exception, so change the intercept logic here.
*/
@Override
@Test
public void testAppendDirectory() throws Exception {
final FileSystem fs = getFileSystem();
final Path folderPath = path("testAppendDirectory");
fs.mkdirs(folderPath);
intercept(FileNotFoundException.class,
() -> fs.append(folderPath));
}
} }

View File

@ -0,0 +1,304 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.fs.azurebfs;
import java.io.IOException;
import java.net.URI;
import java.util.Hashtable;
import java.util.UUID;
import java.util.concurrent.Callable;
import com.google.common.base.Preconditions;
import org.junit.After;
import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.Rule;
import org.junit.rules.TestName;
import org.junit.rules.Timeout;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.azure.AbstractWasbTestWithTimeout;
import org.apache.hadoop.fs.azure.AzureNativeFileSystemStore;
import org.apache.hadoop.fs.azure.NativeAzureFileSystem;
import org.apache.hadoop.fs.azure.integration.AzureTestConstants;
import org.apache.hadoop.fs.azure.metrics.AzureFileSystemInstrumentation;
import org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys;
import org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes;
import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException;
import org.apache.hadoop.fs.azurebfs.utils.UriUtils;
import org.apache.hadoop.fs.contract.ContractTestUtils;
import org.apache.hadoop.io.IOUtils;
import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.*;
import static org.apache.hadoop.fs.azurebfs.contracts.services.AzureServiceErrorCode.FILE_SYSTEM_NOT_FOUND;
import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.*;
import static org.apache.hadoop.test.LambdaTestUtils.intercept;
import static org.junit.Assume.assumeTrue;
import static org.hamcrest.CoreMatchers.*;
/**
* Base for AzureBlobFileSystem Integration tests.
*
* <I>Important: This is for integration tests only.</I>
*/
public abstract class AbstractAbfsIntegrationTest extends
AbstractWasbTestWithTimeout {
private static final Logger LOG =
LoggerFactory.getLogger(AbstractAbfsIntegrationTest.class);
private final boolean isEmulator;
private NativeAzureFileSystem wasb;
private AzureBlobFileSystem abfs;
private String abfsScheme;
private Configuration configuration;
private String fileSystemName;
private String accountName;
private String testUrl;
protected AbstractAbfsIntegrationTest(final boolean secure) {
this(secure ? FileSystemUriSchemes.ABFS_SECURE_SCHEME : FileSystemUriSchemes.ABFS_SCHEME);
}
protected AbstractAbfsIntegrationTest() {
this(FileSystemUriSchemes.ABFS_SCHEME);
}
private AbstractAbfsIntegrationTest(final String scheme) {
abfsScheme = scheme;
fileSystemName = ABFS_TEST_CONTAINER_PREFIX + UUID.randomUUID().toString();
configuration = new Configuration();
configuration.addResource(ABFS_TEST_RESOURCE_XML);
String accountName = configuration.get(FS_AZURE_TEST_ACCOUNT_NAME, "");
assumeTrue("Not set: " + FS_AZURE_TEST_ACCOUNT_NAME,
!accountName.isEmpty());
assertThat("The key in " + FS_AZURE_TEST_ACCOUNT_KEY_PREFIX
+ " is not bound to an ABFS account",
accountName, containsString("dfs.core.windows.net"));
String fullKey = FS_AZURE_TEST_ACCOUNT_KEY_PREFIX
+ accountName;
assumeTrue("Not set: " + fullKey,
configuration.get(fullKey) != null);
final String abfsUrl = this.getFileSystemName() + "@" + this.getAccountName();
URI defaultUri = null;
try {
defaultUri = new URI(abfsScheme, abfsUrl, null, null, null);
} catch (Exception ex) {
throw new AssertionError(ex);
}
this.testUrl = defaultUri.toString();
configuration.set(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY, defaultUri.toString());
configuration.setBoolean(AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION, true);
this.isEmulator = this.configuration.getBoolean(FS_AZURE_EMULATOR_ENABLED, false);
this.accountName = this.configuration.get(FS_AZURE_TEST_ACCOUNT_NAME);
}
@Before
public void setup() throws Exception {
//Create filesystem first to make sure getWasbFileSystem() can return an existing filesystem.
createFileSystem();
if (!isEmulator) {
final URI wasbUri = new URI(abfsUrlToWasbUrl(getTestUrl()));
final AzureNativeFileSystemStore azureNativeFileSystemStore =
new AzureNativeFileSystemStore();
azureNativeFileSystemStore.initialize(
wasbUri,
getConfiguration(),
new AzureFileSystemInstrumentation(getConfiguration()));
wasb = new NativeAzureFileSystem(azureNativeFileSystemStore);
wasb.initialize(wasbUri, configuration);
}
}
@After
public void teardown() throws Exception {
try {
IOUtils.closeStream(wasb);
wasb = null;
if (abfs == null) {
return;
}
final AzureBlobFileSystemStore abfsStore = abfs.getAbfsStore();
abfsStore.deleteFilesystem();
AbfsRestOperationException ex = intercept(
AbfsRestOperationException.class,
new Callable<Hashtable<String, String>>() {
@Override
public Hashtable<String, String> call() throws Exception {
return abfsStore.getFilesystemProperties();
}
});
if (FILE_SYSTEM_NOT_FOUND.getStatusCode() != ex.getStatusCode()) {
LOG.warn("Deleted test filesystem may still exist: {}", abfs, ex);
}
} catch (Exception e) {
LOG.warn("During cleanup: {}", e, e);
} finally {
IOUtils.closeStream(abfs);
abfs = null;
}
}
public AzureBlobFileSystem getFileSystem() throws IOException {
return abfs;
}
/**
* Creates the filesystem; updates the {@link #abfs} field.
* @return the created filesystem.
* @throws IOException failure during create/init.
*/
public AzureBlobFileSystem createFileSystem() throws IOException {
Preconditions.checkState(abfs == null,
"existing ABFS instance exists: %s", abfs);
abfs = (AzureBlobFileSystem) FileSystem.newInstance(configuration);
return abfs;
}
protected NativeAzureFileSystem getWasbFileSystem() {
return wasb;
}
protected String getHostName() {
return configuration.get(FS_AZURE_TEST_HOST_NAME);
}
protected void setTestUrl(String testUrl) {
this.testUrl = testUrl;
}
protected String getTestUrl() {
return testUrl;
}
protected void setFileSystemName(String fileSystemName) {
this.fileSystemName = fileSystemName;
}
protected String getFileSystemName() {
return fileSystemName;
}
protected String getAccountName() {
return configuration.get(FS_AZURE_TEST_ACCOUNT_NAME);
}
protected String getAccountKey() {
return configuration.get(
FS_AZURE_TEST_ACCOUNT_KEY_PREFIX
+ getAccountName());
}
protected Configuration getConfiguration() {
return configuration;
}
protected boolean isEmulator() {
return isEmulator;
}
/**
* Write a buffer to a file.
* @param path path
* @param buffer buffer
* @throws IOException failure
*/
protected void write(Path path, byte[] buffer) throws IOException {
ContractTestUtils.writeDataset(getFileSystem(), path, buffer, buffer.length,
CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_DEFAULT, false);
}
/**
* Touch a file in the test store. Will overwrite any existing file.
* @param path path
* @throws IOException failure.
*/
protected void touch(Path path) throws IOException {
ContractTestUtils.touch(getFileSystem(), path);
}
protected static String wasbUrlToAbfsUrl(final String wasbUrl) {
return convertTestUrls(
wasbUrl, FileSystemUriSchemes.WASB_SCHEME, FileSystemUriSchemes.WASB_SECURE_SCHEME, FileSystemUriSchemes.WASB_DNS_PREFIX,
FileSystemUriSchemes.ABFS_SCHEME, FileSystemUriSchemes.ABFS_SECURE_SCHEME, FileSystemUriSchemes.ABFS_DNS_PREFIX);
}
protected static String abfsUrlToWasbUrl(final String abfsUrl) {
return convertTestUrls(
abfsUrl, FileSystemUriSchemes.ABFS_SCHEME, FileSystemUriSchemes.ABFS_SECURE_SCHEME, FileSystemUriSchemes.ABFS_DNS_PREFIX,
FileSystemUriSchemes.WASB_SCHEME, FileSystemUriSchemes.WASB_SECURE_SCHEME, FileSystemUriSchemes.WASB_DNS_PREFIX);
}
private static String convertTestUrls(
final String url,
final String fromNonSecureScheme,
final String fromSecureScheme,
final String fromDnsPrefix,
final String toNonSecureScheme,
final String toSecureScheme,
final String toDnsPrefix) {
String data = null;
if (url.startsWith(fromNonSecureScheme + "://")) {
data = url.replace(fromNonSecureScheme + "://", toNonSecureScheme + "://");
} else if (url.startsWith(fromSecureScheme + "://")) {
data = url.replace(fromSecureScheme + "://", toSecureScheme + "://");
}
if (data != null) {
data = data.replace("." + fromDnsPrefix + ".",
"." + toDnsPrefix + ".");
}
return data;
}
public Path getTestPath() {
Path path = new Path(UriUtils.generateUniqueTestPath());
return path;
}
/**
* Create a path under the test path provided by
* {@link #getTestPath()}.
* @param filepath path string in
* @return a path qualified by the test filesystem
* @throws IOException IO problems
*/
protected Path path(String filepath) throws IOException {
return getFileSystem().makeQualified(
new Path(getTestPath(), filepath));
}
}

View File

@ -0,0 +1,53 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.fs.azurebfs;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.fs.azure.integration.AzureTestConstants;
import static org.apache.hadoop.fs.azure.integration.AzureTestUtils.assumeScaleTestsEnabled;
/**
* Integration tests at bigger scale; configurable as to
* size, off by default.
*/
public class AbstractAbfsScaleTest extends AbstractAbfsIntegrationTest {
protected static final Logger LOG =
LoggerFactory.getLogger(AbstractAbfsScaleTest.class);
@Override
protected int getTestTimeoutMillis() {
return AzureTestConstants.SCALE_TEST_TIMEOUT_MILLIS;
}
@Override
public void setup() throws Exception {
super.setup();
LOG.debug("Scale test operation count = {}", getOperationCount());
assumeScaleTestsEnabled(getConfiguration());
}
protected long getOperationCount() {
return getConfiguration().getLong(AzureTestConstants.KEY_OPERATION_COUNT,
AzureTestConstants.DEFAULT_OPERATION_COUNT);
}
}

View File

@ -1,206 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.fs.azurebfs;
import java.net.URI;
import java.util.UUID;
import java.util.concurrent.Callable;
import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException;
import org.junit.After;
import org.junit.Assert;
import org.junit.Before;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.azure.AzureNativeFileSystemStore;
import org.apache.hadoop.fs.azure.NativeAzureFileSystem;
import org.apache.hadoop.fs.azure.metrics.AzureFileSystemInstrumentation;
import org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys;
import org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes;
import org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys;
import static org.apache.hadoop.fs.azurebfs.contracts.services.AzureServiceErrorCode.FILE_SYSTEM_NOT_FOUND;
import static org.apache.hadoop.test.LambdaTestUtils.intercept;
import static org.junit.Assert.assertEquals;
import static org.junit.Assume.assumeNotNull;
/**
* Provide dependencies for AzureBlobFileSystem tests.
*/
public abstract class DependencyInjectedTest {
private final boolean isEmulator;
private NativeAzureFileSystem wasb;
private String abfsScheme;
private Configuration configuration;
private String fileSystemName;
private String accountName;
private String testUrl;
public static final String TEST_CONTAINER_PREFIX = "abfs-testcontainer-";
public DependencyInjectedTest(final boolean secure) {
this(secure ? FileSystemUriSchemes.ABFS_SECURE_SCHEME : FileSystemUriSchemes.ABFS_SCHEME);
}
protected DependencyInjectedTest() {
this(FileSystemUriSchemes.ABFS_SCHEME);
}
private DependencyInjectedTest(final String scheme) {
abfsScheme = scheme;
fileSystemName = TEST_CONTAINER_PREFIX + UUID.randomUUID().toString();
configuration = new Configuration();
configuration.addResource("azure-bfs-test.xml");
assumeNotNull(configuration.get(TestConfigurationKeys.FS_AZURE_TEST_ACCOUNT_NAME));
assumeNotNull(configuration.get(TestConfigurationKeys.FS_AZURE_TEST_ACCOUNT_KEY_PREFIX + configuration.get(TestConfigurationKeys
.FS_AZURE_TEST_ACCOUNT_NAME)));
final String abfsUrl = this.getFileSystemName() + "@" + this.getAccountName();
URI defaultUri = null;
try {
defaultUri = new URI(abfsScheme, abfsUrl, null, null, null);
} catch (Exception ex) {
Assert.fail(ex.getMessage());
}
this.testUrl = defaultUri.toString();
configuration.set(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY, defaultUri.toString());
configuration.setBoolean(ConfigurationKeys.AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION, true);
this.isEmulator = this.configuration.getBoolean(ConfigurationKeys.FS_AZURE_EMULATOR_ENABLED, false);
this.accountName = this.configuration.get(TestConfigurationKeys.FS_AZURE_TEST_ACCOUNT_NAME);
}
@Before
public void initialize() throws Exception {
//Create filesystem first to make sure getWasbFileSystem() can return an existed filesystem.
this.getFileSystem();
if (!this.isEmulator) {
final URI wasbUri = new URI(abfsUrlToWasbUrl(this.getTestUrl()));
final AzureNativeFileSystemStore azureNativeFileSystemStore = new AzureNativeFileSystemStore();
azureNativeFileSystemStore.initialize(
wasbUri,
this.getConfiguration(),
new AzureFileSystemInstrumentation(this.getConfiguration()));
this.wasb = new NativeAzureFileSystem(azureNativeFileSystemStore);
this.wasb.initialize(wasbUri, configuration);
}
}
@After
public void testCleanup() throws Exception {
if (this.wasb != null) {
this.wasb.close();
}
FileSystem.closeAll();
final AzureBlobFileSystem fs = this.getFileSystem();
final AzureBlobFileSystemStore abfsStore = fs.getAbfsStore();
abfsStore.deleteFilesystem();
AbfsRestOperationException ex = intercept(
AbfsRestOperationException.class,
new Callable<Void>() {
@Override
public Void call() throws Exception {
fs.getAbfsStore().getFilesystemProperties();
return null;
}
});
assertEquals(FILE_SYSTEM_NOT_FOUND.getStatusCode(), ex.getStatusCode());
}
public AzureBlobFileSystem getFileSystem() throws Exception {
return (AzureBlobFileSystem) FileSystem.get(this.configuration);
}
protected NativeAzureFileSystem getWasbFileSystem() {
return this.wasb;
}
protected String getHostName() {
return configuration.get(TestConfigurationKeys.FS_AZURE_TEST_HOST_NAME);
}
protected void updateTestUrl(String testUrl) {
this.testUrl = testUrl;
}
protected String getTestUrl() {
return testUrl;
}
protected void updateFileSystemName(String fileSystemName) {
this.fileSystemName = fileSystemName;
}
protected String getFileSystemName() {
return fileSystemName;
}
protected String getAccountName() {
return configuration.get(TestConfigurationKeys.FS_AZURE_TEST_ACCOUNT_NAME);
}
protected String getAccountKey() {
return configuration.get(
TestConfigurationKeys.FS_AZURE_TEST_ACCOUNT_KEY_PREFIX
+ getAccountName());
}
protected Configuration getConfiguration() {
return this.configuration;
}
protected boolean isEmulator() {
return isEmulator;
}
protected static String wasbUrlToAbfsUrl(final String wasbUrl) {
return convertTestUrls(
wasbUrl, FileSystemUriSchemes.WASB_SCHEME, FileSystemUriSchemes.WASB_SECURE_SCHEME, FileSystemUriSchemes.WASB_DNS_PREFIX,
FileSystemUriSchemes.ABFS_SCHEME, FileSystemUriSchemes.ABFS_SECURE_SCHEME, FileSystemUriSchemes.ABFS_DNS_PREFIX);
}
protected static String abfsUrlToWasbUrl(final String abfsUrl) {
return convertTestUrls(
abfsUrl, FileSystemUriSchemes.ABFS_SCHEME, FileSystemUriSchemes.ABFS_SECURE_SCHEME, FileSystemUriSchemes.ABFS_DNS_PREFIX,
FileSystemUriSchemes.WASB_SCHEME, FileSystemUriSchemes.WASB_SECURE_SCHEME, FileSystemUriSchemes.WASB_DNS_PREFIX);
}
private static String convertTestUrls(
final String url, final String fromNonSecureScheme, final String fromSecureScheme, final String fromDnsPrefix,
final String toNonSecureScheme, final String toSecureScheme, final String toDnsPrefix) {
String data = null;
if (url.startsWith(fromNonSecureScheme + "://")) {
data = url.replace(fromNonSecureScheme + "://", toNonSecureScheme + "://");
} else if (url.startsWith(fromSecureScheme + "://")) {
data = url.replace(fromSecureScheme + "://", toSecureScheme + "://");
}
data = data.replace("." + fromDnsPrefix + ".", "." + toDnsPrefix + ".");
return data;
}
}

View File

@ -25,13 +25,13 @@ import org.junit.Test;
import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.contract.ContractTestUtils;
import static org.junit.Assert.assertEquals;
/** /**
* Test append operations. * Test append operations.
*/ */
public class ITestAzureBlobFileSystemAppend extends DependencyInjectedTest { public class ITestAzureBlobFileSystemAppend extends
AbstractAbfsIntegrationTest {
private static final Path TEST_FILE_PATH = new Path("testfile"); private static final Path TEST_FILE_PATH = new Path("testfile");
private static final Path TEST_FOLDER_PATH = new Path("testFolder"); private static final Path TEST_FOLDER_PATH = new Path("testFolder");
public ITestAzureBlobFileSystemAppend() { public ITestAzureBlobFileSystemAppend() {
@ -40,7 +40,7 @@ public class ITestAzureBlobFileSystemAppend extends DependencyInjectedTest {
@Test(expected = FileNotFoundException.class) @Test(expected = FileNotFoundException.class)
public void testAppendDirShouldFail() throws Exception { public void testAppendDirShouldFail() throws Exception {
final AzureBlobFileSystem fs = this.getFileSystem(); final AzureBlobFileSystem fs = getFileSystem();
final Path filePath = TEST_FILE_PATH; final Path filePath = TEST_FILE_PATH;
fs.mkdirs(filePath); fs.mkdirs(filePath);
fs.append(filePath, 0); fs.append(filePath, 0);
@ -48,21 +48,21 @@ public class ITestAzureBlobFileSystemAppend extends DependencyInjectedTest {
@Test @Test
public void testAppendWithLength0() throws Exception { public void testAppendWithLength0() throws Exception {
final AzureBlobFileSystem fs = this.getFileSystem(); final AzureBlobFileSystem fs = getFileSystem();
FSDataOutputStream stream = fs.create(TEST_FILE_PATH); try(FSDataOutputStream stream = fs.create(TEST_FILE_PATH)) {
final byte[] b = new byte[1024]; final byte[] b = new byte[1024];
new Random().nextBytes(b); new Random().nextBytes(b);
stream.write(b, 1000, 0); stream.write(b, 1000, 0);
assertEquals(0, stream.getPos()); assertEquals(0, stream.getPos());
} }
}
@Test(expected = FileNotFoundException.class) @Test(expected = FileNotFoundException.class)
public void testAppendFileAfterDelete() throws Exception { public void testAppendFileAfterDelete() throws Exception {
final AzureBlobFileSystem fs = this.getFileSystem(); final AzureBlobFileSystem fs = getFileSystem();
final Path filePath = TEST_FILE_PATH; final Path filePath = TEST_FILE_PATH;
fs.create(filePath); ContractTestUtils.touch(fs, filePath);
fs.delete(filePath, false); fs.delete(filePath, false);
fs.append(filePath); fs.append(filePath);
@ -70,7 +70,7 @@ public class ITestAzureBlobFileSystemAppend extends DependencyInjectedTest {
@Test(expected = FileNotFoundException.class) @Test(expected = FileNotFoundException.class)
public void testAppendDirectory() throws Exception { public void testAppendDirectory() throws Exception {
final AzureBlobFileSystem fs = this.getFileSystem(); final AzureBlobFileSystem fs = getFileSystem();
final Path folderPath = TEST_FOLDER_PATH; final Path folderPath = TEST_FOLDER_PATH;
fs.mkdirs(folderPath); fs.mkdirs(folderPath);
fs.append(folderPath); fs.append(folderPath);

View File

@ -27,13 +27,11 @@ import org.junit.Test;
import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue;
/** /**
* Test AzureBlobFileSystem back compatibility with WASB. * Test AzureBlobFileSystem back compatibility with WASB.
*/ */
public class ITestAzureBlobFileSystemBackCompat extends DependencyInjectedTest { public class ITestAzureBlobFileSystemBackCompat extends
AbstractAbfsIntegrationTest {
public ITestAzureBlobFileSystemBackCompat() { public ITestAzureBlobFileSystemBackCompat() {
super(); super();
} }
@ -54,13 +52,13 @@ public class ITestAzureBlobFileSystemBackCompat extends DependencyInjectedTest {
blockBlob.uploadText(""); blockBlob.uploadText("");
FileStatus[] fileStatuses = fs.listStatus(new Path("/test/10/")); FileStatus[] fileStatuses = fs.listStatus(new Path("/test/10/"));
assertEquals(fileStatuses.length, 2); assertEquals(2, fileStatuses.length);
assertEquals(fileStatuses[0].getPath().getName(), "10"); assertEquals("10", fileStatuses[0].getPath().getName());
assertTrue(fileStatuses[0].isDirectory()); assertTrue(fileStatuses[0].isDirectory());
assertEquals(fileStatuses[0].getLen(), 0); assertEquals(0, fileStatuses[0].getLen());
assertEquals(fileStatuses[1].getPath().getName(), "123"); assertEquals("123", fileStatuses[1].getPath().getName());
assertTrue(fileStatuses[1].isDirectory()); assertTrue(fileStatuses[1].isDirectory());
assertEquals(fileStatuses[1].getLen(), 0); assertEquals(0, fileStatuses[1].getLen());
} }
private String getBlobConnectionString() { private String getBlobConnectionString() {

View File

@ -33,30 +33,29 @@ import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.FileUtil; import org.apache.hadoop.fs.FileUtil;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import static org.junit.Assert.assertEquals; import static org.apache.hadoop.fs.contract.ContractTestUtils.assertIsFile;
import static org.junit.Assert.assertTrue;
/** /**
* Test copy operation. * Test copy operation.
*/ */
public class ITestAzureBlobFileSystemCopy extends DependencyInjectedTest { public class ITestAzureBlobFileSystemCopy extends AbstractAbfsIntegrationTest {
public ITestAzureBlobFileSystemCopy() { public ITestAzureBlobFileSystemCopy() {
super(); super();
} }
@Test @Test
public void testCopyFromLocalFileSystem() throws Exception { public void testCopyFromLocalFileSystem() throws Exception {
final AzureBlobFileSystem fs = this.getFileSystem(); final AzureBlobFileSystem fs = getFileSystem();
Path localFilePath = new Path(System.getProperty("test.build.data", Path localFilePath = new Path(System.getProperty("test.build.data",
"azure_test")); "azure_test"));
FileSystem localFs = FileSystem.get(new Configuration()); FileSystem localFs = FileSystem.getLocal(new Configuration());
localFs.delete(localFilePath, true); localFs.delete(localFilePath, true);
try { try {
writeString(localFs, localFilePath, "Testing"); writeString(localFs, localFilePath, "Testing");
Path dstPath = new Path("copiedFromLocal"); Path dstPath = new Path("copiedFromLocal");
assertTrue(FileUtil.copy(localFs, localFilePath, fs, dstPath, false, assertTrue(FileUtil.copy(localFs, localFilePath, fs, dstPath, false,
fs.getConf())); fs.getConf()));
assertTrue(fs.exists(dstPath)); assertIsFile(fs, dstPath);
assertEquals("Testing", readString(fs, dstPath)); assertEquals("Testing", readString(fs, dstPath));
fs.delete(dstPath, true); fs.delete(dstPath, true);
} finally { } finally {
@ -65,36 +64,32 @@ public class ITestAzureBlobFileSystemCopy extends DependencyInjectedTest {
} }
private String readString(FileSystem fs, Path testFile) throws IOException { private String readString(FileSystem fs, Path testFile) throws IOException {
FSDataInputStream inputStream = fs.open(testFile); return readString(fs.open(testFile));
String ret = readString(inputStream);
inputStream.close();
return ret;
} }
private String readString(FSDataInputStream inputStream) throws IOException { private String readString(FSDataInputStream inputStream) throws IOException {
BufferedReader reader = new BufferedReader(new InputStreamReader( try (BufferedReader reader = new BufferedReader(new InputStreamReader(
inputStream)); inputStream))) {
final int bufferSize = 1024; final int bufferSize = 1024;
char[] buffer = new char[bufferSize]; char[] buffer = new char[bufferSize];
int count = reader.read(buffer, 0, bufferSize); int count = reader.read(buffer, 0, bufferSize);
if (count > bufferSize) { if (count > bufferSize) {
throw new IOException("Exceeded buffer size"); throw new IOException("Exceeded buffer size");
} }
inputStream.close();
return new String(buffer, 0, count); return new String(buffer, 0, count);
} }
}
private void writeString(FileSystem fs, Path path, String value) private void writeString(FileSystem fs, Path path, String value)
throws IOException { throws IOException {
FSDataOutputStream outputStream = fs.create(path, true); writeString(fs.create(path, true), value);
writeString(outputStream, value);
} }
private void writeString(FSDataOutputStream outputStream, String value) private void writeString(FSDataOutputStream outputStream, String value)
throws IOException { throws IOException {
BufferedWriter writer = new BufferedWriter(new OutputStreamWriter( try(BufferedWriter writer = new BufferedWriter(
outputStream)); new OutputStreamWriter(outputStream))) {
writer.write(value); writer.write(value);
writer.close(); }
} }
} }

View File

@ -24,18 +24,17 @@ import java.util.EnumSet;
import org.junit.Test; import org.junit.Test;
import org.apache.hadoop.fs.CreateFlag; import org.apache.hadoop.fs.CreateFlag;
import org.apache.hadoop.fs.FileAlreadyExistsException; import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.permission.FsPermission; import org.apache.hadoop.fs.permission.FsPermission;
import static org.junit.Assert.assertNotNull; import static org.apache.hadoop.fs.contract.ContractTestUtils.assertIsFile;
import static org.junit.Assert.assertTrue;
/** /**
* Test create operation. * Test create operation.
*/ */
public class ITestAzureBlobFileSystemCreate extends DependencyInjectedTest { public class ITestAzureBlobFileSystemCreate extends
AbstractAbfsIntegrationTest {
private static final Path TEST_FILE_PATH = new Path("testfile"); private static final Path TEST_FILE_PATH = new Path("testfile");
private static final Path TEST_FOLDER_PATH = new Path("testFolder"); private static final Path TEST_FOLDER_PATH = new Path("testFolder");
private static final String TEST_CHILD_FILE = "childFile"; private static final String TEST_CHILD_FILE = "childFile";
@ -43,68 +42,65 @@ public class ITestAzureBlobFileSystemCreate extends DependencyInjectedTest {
super(); super();
} }
@Test(expected = FileAlreadyExistsException.class)
public void testCreateFileWithExistingDir() throws Exception {
final AzureBlobFileSystem fs = this.getFileSystem();
fs.mkdirs(TEST_FOLDER_PATH);
fs.create(TEST_FOLDER_PATH);
}
@Test @Test
public void testEnsureFileCreated() throws Exception { public void testEnsureFileCreatedImmediately() throws Exception {
final AzureBlobFileSystem fs = this.getFileSystem(); final AzureBlobFileSystem fs = getFileSystem();
fs.create(TEST_FILE_PATH); FSDataOutputStream out = fs.create(TEST_FILE_PATH);
try {
FileStatus fileStatus = fs.getFileStatus(TEST_FILE_PATH); assertIsFile(fs, TEST_FILE_PATH);
assertNotNull(fileStatus); } finally {
out.close();
}
assertIsFile(fs, TEST_FILE_PATH);
} }
@Test @Test
@SuppressWarnings("deprecation") @SuppressWarnings("deprecation")
public void testCreateNonRecursive() throws Exception { public void testCreateNonRecursive() throws Exception {
final AzureBlobFileSystem fs = this.getFileSystem(); final AzureBlobFileSystem fs = getFileSystem();
Path testFile = new Path(TEST_FOLDER_PATH, TEST_CHILD_FILE); Path testFile = new Path(TEST_FOLDER_PATH, TEST_CHILD_FILE);
try { try {
fs.createNonRecursive(testFile, true, 1024, (short) 1, 1024, null); fs.createNonRecursive(testFile, true, 1024, (short) 1, 1024, null);
assertTrue("Should've thrown", false); fail("Should've thrown");
} catch (FileNotFoundException e) { } catch (FileNotFoundException expected) {
} }
fs.mkdirs(TEST_FOLDER_PATH); fs.mkdirs(TEST_FOLDER_PATH);
fs.createNonRecursive(testFile, true, 1024, (short) 1, 1024, null) fs.createNonRecursive(testFile, true, 1024, (short) 1, 1024, null)
.close(); .close();
assertTrue(fs.exists(testFile)); assertIsFile(fs, testFile);
} }
@Test @Test
@SuppressWarnings("deprecation") @SuppressWarnings("deprecation")
public void testCreateNonRecursive1() throws Exception { public void testCreateNonRecursive1() throws Exception {
final AzureBlobFileSystem fs = this.getFileSystem(); final AzureBlobFileSystem fs = getFileSystem();
Path testFile = new Path(TEST_FOLDER_PATH, TEST_CHILD_FILE); Path testFile = new Path(TEST_FOLDER_PATH, TEST_CHILD_FILE);
try { try {
fs.createNonRecursive(testFile, FsPermission.getDefault(), EnumSet.of(CreateFlag.CREATE, CreateFlag.OVERWRITE), 1024, (short) 1, 1024, null); fs.createNonRecursive(testFile, FsPermission.getDefault(), EnumSet.of(CreateFlag.CREATE, CreateFlag.OVERWRITE), 1024, (short) 1, 1024, null);
assertTrue("Should've thrown", false); fail("Should've thrown");
} catch (FileNotFoundException e) { } catch (FileNotFoundException expected) {
} }
fs.mkdirs(TEST_FOLDER_PATH); fs.mkdirs(TEST_FOLDER_PATH);
fs.createNonRecursive(testFile, true, 1024, (short) 1, 1024, null) fs.createNonRecursive(testFile, true, 1024, (short) 1, 1024, null)
.close(); .close();
assertTrue(fs.exists(testFile)); assertIsFile(fs, testFile);
} }
@Test @Test
@SuppressWarnings("deprecation") @SuppressWarnings("deprecation")
public void testCreateNonRecursive2() throws Exception { public void testCreateNonRecursive2() throws Exception {
final AzureBlobFileSystem fs = this.getFileSystem(); final AzureBlobFileSystem fs = getFileSystem();
Path testFile = new Path(TEST_FOLDER_PATH, TEST_CHILD_FILE); Path testFile = new Path(TEST_FOLDER_PATH, TEST_CHILD_FILE);
try { try {
fs.createNonRecursive(testFile, FsPermission.getDefault(), false, 1024, (short) 1, 1024, null); fs.createNonRecursive(testFile, FsPermission.getDefault(), false, 1024, (short) 1, 1024, null);
assertTrue("Should've thrown", false); fail("Should've thrown");
} catch (FileNotFoundException e) { } catch (FileNotFoundException e) {
} }
fs.mkdirs(TEST_FOLDER_PATH); fs.mkdirs(TEST_FOLDER_PATH);
fs.createNonRecursive(testFile, true, 1024, (short) 1, 1024, null) fs.createNonRecursive(testFile, true, 1024, (short) 1, 1024, null)
.close(); .close();
assertTrue(fs.exists(testFile)); assertIsFile(fs, testFile);
} }
} }

View File

@ -28,71 +28,79 @@ import java.util.concurrent.Future;
import org.junit.Test; import org.junit.Test;
import org.apache.hadoop.fs.FileAlreadyExistsException;
import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import static org.junit.Assert.assertEquals; import static org.apache.hadoop.fs.contract.ContractTestUtils.assertDeleted;
import static org.apache.hadoop.fs.contract.ContractTestUtils.assertPathDoesNotExist;
import static org.apache.hadoop.test.LambdaTestUtils.intercept;
/** /**
* Test delete operation. * Test delete operation.
*/ */
public class ITestAzureBlobFileSystemDelete extends DependencyInjectedTest { public class ITestAzureBlobFileSystemDelete extends
AbstractAbfsIntegrationTest {
public ITestAzureBlobFileSystemDelete() { public ITestAzureBlobFileSystemDelete() {
super(); super();
} }
@Test @Test
public void testDeleteRoot() throws Exception { public void testDeleteRoot() throws Exception {
final AzureBlobFileSystem fs = this.getFileSystem(); final AzureBlobFileSystem fs = getFileSystem();
fs.mkdirs(new Path("/testFolder0")); fs.mkdirs(new Path("/testFolder0"));
fs.mkdirs(new Path("/testFolder1")); fs.mkdirs(new Path("/testFolder1"));
fs.mkdirs(new Path("/testFolder2")); fs.mkdirs(new Path("/testFolder2"));
fs.create(new Path("/testFolder1/testfile")); touch(new Path("/testFolder1/testfile"));
fs.create(new Path("/testFolder1/testfile2")); touch(new Path("/testFolder1/testfile2"));
fs.create(new Path("/testFolder1/testfile3")); touch(new Path("/testFolder1/testfile3"));
FileStatus[] ls = fs.listStatus(new Path("/")); Path root = new Path("/");
assertEquals(4, ls.length); // and user dir FileStatus[] ls = fs.listStatus(root);
assertEquals(3, ls.length);
fs.delete(new Path("/"), true); fs.delete(root, true);
ls = fs.listStatus(new Path("/")); ls = fs.listStatus(root);
assertEquals(0, ls.length); assertEquals("listing size", 0, ls.length);
} }
@Test(expected = FileNotFoundException.class) @Test()
public void testOpenFileAfterDelete() throws Exception { public void testOpenFileAfterDelete() throws Exception {
final AzureBlobFileSystem fs = this.getFileSystem(); final AzureBlobFileSystem fs = getFileSystem();
fs.create(new Path("/testFile")); Path testfile = new Path("/testFile");
fs.delete(new Path("/testFile"), false); touch(testfile);
assertDeleted(fs, testfile, false);
fs.open(new Path("/testFile")); intercept(FileNotFoundException.class,
() -> fs.open(testfile));
} }
@Test(expected = FileNotFoundException.class) @Test
public void testEnsureFileIsDeleted() throws Exception { public void testEnsureFileIsDeleted() throws Exception {
final AzureBlobFileSystem fs = this.getFileSystem(); final AzureBlobFileSystem fs = getFileSystem();
fs.create(new Path("testfile")); Path testfile = new Path("testfile");
fs.delete(new Path("testfile"), false); touch(testfile);
assertDeleted(fs, testfile, false);
fs.getFileStatus(new Path("testfile")); assertPathDoesNotExist(fs, "deleted", testfile);
} }
@Test(expected = FileNotFoundException.class) @Test
public void testDeleteDirectory() throws Exception { public void testDeleteDirectory() throws Exception {
final AzureBlobFileSystem fs = this.getFileSystem(); final AzureBlobFileSystem fs = getFileSystem();
fs.mkdirs(new Path("testfile")); Path dir = new Path("testfile");
fs.mkdirs(dir);
fs.mkdirs(new Path("testfile/test1")); fs.mkdirs(new Path("testfile/test1"));
fs.mkdirs(new Path("testfile/test1/test2")); fs.mkdirs(new Path("testfile/test1/test2"));
fs.delete(new Path("testfile"), true); assertDeleted(fs, dir, true);
fs.getFileStatus(new Path("testfile")); assertPathDoesNotExist(fs, "deleted", dir);
} }
@Test(expected = FileNotFoundException.class) @Test
public void testDeleteFirstLevelDirectory() throws Exception { public void testDeleteFirstLevelDirectory() throws Exception {
final AzureBlobFileSystem fs = this.getFileSystem(); final AzureBlobFileSystem fs = getFileSystem();
final List<Future> tasks = new ArrayList<>(); final List<Future<Void>> tasks = new ArrayList<>();
ExecutorService es = Executors.newFixedThreadPool(10); ExecutorService es = Executors.newFixedThreadPool(10);
for (int i = 0; i < 1000; i++) { for (int i = 0; i < 1000; i++) {
@ -100,7 +108,7 @@ public class ITestAzureBlobFileSystemDelete extends DependencyInjectedTest {
Callable<Void> callable = new Callable<Void>() { Callable<Void> callable = new Callable<Void>() {
@Override @Override
public Void call() throws Exception { public Void call() throws Exception {
fs.create(fileName); touch(fileName);
return null; return null;
} }
}; };
@ -113,7 +121,12 @@ public class ITestAzureBlobFileSystemDelete extends DependencyInjectedTest {
} }
es.shutdownNow(); es.shutdownNow();
fs.delete(new Path("/test"), true); Path dir = new Path("/test");
fs.getFileStatus(new Path("/test")); // first try a non-recursive delete, expect failure
intercept(FileAlreadyExistsException.class,
() -> fs.delete(dir, false));
assertDeleted(fs, dir, true);
assertPathDoesNotExist(fs, "deleted", dir);
} }
} }

View File

@ -38,7 +38,7 @@ import static org.junit.Assert.assertArrayEquals;
/** /**
* Test end to end between ABFS client and ABFS server. * Test end to end between ABFS client and ABFS server.
*/ */
public class ITestAzureBlobFileSystemE2E extends DependencyInjectedTest { public class ITestAzureBlobFileSystemE2E extends AbstractAbfsIntegrationTest {
private static final Path TEST_FILE = new Path("testfile"); private static final Path TEST_FILE = new Path("testfile");
private static final int TEST_BYTE = 100; private static final int TEST_BYTE = 100;
private static final int TEST_OFFSET = 100; private static final int TEST_OFFSET = 100;
@ -53,11 +53,11 @@ public class ITestAzureBlobFileSystemE2E extends DependencyInjectedTest {
@Test @Test
public void testWriteOneByteToFile() throws Exception { public void testWriteOneByteToFile() throws Exception {
final AzureBlobFileSystem fs = this.getFileSystem(); final AzureBlobFileSystem fs = getFileSystem();
FSDataOutputStream stream = fs.create(TEST_FILE);
try(FSDataOutputStream stream = fs.create(TEST_FILE)) {
stream.write(TEST_BYTE); stream.write(TEST_BYTE);
stream.close(); }
FileStatus fileStatus = fs.getFileStatus(TEST_FILE); FileStatus fileStatus = fs.getFileStatus(TEST_FILE);
assertEquals(1, fileStatus.getLen()); assertEquals(1, fileStatus.getLen());
@ -65,52 +65,52 @@ public class ITestAzureBlobFileSystemE2E extends DependencyInjectedTest {
@Test @Test
public void testReadWriteBytesToFile() throws Exception { public void testReadWriteBytesToFile() throws Exception {
final AzureBlobFileSystem fs = this.getFileSystem(); final AzureBlobFileSystem fs = getFileSystem();
testWriteOneByteToFile(); testWriteOneByteToFile();
FSDataInputStream inputStream = fs.open(TEST_FILE, TEST_DEFAULT_BUFFER_SIZE); try(FSDataInputStream inputStream = fs.open(TEST_FILE,
int i = inputStream.read(); TEST_DEFAULT_BUFFER_SIZE)) {
inputStream.close(); assertEquals(TEST_BYTE, inputStream.read());
}
assertEquals(TEST_BYTE, i);
} }
@Test (expected = IOException.class) @Test (expected = IOException.class)
public void testOOBWrites() throws Exception { public void testOOBWrites() throws Exception {
final AzureBlobFileSystem fs = this.getFileSystem(); final AzureBlobFileSystem fs = getFileSystem();
int readBufferSize = fs.getAbfsStore().getAbfsConfiguration().getReadBufferSize(); int readBufferSize = fs.getAbfsStore().getAbfsConfiguration().getReadBufferSize();
fs.create(TEST_FILE);
FSDataOutputStream writeStream = fs.create(TEST_FILE);
byte[] bytesToRead = new byte[readBufferSize]; byte[] bytesToRead = new byte[readBufferSize];
final byte[] b = new byte[2 * readBufferSize]; final byte[] b = new byte[2 * readBufferSize];
new Random().nextBytes(b); new Random().nextBytes(b);
try(FSDataOutputStream writeStream = fs.create(TEST_FILE)) {
writeStream.write(b); writeStream.write(b);
writeStream.flush(); writeStream.flush();
writeStream.close(); }
FSDataInputStream readStream = fs.open(TEST_FILE); try (FSDataInputStream readStream = fs.open(TEST_FILE)) {
readStream.read(bytesToRead, 0, readBufferSize); assertEquals(readBufferSize,
readStream.read(bytesToRead, 0, readBufferSize));
writeStream = fs.create(TEST_FILE); try (FSDataOutputStream writeStream = fs.create(TEST_FILE)) {
writeStream.write(b); writeStream.write(b);
writeStream.flush(); writeStream.flush();
writeStream.close(); }
readStream.read(bytesToRead, 0, readBufferSize); assertEquals(readBufferSize,
readStream.close(); readStream.read(bytesToRead, 0, readBufferSize));
}
} }
@Test @Test
public void testWriteWithBufferOffset() throws Exception { public void testWriteWithBufferOffset() throws Exception {
final AzureBlobFileSystem fs = this.getFileSystem(); final AzureBlobFileSystem fs = getFileSystem();
final FSDataOutputStream stream = fs.create(TEST_FILE);
final byte[] b = new byte[1024 * 1000]; final byte[] b = new byte[1024 * 1000];
new Random().nextBytes(b); new Random().nextBytes(b);
try(final FSDataOutputStream stream = fs.create(TEST_FILE)) {
stream.write(b, TEST_OFFSET, b.length - TEST_OFFSET); stream.write(b, TEST_OFFSET, b.length - TEST_OFFSET);
stream.close(); }
final byte[] r = new byte[TEST_DEFAULT_READ_BUFFER_SIZE]; final byte[] r = new byte[TEST_DEFAULT_READ_BUFFER_SIZE];
FSDataInputStream inputStream = fs.open(TEST_FILE, TEST_DEFAULT_BUFFER_SIZE); FSDataInputStream inputStream = fs.open(TEST_FILE, TEST_DEFAULT_BUFFER_SIZE);
@ -124,13 +124,11 @@ public class ITestAzureBlobFileSystemE2E extends DependencyInjectedTest {
@Test @Test
public void testReadWriteHeavyBytesToFileWithSmallerChunks() throws Exception { public void testReadWriteHeavyBytesToFileWithSmallerChunks() throws Exception {
final AzureBlobFileSystem fs = this.getFileSystem(); final AzureBlobFileSystem fs = getFileSystem();
final FSDataOutputStream stream = fs.create(TEST_FILE);
final byte[] writeBuffer = new byte[5 * 1000 * 1024]; final byte[] writeBuffer = new byte[5 * 1000 * 1024];
new Random().nextBytes(writeBuffer); new Random().nextBytes(writeBuffer);
stream.write(writeBuffer); write(TEST_FILE, writeBuffer);
stream.close();
final byte[] readBuffer = new byte[5 * 1000 * 1024]; final byte[] readBuffer = new byte[5 * 1000 * 1024];
FSDataInputStream inputStream = fs.open(TEST_FILE, TEST_DEFAULT_BUFFER_SIZE); FSDataInputStream inputStream = fs.open(TEST_FILE, TEST_DEFAULT_BUFFER_SIZE);

View File

@ -26,7 +26,6 @@ import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors; import java.util.concurrent.Executors;
import java.util.concurrent.Future; import java.util.concurrent.Future;
import org.junit.Assert;
import org.junit.Test; import org.junit.Test;
import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FSDataInputStream;
@ -35,28 +34,24 @@ import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import static org.junit.Assert.assertArrayEquals;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertNotEquals;
/** /**
* Test end to end between ABFS client and ABFS server with heavy traffic. * Test end to end between ABFS client and ABFS server with heavy traffic.
*/ */
public class ITestAzureBlobFileSystemE2EScale extends DependencyInjectedTest { public class ITestAzureBlobFileSystemE2EScale extends
AbstractAbfsScaleTest {
private static final int TEN = 10; private static final int TEN = 10;
private static final int ONE_THOUSAND = 1000; private static final int ONE_THOUSAND = 1000;
private static final int BASE_SIZE = 1024; private static final int BASE_SIZE = 1024;
private static final int ONE_MB = 1024 * 1024; private static final int ONE_MB = 1024 * 1024;
private static final int DEFAULT_WRITE_TIMES = 100; private static final int DEFAULT_WRITE_TIMES = 100;
private static final Path TEST_FILE = new Path("testfile"); private static final Path TEST_FILE = new Path("ITestAzureBlobFileSystemE2EScale");
public ITestAzureBlobFileSystemE2EScale() { public ITestAzureBlobFileSystemE2EScale() {
super();
} }
@Test @Test
public void testWriteHeavyBytesToFile() throws Exception { public void testWriteHeavyBytesToFileAcrossThreads() throws Exception {
final AzureBlobFileSystem fs = this.getFileSystem(); final AzureBlobFileSystem fs = getFileSystem();
final FSDataOutputStream stream = fs.create(TEST_FILE); final FSDataOutputStream stream = fs.create(TEST_FILE);
ExecutorService es = Executors.newFixedThreadPool(TEN); ExecutorService es = Executors.newFixedThreadPool(TEN);
@ -65,7 +60,8 @@ public class ITestAzureBlobFileSystemE2EScale extends DependencyInjectedTest {
new Random().nextBytes(b); new Random().nextBytes(b);
List<Future<Void>> tasks = new ArrayList<>(); List<Future<Void>> tasks = new ArrayList<>();
for (int i = 0; i < DEFAULT_WRITE_TIMES; i++) { int operationCount = DEFAULT_WRITE_TIMES;
for (int i = 0; i < operationCount; i++) {
Callable<Void> callable = new Callable<Void>() { Callable<Void> callable = new Callable<Void>() {
@Override @Override
public Void call() throws Exception { public Void call() throws Exception {
@ -86,48 +82,38 @@ public class ITestAzureBlobFileSystemE2EScale extends DependencyInjectedTest {
es.shutdownNow(); es.shutdownNow();
FileStatus fileStatus = fs.getFileStatus(TEST_FILE); FileStatus fileStatus = fs.getFileStatus(TEST_FILE);
assertEquals(testWriteBufferSize * DEFAULT_WRITE_TIMES, fileStatus.getLen()); assertEquals(testWriteBufferSize * operationCount, fileStatus.getLen());
}
@Test
public void testReadWriteHeavyBytesToFile() throws Exception {
final AzureBlobFileSystem fs = this.getFileSystem();
final FSDataOutputStream stream = fs.create(TEST_FILE);
int testBufferSize = 5 * TEN * ONE_THOUSAND * BASE_SIZE;
final byte[] b = new byte[testBufferSize];
new Random().nextBytes(b);
stream.write(b);
stream.close();
final byte[] r = new byte[testBufferSize];
FSDataInputStream inputStream = fs.open(TEST_FILE, 4 * ONE_MB);
int result = inputStream.read(r);
inputStream.close();
assertNotEquals(-1, result);
assertArrayEquals(r, b);
} }
@Test @Test
public void testReadWriteHeavyBytesToFileWithStatistics() throws Exception { public void testReadWriteHeavyBytesToFileWithStatistics() throws Exception {
final AzureBlobFileSystem fs = this.getFileSystem(); final AzureBlobFileSystem fs = getFileSystem();
final FSDataOutputStream stream = fs.create(TEST_FILE); final FileSystem.Statistics abfsStatistics;
final FileSystem.Statistics abfsStatistics = fs.getFsStatistics(); int testBufferSize;
final byte[] sourceData;
try(final FSDataOutputStream stream = fs.create(TEST_FILE)) {
abfsStatistics = fs.getFsStatistics();
abfsStatistics.reset(); abfsStatistics.reset();
int testBufferSize = 5 * TEN * ONE_THOUSAND * BASE_SIZE; testBufferSize = 5 * TEN * ONE_THOUSAND * BASE_SIZE;
final byte[] b = new byte[testBufferSize]; sourceData = new byte[testBufferSize];
new Random().nextBytes(b); new Random().nextBytes(sourceData);
stream.write(b); stream.write(sourceData);
stream.close(); }
final byte[] r = new byte[testBufferSize]; final byte[] remoteData = new byte[testBufferSize];
FSDataInputStream inputStream = fs.open(TEST_FILE, 4 * ONE_MB); int bytesRead;
inputStream.read(r); try (FSDataInputStream inputStream = fs.open(TEST_FILE, 4 * ONE_MB)) {
inputStream.close(); bytesRead = inputStream.read(remoteData);
}
String stats = abfsStatistics.toString();
assertEquals("Bytes read in " + stats,
remoteData.length, abfsStatistics.getBytesRead());
assertEquals("bytes written in " + stats,
sourceData.length, abfsStatistics.getBytesWritten());
assertEquals("bytesRead from read() call", testBufferSize, bytesRead );
assertArrayEquals("round tripped data", sourceData, remoteData);
Assert.assertEquals(r.length, abfsStatistics.getBytesRead());
Assert.assertEquals(b.length, abfsStatistics.getBytesWritten());
} }
} }

View File

@ -18,6 +18,8 @@
package org.apache.hadoop.fs.azurebfs; package org.apache.hadoop.fs.azurebfs;
import java.io.IOException;
import org.junit.Test; import org.junit.Test;
import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileStatus;
@ -25,12 +27,11 @@ import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.permission.FsAction; import org.apache.hadoop.fs.permission.FsAction;
import org.apache.hadoop.fs.permission.FsPermission; import org.apache.hadoop.fs.permission.FsPermission;
import static org.junit.Assert.assertEquals;
/** /**
* Test FileStatus. * Test FileStatus.
*/ */
public class ITestAzureBlobFileSystemFileStatus extends DependencyInjectedTest { public class ITestAzureBlobFileSystemFileStatus extends
AbstractAbfsIntegrationTest {
private static final Path TEST_FILE = new Path("testFile"); private static final Path TEST_FILE = new Path("testFile");
private static final Path TEST_FOLDER = new Path("testDir"); private static final Path TEST_FOLDER = new Path("testDir");
public ITestAzureBlobFileSystemFileStatus() { public ITestAzureBlobFileSystemFileStatus() {
@ -41,24 +42,38 @@ public class ITestAzureBlobFileSystemFileStatus extends DependencyInjectedTest {
public void testEnsureStatusWorksForRoot() throws Exception { public void testEnsureStatusWorksForRoot() throws Exception {
final AzureBlobFileSystem fs = this.getFileSystem(); final AzureBlobFileSystem fs = this.getFileSystem();
fs.getFileStatus(new Path("/")); Path root = new Path("/");
fs.listStatus(new Path("/")); FileStatus[] rootls = fs.listStatus(root);
assertEquals("root listing", 0, rootls.length);
} }
@Test @Test
public void testFileStatusPermissionsAndOwnerAndGroup() throws Exception { public void testFileStatusPermissionsAndOwnerAndGroup() throws Exception {
final AzureBlobFileSystem fs = this.getFileSystem(); final AzureBlobFileSystem fs = this.getFileSystem();
fs.create(TEST_FILE); touch(TEST_FILE);
validateStatus(fs, TEST_FILE);
}
private FileStatus validateStatus(final AzureBlobFileSystem fs, final Path name)
throws IOException {
FileStatus fileStatus = fs.getFileStatus(name);
String errorInStatus = "error in " + fileStatus + " from " + fs;
assertEquals(errorInStatus + ": permission",
new FsPermission(FsAction.ALL, FsAction.ALL, FsAction.ALL),
fileStatus.getPermission());
assertEquals(errorInStatus + ": owner",
fs.getOwnerUser(), fileStatus.getOwner());
assertEquals(errorInStatus + ": group",
fs.getOwnerUserPrimaryGroup(), fileStatus.getGroup());
return fileStatus;
}
@Test
public void testFolderStatusPermissionsAndOwnerAndGroup() throws Exception {
final AzureBlobFileSystem fs = this.getFileSystem();
fs.mkdirs(TEST_FOLDER); fs.mkdirs(TEST_FOLDER);
FileStatus fileStatus = fs.getFileStatus(TEST_FILE); validateStatus(fs, TEST_FOLDER);
assertEquals(new FsPermission(FsAction.ALL, FsAction.ALL, FsAction.ALL), fileStatus.getPermission());
assertEquals(fs.getOwnerUser(), fileStatus.getGroup());
assertEquals(fs.getOwnerUserPrimaryGroup(), fileStatus.getOwner());
fileStatus = fs.getFileStatus(TEST_FOLDER);
assertEquals(new FsPermission(FsAction.ALL, FsAction.ALL, FsAction.ALL), fileStatus.getPermission());
assertEquals(fs.getOwnerUser(), fileStatus.getGroup());
assertEquals(fs.getOwnerUserPrimaryGroup(), fileStatus.getOwner());
} }
} }

View File

@ -34,14 +34,10 @@ import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import static org.junit.Assert.assertArrayEquals;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertNotEquals;
/** /**
* Test flush operation. * Test flush operation.
*/ */
public class ITestAzureBlobFileSystemFlush extends DependencyInjectedTest { public class ITestAzureBlobFileSystemFlush extends AbstractAbfsScaleTest {
private static final int BASE_SIZE = 1024; private static final int BASE_SIZE = 1024;
private static final int ONE_THOUSAND = 1000; private static final int ONE_THOUSAND = 1000;
private static final int TEST_BUFFER_SIZE = 5 * ONE_THOUSAND * BASE_SIZE; private static final int TEST_BUFFER_SIZE = 5 * ONE_THOUSAND * BASE_SIZE;
@ -56,11 +52,11 @@ public class ITestAzureBlobFileSystemFlush extends DependencyInjectedTest {
} }
@Test @Test
public void testAbfsOutputStreamAsyncFlushWithRetainUncommitedData() throws Exception { public void testAbfsOutputStreamAsyncFlushWithRetainUncommittedData() throws Exception {
final AzureBlobFileSystem fs = this.getFileSystem(); final AzureBlobFileSystem fs = getFileSystem();
final FSDataOutputStream stream = fs.create(TEST_FILE_PATH); final byte[] b;
try(final FSDataOutputStream stream = fs.create(TEST_FILE_PATH)) {
final byte[] b = new byte[TEST_BUFFER_SIZE]; b = new byte[TEST_BUFFER_SIZE];
new Random().nextBytes(b); new Random().nextBytes(b);
for (int i = 0; i < 2; i++) { for (int i = 0; i < 2; i++) {
@ -71,28 +67,25 @@ public class ITestAzureBlobFileSystemFlush extends DependencyInjectedTest {
Thread.sleep(10); Thread.sleep(10);
} }
} }
}
stream.close();
final byte[] r = new byte[TEST_BUFFER_SIZE]; final byte[] r = new byte[TEST_BUFFER_SIZE];
FSDataInputStream inputStream = fs.open(TEST_FILE_PATH, 4 * ONE_MB); try(FSDataInputStream inputStream = fs.open(TEST_FILE_PATH, 4 * ONE_MB)) {
while (inputStream.available() != 0) { while (inputStream.available() != 0) {
int result = inputStream.read(r); int result = inputStream.read(r);
assertNotEquals(-1, result); assertNotEquals("read returned -1", -1, result);
assertArrayEquals(r, b); assertArrayEquals("buffer read from stream", r, b);
}
} }
inputStream.close();
} }
@Test @Test
public void testAbfsOutputStreamSyncFlush() throws Exception { public void testAbfsOutputStreamSyncFlush() throws Exception {
final AzureBlobFileSystem fs = this.getFileSystem(); final AzureBlobFileSystem fs = getFileSystem();
final FSDataOutputStream stream = fs.create(TEST_FILE_PATH); final byte[] b;
try(final FSDataOutputStream stream = fs.create(TEST_FILE_PATH)) {
final byte[] b = new byte[TEST_BUFFER_SIZE]; b = new byte[TEST_BUFFER_SIZE];
new Random().nextBytes(b); new Random().nextBytes(b);
stream.write(b); stream.write(b);
@ -101,27 +94,28 @@ public class ITestAzureBlobFileSystemFlush extends DependencyInjectedTest {
stream.hflush(); stream.hflush();
Thread.sleep(10); Thread.sleep(10);
} }
stream.close(); }
final byte[] r = new byte[TEST_BUFFER_SIZE]; final byte[] r = new byte[TEST_BUFFER_SIZE];
FSDataInputStream inputStream = fs.open(TEST_FILE_PATH, 4 * ONE_MB); try(FSDataInputStream inputStream = fs.open(TEST_FILE_PATH, 4 * ONE_MB)) {
int result = inputStream.read(r); int result = inputStream.read(r);
assertNotEquals(-1, result); assertNotEquals(-1, result);
assertArrayEquals(r, b); assertArrayEquals(r, b);
}
inputStream.close();
} }
@Test @Test
public void testWriteHeavyBytesToFileSyncFlush() throws Exception { public void testWriteHeavyBytesToFileSyncFlush() throws Exception {
final AzureBlobFileSystem fs = this.getFileSystem(); final AzureBlobFileSystem fs = getFileSystem();
final FSDataOutputStream stream = fs.create(TEST_FILE_PATH); final FileSystem.Statistics abfsStatistics;
final FileSystem.Statistics abfsStatistics = fs.getFsStatistics(); ExecutorService es;
try(final FSDataOutputStream stream = fs.create(TEST_FILE_PATH)) {
abfsStatistics = fs.getFsStatistics();
abfsStatistics.reset(); abfsStatistics.reset();
ExecutorService es = Executors.newFixedThreadPool(10); es = Executors.newFixedThreadPool(10);
final byte[] b = new byte[TEST_BUFFER_SIZE]; final byte[] b = new byte[TEST_BUFFER_SIZE];
new Random().nextBytes(b); new Random().nextBytes(b);
@ -152,20 +146,21 @@ public class ITestAzureBlobFileSystemFlush extends DependencyInjectedTest {
} }
tasks.clear(); tasks.clear();
stream.close(); }
es.shutdownNow(); es.shutdownNow();
FileStatus fileStatus = fs.getFileStatus(TEST_FILE_PATH); FileStatus fileStatus = fs.getFileStatus(TEST_FILE_PATH);
assertEquals((long) TEST_BUFFER_SIZE * FLUSH_TIMES, fileStatus.getLen()); long expectedWrites = (long) TEST_BUFFER_SIZE * FLUSH_TIMES;
assertEquals((long) TEST_BUFFER_SIZE * FLUSH_TIMES, abfsStatistics.getBytesWritten()); assertEquals("Wrong file length in " + fileStatus, expectedWrites, fileStatus.getLen());
assertEquals("wrong bytes Written count in " + abfsStatistics,
expectedWrites, abfsStatistics.getBytesWritten());
} }
@Test @Test
public void testWriteHeavyBytesToFileAsyncFlush() throws Exception { public void testWriteHeavyBytesToFileAsyncFlush() throws Exception {
final AzureBlobFileSystem fs = this.getFileSystem(); final AzureBlobFileSystem fs = getFileSystem();
fs.create(TEST_FILE_PATH);
final FSDataOutputStream stream = fs.create(TEST_FILE_PATH);
ExecutorService es = Executors.newFixedThreadPool(10); ExecutorService es = Executors.newFixedThreadPool(10);
try(final FSDataOutputStream stream = fs.create(TEST_FILE_PATH)) {
final byte[] b = new byte[TEST_BUFFER_SIZE]; final byte[] b = new byte[TEST_BUFFER_SIZE];
new Random().nextBytes(b); new Random().nextBytes(b);
@ -195,7 +190,7 @@ public class ITestAzureBlobFileSystemFlush extends DependencyInjectedTest {
} }
Thread.sleep(THREAD_SLEEP_TIME); Thread.sleep(THREAD_SLEEP_TIME);
tasks.clear(); tasks.clear();
stream.close(); }
es.shutdownNow(); es.shutdownNow();
FileStatus fileStatus = fs.getFileStatus(TEST_FILE_PATH); FileStatus fileStatus = fs.getFileStatus(TEST_FILE_PATH);

View File

@ -22,29 +22,32 @@ import java.io.FileNotFoundException;
import org.junit.Test; import org.junit.Test;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys; import org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys;
/** /**
* Test filesystem initialization and creation. * Test filesystem initialization and creation.
*/ */
public class ITestAzureBlobFileSystemInitAndCreate extends DependencyInjectedTest { public class ITestAzureBlobFileSystemInitAndCreate extends
AbstractAbfsIntegrationTest {
public ITestAzureBlobFileSystemInitAndCreate() { public ITestAzureBlobFileSystemInitAndCreate() {
super();
this.getConfiguration().unset(ConfigurationKeys.AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION); this.getConfiguration().unset(ConfigurationKeys.AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION);
} }
@Override @Override
public void initialize() { public void setup() {
} }
@Override @Override
public void testCleanup() { public void teardown() {
} }
@Test (expected = FileNotFoundException.class) @Test (expected = FileNotFoundException.class)
public void ensureFilesystemWillNotBeCreatedIfCreationConfigIsNotSet() throws Exception { public void ensureFilesystemWillNotBeCreatedIfCreationConfigIsNotSet() throws Exception {
super.initialize(); super.setup();
this.getFileSystem(); final AzureBlobFileSystem fs = this.getFileSystem();
FileStatus[] fileStatuses = fs.listStatus(new Path("/"));
} }
} }

View File

@ -26,20 +26,21 @@ import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors; import java.util.concurrent.Executors;
import java.util.concurrent.Future; import java.util.concurrent.Future;
import org.junit.Assert;
import org.junit.Test; import org.junit.Test;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.LocatedFileStatus;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.contract.ContractTestUtils;
import static org.junit.Assert.assertEquals; import static org.apache.hadoop.test.LambdaTestUtils.intercept;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.assertFalse;
/** /**
* Test listStatus operation. * Test listStatus operation.
*/ */
public class ITestAzureBlobFileSystemListStatus extends DependencyInjectedTest { public class ITestAzureBlobFileSystemListStatus extends
AbstractAbfsIntegrationTest {
private static final int TEST_FILES_NUMBER = 6000; private static final int TEST_FILES_NUMBER = 6000;
public ITestAzureBlobFileSystemListStatus() { public ITestAzureBlobFileSystemListStatus() {
super(); super();
@ -47,8 +48,8 @@ public class ITestAzureBlobFileSystemListStatus extends DependencyInjectedTest {
@Test @Test
public void testListPath() throws Exception { public void testListPath() throws Exception {
final AzureBlobFileSystem fs = this.getFileSystem(); final AzureBlobFileSystem fs = getFileSystem();
final List<Future> tasks = new ArrayList<>(); final List<Future<Void>> tasks = new ArrayList<>();
ExecutorService es = Executors.newFixedThreadPool(10); ExecutorService es = Executors.newFixedThreadPool(10);
for (int i = 0; i < TEST_FILES_NUMBER; i++) { for (int i = 0; i < TEST_FILES_NUMBER; i++) {
@ -56,7 +57,7 @@ public class ITestAzureBlobFileSystemListStatus extends DependencyInjectedTest {
Callable<Void> callable = new Callable<Void>() { Callable<Void> callable = new Callable<Void>() {
@Override @Override
public Void call() throws Exception { public Void call() throws Exception {
fs.create(fileName); touch(fileName);
return null; return null;
} }
}; };
@ -70,63 +71,101 @@ public class ITestAzureBlobFileSystemListStatus extends DependencyInjectedTest {
es.shutdownNow(); es.shutdownNow();
FileStatus[] files = fs.listStatus(new Path("/")); FileStatus[] files = fs.listStatus(new Path("/"));
Assert.assertEquals(files.length, TEST_FILES_NUMBER + 1 /* user directory */); assertEquals(TEST_FILES_NUMBER, files.length /* user directory */);
} }
/**
* Creates a file, verifies that listStatus returns it,
* even while the file is still open for writing.
*/
@Test @Test
public void testListFileVsListDir() throws Exception { public void testListFileVsListDir() throws Exception {
final AzureBlobFileSystem fs = this.getFileSystem(); final AzureBlobFileSystem fs = getFileSystem();
fs.create(new Path("/testFile")); Path path = new Path("/testFile");
try(FSDataOutputStream ignored = fs.create(path)) {
FileStatus[] testFiles = fs.listStatus(new Path("/testFile")); FileStatus[] testFiles = fs.listStatus(path);
Assert.assertEquals(testFiles.length, 1); assertEquals("length of test files", 1, testFiles.length);
Assert.assertFalse(testFiles[0].isDirectory()); FileStatus status = testFiles[0];
assertIsFileReference(status);
}
} }
@Test @Test
public void testListFileVsListDir2() throws Exception { public void testListFileVsListDir2() throws Exception {
final AzureBlobFileSystem fs = this.getFileSystem(); final AzureBlobFileSystem fs = getFileSystem();
fs.mkdirs(new Path("/testFolder")); fs.mkdirs(new Path("/testFolder"));
fs.mkdirs(new Path("/testFolder/testFolder2")); fs.mkdirs(new Path("/testFolder/testFolder2"));
fs.mkdirs(new Path("/testFolder/testFolder2/testFolder3")); fs.mkdirs(new Path("/testFolder/testFolder2/testFolder3"));
fs.create(new Path("/testFolder/testFolder2/testFolder3/testFile")); Path testFile0Path = new Path("/testFolder/testFolder2/testFolder3/testFile");
ContractTestUtils.touch(fs, testFile0Path);
FileStatus[] testFiles = fs.listStatus(new Path("/testFolder/testFolder2/testFolder3/testFile")); FileStatus[] testFiles = fs.listStatus(testFile0Path);
Assert.assertEquals(testFiles.length, 1); assertEquals("Wrong listing size of file " + testFile0Path,
Assert.assertEquals(testFiles[0].getPath(), new Path(this.getTestUrl(), 1, testFiles.length);
"/testFolder/testFolder2/testFolder3/testFile")); FileStatus file0 = testFiles[0];
Assert.assertFalse(testFiles[0].isDirectory()); assertEquals("Wrong path for " + file0,
new Path(getTestUrl(), "/testFolder/testFolder2/testFolder3/testFile"),
file0.getPath());
assertIsFileReference(file0);
} }
@Test(expected = FileNotFoundException.class) @Test(expected = FileNotFoundException.class)
public void testListNonExistentDir() throws Exception { public void testListNonExistentDir() throws Exception {
final AzureBlobFileSystem fs = this.getFileSystem(); final AzureBlobFileSystem fs = getFileSystem();
fs.listStatus(new Path("/testFile/")); fs.listStatus(new Path("/testFile/"));
} }
@Test @Test
public void testListFiles() throws Exception { public void testListFiles() throws Exception {
final AzureBlobFileSystem fs = this.getFileSystem(); final AzureBlobFileSystem fs = getFileSystem();
fs.mkdirs(new Path("/test")); Path testDir = new Path("/test");
fs.mkdirs(testDir);
FileStatus[] fileStatuses = fs.listStatus(new Path("/")); FileStatus[] fileStatuses = fs.listStatus(new Path("/"));
assertEquals(fileStatuses.length, 2); assertEquals(1, fileStatuses.length);
fs.mkdirs(new Path("/test/sub")); fs.mkdirs(new Path("/test/sub"));
fileStatuses = fs.listStatus(new Path("/test")); fileStatuses = fs.listStatus(testDir);
assertEquals(fileStatuses.length, 1); assertEquals(1, fileStatuses.length);
assertEquals(fileStatuses[0].getPath().getName(), "sub"); assertEquals("sub", fileStatuses[0].getPath().getName());
assertTrue(fileStatuses[0].isDirectory()); assertIsDirectoryReference(fileStatuses[0]);
assertEquals(fileStatuses[0].getLen(), 0); Path childF = fs.makeQualified(new Path("/test/f"));
touch(childF);
fileStatuses = fs.listStatus(testDir);
assertEquals(2, fileStatuses.length);
final FileStatus childStatus = fileStatuses[0];
assertEquals(childF, childStatus.getPath());
assertEquals("f", childStatus.getPath().getName());
assertIsFileReference(childStatus);
assertEquals(0, childStatus.getLen());
final FileStatus status1 = fileStatuses[1];
assertEquals("sub", status1.getPath().getName());
assertIsDirectoryReference(status1);
// look at the child through getFileStatus
LocatedFileStatus locatedChildStatus = fs.listFiles(childF, false).next();
assertIsFileReference(locatedChildStatus);
fs.create(new Path("/test/f")); fs.delete(testDir, true);
fileStatuses = fs.listStatus(new Path("/test")); intercept(FileNotFoundException.class,
assertEquals(fileStatuses.length, 2); () -> fs.listFiles(childF, false).next());
assertEquals(fileStatuses[0].getPath().getName(), "f");
assertFalse(fileStatuses[0].isDirectory()); // do some final checks on the status (failing due to version checks)
assertEquals(fileStatuses[0].getLen(), 0); assertEquals("Path mismatch of " + locatedChildStatus,
assertEquals(fileStatuses[1].getPath().getName(), "sub"); childF, locatedChildStatus.getPath());
assertTrue(fileStatuses[1].isDirectory()); assertEquals("locatedstatus.equals(status)",
assertEquals(fileStatuses[1].getLen(), 0); locatedChildStatus, childStatus);
assertEquals("status.equals(locatedstatus)",
childStatus, locatedChildStatus);
}
private void assertIsDirectoryReference(FileStatus status) {
assertTrue("Not a directory: " + status, status.isDirectory());
assertFalse("Not a directory: " + status, status.isFile());
assertEquals(0, status.getLen());
}
private void assertIsFileReference(FileStatus status) {
assertFalse("Not a file: " + status, status.isDirectory());
assertTrue("Not a file: " + status, status.isFile());
} }
} }

View File

@ -18,71 +18,30 @@
package org.apache.hadoop.fs.azurebfs; package org.apache.hadoop.fs.azurebfs;
import java.util.concurrent.Callable;
import org.junit.Test; import org.junit.Test;
import org.apache.hadoop.fs.FileAlreadyExistsException;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import static org.apache.hadoop.test.LambdaTestUtils.intercept; import static org.apache.hadoop.fs.contract.ContractTestUtils.assertMkdirs;
import static org.junit.Assert.assertTrue;
/** /**
* Test mkdir operation. * Test mkdir operation.
*/ */
public class ITestAzureBlobFileSystemMkDir extends DependencyInjectedTest { public class ITestAzureBlobFileSystemMkDir extends AbstractAbfsIntegrationTest {
public ITestAzureBlobFileSystemMkDir() { public ITestAzureBlobFileSystemMkDir() {
super(); super();
} }
@Test @Test
public void testCreateDirWithExistingDir() throws Exception { public void testCreateDirWithExistingDir() throws Exception {
final AzureBlobFileSystem fs = this.getFileSystem(); final AzureBlobFileSystem fs = getFileSystem();
assertTrue(fs.mkdirs(new Path("testFolder"))); Path path = new Path("testFolder");
assertTrue(fs.mkdirs(new Path("testFolder"))); assertMkdirs(fs, path);
} assertMkdirs(fs, path);
@Test(expected = FileAlreadyExistsException.class)
public void createDirectoryUnderFile() throws Exception {
final AzureBlobFileSystem fs = this.getFileSystem();
fs.create(new Path("testFile"));
fs.mkdirs(new Path("testFile/TestDirectory"));
}
@Test
public void testCreateDirectoryOverExistingFiles() throws Exception {
final AzureBlobFileSystem fs = this.getFileSystem();
fs.create(new Path("/testPath"));
FileAlreadyExistsException ex = intercept(
FileAlreadyExistsException.class,
new Callable<Void>() {
@Override
public Void call() throws Exception {
fs.mkdirs(new Path("/testPath"));
return null;
}
});
assertTrue(ex instanceof FileAlreadyExistsException);
fs.create(new Path("/testPath1/file1"));
ex = intercept(
FileAlreadyExistsException.class,
new Callable<Void>() {
@Override
public Void call() throws Exception {
fs.mkdirs(new Path("/testPath1/file1"));
return null;
}
});
assertTrue(ex instanceof FileAlreadyExistsException);
} }
@Test @Test
public void testCreateRoot() throws Exception { public void testCreateRoot() throws Exception {
final AzureBlobFileSystem fs = this.getFileSystem(); assertMkdirs(getFileSystem(), new Path("/"));
assertTrue(fs.mkdirs(new Path("/")));
} }
} }

View File

@ -1,41 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.fs.azurebfs;
import java.io.FileNotFoundException;
import org.junit.Test;
import org.apache.hadoop.fs.Path;
/**
* Test open operation.
*/
public class ITestAzureBlobFileSystemOpen extends DependencyInjectedTest {
public ITestAzureBlobFileSystemOpen() throws Exception {
super();
}
@Test(expected = FileNotFoundException.class)
public void testOpenDirectory() throws Exception {
final AzureBlobFileSystem fs = this.getFileSystem();
fs.mkdirs(new Path("testFolder"));
fs.open(new Path("testFolder"));
}
}

View File

@ -18,32 +18,31 @@
package org.apache.hadoop.fs.azurebfs; package org.apache.hadoop.fs.azurebfs;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FSExceptionMessages;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.azure.NativeAzureFileSystem;
import org.apache.hadoop.fs.contract.ContractTestUtils;
import org.junit.Test;
import java.io.EOFException; import java.io.EOFException;
import java.io.IOException; import java.io.IOException;
import java.util.Random; import java.util.Random;
import java.util.concurrent.Callable; import java.util.concurrent.Callable;
import static org.apache.hadoop.test.LambdaTestUtils.intercept; import org.junit.Test;
import static org.junit.Assert.assertEquals; import org.slf4j.Logger;
import static org.junit.Assert.assertTrue; import org.slf4j.LoggerFactory;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertArrayEquals;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FSExceptionMessages;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.azure.NativeAzureFileSystem;
import org.apache.hadoop.fs.contract.ContractTestUtils;
import static org.apache.hadoop.test.LambdaTestUtils.intercept;
/** /**
* Test random read operation. * Test random read operation.
*/ */
public class ITestAzureBlobFileSystemRandomRead extends DependencyInjectedTest { public class ITestAzureBlobFileSystemRandomRead extends
AbstractAbfsScaleTest {
private static final int KILOBYTE = 1024; private static final int KILOBYTE = 1024;
private static final int MEGABYTE = KILOBYTE * KILOBYTE; private static final int MEGABYTE = KILOBYTE * KILOBYTE;
private static final long TEST_FILE_SIZE = 8 * MEGABYTE; private static final long TEST_FILE_SIZE = 8 * MEGABYTE;
@ -62,6 +61,9 @@ public class ITestAzureBlobFileSystemRandomRead extends DependencyInjectedTest {
private static final String ABFS = "ABFS"; private static final String ABFS = "ABFS";
private static long testFileLength = 0; private static long testFileLength = 0;
private static final Logger LOG =
LoggerFactory.getLogger(ITestAzureBlobFileSystemRandomRead.class);
public ITestAzureBlobFileSystemRandomRead() throws Exception { public ITestAzureBlobFileSystemRandomRead() throws Exception {
super(); super();
} }
@ -76,7 +78,7 @@ public class ITestAzureBlobFileSystemRandomRead extends DependencyInjectedTest {
// forward seek and read a kilobyte into first kilobyte of bufferV2 // forward seek and read a kilobyte into first kilobyte of bufferV2
inputStream.seek(5 * MEGABYTE); inputStream.seek(5 * MEGABYTE);
int numBytesRead = inputStream.read(buffer, 0, KILOBYTE); int numBytesRead = inputStream.read(buffer, 0, KILOBYTE);
assertEquals(KILOBYTE, numBytesRead); assertEquals("Wrong number of bytes read", KILOBYTE, numBytesRead);
int len = MEGABYTE; int len = MEGABYTE;
int offset = buffer.length - len; int offset = buffer.length - len;
@ -84,7 +86,7 @@ public class ITestAzureBlobFileSystemRandomRead extends DependencyInjectedTest {
// reverse seek and read a megabyte into last megabyte of bufferV1 // reverse seek and read a megabyte into last megabyte of bufferV1
inputStream.seek(3 * MEGABYTE); inputStream.seek(3 * MEGABYTE);
numBytesRead = inputStream.read(buffer, offset, len); numBytesRead = inputStream.read(buffer, offset, len);
assertEquals(len, numBytesRead); assertEquals("Wrong number of bytes read after seek", len, numBytesRead);
} }
} }
@ -391,7 +393,7 @@ public class ITestAzureBlobFileSystemRandomRead extends DependencyInjectedTest {
afterSeekElapsedMs = sequentialRead(ABFS, afterSeekElapsedMs = sequentialRead(ABFS,
this.getFileSystem(), true); this.getFileSystem(), true);
ratio = afterSeekElapsedMs / beforeSeekElapsedMs; ratio = afterSeekElapsedMs / beforeSeekElapsedMs;
System.out.println((String.format( LOG.info((String.format(
"beforeSeekElapsedMs=%1$d, afterSeekElapsedMs=%2$d, ratio=%3$.2f", "beforeSeekElapsedMs=%1$d, afterSeekElapsedMs=%2$d, ratio=%3$.2f",
(long) beforeSeekElapsedMs, (long) beforeSeekElapsedMs,
(long) afterSeekElapsedMs, (long) afterSeekElapsedMs,
@ -425,7 +427,7 @@ public class ITestAzureBlobFileSystemRandomRead extends DependencyInjectedTest {
ratio = v2ElapsedMs / v1ElapsedMs; ratio = v2ElapsedMs / v1ElapsedMs;
System.out.println(String.format( LOG.info(String.format(
"v1ElapsedMs=%1$d, v2ElapsedMs=%2$d, ratio=%3$.2f", "v1ElapsedMs=%1$d, v2ElapsedMs=%2$d, ratio=%3$.2f",
(long) v1ElapsedMs, (long) v1ElapsedMs,
(long) v2ElapsedMs, (long) v2ElapsedMs,
@ -464,7 +466,7 @@ public class ITestAzureBlobFileSystemRandomRead extends DependencyInjectedTest {
} }
long elapsedTimeMs = timer.elapsedTimeMs(); long elapsedTimeMs = timer.elapsedTimeMs();
System.out.println(String.format( LOG.info(String.format(
"v%1$s: bytesRead=%2$d, elapsedMs=%3$d, Mbps=%4$.2f," "v%1$s: bytesRead=%2$d, elapsedMs=%3$d, Mbps=%4$.2f,"
+ " afterReverseSeek=%5$s", + " afterReverseSeek=%5$s",
version, version,
@ -496,7 +498,7 @@ public class ITestAzureBlobFileSystemRandomRead extends DependencyInjectedTest {
} while (bytesRead > 0 && totalBytesRead < minBytesToRead); } while (bytesRead > 0 && totalBytesRead < minBytesToRead);
long elapsedTimeMs = timer.elapsedTimeMs(); long elapsedTimeMs = timer.elapsedTimeMs();
inputStream.close(); inputStream.close();
System.out.println(String.format( LOG.info(String.format(
"v%1$d: totalBytesRead=%2$d, elapsedTimeMs=%3$d, Mbps=%4$.2f", "v%1$d: totalBytesRead=%2$d, elapsedTimeMs=%3$d, Mbps=%4$.2f",
version, version,
totalBytesRead, totalBytesRead,
@ -535,7 +537,7 @@ public class ITestAzureBlobFileSystemRandomRead extends DependencyInjectedTest {
character = (character == 'z') ? 'a' : (char) ((int) character + 1); character = (character == 'z') ? 'a' : (char) ((int) character + 1);
} }
System.out.println(String.format("Creating test file %s of size: %d ", TEST_FILE_PATH, TEST_FILE_SIZE)); LOG.info(String.format("Creating test file %s of size: %d ", TEST_FILE_PATH, TEST_FILE_SIZE));
ContractTestUtils.NanoTimer timer = new ContractTestUtils.NanoTimer(); ContractTestUtils.NanoTimer timer = new ContractTestUtils.NanoTimer();
try (FSDataOutputStream outputStream = fs.create(TEST_FILE_PATH)) { try (FSDataOutputStream outputStream = fs.create(TEST_FILE_PATH)) {
@ -544,7 +546,7 @@ public class ITestAzureBlobFileSystemRandomRead extends DependencyInjectedTest {
outputStream.write(buffer); outputStream.write(buffer);
bytesWritten += buffer.length; bytesWritten += buffer.length;
} }
System.out.println(String.format("Closing stream %s", outputStream)); LOG.info("Closing stream {}", outputStream);
ContractTestUtils.NanoTimer closeTimer ContractTestUtils.NanoTimer closeTimer
= new ContractTestUtils.NanoTimer(); = new ContractTestUtils.NanoTimer();
outputStream.close(); outputStream.close();

View File

@ -18,7 +18,6 @@
package org.apache.hadoop.fs.azurebfs; package org.apache.hadoop.fs.azurebfs;
import java.io.FileNotFoundException;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.List; import java.util.List;
import java.util.concurrent.Callable; import java.util.concurrent.Callable;
@ -26,93 +25,74 @@ import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors; import java.util.concurrent.Executors;
import java.util.concurrent.Future; import java.util.concurrent.Future;
import org.junit.Assert;
import org.junit.Test; import org.junit.Test;
import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import static org.junit.Assert.assertFalse; import static org.apache.hadoop.fs.contract.ContractTestUtils.assertIsDirectory;
import static org.junit.Assert.assertTrue; import static org.apache.hadoop.fs.contract.ContractTestUtils.assertMkdirs;
import static org.junit.Assert.assertEquals; import static org.apache.hadoop.fs.contract.ContractTestUtils.assertPathDoesNotExist;
import static org.junit.Assert.assertNotNull; import static org.apache.hadoop.fs.contract.ContractTestUtils.assertRenameOutcome;
import static org.apache.hadoop.fs.contract.ContractTestUtils.assertIsFile;
/** /**
* Test rename operation. * Test rename operation.
*/ */
public class ITestAzureBlobFileSystemRename extends DependencyInjectedTest { public class ITestAzureBlobFileSystemRename extends
AbstractAbfsIntegrationTest {
public ITestAzureBlobFileSystemRename() { public ITestAzureBlobFileSystemRename() {
super();
} }
@Test(expected = FileNotFoundException.class) @Test
public void testEnsureFileIsRenamed() throws Exception { public void testEnsureFileIsRenamed() throws Exception {
final AzureBlobFileSystem fs = this.getFileSystem(); final AzureBlobFileSystem fs = getFileSystem();
fs.create(new Path("testfile")); Path src = path("testEnsureFileIsRenamed-src");
fs.rename(new Path("testfile"), new Path("testfile2")); touch(src);
Path dest = path("testEnsureFileIsRenamed-dest");
fs.delete(dest, true);
assertRenameOutcome(fs, src, dest, true);
FileStatus fileStatus = fs.getFileStatus(new Path("testfile2")); assertIsFile(fs, dest);
assertNotNull(fileStatus); assertPathDoesNotExist(fs, "expected renamed", src);
fs.getFileStatus(new Path("testfile"));
} }
@Test @Test
public void testRenameFile() throws Exception { public void testRenameFileUnderDir() throws Exception {
final AzureBlobFileSystem fs = this.getFileSystem(); final AzureBlobFileSystem fs = getFileSystem();
fs.mkdirs(new Path("/testSrc")); Path sourceDir = new Path("/testSrc");
fs.create(new Path("/testSrc/file1")); assertMkdirs(fs, sourceDir);
String filename = "file1";
Path file1 = new Path(sourceDir, filename);
touch(file1);
fs.rename(new Path("/testSrc"), new Path("/testDst")); Path destDir = new Path("/testDst");
FileStatus[] fileStatus = fs.listStatus(new Path("/testDst")); assertRenameOutcome(fs, sourceDir, destDir, true);
assertNotNull(fileStatus); FileStatus[] fileStatus = fs.listStatus(destDir);
assertNotNull("Null file status", fileStatus);
FileStatus status = fileStatus[0];
assertEquals("Wrong filename in " + status,
filename, status.getPath().getName());
} }
@Test @Test
public void testRenameFileUsingUnicode() throws Exception {
final AzureBlobFileSystem fs = this.getFileSystem();
//known issue: ListStatus operation to folders/files whose name contains '?' will fail
//This is because Auto rest client didn't encode '?' in the uri query parameters
String[] folders1 = new String[]{"/%2c%26", "/ÖáΠ⇒", "/A +B", "/A~`!@#$%^&*()-_+={};:'>,<B"};
String[] folders2 = new String[]{"/abcÖ⇒123", "/abcÖáΠ⇒123", "/B+ C", "/B~`!@#$%^&*()-_+={};:'>,<C"};
String[] files = new String[]{"/%2c%27", "/中文", "/C +D", "/C~`!@#$%^&*()-_+={};:'>,<D"};
for (int i = 0; i < 4; i++) {
Path folderPath1 = new Path(folders1[i]);
assertTrue(fs.mkdirs(folderPath1));
assertTrue(fs.exists(folderPath1));
Path filePath = new Path(folders1[i] + files[i]);
fs.create(filePath);
assertTrue(fs.exists(filePath));
Path folderPath2 = new Path(folders2[i]);
fs.rename(folderPath1, folderPath2);
assertFalse(fs.exists(folderPath1));
assertTrue(fs.exists(folderPath2));
FileStatus[] fileStatus = fs.listStatus(folderPath2);
assertEquals("/" + fileStatus[0].getPath().getName(), files[i]);
assertNotNull(fileStatus);
}
}
@Test(expected = FileNotFoundException.class)
public void testRenameDirectory() throws Exception { public void testRenameDirectory() throws Exception {
final AzureBlobFileSystem fs = this.getFileSystem(); final AzureBlobFileSystem fs = getFileSystem();
fs.mkdirs(new Path("testDir")); fs.mkdirs(new Path("testDir"));
fs.mkdirs(new Path("testDir/test1")); Path test1 = new Path("testDir/test1");
fs.mkdirs(test1);
fs.mkdirs(new Path("testDir/test1/test2")); fs.mkdirs(new Path("testDir/test1/test2"));
fs.mkdirs(new Path("testDir/test1/test2/test3")); fs.mkdirs(new Path("testDir/test1/test2/test3"));
Assert.assertTrue(fs.rename(new Path("testDir/test1"), new Path("testDir/test10"))); assertRenameOutcome(fs, test1,
fs.getFileStatus(new Path("testDir/test1")); new Path("testDir/test10"), true);
assertPathDoesNotExist(fs, "rename source dir", test1 );
} }
@Test(expected = FileNotFoundException.class) @Test
public void testRenameFirstLevelDirectory() throws Exception { public void testRenameFirstLevelDirectory() throws Exception {
final AzureBlobFileSystem fs = this.getFileSystem(); final AzureBlobFileSystem fs = getFileSystem();
final List<Future> tasks = new ArrayList<>(); final List<Future<Void>> tasks = new ArrayList<>();
ExecutorService es = Executors.newFixedThreadPool(10); ExecutorService es = Executors.newFixedThreadPool(10);
for (int i = 0; i < 1000; i++) { for (int i = 0; i < 1000; i++) {
@ -120,7 +100,7 @@ public class ITestAzureBlobFileSystemRename extends DependencyInjectedTest {
Callable<Void> callable = new Callable<Void>() { Callable<Void> callable = new Callable<Void>() {
@Override @Override
public Void call() throws Exception { public Void call() throws Exception {
fs.create(fileName); touch(fileName);
return null; return null;
} }
}; };
@ -133,20 +113,25 @@ public class ITestAzureBlobFileSystemRename extends DependencyInjectedTest {
} }
es.shutdownNow(); es.shutdownNow();
fs.rename(new Path("/test"), new Path("/renamedDir")); Path source = new Path("/test");
Path dest = new Path("/renamedDir");
assertRenameOutcome(fs, source, dest, true);
FileStatus[] files = fs.listStatus(new Path("/renamedDir")); FileStatus[] files = fs.listStatus(dest);
Assert.assertEquals(files.length, 1000); assertEquals("Wrong number of files in listing", 1000, files.length);
fs.getFileStatus(new Path("/test")); assertPathDoesNotExist(fs, "rename source dir", source);
} }
@Test @Test
public void testRenameRoot() throws Exception { public void testRenameRoot() throws Exception {
final AzureBlobFileSystem fs = this.getFileSystem(); final AzureBlobFileSystem fs = getFileSystem();
boolean renamed = fs.rename(new Path("/"), new Path("/ddd")); assertRenameOutcome(fs,
assertFalse(renamed); new Path("/"),
new Path("/testRenameRoot"),
renamed = fs.rename(new Path(fs.getUri().toString() + "/"), new Path(fs.getUri().toString() + "/s")); false);
assertFalse(renamed); assertRenameOutcome(fs,
new Path(fs.getUri().toString() + "/"),
new Path(fs.getUri().toString() + "/s"),
false);
} }
} }

View File

@ -0,0 +1,98 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.fs.azurebfs;
import java.util.Arrays;
import org.junit.Test;
import org.junit.runner.RunWith;
import org.junit.runners.Parameterized;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.Path;
import static org.apache.hadoop.fs.contract.ContractTestUtils.assertIsDirectory;
import static org.apache.hadoop.fs.contract.ContractTestUtils.assertIsFile;
import static org.apache.hadoop.fs.contract.ContractTestUtils.assertMkdirs;
import static org.apache.hadoop.fs.contract.ContractTestUtils.assertPathDoesNotExist;
import static org.apache.hadoop.fs.contract.ContractTestUtils.assertPathExists;
import static org.apache.hadoop.fs.contract.ContractTestUtils.assertRenameOutcome;
/**
* Parameterized test of rename operations of unicode paths.
*/
@RunWith(Parameterized.class)
public class ITestAzureBlobFileSystemRenameUnicode extends
AbstractAbfsIntegrationTest {
@Parameterized.Parameter
public String srcDir;
@Parameterized.Parameter(1)
public String destDir;
@Parameterized.Parameter(2)
public String filename;
@Parameterized.Parameters
public static Iterable<Object[]> params() {
return Arrays.asList(
new Object[][]{
{"/src", "/dest", "filename"},
{"/%2c%26", "/abcÖ⇒123", "%2c%27"},
{"/ÖáΠ⇒", "/abcÖáΠ⇒123", "中文"},
{"/A +B", "/B+ C", "C +D"},
{
"/A~`!@#$%^&*()-_+={};:'>,<B",
"/B~`!@#$%^&*()-_+={};:'>,<C",
"C~`!@#$%^&*()-_+={};:'>,<D"
}
});
}
public ITestAzureBlobFileSystemRenameUnicode() {
}
/**
* Known issue: ListStatus operation to folders/files whose name contains '?' will fail.
* This is because Auto rest client didn't encode '?' in the uri query parameters
*/
@Test
public void testRenameFileUsingUnicode() throws Exception {
final AzureBlobFileSystem fs = getFileSystem();
Path folderPath1 = new Path(srcDir);
assertMkdirs(fs, folderPath1);
assertIsDirectory(fs, folderPath1);
Path filePath = new Path(folderPath1 + "/" + filename);
touch(filePath);
assertIsFile(fs, filePath);
Path folderPath2 = new Path(destDir);
assertRenameOutcome(fs, folderPath1, folderPath2, true);
assertPathDoesNotExist(fs, "renamed", folderPath1);
assertIsDirectory(fs, folderPath2);
assertPathExists(fs, "renamed file", new Path(folderPath2 + "/" + filename));
FileStatus[] fileStatus = fs.listStatus(folderPath2);
assertNotNull(fileStatus);
assertTrue("Empty listing returned from listStatus(\"" + folderPath2 + "\")",
fileStatus.length > 0);
assertEquals(fileStatus[0].getPath().getName(), filename);
}
}

View File

@ -20,9 +20,9 @@ package org.apache.hadoop.fs.azurebfs;
import java.net.URI; import java.net.URI;
import org.junit.Assert;
import org.junit.Test; import org.junit.Test;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.CommonConfigurationKeysPublic; import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes; import org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes;
@ -30,30 +30,45 @@ import org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes;
/** /**
* Test AzureBlobFileSystem initialization. * Test AzureBlobFileSystem initialization.
*/ */
public class ITestFileSystemInitialization extends DependencyInjectedTest { public class ITestFileSystemInitialization extends AbstractAbfsIntegrationTest {
public ITestFileSystemInitialization() { public ITestFileSystemInitialization() {
super(); super();
} }
@Test @Test
public void ensureAzureBlobFileSystemIsInitialized() throws Exception { public void ensureAzureBlobFileSystemIsInitialized() throws Exception {
final FileSystem fs = this.getFileSystem(); final AzureBlobFileSystem fs = getFileSystem();
final String accountName = this.getAccountName(); final String accountName = getAccountName();
final String filesystem = this.getFileSystemName(); final String filesystem = getFileSystemName();
Assert.assertEquals(fs.getUri(), new URI(FileSystemUriSchemes.ABFS_SCHEME, filesystem + "@" + accountName, null, null, null)); assertEquals(fs.getUri(),
Assert.assertNotNull(fs.getWorkingDirectory()); new URI(FileSystemUriSchemes.ABFS_SCHEME,
filesystem + "@" + accountName,
null,
null,
null));
assertNotNull("working directory", fs.getWorkingDirectory());
} }
@Test @Test
public void ensureSecureAzureBlobFileSystemIsInitialized() throws Exception { public void ensureSecureAzureBlobFileSystemIsInitialized() throws Exception {
final String accountName = this.getAccountName(); final String accountName = getAccountName();
final String filesystem = this.getFileSystemName(); final String filesystem = getFileSystemName();
final URI defaultUri = new URI(FileSystemUriSchemes.ABFS_SECURE_SCHEME, filesystem + "@" + accountName, null, null, null); final URI defaultUri = new URI(FileSystemUriSchemes.ABFS_SECURE_SCHEME,
this.getConfiguration().set(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY, defaultUri.toString()); filesystem + "@" + accountName,
null,
null,
null);
Configuration conf = getConfiguration();
conf.set(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY, defaultUri.toString());
final FileSystem fs = this.getFileSystem(); try(SecureAzureBlobFileSystem fs = (SecureAzureBlobFileSystem) FileSystem.newInstance(conf)) {
Assert.assertEquals(fs.getUri(), new URI(FileSystemUriSchemes.ABFS_SECURE_SCHEME, filesystem + "@" + accountName, null, null, null)); assertEquals(fs.getUri(), new URI(FileSystemUriSchemes.ABFS_SECURE_SCHEME,
Assert.assertNotNull(fs.getWorkingDirectory()); filesystem + "@" + accountName,
null,
null,
null));
assertNotNull("working directory", fs.getWorkingDirectory());
}
} }
} }

View File

@ -20,7 +20,6 @@ package org.apache.hadoop.fs.azurebfs;
import java.util.Hashtable; import java.util.Hashtable;
import org.junit.Assert;
import org.junit.Ignore; import org.junit.Ignore;
import org.junit.Test; import org.junit.Test;
@ -34,31 +33,29 @@ import static org.junit.Assert.assertEquals;
/** /**
* Test FileSystemProperties. * Test FileSystemProperties.
*/ */
public class ITestFileSystemProperties extends DependencyInjectedTest { public class ITestFileSystemProperties extends AbstractAbfsIntegrationTest {
private static final int TEST_DATA = 100; private static final int TEST_DATA = 100;
private static final Path TEST_PATH = new Path("/testfile"); private static final Path TEST_PATH = new Path("/testfile");
public ITestFileSystemProperties() { public ITestFileSystemProperties() {
super();
} }
@Test @Test
public void testReadWriteBytesToFileAndEnsureThreadPoolCleanup() throws Exception { public void testReadWriteBytesToFileAndEnsureThreadPoolCleanup() throws Exception {
final AzureBlobFileSystem fs = this.getFileSystem(); final AzureBlobFileSystem fs = getFileSystem();
testWriteOneByteToFileAndEnsureThreadPoolCleanup(); testWriteOneByteToFileAndEnsureThreadPoolCleanup();
FSDataInputStream inputStream = fs.open(TEST_PATH, 4 * 1024 * 1024); try(FSDataInputStream inputStream = fs.open(TEST_PATH, 4 * 1024 * 1024)) {
int i = inputStream.read(); int i = inputStream.read();
assertEquals(TEST_DATA, i); assertEquals(TEST_DATA, i);
} }
}
@Test @Test
public void testWriteOneByteToFileAndEnsureThreadPoolCleanup() throws Exception { public void testWriteOneByteToFileAndEnsureThreadPoolCleanup() throws Exception {
final AzureBlobFileSystem fs = this.getFileSystem(); final AzureBlobFileSystem fs = getFileSystem();
FSDataOutputStream stream = fs.create(TEST_PATH); try(FSDataOutputStream stream = fs.create(TEST_PATH)) {
stream.write(TEST_DATA); stream.write(TEST_DATA);
stream.close(); }
FileStatus fileStatus = fs.getFileStatus(TEST_PATH); FileStatus fileStatus = fs.getFileStatus(TEST_PATH);
assertEquals(1, fileStatus.getLen()); assertEquals(1, fileStatus.getLen());
@ -67,60 +64,60 @@ public class ITestFileSystemProperties extends DependencyInjectedTest {
@Test @Test
@Ignore("JDK7 doesn't support PATCH, so PUT is used. Fix is applied in latest test tenant") @Ignore("JDK7 doesn't support PATCH, so PUT is used. Fix is applied in latest test tenant")
public void testBase64FileSystemProperties() throws Exception { public void testBase64FileSystemProperties() throws Exception {
final AzureBlobFileSystem fs = this.getFileSystem(); final AzureBlobFileSystem fs = getFileSystem();
final Hashtable<String, String> properties = new Hashtable<>(); final Hashtable<String, String> properties = new Hashtable<>();
properties.put("key", "{ value: value }"); properties.put("key", "{ value: value }");
fs.getAbfsStore().setFilesystemProperties(properties); fs.getAbfsStore().setFilesystemProperties(properties);
Hashtable<String, String> fetchedProperties = fs.getAbfsStore().getFilesystemProperties(); Hashtable<String, String> fetchedProperties = fs.getAbfsStore().getFilesystemProperties();
Assert.assertEquals(properties, fetchedProperties); assertEquals(properties, fetchedProperties);
} }
@Test @Test
public void testBase64PathProperties() throws Exception { public void testBase64PathProperties() throws Exception {
final AzureBlobFileSystem fs = this.getFileSystem(); final AzureBlobFileSystem fs = getFileSystem();
final Hashtable<String, String> properties = new Hashtable<>(); final Hashtable<String, String> properties = new Hashtable<>();
properties.put("key", "{ value: valueTest }"); properties.put("key", "{ value: valueTest }");
fs.create(TEST_PATH); touch(TEST_PATH);
fs.getAbfsStore().setPathProperties(TEST_PATH, properties); fs.getAbfsStore().setPathProperties(TEST_PATH, properties);
Hashtable<String, String> fetchedProperties = Hashtable<String, String> fetchedProperties =
fs.getAbfsStore().getPathProperties(TEST_PATH); fs.getAbfsStore().getPathProperties(TEST_PATH);
Assert.assertEquals(properties, fetchedProperties); assertEquals(properties, fetchedProperties);
} }
@Test (expected = Exception.class) @Test (expected = Exception.class)
public void testBase64InvalidFileSystemProperties() throws Exception { public void testBase64InvalidFileSystemProperties() throws Exception {
final AzureBlobFileSystem fs = this.getFileSystem(); final AzureBlobFileSystem fs = getFileSystem();
final Hashtable<String, String> properties = new Hashtable<>(); final Hashtable<String, String> properties = new Hashtable<>();
properties.put("key", "{ value: value歲 }"); properties.put("key", "{ value: value歲 }");
fs.getAbfsStore().setFilesystemProperties(properties); fs.getAbfsStore().setFilesystemProperties(properties);
Hashtable<String, String> fetchedProperties = fs.getAbfsStore().getFilesystemProperties(); Hashtable<String, String> fetchedProperties = fs.getAbfsStore().getFilesystemProperties();
Assert.assertEquals(properties, fetchedProperties); assertEquals(properties, fetchedProperties);
} }
@Test (expected = Exception.class) @Test (expected = Exception.class)
public void testBase64InvalidPathProperties() throws Exception { public void testBase64InvalidPathProperties() throws Exception {
final AzureBlobFileSystem fs = this.getFileSystem(); final AzureBlobFileSystem fs = getFileSystem();
final Hashtable<String, String> properties = new Hashtable<>(); final Hashtable<String, String> properties = new Hashtable<>();
properties.put("key", "{ value: valueTest兩 }"); properties.put("key", "{ value: valueTest兩 }");
fs.create(TEST_PATH); touch(TEST_PATH);
fs.getAbfsStore().setPathProperties(TEST_PATH, properties); fs.getAbfsStore().setPathProperties(TEST_PATH, properties);
Hashtable<String, String> fetchedProperties = fs.getAbfsStore().getPathProperties(TEST_PATH); Hashtable<String, String> fetchedProperties = fs.getAbfsStore().getPathProperties(TEST_PATH);
Assert.assertEquals(properties, fetchedProperties); assertEquals(properties, fetchedProperties);
} }
@Test @Test
public void testSetFileSystemProperties() throws Exception { public void testSetFileSystemProperties() throws Exception {
final AzureBlobFileSystem fs = this.getFileSystem(); final AzureBlobFileSystem fs = getFileSystem();
final Hashtable<String, String> properties = new Hashtable<>(); final Hashtable<String, String> properties = new Hashtable<>();
properties.put("containerForDevTest", "true"); properties.put("containerForDevTest", "true");
fs.getAbfsStore().setFilesystemProperties(properties); fs.getAbfsStore().setFilesystemProperties(properties);
Hashtable<String, String> fetchedProperties = fs.getAbfsStore().getFilesystemProperties(); Hashtable<String, String> fetchedProperties = fs.getAbfsStore().getFilesystemProperties();
Assert.assertEquals(properties, fetchedProperties); assertEquals(properties, fetchedProperties);
} }
} }

View File

@ -20,10 +20,9 @@ package org.apache.hadoop.fs.azurebfs;
import java.net.URI; import java.net.URI;
import org.junit.Assert;
import org.junit.Test; import org.junit.Test;
import org.apache.hadoop.fs.AbstractFileSystem; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.CommonConfigurationKeysPublic; import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
import org.apache.hadoop.fs.FileContext; import org.apache.hadoop.fs.FileContext;
import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileSystem;
@ -31,33 +30,76 @@ import org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes;
/** /**
* Test AzureBlobFileSystem registration. * Test AzureBlobFileSystem registration.
* Use casts to have interesting stack traces on failures.
*/ */
public class ITestFileSystemRegistration extends DependencyInjectedTest { public class ITestFileSystemRegistration extends AbstractAbfsIntegrationTest {
protected static final String ABFS = "org.apache.hadoop.fs.azurebfs.Abfs";
protected static final String ABFSS = "org.apache.hadoop.fs.azurebfs.Abfss";
public ITestFileSystemRegistration() throws Exception { public ITestFileSystemRegistration() throws Exception {
super(); }
private void assertConfigMatches(Configuration conf, String key, String expected) {
String v = conf.get(key);
assertNotNull("No value for key " + key, v);
assertEquals("Wrong value for key " + key, expected, v);
}
@Test
public void testAbfsFileSystemRegistered() throws Throwable {
assertConfigMatches(new Configuration(true),
"fs.abfs.impl",
"org.apache.hadoop.fs.azurebfs.AzureBlobFileSystem");
}
@Test
public void testSecureAbfsFileSystemRegistered() throws Throwable {
assertConfigMatches(new Configuration(true),
"fs.abfss.impl",
"org.apache.hadoop.fs.azurebfs.SecureAzureBlobFileSystem");
}
@Test
public void testAbfsFileContextRegistered() throws Throwable {
assertConfigMatches(new Configuration(true),
"fs.AbstractFileSystem.abfs.impl",
ABFS);
}
@Test
public void testSecureAbfsFileContextRegistered() throws Throwable {
assertConfigMatches(new Configuration(true),
"fs.AbstractFileSystem.abfss.impl",
ABFSS);
} }
@Test @Test
public void ensureAzureBlobFileSystemIsDefaultFileSystem() throws Exception { public void ensureAzureBlobFileSystemIsDefaultFileSystem() throws Exception {
FileSystem fs = FileSystem.get(this.getConfiguration()); AzureBlobFileSystem fs = (AzureBlobFileSystem) FileSystem.get(getConfiguration());
Assert.assertTrue(fs instanceof AzureBlobFileSystem); assertNotNull("filesystem", fs);
AbstractFileSystem afs = FileContext.getFileContext(this.getConfiguration()).getDefaultFileSystem(); Abfs afs = (Abfs) FileContext.getFileContext(getConfiguration()).getDefaultFileSystem();
Assert.assertTrue(afs instanceof Abfs); assertNotNull("filecontext", afs);
} }
@Test @Test
public void ensureSecureAzureBlobFileSystemIsDefaultFileSystem() throws Exception { public void ensureSecureAzureBlobFileSystemIsDefaultFileSystem() throws Exception {
final String accountName = this.getAccountName(); final String accountName = getAccountName();
final String fileSystemName = this.getFileSystemName(); final String fileSystemName = getFileSystemName();
final URI defaultUri = new URI(FileSystemUriSchemes.ABFS_SECURE_SCHEME, fileSystemName + "@" + accountName, null, null, null); final URI defaultUri = new URI(FileSystemUriSchemes.ABFS_SECURE_SCHEME,
this.getConfiguration().set(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY, defaultUri.toString()); fileSystemName + "@" + accountName,
null,
null,
null);
getConfiguration().set(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY,
defaultUri.toString());
FileSystem fs = FileSystem.get(this.getConfiguration()); SecureAzureBlobFileSystem fs = (SecureAzureBlobFileSystem) FileSystem.get(
Assert.assertTrue(fs instanceof SecureAzureBlobFileSystem); getConfiguration());
assertNotNull("filesystem", fs);
AbstractFileSystem afs = FileContext.getFileContext(this.getConfiguration()).getDefaultFileSystem(); Abfss afs = (Abfss) FileContext.getFileContext(getConfiguration()).getDefaultFileSystem();
Assert.assertTrue(afs instanceof Abfss); assertNotNull("filecontext", afs);
} }
} }

View File

@ -17,59 +17,62 @@
*/ */
package org.apache.hadoop.fs.azurebfs; package org.apache.hadoop.fs.azurebfs;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.azure.NativeAzureFileSystem;
import org.junit.Assert;
import org.junit.Assume;
import org.junit.Test;
import java.io.BufferedReader; import java.io.BufferedReader;
import java.io.InputStreamReader; import java.io.InputStreamReader;
import static junit.framework.TestCase.assertEquals; import org.junit.Assume;
import static junit.framework.TestCase.assertFalse; import org.junit.Test;
import static junit.framework.TestCase.assertTrue; import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.azure.NativeAzureFileSystem;
import org.apache.hadoop.fs.contract.ContractTestUtils;
import static org.apache.hadoop.fs.contract.ContractTestUtils.assertDeleted;
import static org.apache.hadoop.fs.contract.ContractTestUtils.assertIsDirectory;
import static org.apache.hadoop.fs.contract.ContractTestUtils.assertMkdirs;
import static org.apache.hadoop.fs.contract.ContractTestUtils.assertPathExists;
/** /**
* Test compatibility between ABFS client and WASB client. * Test compatibility between ABFS client and WASB client.
*/ */
public class ITestWasbAbfsCompatibility extends DependencyInjectedTest { public class ITestWasbAbfsCompatibility extends AbstractAbfsIntegrationTest {
private static final String WASB_TEST_CONTEXT = "wasb test file"; private static final String WASB_TEST_CONTEXT = "wasb test file";
private static final String ABFS_TEST_CONTEXT = "abfs test file"; private static final String ABFS_TEST_CONTEXT = "abfs test file";
private static final String TEST_CONTEXT = "THIS IS FOR TEST"; private static final String TEST_CONTEXT = "THIS IS FOR TEST";
public ITestWasbAbfsCompatibility() throws Exception { private static final Logger LOG =
super(); LoggerFactory.getLogger(ITestWasbAbfsCompatibility.class);
Assume.assumeFalse(this.isEmulator()); public ITestWasbAbfsCompatibility() throws Exception {
Assume.assumeFalse("Emulator is not supported", isEmulator());
} }
@Test @Test
public void testListFileStatus() throws Exception { public void testListFileStatus() throws Exception {
// crate file using abfs // crate file using abfs
AzureBlobFileSystem fs = this.getFileSystem(); AzureBlobFileSystem fs = getFileSystem();
NativeAzureFileSystem wasb = this.getWasbFileSystem(); NativeAzureFileSystem wasb = getWasbFileSystem();
Path path1 = new Path("/testfiles/~12/!008/3/abFsTestfile"); Path path1 = new Path("/testfiles/~12/!008/3/abFsTestfile");
FSDataOutputStream abfsStream = fs.create(path1, true); try(FSDataOutputStream abfsStream = fs.create(path1, true)) {
abfsStream.write(ABFS_TEST_CONTEXT.getBytes()); abfsStream.write(ABFS_TEST_CONTEXT.getBytes());
abfsStream.flush(); abfsStream.flush();
abfsStream.hsync(); abfsStream.hsync();
abfsStream.close(); }
// create file using wasb // create file using wasb
Path path2 = new Path("/testfiles/~12/!008/3/nativeFsTestfile"); Path path2 = new Path("/testfiles/~12/!008/3/nativeFsTestfile");
System.out.println(wasb.getUri()); LOG.info("{}", wasb.getUri());
FSDataOutputStream nativeFsStream = wasb.create(path2, true); try(FSDataOutputStream nativeFsStream = wasb.create(path2, true)) {
nativeFsStream.write(WASB_TEST_CONTEXT.getBytes()); nativeFsStream.write(WASB_TEST_CONTEXT.getBytes());
nativeFsStream.flush(); nativeFsStream.flush();
nativeFsStream.hsync(); nativeFsStream.hsync();
nativeFsStream.close(); }
// list file using abfs and wasb // list file using abfs and wasb
FileStatus[] abfsFileStatus = fs.listStatus(new Path("/testfiles/~12/!008/3/")); FileStatus[] abfsFileStatus = fs.listStatus(new Path("/testfiles/~12/!008/3/"));
FileStatus[] nativeFsFileStatus = wasb.listStatus(new Path("/testfiles/~12/!008/3/")); FileStatus[] nativeFsFileStatus = wasb.listStatus(new Path("/testfiles/~12/!008/3/"));
@ -83,52 +86,34 @@ public class ITestWasbAbfsCompatibility extends DependencyInjectedTest {
boolean[] createFileWithAbfs = new boolean[]{false, true, false, true}; boolean[] createFileWithAbfs = new boolean[]{false, true, false, true};
boolean[] readFileWithAbfs = new boolean[]{false, true, true, false}; boolean[] readFileWithAbfs = new boolean[]{false, true, true, false};
AzureBlobFileSystem abfs = this.getFileSystem(); AzureBlobFileSystem abfs = getFileSystem();
NativeAzureFileSystem wasb = this.getWasbFileSystem(); NativeAzureFileSystem wasb = getWasbFileSystem();
FileSystem fs;
BufferedReader br = null;
for (int i = 0; i< 4; i++) { for (int i = 0; i< 4; i++) {
try {
Path path = new Path("/testfiles/~12/!008/testfile" + i); Path path = new Path("/testfiles/~12/!008/testfile" + i);
if (createFileWithAbfs[i]) { final FileSystem createFs = createFileWithAbfs[i] ? abfs : wasb;
fs = abfs;
} else {
fs = wasb;
}
// Write // Write
FSDataOutputStream nativeFsStream = fs.create(path, true); try(FSDataOutputStream nativeFsStream = createFs.create(path, true)) {
nativeFsStream.write(TEST_CONTEXT.getBytes()); nativeFsStream.write(TEST_CONTEXT.getBytes());
nativeFsStream.flush(); nativeFsStream.flush();
nativeFsStream.hsync(); nativeFsStream.hsync();
nativeFsStream.close(); }
// Check file status // Check file status
assertEquals(true, fs.exists(path)); ContractTestUtils.assertIsFile(createFs, path);
assertEquals(false, fs.getFileStatus(path).isDirectory());
// Read // Read
if (readFileWithAbfs[i]) { final FileSystem readFs = readFileWithAbfs[i] ? abfs : wasb;
fs = abfs;
} else { try(BufferedReader br =new BufferedReader(new InputStreamReader(readFs.open(path)))) {
fs = wasb;
}
FSDataInputStream inputStream = fs.open(path);
br = new BufferedReader(new InputStreamReader(fs.open(path)));
String line = br.readLine(); String line = br.readLine();
assertEquals(TEST_CONTEXT, line); assertEquals("Wrong text from " + readFs,
TEST_CONTEXT, line);
}
// Remove file // Remove file
fs.delete(path, true); assertDeleted(readFs, path, true);
assertFalse(fs.exists(path));
} catch (Exception e) {
e.printStackTrace();
} finally {
if (br != null) {
br.close();
}
}
} }
} }
@ -137,32 +122,22 @@ public class ITestWasbAbfsCompatibility extends DependencyInjectedTest {
boolean[] createDirWithAbfs = new boolean[]{false, true, false, true}; boolean[] createDirWithAbfs = new boolean[]{false, true, false, true};
boolean[] readDirWithAbfs = new boolean[]{false, true, true, false}; boolean[] readDirWithAbfs = new boolean[]{false, true, true, false};
AzureBlobFileSystem abfs = this.getFileSystem(); AzureBlobFileSystem abfs = getFileSystem();
NativeAzureFileSystem wasb = this.getWasbFileSystem(); NativeAzureFileSystem wasb = getWasbFileSystem();
FileSystem fs;
for (int i = 0; i < 4; i++) { for (int i = 0; i < 4; i++) {
Path path = new Path("/testDir/t" + i); Path path = new Path("/testDir/t" + i);
//create //create
if (createDirWithAbfs[i]) { final FileSystem createFs = createDirWithAbfs[i] ? abfs : wasb;
fs = abfs; assertTrue(createFs.mkdirs(path));
} else {
fs = wasb;
}
assertTrue(fs.mkdirs(path));
//check //check
assertTrue(fs.exists(path)); assertPathExists(createFs, "Created dir not found with " + createFs, path);
//read //read
if (readDirWithAbfs[i]) { final FileSystem readFs = readDirWithAbfs[i] ? abfs : wasb;
fs = abfs; assertPathExists(readFs, "Created dir not found with " + readFs,
} else { path);
fs = wasb; assertIsDirectory(readFs, path);
} assertDeleted(readFs, path, true);
assertTrue(fs.exists(path));
FileStatus dirStatus = fs.getFileStatus(path);
assertTrue(dirStatus.isDirectory());
fs.delete(path, true);
assertFalse(fs.exists(path));
} }
} }
@ -171,17 +146,18 @@ public class ITestWasbAbfsCompatibility extends DependencyInjectedTest {
public void testUrlConversion(){ public void testUrlConversion(){
String abfsUrl = "abfs://abcde-1111-1111-1111-1111@xxxx.dfs.xxx.xxx.xxxx.xxxx"; String abfsUrl = "abfs://abcde-1111-1111-1111-1111@xxxx.dfs.xxx.xxx.xxxx.xxxx";
String wabsUrl = "wasb://abcde-1111-1111-1111-1111@xxxx.blob.xxx.xxx.xxxx.xxxx"; String wabsUrl = "wasb://abcde-1111-1111-1111-1111@xxxx.blob.xxx.xxx.xxxx.xxxx";
Assert.assertEquals(abfsUrl, wasbUrlToAbfsUrl(wabsUrl)); assertEquals(abfsUrl, wasbUrlToAbfsUrl(wabsUrl));
Assert.assertEquals(wabsUrl, abfsUrlToWasbUrl(abfsUrl)); assertEquals(wabsUrl, abfsUrlToWasbUrl(abfsUrl));
} }
@Test @Test
public void testSetWorkingDirectory() throws Exception { public void testSetWorkingDirectory() throws Exception {
//create folders //create folders
AzureBlobFileSystem abfs = this.getFileSystem(); AzureBlobFileSystem abfs = getFileSystem();
NativeAzureFileSystem wasb = this.getWasbFileSystem(); NativeAzureFileSystem wasb = getWasbFileSystem();
assertTrue(abfs.mkdirs(new Path("/d1/d2/d3/d4"))); Path d1d4 = new Path("/d1/d2/d3/d4");
assertMkdirs(abfs, d1d4);
//set working directory to path1 //set working directory to path1
Path path1 = new Path("/d1/d2"); Path path1 = new Path("/d1/d2");
@ -195,7 +171,7 @@ public class ITestWasbAbfsCompatibility extends DependencyInjectedTest {
wasb.setWorkingDirectory(path2); wasb.setWorkingDirectory(path2);
abfs.setWorkingDirectory(path2); abfs.setWorkingDirectory(path2);
Path path3 = new Path("/d1/d2/d3/d4"); Path path3 = d1d4;
assertEquals(path3, wasb.getWorkingDirectory()); assertEquals(path3, wasb.getWorkingDirectory());
assertEquals(path3, abfs.getWorkingDirectory()); assertEquals(path3, abfs.getWorkingDirectory());
} }

View File

@ -18,14 +18,9 @@
package org.apache.hadoop.fs.azurebfs.constants; package org.apache.hadoop.fs.azurebfs.constants;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
/** /**
* Responsible to keep all the Azure Blob File System configurations keys in Hadoop configuration file. * Responsible to keep all the Azure Blob File System configurations keys in Hadoop configuration file.
*/ */
@InterfaceAudience.Public
@InterfaceStability.Evolving
public final class TestConfigurationKeys { public final class TestConfigurationKeys {
public static final String FS_AZURE_TEST_ACCOUNT_NAME = "fs.azure.test.account.name"; public static final String FS_AZURE_TEST_ACCOUNT_NAME = "fs.azure.test.account.name";
public static final String FS_AZURE_TEST_ACCOUNT_KEY_PREFIX = "fs.azure.test.account.key."; public static final String FS_AZURE_TEST_ACCOUNT_KEY_PREFIX = "fs.azure.test.account.key.";
@ -33,5 +28,9 @@ public final class TestConfigurationKeys {
public static final String FS_AZURE_TEST_HOST_PORT = "fs.azure.test.host.port"; public static final String FS_AZURE_TEST_HOST_PORT = "fs.azure.test.host.port";
public static final String FS_AZURE_CONTRACT_TEST_URI = "fs.contract.test.fs.abfs"; public static final String FS_AZURE_CONTRACT_TEST_URI = "fs.contract.test.fs.abfs";
public static final String ABFS_TEST_RESOURCE_XML = "azure-bfs-test.xml";
public static final String ABFS_TEST_CONTAINER_PREFIX = "abfs-testcontainer-";
private TestConfigurationKeys() {} private TestConfigurationKeys() {}
} }

View File

@ -22,36 +22,37 @@ import java.net.URI;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.CommonConfigurationKeysPublic; import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
import org.apache.hadoop.fs.azurebfs.DependencyInjectedTest; import org.apache.hadoop.fs.azurebfs.AbstractAbfsIntegrationTest;
import org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes; import org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes;
import org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys; import org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys;
/** /**
* Dependency inject for ABFS contract tests. * Bind ABFS contract tests to the Azure test setup/teardown.
*/ */
public class DependencyInjectedContractTest extends DependencyInjectedTest { public class ABFSContractTestBinding extends AbstractAbfsIntegrationTest {
private final URI testUri; private final URI testUri;
public DependencyInjectedContractTest(final boolean secure) throws Exception { public ABFSContractTestBinding(final boolean secure) throws Exception {
this(secure, true); this(secure, true);
} }
public DependencyInjectedContractTest(final boolean secure, final boolean useExistedFileSystem) throws Exception{ public ABFSContractTestBinding(final boolean secure,
final boolean useExistingFileSystem) throws Exception{
super(secure); super(secure);
if (useExistedFileSystem) { if (useExistingFileSystem) {
Configuration configuration = getConfiguration(); Configuration configuration = getConfiguration();
String testUrl = configuration.get(TestConfigurationKeys.FS_AZURE_CONTRACT_TEST_URI); String testUrl = configuration.get(TestConfigurationKeys.FS_AZURE_CONTRACT_TEST_URI);
if (secure) { if (secure) {
testUrl = testUrl.replaceFirst(FileSystemUriSchemes.ABFS_SCHEME, FileSystemUriSchemes.ABFS_SECURE_SCHEME); testUrl = testUrl.replaceFirst(FileSystemUriSchemes.ABFS_SCHEME, FileSystemUriSchemes.ABFS_SECURE_SCHEME);
} }
updateTestUrl(testUrl); setTestUrl(testUrl);
this.testUri = new URI(testUrl); this.testUri = new URI(testUrl);
//Get container for contract tests //Get container for contract tests
configuration.set(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY, this.testUri.toString()); configuration.set(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY, this.testUri.toString());
String[] splitAuthority = this.testUri.getAuthority().split("\\@"); String[] splitAuthority = this.testUri.getAuthority().split("\\@");
updateFileSystemName(splitAuthority[0]); setFileSystemName(splitAuthority[0]);
} else { } else {
this.testUri = new URI(super.getTestUrl()); this.testUri = new URI(super.getTestUrl());
} }

View File

@ -29,12 +29,12 @@ import org.apache.hadoop.fs.contract.AbstractBondedFSContract;
* identifier, if defined. This guarantees paths unique to tests * identifier, if defined. This guarantees paths unique to tests
* running in parallel. * running in parallel.
*/ */
public class ITestAbfsFileSystemContract extends AbstractBondedFSContract { public class AbfsFileSystemContract extends AbstractBondedFSContract {
public static final String CONTRACT_XML = "abfs.xml"; public static final String CONTRACT_XML = "abfs.xml";
private final boolean isSecure; private final boolean isSecure;
protected ITestAbfsFileSystemContract(final Configuration conf, boolean secure) { protected AbfsFileSystemContract(final Configuration conf, boolean secure) {
super(conf); super(conf);
//insert the base features //insert the base features
addConfResource(CONTRACT_XML); addConfResource(CONTRACT_XML);
@ -43,12 +43,23 @@ public class ITestAbfsFileSystemContract extends AbstractBondedFSContract {
@Override @Override
public String getScheme() { public String getScheme() {
return isSecure ? FileSystemUriSchemes.ABFS_SECURE_SCHEME : FileSystemUriSchemes.ABFS_SCHEME; return isSecure ?
FileSystemUriSchemes.ABFS_SECURE_SCHEME
: FileSystemUriSchemes.ABFS_SCHEME;
} }
@Override @Override
public Path getTestPath() { public Path getTestPath() {
Path path = new Path(UriUtils.generateUniqueTestPath()); return new Path(UriUtils.generateUniqueTestPath());
return path; }
@Override
public String toString() {
final StringBuilder sb = new StringBuilder(
"AbfsFileSystemContract{");
sb.append("isSecure=").append(isSecure);
sb.append(super.toString());
sb.append('}');
return sb.toString();
} }
} }

View File

@ -20,6 +20,7 @@ package org.apache.hadoop.fs.azurebfs.contract;
import java.util.Arrays; import java.util.Arrays;
import org.junit.Test;
import org.junit.runner.RunWith; import org.junit.runner.RunWith;
import org.junit.runners.Parameterized; import org.junit.runners.Parameterized;
@ -40,30 +41,31 @@ public class ITestAbfsFileSystemContractAppend extends AbstractContractAppendTes
} }
private final boolean isSecure; private final boolean isSecure;
private final DependencyInjectedContractTest dependencyInjectedContractTest; private final ABFSContractTestBinding binding;
public ITestAbfsFileSystemContractAppend(final boolean secure) throws Exception { public ITestAbfsFileSystemContractAppend(final boolean secure) throws Exception {
this.isSecure = secure; this.isSecure = secure;
dependencyInjectedContractTest = new DependencyInjectedContractTest(this.isSecure); binding = new ABFSContractTestBinding(this.isSecure);
} }
@Override @Override
public void setup() throws Exception { public void setup() throws Exception {
dependencyInjectedContractTest.initialize(); binding.setup();
super.setup(); super.setup();
} }
@Override @Override
protected Configuration createConfiguration() { protected Configuration createConfiguration() {
return this.dependencyInjectedContractTest.getConfiguration(); return binding.getConfiguration();
} }
@Override @Override
protected AbstractFSContract createContract(final Configuration conf) { protected AbstractFSContract createContract(final Configuration conf) {
return new ITestAbfsFileSystemContract(conf, this.isSecure); return new AbfsFileSystemContract(conf, isSecure);
} }
@Override @Override
@Test
public void testRenameFileBeingAppended() throws Throwable { public void testRenameFileBeingAppended() throws Throwable {
skip("Skipping as renaming an opened file is not supported"); skip("Skipping as renaming an opened file is not supported");
} }

View File

@ -37,26 +37,26 @@ public class ITestAbfsFileSystemContractConcat extends AbstractContractConcatTes
} }
private final boolean isSecure; private final boolean isSecure;
private final DependencyInjectedContractTest dependencyInjectedContractTest; private final ABFSContractTestBinding binding;
public ITestAbfsFileSystemContractConcat(final boolean secure) throws Exception { public ITestAbfsFileSystemContractConcat(final boolean secure) throws Exception {
this.isSecure = secure; isSecure = secure;
dependencyInjectedContractTest = new DependencyInjectedContractTest(isSecure); binding = new ABFSContractTestBinding(isSecure);
} }
@Override @Override
public void setup() throws Exception { public void setup() throws Exception {
dependencyInjectedContractTest.initialize(); binding.setup();
super.setup(); super.setup();
} }
@Override @Override
protected Configuration createConfiguration() { protected Configuration createConfiguration() {
return this.dependencyInjectedContractTest.getConfiguration(); return binding.getConfiguration();
} }
@Override @Override
protected AbstractFSContract createContract(final Configuration conf) { protected AbstractFSContract createContract(final Configuration conf) {
return new ITestAbfsFileSystemContract(conf, this.isSecure); return new AbfsFileSystemContract(conf, isSecure);
} }
} }

View File

@ -38,26 +38,26 @@ public class ITestAbfsFileSystemContractCreate extends AbstractContractCreateTes
} }
private final boolean isSecure; private final boolean isSecure;
private final DependencyInjectedContractTest dependencyInjectedContractTest; private final ABFSContractTestBinding binding;
public ITestAbfsFileSystemContractCreate(final boolean secure) throws Exception { public ITestAbfsFileSystemContractCreate(final boolean secure) throws Exception {
this.isSecure = secure; this.isSecure = secure;
dependencyInjectedContractTest = new DependencyInjectedContractTest(this.isSecure); binding = new ABFSContractTestBinding(this.isSecure);
} }
@Override @Override
public void setup() throws Exception { public void setup() throws Exception {
dependencyInjectedContractTest.initialize(); binding.setup();
super.setup(); super.setup();
} }
@Override @Override
protected Configuration createConfiguration() { protected Configuration createConfiguration() {
return this.dependencyInjectedContractTest.getConfiguration(); return binding.getConfiguration();
} }
@Override @Override
protected AbstractFSContract createContract(final Configuration conf) { protected AbstractFSContract createContract(final Configuration conf) {
return new ITestAbfsFileSystemContract(conf, this.isSecure); return new AbfsFileSystemContract(conf, isSecure);
} }
} }

View File

@ -38,26 +38,26 @@ public class ITestAbfsFileSystemContractDelete extends AbstractContractDeleteTes
} }
private final boolean isSecure; private final boolean isSecure;
private final DependencyInjectedContractTest dependencyInjectedContractTest; private final ABFSContractTestBinding binding;
public ITestAbfsFileSystemContractDelete(final boolean secure) throws Exception { public ITestAbfsFileSystemContractDelete(final boolean secure) throws Exception {
this.isSecure = secure; this.isSecure = secure;
dependencyInjectedContractTest = new DependencyInjectedContractTest(isSecure); binding = new ABFSContractTestBinding(isSecure);
} }
@Override @Override
public void setup() throws Exception { public void setup() throws Exception {
dependencyInjectedContractTest.initialize(); binding.setup();
super.setup(); super.setup();
} }
@Override @Override
protected Configuration createConfiguration() { protected Configuration createConfiguration() {
return this.dependencyInjectedContractTest.getConfiguration(); return binding.getConfiguration();
} }
@Override @Override
protected AbstractFSContract createContract(final Configuration conf) { protected AbstractFSContract createContract(final Configuration conf) {
return new ITestAbfsFileSystemContract(conf, this.isSecure); return new AbfsFileSystemContract(conf, isSecure);
} }
} }

View File

@ -25,20 +25,20 @@ import org.apache.hadoop.tools.contract.AbstractContractDistCpTest;
* Contract test for distCp operation. * Contract test for distCp operation.
*/ */
public class ITestAbfsFileSystemContractDistCp extends AbstractContractDistCpTest { public class ITestAbfsFileSystemContractDistCp extends AbstractContractDistCpTest {
private final DependencyInjectedContractTest dependencyInjectedContractTest; private final ABFSContractTestBinding binding;
public ITestAbfsFileSystemContractDistCp() throws Exception { public ITestAbfsFileSystemContractDistCp() throws Exception {
dependencyInjectedContractTest = new DependencyInjectedContractTest(false); binding = new ABFSContractTestBinding(false);
} }
@Override @Override
public void setup() throws Exception { public void setup() throws Exception {
dependencyInjectedContractTest.initialize(); binding.setup();
super.setup(); super.setup();
} }
@Override @Override
protected ITestAbfsFileSystemContract createContract(Configuration conf) { protected AbfsFileSystemContract createContract(Configuration conf) {
return new ITestAbfsFileSystemContract(conf, false); return new AbfsFileSystemContract(conf, false);
} }
} }

View File

@ -37,26 +37,26 @@ public class ITestAbfsFileSystemContractGetFileStatus extends AbstractContractGe
} }
private final boolean isSecure; private final boolean isSecure;
private final DependencyInjectedContractTest dependencyInjectedContractTest; private final ABFSContractTestBinding binding;
public ITestAbfsFileSystemContractGetFileStatus(final boolean secure) throws Exception { public ITestAbfsFileSystemContractGetFileStatus(final boolean secure) throws Exception {
this.isSecure = secure; this.isSecure = secure;
dependencyInjectedContractTest = new DependencyInjectedContractTest(isSecure); binding = new ABFSContractTestBinding(isSecure);
} }
@Override @Override
public void setup() throws Exception { public void setup() throws Exception {
dependencyInjectedContractTest.initialize(); binding.setup();
super.setup(); super.setup();
} }
@Override @Override
protected Configuration createConfiguration() { protected Configuration createConfiguration() {
return this.dependencyInjectedContractTest.getConfiguration(); return this.binding.getConfiguration();
} }
@Override @Override
protected AbstractFSContract createContract(final Configuration conf) { protected AbstractFSContract createContract(final Configuration conf) {
return new ITestAbfsFileSystemContract(conf, this.isSecure); return new AbfsFileSystemContract(conf, this.isSecure);
} }
} }

View File

@ -38,26 +38,26 @@ public class ITestAbfsFileSystemContractMkdir extends AbstractContractMkdirTest
} }
private final boolean isSecure; private final boolean isSecure;
private final DependencyInjectedContractTest dependencyInjectedContractTest; private final ABFSContractTestBinding binding;
public ITestAbfsFileSystemContractMkdir(final boolean secure) throws Exception { public ITestAbfsFileSystemContractMkdir(final boolean secure) throws Exception {
this.isSecure = secure; this.isSecure = secure;
dependencyInjectedContractTest = new DependencyInjectedContractTest(secure); binding = new ABFSContractTestBinding(secure);
} }
@Override @Override
public void setup() throws Exception { public void setup() throws Exception {
dependencyInjectedContractTest.initialize(); binding.setup();
super.setup(); super.setup();
} }
@Override @Override
protected Configuration createConfiguration() { protected Configuration createConfiguration() {
return this.dependencyInjectedContractTest.getConfiguration(); return binding.getConfiguration();
} }
@Override @Override
protected AbstractFSContract createContract(final Configuration conf) { protected AbstractFSContract createContract(final Configuration conf) {
return new ITestAbfsFileSystemContract(conf, this.isSecure); return new AbfsFileSystemContract(conf, isSecure);
} }
} }

View File

@ -38,26 +38,26 @@ public class ITestAbfsFileSystemContractOpen extends AbstractContractOpenTest {
} }
private final boolean isSecure; private final boolean isSecure;
private final DependencyInjectedContractTest dependencyInjectedContractTest; private final ABFSContractTestBinding binding;
public ITestAbfsFileSystemContractOpen(final boolean secure) throws Exception { public ITestAbfsFileSystemContractOpen(final boolean secure) throws Exception {
this.isSecure = secure; this.isSecure = secure;
dependencyInjectedContractTest = new DependencyInjectedContractTest(this.isSecure); binding = new ABFSContractTestBinding(this.isSecure);
} }
@Override @Override
public void setup() throws Exception { public void setup() throws Exception {
dependencyInjectedContractTest.initialize(); binding.setup();
super.setup(); super.setup();
} }
@Override @Override
protected Configuration createConfiguration() { protected Configuration createConfiguration() {
return this.dependencyInjectedContractTest.getConfiguration(); return binding.getConfiguration();
} }
@Override @Override
protected AbstractFSContract createContract(final Configuration conf) { protected AbstractFSContract createContract(final Configuration conf) {
return new ITestAbfsFileSystemContract(conf, this.isSecure); return new AbfsFileSystemContract(conf, isSecure);
} }
} }

View File

@ -38,26 +38,26 @@ public class ITestAbfsFileSystemContractRename extends AbstractContractRenameTes
} }
private final boolean isSecure; private final boolean isSecure;
private final DependencyInjectedContractTest dependencyInjectedContractTest; private final ABFSContractTestBinding binding;
public ITestAbfsFileSystemContractRename(final boolean secure) throws Exception { public ITestAbfsFileSystemContractRename(final boolean secure) throws Exception {
this.isSecure = secure; this.isSecure = secure;
dependencyInjectedContractTest = new DependencyInjectedContractTest(this.isSecure); binding = new ABFSContractTestBinding(this.isSecure);
} }
@Override @Override
public void setup() throws Exception { public void setup() throws Exception {
dependencyInjectedContractTest.initialize(); binding.setup();
super.setup(); super.setup();
} }
@Override @Override
protected Configuration createConfiguration() { protected Configuration createConfiguration() {
return this.dependencyInjectedContractTest.getConfiguration(); return binding.getConfiguration();
} }
@Override @Override
protected AbstractFSContract createContract(final Configuration conf) { protected AbstractFSContract createContract(final Configuration conf) {
return new ITestAbfsFileSystemContract(conf, this.isSecure); return new AbfsFileSystemContract(conf, isSecure);
} }
} }

View File

@ -37,27 +37,27 @@ public class ITestAbfsFileSystemContractRootDirectory extends AbstractContractRo
} }
private final boolean isSecure; private final boolean isSecure;
private final DependencyInjectedContractTest dependencyInjectedContractTest; private final ABFSContractTestBinding binding;
public ITestAbfsFileSystemContractRootDirectory(final boolean secure) throws Exception { public ITestAbfsFileSystemContractRootDirectory(final boolean secure) throws Exception {
this.isSecure = secure; this.isSecure = secure;
dependencyInjectedContractTest = new DependencyInjectedContractTest(secure); binding = new ABFSContractTestBinding(secure);
} }
@Override @Override
public void setup() throws Exception { public void setup() throws Exception {
dependencyInjectedContractTest.initialize(); binding.setup();
super.setup(); super.setup();
} }
@Override @Override
protected Configuration createConfiguration() { protected Configuration createConfiguration() {
return this.dependencyInjectedContractTest.getConfiguration(); return binding.getConfiguration();
} }
@Override @Override
protected AbstractFSContract createContract(final Configuration conf) { protected AbstractFSContract createContract(final Configuration conf) {
return new ITestAbfsFileSystemContract(conf, this.isSecure); return new AbfsFileSystemContract(conf, isSecure);
} }
@Override @Override

View File

@ -25,20 +25,20 @@ import org.apache.hadoop.tools.contract.AbstractContractDistCpTest;
* Contract test for secure distCP operation. * Contract test for secure distCP operation.
*/ */
public class ITestAbfsFileSystemContractSecureDistCp extends AbstractContractDistCpTest { public class ITestAbfsFileSystemContractSecureDistCp extends AbstractContractDistCpTest {
private final DependencyInjectedContractTest dependencyInjectedContractTest; private final ABFSContractTestBinding binding;
public ITestAbfsFileSystemContractSecureDistCp() throws Exception { public ITestAbfsFileSystemContractSecureDistCp() throws Exception {
dependencyInjectedContractTest = new DependencyInjectedContractTest(true); binding = new ABFSContractTestBinding(true);
} }
@Override @Override
public void setup() throws Exception { public void setup() throws Exception {
dependencyInjectedContractTest.initialize(); binding.setup();
super.setup(); super.setup();
} }
@Override @Override
protected ITestAbfsFileSystemContract createContract(Configuration conf) { protected AbfsFileSystemContract createContract(Configuration conf) {
return new ITestAbfsFileSystemContract(conf, true); return new AbfsFileSystemContract(conf, true);
} }
} }

View File

@ -38,26 +38,26 @@ public class ITestAbfsFileSystemContractSeek extends AbstractContractSeekTest{
} }
private final boolean isSecure; private final boolean isSecure;
private final DependencyInjectedContractTest dependencyInjectedContractTest; private final ABFSContractTestBinding binding;
public ITestAbfsFileSystemContractSeek(final boolean secure) throws Exception { public ITestAbfsFileSystemContractSeek(final boolean secure) throws Exception {
this.isSecure = secure; this.isSecure = secure;
dependencyInjectedContractTest = new DependencyInjectedContractTest(this.isSecure); binding = new ABFSContractTestBinding(this.isSecure);
} }
@Override @Override
public void setup() throws Exception { public void setup() throws Exception {
dependencyInjectedContractTest.initialize(); binding.setup();
super.setup(); super.setup();
} }
@Override @Override
protected Configuration createConfiguration() { protected Configuration createConfiguration() {
return this.dependencyInjectedContractTest.getConfiguration(); return binding.getConfiguration();
} }
@Override @Override
protected AbstractFSContract createContract(final Configuration conf) { protected AbstractFSContract createContract(final Configuration conf) {
return new ITestAbfsFileSystemContract(conf, this.isSecure); return new AbfsFileSystemContract(conf, isSecure);
} }
} }

View File

@ -37,26 +37,26 @@ public class ITestAbfsFileSystemContractSetTimes extends AbstractContractSetTime
} }
private final boolean isSecure; private final boolean isSecure;
private final DependencyInjectedContractTest dependencyInjectedContractTest; private final ABFSContractTestBinding binding;
public ITestAbfsFileSystemContractSetTimes(final boolean secure) throws Exception { public ITestAbfsFileSystemContractSetTimes(final boolean secure) throws Exception {
this.isSecure = secure; this.isSecure = secure;
dependencyInjectedContractTest = new DependencyInjectedContractTest(this.isSecure); binding = new ABFSContractTestBinding(this.isSecure);
} }
@Override @Override
public void setup() throws Exception { public void setup() throws Exception {
dependencyInjectedContractTest.initialize(); binding.setup();
super.setup(); super.setup();
} }
@Override @Override
protected Configuration createConfiguration() { protected Configuration createConfiguration() {
return this.dependencyInjectedContractTest.getConfiguration(); return binding.getConfiguration();
} }
@Override @Override
protected AbstractFSContract createContract(final Configuration conf) { protected AbstractFSContract createContract(final Configuration conf) {
return new ITestAbfsFileSystemContract(conf, this.isSecure); return new AbfsFileSystemContract(conf, isSecure);
} }
} }

View File

@ -22,8 +22,8 @@ import java.io.IOException;
import org.apache.hadoop.fs.FileSystemContractBaseTest; import org.apache.hadoop.fs.FileSystemContractBaseTest;
import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.contract.ContractTestUtils;
import org.junit.After;
import org.junit.Before; import org.junit.Before;
import org.junit.Ignore; import org.junit.Ignore;
import org.junit.Test; import org.junit.Test;
@ -35,26 +35,31 @@ import static org.junit.Assert.assertTrue;
* Basic Contract test for Azure BlobFileSystem. * Basic Contract test for Azure BlobFileSystem.
*/ */
public class ITestAzureBlobFileSystemBasics extends FileSystemContractBaseTest { public class ITestAzureBlobFileSystemBasics extends FileSystemContractBaseTest {
private final DependencyInjectedContractTest dependencyInjectedContractTest; private final ABFSContractTestBinding binding;
public ITestAzureBlobFileSystemBasics() throws Exception { public ITestAzureBlobFileSystemBasics() throws Exception {
// If all contract tests are running in parallel, some root level tests in FileSystemContractBaseTest will fail // If all contract tests are running in parallel, some root level tests in FileSystemContractBaseTest will fail
// due to the race condition. Hence for this contract test it should be tested in different container // due to the race condition. Hence for this contract test it should be tested in different container
dependencyInjectedContractTest = new DependencyInjectedContractTest(false, false); binding = new ABFSContractTestBinding(false, false);
} }
@Before @Before
public void setUp() throws Exception { public void setUp() throws Exception {
this.dependencyInjectedContractTest.initialize(); binding.setup();
fs = this.dependencyInjectedContractTest.getFileSystem(); fs = binding.getFileSystem();
} }
@After @Override
public void testCleanup() throws Exception { public void tearDown() throws Exception {
// This contract test is not using existing container for test, // This contract test is not using existing container for test,
// instead it creates its own temp container for test, hence we need to destroy // instead it creates its own temp container for test, hence we need to destroy
// it after the test. // it after the test.
this.dependencyInjectedContractTest.testCleanup(); try {
super.tearDown();
} finally {
binding.teardown();
}
} }
@Test @Test
@ -82,7 +87,7 @@ public class ITestAzureBlobFileSystemBasics extends FileSystemContractBaseTest {
Path filePath = path("testListStatus/file"); Path filePath = path("testListStatus/file");
assertTrue(fs.mkdirs(folderPath)); assertTrue(fs.mkdirs(folderPath));
fs.create(filePath); ContractTestUtils.touch(fs, filePath);
FileStatus[] listFolderStatus; FileStatus[] listFolderStatus;
listFolderStatus = fs.listStatus(path("testListStatus")); listFolderStatus = fs.listStatus(path("testListStatus"));

View File

@ -19,10 +19,11 @@
package org.apache.hadoop.fs.azurebfs.diagnostics; package org.apache.hadoop.fs.azurebfs.diagnostics;
import org.apache.commons.codec.Charsets; import org.apache.commons.codec.Charsets;
import org.junit.Assert;
import org.junit.Test; import org.junit.Test;
import org.apache.hadoop.fs.azurebfs.DependencyInjectedTest;
import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidConfigurationValueException; import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidConfigurationValueException;
import org.apache.commons.codec.binary.Base64; import org.apache.commons.codec.binary.Base64;
@ -36,7 +37,7 @@ import static org.junit.Assert.assertEquals;
/** /**
* Test configuration validators. * Test configuration validators.
*/ */
public class TestConfigurationValidators extends DependencyInjectedTest { public class TestConfigurationValidators extends Assert {
private static final String FAKE_KEY = "FakeKey"; private static final String FAKE_KEY = "FakeKey";

View File

@ -18,61 +18,74 @@
package org.apache.hadoop.fs.azurebfs.services; package org.apache.hadoop.fs.azurebfs.services;
import java.util.Arrays;
import java.util.Random; import java.util.Random;
import org.junit.Test;
import org.junit.runner.RunWith;
import org.junit.runners.Parameterized;
import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.azurebfs.AbstractAbfsScaleTest;
import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystem; import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystem;
import org.apache.hadoop.fs.azurebfs.DependencyInjectedTest;
import org.junit.Test;
import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.MIN_BUFFER_SIZE;
import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.DEFAULT_READ_BUFFER_SIZE; import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.DEFAULT_READ_BUFFER_SIZE;
import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.MAX_BUFFER_SIZE; import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.MAX_BUFFER_SIZE;
import static org.junit.Assert.assertArrayEquals; import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.MIN_BUFFER_SIZE;
import static org.junit.Assert.assertNotEquals;
/** /**
* Test read, write and seek. * Test read, write and seek.
* Uses package-private methods in AbfsConfiguration, which is why it is in
* this package.
*/ */
public class ITestReadWriteAndSeek extends DependencyInjectedTest { @RunWith(Parameterized.class)
public class ITestAbfsReadWriteAndSeek extends AbstractAbfsScaleTest {
private static final Path TEST_PATH = new Path("/testfile"); private static final Path TEST_PATH = new Path("/testfile");
public ITestReadWriteAndSeek() {
super(); @Parameterized.Parameters(name = "Size={0}")
public static Iterable<Object[]> sizes() {
return Arrays.asList(new Object[][]{{MIN_BUFFER_SIZE},
{DEFAULT_READ_BUFFER_SIZE},
{MAX_BUFFER_SIZE}});
}
private final int size;
public ITestAbfsReadWriteAndSeek(final int size) {
this.size = size;
} }
@Test @Test
public void testReadAndWriteWithDifferentBufferSizesAndSeek() throws Exception { public void testReadAndWriteWithDifferentBufferSizesAndSeek() throws Exception {
testReadWriteAndSeek(MIN_BUFFER_SIZE); testReadWriteAndSeek(size);
testReadWriteAndSeek(DEFAULT_READ_BUFFER_SIZE);
testReadWriteAndSeek(MAX_BUFFER_SIZE);
} }
private void testReadWriteAndSeek(int bufferSize) throws Exception { private void testReadWriteAndSeek(int bufferSize) throws Exception {
final AzureBlobFileSystem fs = this.getFileSystem(); final AzureBlobFileSystem fs = getFileSystem();
final AbfsConfiguration abfsConfiguration = new AbfsConfiguration(this.getConfiguration()); final AbfsConfiguration abfsConfiguration = new AbfsConfiguration(getConfiguration());
fs.create(TEST_PATH);
abfsConfiguration.setWriteBufferSize(bufferSize); abfsConfiguration.setWriteBufferSize(bufferSize);
abfsConfiguration.setReadBufferSize(bufferSize); abfsConfiguration.setReadBufferSize(bufferSize);
final FSDataOutputStream stream = fs.create(TEST_PATH);
final byte[] b = new byte[2 * bufferSize]; final byte[] b = new byte[2 * bufferSize];
new Random().nextBytes(b); new Random().nextBytes(b);
try(final FSDataOutputStream stream = fs.create(TEST_PATH)) {
stream.write(b); stream.write(b);
stream.close(); }
final byte[] r = new byte[2 * bufferSize]; final byte[] readBuffer = new byte[2 * bufferSize];
final FSDataInputStream inputStream = fs.open(TEST_PATH); int result;
try(final FSDataInputStream inputStream = fs.open(TEST_PATH)) {
inputStream.seek(bufferSize); inputStream.seek(bufferSize);
int result = inputStream.read(r, bufferSize, bufferSize); result = inputStream.read(readBuffer, bufferSize, bufferSize);
assertNotEquals(-1, result); assertNotEquals(-1, result);
inputStream.seek(0); inputStream.seek(0);
result = inputStream.read(r, 0, bufferSize); result = inputStream.read(readBuffer, 0, bufferSize);
assertNotEquals(-1, result); }
assertArrayEquals(r, b); assertNotEquals("data read in final read()", -1, result);
assertArrayEquals(readBuffer, b);
} }
} }

View File

@ -48,7 +48,7 @@ import org.junit.Test;
* Test ConfigurationServiceFieldsValidation. * Test ConfigurationServiceFieldsValidation.
*/ */
public class TestAbfsConfigurationFieldsValidation { public class TestAbfsConfigurationFieldsValidation {
private AbfsConfiguration abfsConfiguration; private final AbfsConfiguration abfsConfiguration;
private static final String INT_KEY= "intKey"; private static final String INT_KEY= "intKey";
private static final String LONG_KEY= "longKey"; private static final String LONG_KEY= "longKey";
@ -89,12 +89,10 @@ public class TestAbfsConfigurationFieldsValidation {
private boolean boolField; private boolean boolField;
public TestAbfsConfigurationFieldsValidation() throws Exception { public TestAbfsConfigurationFieldsValidation() throws Exception {
super();
Base64 base64 = new Base64(); Base64 base64 = new Base64();
this.encodedString = new String(base64.encode("base64Value".getBytes(Charsets.UTF_8)), Charsets.UTF_8); this.encodedString = new String(base64.encode("base64Value".getBytes(Charsets.UTF_8)), Charsets.UTF_8);
this.encodedAccountKey = new String(base64.encode("someAccountKey".getBytes(Charsets.UTF_8)), Charsets.UTF_8); this.encodedAccountKey = new String(base64.encode("someAccountKey".getBytes(Charsets.UTF_8)), Charsets.UTF_8);
Configuration configuration = new Configuration(); Configuration configuration = new Configuration(false);
configuration.addResource("azure-bfs-test.xml");
configuration.set(INT_KEY, "1234565"); configuration.set(INT_KEY, "1234565");
configuration.set(LONG_KEY, "4194304"); configuration.set(LONG_KEY, "4194304");
configuration.set(STRING_KEY, "stringValue"); configuration.set(STRING_KEY, "stringValue");

View File

@ -24,7 +24,7 @@ import com.microsoft.azure.storage.blob.CloudBlobContainer;
import org.apache.hadoop.fs.azure.AzureBlobStorageTestAccount; import org.apache.hadoop.fs.azure.AzureBlobStorageTestAccount;
import org.junit.Test; import org.junit.Test;
import static org.apache.hadoop.fs.azurebfs.DependencyInjectedTest.TEST_CONTAINER_PREFIX; import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.ABFS_TEST_CONTAINER_PREFIX;
/** /**
* If unit tests were interrupted and crushed accidentally, the test containers won't be deleted. * If unit tests were interrupted and crushed accidentally, the test containers won't be deleted.
@ -38,7 +38,7 @@ public class CleanUpAbfsTestContainer {
CloudStorageAccount storageAccount = AzureBlobStorageTestAccount.createTestAccount(); CloudStorageAccount storageAccount = AzureBlobStorageTestAccount.createTestAccount();
CloudBlobClient blobClient = storageAccount.createCloudBlobClient(); CloudBlobClient blobClient = storageAccount.createCloudBlobClient();
Iterable<CloudBlobContainer> containers Iterable<CloudBlobContainer> containers
= blobClient.listContainers(TEST_CONTAINER_PREFIX); = blobClient.listContainers(ABFS_TEST_CONTAINER_PREFIX);
for (CloudBlobContainer container : containers) { for (CloudBlobContainer container : containers) {
count++; count++;
System.out.println(String.format("Container %s URI %s", System.out.println(String.format("Container %s URI %s",
@ -54,7 +54,7 @@ public class CleanUpAbfsTestContainer {
CloudStorageAccount storageAccount = AzureBlobStorageTestAccount.createTestAccount(); CloudStorageAccount storageAccount = AzureBlobStorageTestAccount.createTestAccount();
CloudBlobClient blobClient = storageAccount.createCloudBlobClient(); CloudBlobClient blobClient = storageAccount.createCloudBlobClient();
Iterable<CloudBlobContainer> containers Iterable<CloudBlobContainer> containers
= blobClient.listContainers(TEST_CONTAINER_PREFIX); = blobClient.listContainers(ABFS_TEST_CONTAINER_PREFIX);
for (CloudBlobContainer container : containers) { for (CloudBlobContainer container : containers) {
System.out.println(String.format("Container %s URI %s", System.out.println(String.format("Container %s URI %s",
container.getName(), container.getName(),

View File

@ -1,3 +1,4 @@
<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
<!-- <!--
~ Licensed to the Apache Software Foundation (ASF) under one ~ Licensed to the Apache Software Foundation (ASF) under one
~ or more contributor license agreements. See the NOTICE file ~ or more contributor license agreements. See the NOTICE file
@ -15,7 +16,6 @@
~ See the License for the specific language governing permissions and ~ See the License for the specific language governing permissions and
~ limitations under the License. ~ limitations under the License.
--> -->
<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
<configuration xmlns:xi="http://www.w3.org/2001/XInclude"> <configuration xmlns:xi="http://www.w3.org/2001/XInclude">
<property> <property>
<name>fs.contract.test.root-tests-enabled</name> <name>fs.contract.test.root-tests-enabled</name>

View File

@ -1,4 +1,5 @@
<?xml version="1.0" encoding="UTF-8"?> <?xml version="1.0" encoding="UTF-8"?>
<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
<!-- <!--
Licensed under the Apache License, Version 2.0 (the "License"); Licensed under the Apache License, Version 2.0 (the "License");
you may not use this file except in compliance with the License. you may not use this file except in compliance with the License.
@ -13,7 +14,6 @@
limitations under the License. limitations under the License.
--> -->
<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
<configuration xmlns:xi="http://www.w3.org/2001/XInclude"> <configuration xmlns:xi="http://www.w3.org/2001/XInclude">
<!-- <!--
@ -63,14 +63,4 @@
<!--<value>false</value>--> <!--<value>false</value>-->
<!--</property>--> <!--</property>-->
<property>
<name>fs.AbstractFileSystem.abfs.impl</name>
<value>org.apache.hadoop.fs.azurebfs.Abfs</value>
</property>
<property>
<name>fs.AbstractFileSystem.abfss.impl</name>
<value>org.apache.hadoop.fs.azurebfs.Abfss</value>
</property>
</configuration> </configuration>

View File

@ -26,3 +26,35 @@ log4j.logger.org.apache.hadoop.fs.azure.AzureFileSystemThreadPoolExecutor=DEBUG
log4j.logger.org.apache.hadoop.fs.azure.BlockBlobAppendStream=DEBUG log4j.logger.org.apache.hadoop.fs.azure.BlockBlobAppendStream=DEBUG
log4j.logger.org.apache.hadoop.fs.azurebfs.contracts.services.TracingService=TRACE log4j.logger.org.apache.hadoop.fs.azurebfs.contracts.services.TracingService=TRACE
log4j.logger.org.apache.hadoop.fs.azurebfs.services.AbfsClient=DEBUG log4j.logger.org.apache.hadoop.fs.azurebfs.services.AbfsClient=DEBUG
# after here: turn off log messages from other parts of the system
# which only clutter test reports.
log4j.logger.org.apache.hadoop.util.NativeCodeLoader=ERROR
log4j.logger.org.apache.hadoop.conf.Configuration.deprecation=WARN
log4j.logger.org.apache.hadoop.util.GSet=WARN
# MiniDFS clusters can be noisy
log4j.logger.org.apache.hadoop.hdfs.server=ERROR
log4j.logger.org.apache.hadoop.metrics2=WARN
log4j.logger.org.apache.hadoop.net.NetworkTopology=WARN
log4j.logger.org.apache.hadoop.util.JvmPauseMonitor=WARN
log4j.logger.org.apache.hadoop.ipc=WARN
log4j.logger.org.apache.hadoop.http=WARN
log4j.logger.org.apache.hadoop.security.authentication.server.AuthenticationFilter=WARN
log4j.logger.org.apache.hadoop.util.HostsFileReader=WARN
log4j.logger.org.apache.commons.beanutils=WARN
log4j.logger.org.apache.hadoop.hdfs.StateChange=WARN
log4j.logger.BlockStateChange=WARN
log4j.logger.org.apache.hadoop.hdfs.DFSUtil=WARN
## YARN can be noisy too
log4j.logger.org.apache.hadoop.yarn.server.resourcemanager.scheduler=WARN
log4j.logger.org.apache.hadoop.yarn.server.nodemanager=WARN
log4j.logger.org.apache.hadoop.yarn.event=WARN
log4j.logger.org.apache.hadoop.yarn.util.ResourceCalculatorPlugin=ERROR
log4j.logger.org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor=WARN
log4j.logger.org.apache.hadoop.mapred.IndexCache=WARN
log4j.logger.org.apache.hadoop.yarn.webapp.WebApps=WARN
log4j.logger.org.apache.hadoop.yarn.server.resourcemanager.security=WARN
log4j.logger.org.apache.hadoop.yarn.util.AbstractLivelinessMonitor=WARN
log4j.logger.org.apache.hadoop.security.token.delegation=WARN
log4j.logger.org.apache.hadoop.mapred.ShuffleHandler=WARN
log4j.logger.org.apache.hadoop.ipc.Server=WARN