HADOOP-15407. HADOOP-15540. Support Windows Azure Storage - Blob file system "ABFS" in Hadoop: Core Commit.

Contributed by Shane Mainali, Thomas Marquardt, Zichen Sun, Georgi Chalakov, Esfandiar Manii, Amit Singh, Dana Kaban, Da Zhou, Junhua Gu, Saher Ahwal, Saurabh Pant, James Baker, Shaoyu Zhang, Lawrence Chen, Kevin Chen and Steve Loughran
This commit is contained in:
Steve Loughran 2018-06-15 18:14:13 +01:00 committed by Thomas Marquardt
parent 3d89c3e73e
commit f044deedbb
129 changed files with 12606 additions and 12 deletions

1
.gitignore vendored
View File

@ -22,6 +22,7 @@ make-build-debug
# Filesystem contract test options and credentials
auth-keys.xml
azure-auth-keys.xml
azure-bfs-auth-keys.xml
# External tool builders
*/.externalToolBuilders

View File

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

View File

@ -113,6 +113,9 @@ public class TestCommonConfigurationFields extends TestConfigurationFieldsBase {
xmlPrefixToSkipCompare.add("fs.wasb.impl");
xmlPrefixToSkipCompare.add("fs.wasbs.impl");
xmlPrefixToSkipCompare.add("fs.azure.");
xmlPrefixToSkipCompare.add("fs.abfs.impl");
xmlPrefixToSkipCompare.add("fs.abfss.impl");
// ADL properties are in a different subtree
// - org.apache.hadoop.hdfs.web.ADLConfKeys

View File

@ -1209,6 +1209,11 @@
<artifactId>jsch</artifactId>
<version>0.1.54</version>
</dependency>
<dependency>
<groupId>org.apache.htrace</groupId>
<artifactId>htrace-core</artifactId>
<version>3.1.0-incubating</version>
</dependency>
<dependency>
<groupId>org.apache.htrace</groupId>
<artifactId>htrace-core4</artifactId>
@ -1344,6 +1349,12 @@
<version>7.0.0</version>
</dependency>
<dependency>
<groupId>org.threadly</groupId>
<artifactId>threadly</artifactId>
<version>4.9.0</version>
</dependency>
<dependency>
<groupId>com.aliyun.oss</groupId>
<artifactId>aliyun-sdk-oss</artifactId>

View File

@ -67,6 +67,7 @@
<!-- To run with the default Sun ruleset,
comment out the configLocation line -->
<configLocation>src/config/checkstyle.xml</configLocation>
<suppressionsLocation>src/config/checkstyle-suppressions.xml</suppressionsLocation>
</configuration>
</plugin>
@ -148,11 +149,23 @@
<scope>provided</scope>
</dependency>
<dependency>
<groupId>org.threadly</groupId>
<artifactId>threadly</artifactId>
<scope>compile</scope>
</dependency>
<dependency>
<groupId>com.fasterxml.jackson.core</groupId>
<artifactId>jackson-core</artifactId>
<scope>compile</scope>
</dependency>
<dependency>
<groupId>com.fasterxml.jackson.core</groupId>
<artifactId>jackson-databind</artifactId>
<scope>compile</scope>
</dependency>
<dependency>
<groupId>org.apache.httpcomponents</groupId>
@ -172,9 +185,39 @@
</exclusions>
</dependency>
<dependency>
<groupId>org.apache.htrace</groupId>
<artifactId>htrace-core</artifactId>
<scope>compile</scope>
</dependency>
<dependency>
<groupId>org.apache.htrace</groupId>
<artifactId>htrace-core4</artifactId>
<scope>compile</scope>
</dependency>
<dependency>
<groupId>com.google.inject</groupId>
<artifactId>guice</artifactId>
<scope>compile</scope>
<!-- we have a dependency on a lower version -->
<exclusions>
<exclusion>
<groupId>com.google.guava</groupId>
<artifactId>guava</artifactId>
</exclusion>
</exclusions>
</dependency>
<dependency>
<groupId>com.google.guava</groupId>
<artifactId>guava</artifactId>
</dependency>
<dependency>
<groupId>joda-time</groupId>
<artifactId>joda-time</artifactId>
<scope>compile</scope>
</dependency>
@ -183,15 +226,8 @@
<artifactId>jetty-util-ajax</artifactId>
<scope>compile</scope>
</dependency>
<!-- dependencies use for test only -->
<dependency>
<groupId>commons-io</groupId>
<artifactId>commons-io</artifactId>
<scope>test</scope>
</dependency>
<!-- dependencies use for test only -->
<dependency>
<groupId>junit</groupId>
<artifactId>junit</artifactId>
@ -229,10 +265,6 @@
<artifactId>mockito-all</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>com.fasterxml.jackson.core</groupId>
<artifactId>jackson-databind</artifactId>
</dependency>
</dependencies>
@ -398,8 +430,10 @@
<exclude>**/ITestNativeAzureFileSystemConcurrencyLive.java</exclude>
<exclude>**/ITestNativeAzureFileSystemLive.java</exclude>
<exclude>**/ITestNativeAzureFSPageBlobLive.java</exclude>
<exclude>**/ITestAzureBlobFileSystemRandomRead.java</exclude>
<exclude>**/ITestWasbRemoteCallHelper.java</exclude>
<exclude>**/ITestBlockBlobInputStream.java</exclude>
<exclude>**/ITestWasbAbfsCompatibility.java</exclude>
</excludes>
</configuration>
</execution>
@ -424,12 +458,15 @@
<fs.azure.scale.test.list.performance.files>${fs.azure.scale.test.list.performance.files}</fs.azure.scale.test.list.performance.files>
</systemPropertyVariables>
<includes>
<include>**/ITestWasbAbfsCompatibility.java</include>
<include>**/ITestFileSystemOperationsExceptionHandlingMultiThreaded.java</include>
<include>**/ITestFileSystemOperationsWithThreads.java</include>
<include>**/ITestOutOfBandAzureBlobOperationsLive.java</include>
<include>**/ITestNativeAzureFileSystemAuthorizationWithOwner.java</include>
<include>**/ITestNativeAzureFileSystemConcurrencyLive.java</include>
<include>**/ITestNativeAzureFileSystemLive.java</include>
<include>**/ITestNativeAzureFSPageBlobLive.java</include>
<include>**/ITestAzureBlobFileSystemRandomRead.java</include>
<include>**/ITestWasbRemoteCallHelper.java</include>
<include>**/ITestBlockBlobInputStream.java</include>
</includes>

View File

@ -0,0 +1,47 @@
<?xml version="1.0" encoding="UTF-8"?>
<!DOCTYPE suppressions PUBLIC
"-//Puppy Crawl//DTD Suppressions 1.0//EN"
"http://www.puppycrawl.com/dtds/suppressions_1_0.dtd">
<!--
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.
-->
<!--
Checkstyle configuration that checks the sun coding conventions from:
- the Java Language Specification at
http://java.sun.com/docs/books/jls/second_edition/html/index.html
- the Sun Code Conventions at http://java.sun.com/docs/codeconv/
- the Javadoc guidelines at
http://java.sun.com/j2se/javadoc/writingdoccomments/index.html
- the JDK Api documentation http://java.sun.com/j2se/docs/api/index.html
- some best practices
Checkstyle is very configurable. Be sure to read the documentation at
http://checkstyle.sf.net (or in your downloaded distribution).
Most Checks are configurable, be sure to consult the documentation.
To completely disable a check, just comment it out or delete it from the file.
Finally, it is worth reading the documentation.
-->
<suppressions>
<suppress checks="ParameterNumber|MagicNumber"
files="org[\\/]apache[\\/]hadoop[\\/]fs[\\/]azurebfs[\\/]services[\\/]AbfsHttpServiceImpl.java"/>
</suppressions>

View File

@ -0,0 +1,48 @@
/**
* 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.net.URISyntaxException;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.DelegateToFileSystem;
import org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes;
/**
* Azure Blob File System implementation of AbstractFileSystem.
* This impl delegates to the old FileSystem
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving
public class Abfs extends DelegateToFileSystem {
Abfs(final URI theUri, final Configuration conf) throws IOException,
URISyntaxException {
super(theUri, new AzureBlobFileSystem(), conf, FileSystemUriSchemes.ABFS_SCHEME, false);
}
@Override
public int getUriDefaultPort() {
return -1;
}
}

View File

@ -0,0 +1,48 @@
/**
* 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.net.URISyntaxException;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.DelegateToFileSystem;
import org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes;
/**
* Azure Blob File System implementation of AbstractFileSystem.
* This impl delegates to the old FileSystem
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving
public class Abfss extends DelegateToFileSystem {
Abfss(final URI theUri, final Configuration conf) throws IOException,
URISyntaxException {
super(theUri, new SecureAzureBlobFileSystem(), conf, FileSystemUriSchemes.ABFS_SECURE_SCHEME, false);
}
@Override
public int getUriDefaultPort() {
return -1;
}
}

View File

@ -0,0 +1,612 @@
/**
* 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.File;
import java.io.FileNotFoundException;
import java.io.IOException;
import java.io.InputStream;
import java.io.OutputStream;
import java.net.HttpURLConnection;
import java.net.URI;
import java.net.URISyntaxException;
import java.util.ArrayList;
import java.util.EnumSet;
import java.util.concurrent.Callable;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.Future;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Preconditions;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.threadly.util.ExceptionUtils;
import org.apache.hadoop.fs.PathIOException;
import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException;
import org.apache.hadoop.fs.azurebfs.services.AbfsServiceProviderImpl;
import org.apache.commons.lang.ArrayUtils;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.BlockLocation;
import org.apache.hadoop.fs.CreateFlag;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileAlreadyExistsException;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations;
import org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes;
import org.apache.hadoop.fs.azurebfs.contracts.services.TracingService;
import org.apache.hadoop.fs.azurebfs.contracts.services.ConfigurationService;
import org.apache.hadoop.fs.azurebfs.contracts.services.AbfsHttpService;
import org.apache.hadoop.fs.azurebfs.contracts.services.AbfsServiceProvider;
import org.apache.hadoop.fs.azurebfs.contracts.services.AzureServiceErrorCode;
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.InvalidUriAuthorityException;
import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidUriException;
import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.util.Progressable;
import org.apache.htrace.core.TraceScope;
/**
* A {@link org.apache.hadoop.fs.FileSystem} for reading and writing files stored on <a
* href="http://store.azure.com/">Windows Azure</a>
*/
@InterfaceAudience.Public
@InterfaceStability.Stable
public class AzureBlobFileSystem extends FileSystem {
public static final Logger LOG = LoggerFactory.getLogger(AzureBlobFileSystem.class);
private URI uri;
private Path workingDir;
private UserGroupInformation userGroupInformation;
private String user;
private String primaryUserGroup;
private AbfsServiceProvider abfsServiceProvider;
private TracingService tracingService;
private AbfsHttpService abfsHttpService;
private ConfigurationService configurationService;
private boolean isClosed;
@Override
public void initialize(URI uri, Configuration configuration)
throws IOException {
uri = ensureAuthority(uri, configuration);
super.initialize(uri, configuration);
setConf(configuration);
try {
this.abfsServiceProvider = AbfsServiceProviderImpl.create(configuration);
this.tracingService = abfsServiceProvider.get(TracingService.class);
this.abfsHttpService = abfsServiceProvider.get(AbfsHttpService.class);
this.configurationService = abfsServiceProvider.get(ConfigurationService.class);
} catch (AzureBlobFileSystemException exception) {
throw new IOException(exception);
}
this.LOG.debug(
"Initializing AzureBlobFileSystem for {}", uri);
this.uri = URI.create(uri.getScheme() + "://" + uri.getAuthority());
this.userGroupInformation = UserGroupInformation.getCurrentUser();
this.user = userGroupInformation.getUserName();
this.primaryUserGroup = userGroupInformation.getPrimaryGroupName();
this.LOG.debug(
"Initializing NativeAzureFileSystem for {}", uri);
this.setWorkingDirectory(this.getHomeDirectory());
if (this.configurationService.getCreateRemoteFileSystemDuringInitialization()) {
this.createFileSystem();
}
this.mkdirs(this.workingDir);
}
public boolean isSecure() {
return false;
}
@Override
public URI getUri() {
return this.uri;
}
@Override
public FSDataInputStream open(final Path path, final int bufferSize) throws IOException {
this.LOG.debug(
"AzureBlobFileSystem.open path: {} bufferSize: {}", path.toString(), bufferSize);
try {
InputStream inputStream = abfsHttpService.openFileForRead(this, makeQualified(path), statistics);
return new FSDataInputStream(inputStream);
} catch(AzureBlobFileSystemException ex) {
checkException(path, ex);
return null;
}
}
@Override
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 {
this.LOG.debug(
"AzureBlobFileSystem.create path: {} permission: {} overwrite: {} bufferSize: {}",
f.toString(),
permission,
overwrite,
blockSize);
try {
OutputStream outputStream = abfsHttpService.createFile(this, makeQualified(f), overwrite);
return new FSDataOutputStream(outputStream, statistics);
} catch(AzureBlobFileSystemException ex) {
checkException(f, ex);
return null;
}
}
@Override
@SuppressWarnings("deprecation")
public FSDataOutputStream createNonRecursive(final Path f, final FsPermission permission,
final boolean overwrite, final int bufferSize, final short replication, final long blockSize,
final Progressable progress) throws IOException {
final Path parent = f.getParent();
final FileStatus parentFileStatus = tryGetFileStatus(parent);
if (parentFileStatus == null) {
throw new FileNotFoundException("Cannot create file "
+ f.getName() + " because parent folder does not exist.");
}
return create(f, permission, overwrite, bufferSize, replication, blockSize, progress);
}
@Override
@SuppressWarnings("deprecation")
public FSDataOutputStream createNonRecursive(final Path f, final FsPermission permission,
final EnumSet<CreateFlag> flags, final int bufferSize, final short replication, final long blockSize,
final Progressable progress) throws IOException {
// Check if file should be appended or overwritten. Assume that the file
// is overwritten on if the CREATE and OVERWRITE create flags are set.
final EnumSet<CreateFlag> createflags =
EnumSet.of(CreateFlag.CREATE, CreateFlag.OVERWRITE);
final boolean overwrite = flags.containsAll(createflags);
// Delegate the create non-recursive call.
return this.createNonRecursive(f, permission, overwrite,
bufferSize, replication, blockSize, progress);
}
@Override
@SuppressWarnings("deprecation")
public FSDataOutputStream createNonRecursive(final Path f,
final boolean overwrite, final int bufferSize, final short replication, final long blockSize,
final Progressable progress) throws IOException {
return this.createNonRecursive(f, FsPermission.getFileDefault(),
overwrite, bufferSize, replication, blockSize, progress);
}
@Override
public FSDataOutputStream append(final Path f, final int bufferSize, final Progressable progress) throws IOException {
this.LOG.debug(
"AzureBlobFileSystem.append path: {} bufferSize: {}",
f.toString(),
bufferSize);
try {
OutputStream outputStream = abfsHttpService.openFileForWrite(this, makeQualified(f), false);
return new FSDataOutputStream(outputStream, statistics);
} catch(AzureBlobFileSystemException ex) {
checkException(f, ex);
return null;
}
}
public boolean rename(final Path src, final Path dst) throws IOException {
this.LOG.debug(
"AzureBlobFileSystem.rename src: {} dst: {}", src.toString(), dst.toString());
Path parentFolder = src.getParent();
if (parentFolder == null) {
return false;
}
final FileStatus dstFileStatus = tryGetFileStatus(dst);
try {
String sourceFileName = src.getName();
Path adjustedDst = dst;
if (dstFileStatus != null) {
if (!dstFileStatus.isDirectory()) {
return src.equals(dst);
}
adjustedDst = new Path(dst, sourceFileName);
}
abfsHttpService.rename(this, makeQualified(src), makeQualified(adjustedDst));
return true;
} catch(AzureBlobFileSystemException ex) {
checkException(
src,
ex,
AzureServiceErrorCode.PATH_ALREADY_EXISTS,
AzureServiceErrorCode.INVALID_RENAME_SOURCE_PATH,
AzureServiceErrorCode.SOURCE_PATH_NOT_FOUND,
AzureServiceErrorCode.INVALID_SOURCE_OR_DESTINATION_RESOURCE_TYPE,
AzureServiceErrorCode.RENAME_DESTINATION_PARENT_PATH_NOT_FOUND);
return false;
}
}
@Override
public boolean delete(final Path f, final boolean recursive) throws IOException {
this.LOG.debug(
"AzureBlobFileSystem.delete path: {} recursive: {}", f.toString(), recursive);
if (f.isRoot()) {
if (!recursive) {
return false;
}
return deleteRoot();
}
try {
abfsHttpService.delete(this, makeQualified(f), recursive);
return true;
} catch (AzureBlobFileSystemException ex) {
checkException(f, ex, AzureServiceErrorCode.PATH_NOT_FOUND);
return false;
}
}
@Override
public FileStatus[] listStatus(final Path f) throws IOException {
this.LOG.debug(
"AzureBlobFileSystem.listStatus path: {}", f.toString());
try {
FileStatus[] result = abfsHttpService.listStatus(this, makeQualified(f));
return result;
} catch (AzureBlobFileSystemException ex) {
checkException(f, ex);
return null;
}
}
@Override
public boolean mkdirs(final Path f, final FsPermission permission) throws IOException {
this.LOG.debug(
"AzureBlobFileSystem.mkdirs path: {} permissions: {}", f.toString(), permission);
final Path parentFolder = f.getParent();
if (parentFolder == null) {
// Cannot create root
return true;
}
try {
abfsHttpService.createDirectory(this, makeQualified(f));
return true;
} catch (AzureBlobFileSystemException ex) {
checkException(f, ex, AzureServiceErrorCode.PATH_ALREADY_EXISTS);
return true;
}
}
@Override
public synchronized void close() throws IOException {
if (isClosed) {
return;
}
super.close();
this.LOG.debug("AzureBlobFileSystem.close");
try {
abfsHttpService.closeFileSystem(this);
} catch (AzureBlobFileSystemException ex) {
checkException(null, ex);
this.isClosed = true;
}
}
@Override
public FileStatus getFileStatus(final Path f) throws IOException {
this.LOG.debug("AzureBlobFileSystem.getFileStatus path: {}", f.toString());
try {
return abfsHttpService.getFileStatus(this, makeQualified(f));
} catch(AzureBlobFileSystemException ex) {
checkException(f, ex);
return null;
}
}
@Override
public Path getWorkingDirectory() {
return this.workingDir;
}
@Override
public void setWorkingDirectory(final Path newDir) {
if (newDir.isAbsolute()) {
this.workingDir = newDir;
} else {
this.workingDir = new Path(workingDir, newDir);
}
}
@Override
public String getScheme() {
return FileSystemUriSchemes.ABFS_SCHEME;
}
@Override
public Path getHomeDirectory() {
return makeQualified(new Path(
FileSystemConfigurations.USER_HOME_DIRECTORY_PREFIX + "/" + this.userGroupInformation.getShortUserName()));
}
/**
* Return an array containing hostnames, offset and size of
* portions of the given file. For ABFS we'll just lie and give
* fake hosts to make sure we get many splits in MR jobs.
*/
@Override
public BlockLocation[] getFileBlockLocations(FileStatus file,
long start, long len) throws IOException {
if (file == null) {
return null;
}
if ((start < 0) || (len < 0)) {
throw new IllegalArgumentException("Invalid start or len parameter");
}
if (file.getLen() < start) {
return new BlockLocation[0];
}
final String blobLocationHost = this.configurationService.getAzureBlockLocationHost();
final String[] name = { blobLocationHost };
final String[] host = { blobLocationHost };
long blockSize = file.getBlockSize();
if (blockSize <= 0) {
throw new IllegalArgumentException(
"The block size for the given file is not a positive number: "
+ blockSize);
}
int numberOfLocations = (int) (len / blockSize)
+ ((len % blockSize == 0) ? 0 : 1);
BlockLocation[] locations = new BlockLocation[numberOfLocations];
for (int i = 0; i < locations.length; i++) {
long currentOffset = start + (i * blockSize);
long currentLength = Math.min(blockSize, start + len - currentOffset);
locations[i] = new BlockLocation(name, host, currentOffset, currentLength);
}
return locations;
}
public String getOwnerUser() {
return user;
}
public String getOwnerUserPrimaryGroup() {
return primaryUserGroup;
}
private boolean deleteRoot() throws IOException {
this.LOG.debug("Deleting root content");
final ExecutorService executorService = Executors.newFixedThreadPool(10);
try {
final FileStatus[] ls = listStatus(makeQualified(new Path(File.separator)));
final ArrayList<Future> deleteTasks = new ArrayList<>();
for (final FileStatus fs : ls) {
final Future deleteTask = executorService.submit(new Callable<Void>() {
@Override
public Void call() throws Exception {
delete(fs.getPath(), fs.isDirectory());
return null;
}
});
deleteTasks.add(deleteTask);
}
for (final Future deleteTask : deleteTasks) {
execute("deleteRoot", new Callable<Void>() {
@Override
public Void call() throws Exception {
deleteTask.get();
return null;
}
});
}
}
finally {
executorService.shutdownNow();
}
return true;
}
private FileStatus tryGetFileStatus(final Path f) {
try {
return getFileStatus(f);
}
catch (IOException ex) {
this.LOG.debug("File not found {}", f.toString());
return null;
}
}
private void createFileSystem() throws IOException {
this.LOG.debug(
"AzureBlobFileSystem.createFileSystem uri: {}", uri);
try {
abfsHttpService.createFilesystem(this);
} catch (AzureBlobFileSystemException ex) {
checkException(null, ex, AzureServiceErrorCode.FILE_SYSTEM_ALREADY_EXISTS);
}
}
private URI ensureAuthority(URI uri, final Configuration conf) {
Preconditions.checkNotNull(uri, "uri");
if (uri.getAuthority() == null) {
final URI defaultUri = FileSystem.getDefaultUri(conf);
if (defaultUri != null && isAbfsScheme(defaultUri.getScheme())) {
try {
// Reconstruct the URI with the authority from the default URI.
uri = new URI(
uri.getScheme(),
defaultUri.getAuthority(),
uri.getPath(),
uri.getQuery(),
uri.getFragment());
} catch (URISyntaxException e) {
// This should never happen.
throw new IllegalArgumentException(new InvalidUriException(uri.toString()));
}
}
}
if (uri.getAuthority() == null) {
throw new IllegalArgumentException(new InvalidUriAuthorityException(uri.toString()));
}
return uri;
}
private boolean isAbfsScheme(final String scheme) {
if (scheme == null) {
return false;
}
if (scheme.equals(FileSystemUriSchemes.ABFS_SCHEME) || scheme.equals(FileSystemUriSchemes.ABFS_SECURE_SCHEME)) {
return true;
}
return false;
}
@VisibleForTesting
<T> FileSystemOperation execute(
final String scopeDescription,
final Callable<T> callableFileOperation) throws IOException {
return execute(scopeDescription, callableFileOperation, null);
}
@VisibleForTesting
<T> FileSystemOperation execute(
final String scopeDescription,
final Callable<T> callableFileOperation,
T defaultResultValue) throws IOException {
final TraceScope traceScope = tracingService.traceBegin(scopeDescription);
try {
final T executionResult = callableFileOperation.call();
return new FileSystemOperation(executionResult, null);
} catch (AbfsRestOperationException abfsRestOperationException) {
return new FileSystemOperation(defaultResultValue, abfsRestOperationException);
} catch (AzureBlobFileSystemException azureBlobFileSystemException) {
tracingService.traceException(traceScope, azureBlobFileSystemException);
throw new IOException(azureBlobFileSystemException);
} catch (Exception exception) {
if (exception instanceof ExecutionException) {
exception = (Exception) ExceptionUtils.getRootCause(exception);
}
final FileSystemOperationUnhandledException fileSystemOperationUnhandledException = new FileSystemOperationUnhandledException(exception);
tracingService.traceException(traceScope, fileSystemOperationUnhandledException);
throw new IOException(fileSystemOperationUnhandledException);
} finally {
tracingService.traceEnd(traceScope);
}
}
private void checkException(final Path path,
final AzureBlobFileSystemException exception,
final AzureServiceErrorCode... allowedErrorCodesList) throws IOException {
if (exception instanceof AbfsRestOperationException) {
AbfsRestOperationException ere = (AbfsRestOperationException) exception;
if (ArrayUtils.contains(allowedErrorCodesList, ere.getErrorCode())) {
return;
}
int statusCode = ere.getStatusCode();
//AbfsRestOperationException.getMessage() contains full error info including path/uri.
if (statusCode == HttpURLConnection.HTTP_NOT_FOUND) {
throw new FileNotFoundException(ere.getMessage());
} else if (statusCode == HttpURLConnection.HTTP_CONFLICT) {
throw new FileAlreadyExistsException(ere.getMessage());
} else {
throw ere;
}
} else {
if (path == null) {
throw exception;
}
// record info of path
throw new PathIOException(path.toString(), exception);
}
}
@VisibleForTesting
FileSystem.Statistics getFsStatistics() {
return this.statistics;
}
@VisibleForTesting
static class FileSystemOperation<T> {
private final T result;
private final AbfsRestOperationException exception;
FileSystemOperation(final T result, final AbfsRestOperationException exception) {
this.result = result;
this.exception = exception;
}
public boolean failed() {
return this.exception != null;
}
}
}

View File

@ -0,0 +1,41 @@
/**
* 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.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes;
/**
* A secure {@link org.apache.hadoop.fs.FileSystem} for reading and writing files stored on <a
* href="http://store.azure.com/">Windows Azure</a>
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving
public class SecureAzureBlobFileSystem extends AzureBlobFileSystem {
@Override
public boolean isSecure() {
return true;
}
@Override
public String getScheme() {
return FileSystemUriSchemes.ABFS_SECURE_SCHEME;
}
}

View File

@ -0,0 +1,76 @@
/**
* 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.constants;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
/**
* Responsible to keep all constant keys used in abfs rest client here
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving
public final class AbfsHttpConstants {
// Abfs Http client constants
public static final String FILESYSTEM = "filesystem";
public static final String FILE = "file";
public static final String DIRECTORY = "directory";
public static final String APPEND_ACTION = "append";
public static final String FLUSH_ACTION = "flush";
public static final String SET_PROPERTIES_ACTION = "setProperties";
public static final String DEFAULT_TIMEOUT = "90";
public static final String JAVA_VERSION = "java.version";
public static final String OS_NAME = "os.name";
public static final String OS_VERSION = "os.version";
public static final String CLIENT_VERSION = "Azure Blob FS/1.0";
// Abfs Http Verb
public static final String HTTP_METHOD_DELETE = "DELETE";
public static final String HTTP_METHOD_GET = "GET";
public static final String HTTP_METHOD_HEAD = "HEAD";
public static final String HTTP_METHOD_PATCH = "PATCH";
public static final String HTTP_METHOD_POST = "POST";
public static final String HTTP_METHOD_PUT = "PUT";
// Abfs generic constants
public static final String SINGLE_WHITE_SPACE = " ";
public static final String EMPTY_STRING = "";
public static final String FORWARD_SLASH = "/";
public static final String DOT = ".";
public static final String PLUS = "+";
public static final String STAR = "*";
public static final String COMMA = ",";
public static final String EQUAL = "=";
public static final String QUESTION_MARK = "?";
public static final String AND_MARK = "&";
public static final String SEMICOLON = ";";
public static final String HTTP_HEADER_PREFIX = "x-ms-";
public static final String PLUS_ENCODE = "%20";
public static final String FORWARD_SLASH_ENCODE = "%2F";
public static final String AZURE_DISTRIBUTED_FILE_SYSTEM_AUTHORITY_DELIMITER = "@";
public static final String UTF_8 = "utf-8";
public static final String GMT_TIMEZONE = "GMT";
public static final String APPLICATION_JSON = "application/json";
public static final String APPLICATION_OCTET_STREAM = "application/octet-stream";
private AbfsHttpConstants() {}
}

View File

@ -0,0 +1,57 @@
/**
* 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.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.
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving
public final class ConfigurationKeys {
public static final String FS_AZURE_ACCOUNT_KEY_PROPERTY_NAME = "fs.azure.account.key.";
public static final String FS_AZURE_ACCOUNT_KEY_PROPERTY_NAME_REGX = "fs\\.azure\\.account\\.key\\.(.*)";
public static final String FS_AZURE_SECURE_MODE = "fs.azure.secure.mode";
// Retry strategy defined by the user
public static final String AZURE_MIN_BACKOFF_INTERVAL = "fs.azure.io.retry.min.backoff.interval";
public static final String AZURE_MAX_BACKOFF_INTERVAL = "fs.azure.io.retry.max.backoff.interval";
public static final String AZURE_BACKOFF_INTERVAL = "fs.azure.io.retry.backoff.interval";
public static final String AZURE_MAX_IO_RETRIES = "fs.azure.io.retry.max.retries";
// Remove this and use common azure storage emulator property for public release.
public static final String FS_AZURE_EMULATOR_ENABLED = "fs.azure.abfs.emulator.enabled";
// Read and write buffer sizes defined by the user
public static final String AZURE_WRITE_BUFFER_SIZE = "fs.azure.write.request.size";
public static final String AZURE_READ_BUFFER_SIZE = "fs.azure.read.request.size";
public static final String AZURE_BLOCK_SIZE_PROPERTY_NAME = "fs.azure.block.size";
public static final String AZURE_BLOCK_LOCATION_HOST_PROPERTY_NAME = "fs.azure.block.location.impersonatedhost";
public static final String AZURE_CONCURRENT_CONNECTION_VALUE_OUT = "fs.azure.concurrentRequestCount.out";
public static final String AZURE_CONCURRENT_CONNECTION_VALUE_IN = "fs.azure.concurrentRequestCount.in";
public static final String AZURE_TOLERATE_CONCURRENT_APPEND = "fs.azure.io.read.tolerate.concurrent.append";
public static final String AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION = "fs.azure.createRemoteFileSystemDuringInitialization";
public static final String FS_AZURE_AUTOTHROTTLING_ENABLE = "fs.azure.autothrottling.enable";
public static final String FS_AZURE_ATOMIC_RENAME_KEY = "fs.azure.atomic.rename.key";
public static final String FS_AZURE_READ_AHEAD_QUEUE_DEPTH = "fs.azure.readaheadqueue.depth";
private ConfigurationKeys() {}
}

View File

@ -0,0 +1,59 @@
/**
* 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.constants;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
/**
* Responsible to keep all the Azure Blob File System related configurations.
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving
public final class FileSystemConfigurations {
public static final String USER_HOME_DIRECTORY_PREFIX = "/user";
// Retry parameter defaults.
public static final int DEFAULT_MIN_BACKOFF_INTERVAL = 3 * 1000; // 3s
public static final int DEFAULT_MAX_BACKOFF_INTERVAL = 30 * 1000; // 30s
public static final int DEFAULT_BACKOFF_INTERVAL = 3 * 1000; // 3s
public static final int DEFAULT_MAX_RETRY_ATTEMPTS = 30;
private static final int ONE_KB = 1024;
private static final int ONE_MB = ONE_KB * ONE_KB;
// Default upload and download buffer size
public static final int DEFAULT_WRITE_BUFFER_SIZE = 4 * ONE_MB; // 4 MB
public static final int DEFAULT_READ_BUFFER_SIZE = 4 * ONE_MB; // 4 MB
public static final int MIN_BUFFER_SIZE = 16 * ONE_KB; // 16 KB
public static final int MAX_BUFFER_SIZE = 100 * ONE_MB; // 100 MB
public static final long MAX_AZURE_BLOCK_SIZE = 512 * 1024 * 1024L;
public static final String AZURE_BLOCK_LOCATION_HOST_DEFAULT = "localhost";
public static final int MAX_CONCURRENT_READ_THREADS = 12;
public static final int MAX_CONCURRENT_WRITE_THREADS = 8;
public static final boolean DEFAULT_READ_TOLERATE_CONCURRENT_APPEND = false;
public static final boolean DEFAULT_AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION = false;
public static final String DEFAULT_FS_AZURE_ATOMIC_RENAME_DIRECTORIES = "/hbase";
public static final int DEFAULT_READ_AHEAD_QUEUE_DEPTH = -1;
private FileSystemConfigurations() {}
}

View File

@ -0,0 +1,42 @@
/**
* 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.constants;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
/**
* Responsible to keep all Azure Blob File System valid URI schemes.
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving
public final class FileSystemUriSchemes {
public static final String ABFS_SCHEME = "abfs";
public static final String ABFS_SECURE_SCHEME = "abfss";
public static final String ABFS_DNS_PREFIX = "dfs";
public static final String HTTP_SCHEME = "http";
public static final String HTTPS_SCHEME = "https";
public static final String WASB_SCHEME = "wasb";
public static final String WASB_SECURE_SCHEME = "wasbs";
public static final String WASB_DNS_PREFIX = "blob";
private FileSystemUriSchemes() {}
}

View File

@ -0,0 +1,57 @@
/**
* 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.constants;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
/**
* Responsible to keep all abfs http headers here
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving
public final class HttpHeaderConfigurations {
public static final String ACCEPT = "Accept";
public static final String ACCEPT_CHARSET = "Accept-Charset";
public static final String AUTHORIZATION = "Authorization";
public static final String IF_MODIFIED_SINCE = "If-Modified-Since";
public static final String IF_UNMODIFIED_SINCE = "If-Unmodified-Since";
public static final String IF_MATCH = "If-Match";
public static final String IF_NONE_MATCH = "If-None-Match";
public static final String CONTENT_LENGTH = "Content-Length";
public static final String CONTENT_ENCODING = "Content-Encoding";
public static final String CONTENT_LANGUAGE = "Content-Language";
public static final String CONTENT_MD5 = "Content-MD5";
public static final String CONTENT_TYPE = "Content-Type";
public static final String RANGE = "Range";
public static final String TRANSFER_ENCODING = "Transfer-Encoding";
public static final String USER_AGENT = "User-Agent";
public static final String X_HTTP_METHOD_OVERRIDE = "X-HTTP-Method-Override";
public static final String X_MS_CLIENT_REQUEST_ID = "x-ms-client-request-id";
public static final String X_MS_DATE = "x-ms-date";
public static final String X_MS_REQUEST_ID = "x-ms-request-id";
public static final String X_MS_VERSION = "x-ms-version";
public static final String X_MS_RESOURCE_TYPE = "x-ms-resource-type";
public static final String X_MS_CONTINUATION = "x-ms-continuation";
public static final String ETAG = "ETag";
public static final String X_MS_PROPERTIES = "x-ms-properties";
public static final String X_MS_RENAME_SOURCE = "x-ms-rename-source";
public static final String LAST_MODIFIED = "Last-Modified";
private HttpHeaderConfigurations() {}
}

View File

@ -0,0 +1,40 @@
/**
* 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.constants;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
/**
* Responsible to keep all Http Query params here
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving
public final class HttpQueryParams {
public static final String QUERY_PARAM_RESOURCE = "resource";
public static final String QUERY_PARAM_DIRECTORY = "directory";
public static final String QUERY_PARAM_CONTINUATION = "continuation";
public static final String QUERY_PARAM_RECURSIVE = "recursive";
public static final String QUERY_PARAM_MAXRESULTS = "maxResults";
public static final String QUERY_PARAM_ACTION = "action";
public static final String QUERY_PARAM_POSITION = "position";
public static final String QUERY_PARAM_TIMEOUT = "timeout";
public static final String QUERY_PARAM_RETAIN_UNCOMMITTED_DATA = "retainUncommittedData";
private HttpQueryParams() {}
}

View File

@ -0,0 +1,22 @@
/*
* 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.
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving
package org.apache.hadoop.fs.azurebfs.constants;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;

View File

@ -0,0 +1,104 @@
/**
* 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.contracts.annotations;
import java.lang.annotation.ElementType;
import java.lang.annotation.Retention;
import java.lang.annotation.RetentionPolicy;
import java.lang.annotation.Target;
import org.apache.hadoop.classification.InterfaceStability;
/**
* Definitions of Annotations for all types of the validators
*/
@InterfaceStability.Evolving
public class ConfigurationValidationAnnotations {
/**
* Describes the requirements when validating the annotated int field
*/
@Target({ ElementType.FIELD })
@Retention(RetentionPolicy.RUNTIME)
public @interface IntegerConfigurationValidatorAnnotation {
String ConfigurationKey();
int MaxValue() default Integer.MAX_VALUE;
int MinValue() default Integer.MIN_VALUE;
int DefaultValue();
boolean ThrowIfInvalid() default false;
}
/**
* Describes the requirements when validating the annotated long field
*/
@Target({ ElementType.FIELD })
@Retention(RetentionPolicy.RUNTIME)
public @interface LongConfigurationValidatorAnnotation {
String ConfigurationKey();
long MaxValue() default Long.MAX_VALUE;
long MinValue() default Long.MIN_VALUE;
long DefaultValue();
boolean ThrowIfInvalid() default false;
}
/**
* Describes the requirements when validating the annotated String field
*/
@Target({ ElementType.FIELD })
@Retention(RetentionPolicy.RUNTIME)
public @interface StringConfigurationValidatorAnnotation {
String ConfigurationKey();
String DefaultValue();
boolean ThrowIfInvalid() default false;
}
/**
* Describes the requirements when validating the annotated String field
*/
@Target({ ElementType.FIELD })
@Retention(RetentionPolicy.RUNTIME)
public @interface Base64StringConfigurationValidatorAnnotation {
String ConfigurationKey();
String DefaultValue();
boolean ThrowIfInvalid() default false;
}
/**
* Describes the requirements when validating the annotated boolean field
*/
@Target({ ElementType.FIELD })
@Retention(RetentionPolicy.RUNTIME)
public @interface BooleanConfigurationValidatorAnnotation {
String ConfigurationKey();
boolean DefaultValue();
boolean ThrowIfInvalid() default false;
}
}

View File

@ -0,0 +1,22 @@
/*
* 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.
*/
@InterfaceAudience.Private
@InterfaceStability.Evolving
package org.apache.hadoop.fs.azurebfs.contracts.annotations;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;

View File

@ -0,0 +1,37 @@
/**
* 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.contracts.diagnostics;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidConfigurationValueException;
/**
* ConfigurationValidator to validate the value of a configuration key
* @param <T> the type of the validator and the validated value
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving
public interface ConfigurationValidator<T> {
/**
* Validates the configValue
* @return validated value of type T
*/
T validate(String configValue) throws InvalidConfigurationValueException;
}

View File

@ -0,0 +1,22 @@
/*
* 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.
*/
@InterfaceAudience.Private
@InterfaceStability.Evolving
package org.apache.hadoop.fs.azurebfs.contracts.diagnostics;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;

View File

@ -0,0 +1,84 @@
/**
* 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.contracts.exceptions;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.fs.azurebfs.contracts.services.AzureServiceErrorCode;
import org.apache.hadoop.fs.azurebfs.services.AbfsHttpOperation;
/**
* Exception to wrap Azure service error responses.
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving
public class AbfsRestOperationException extends AzureBlobFileSystemException {
private final int statusCode;
private final AzureServiceErrorCode errorCode;
private final String errorMessage;
public AbfsRestOperationException(
final int statusCode,
final String errorCode,
final String errorMessage,
final Exception innerException) {
super("Status code: " + statusCode + " error code: " + errorCode + " error message: " + errorMessage, innerException);
this.statusCode = statusCode;
this.errorCode = AzureServiceErrorCode.getAzureServiceCode(this.statusCode, errorCode);
this.errorMessage = errorMessage;
}
public AbfsRestOperationException(
final int statusCode,
final String errorCode,
final String errorMessage,
final Exception innerException,
final AbfsHttpOperation abfsHttpOperation) {
super(formatMessage(abfsHttpOperation));
this.statusCode = statusCode;
this.errorCode = AzureServiceErrorCode.getAzureServiceCode(this.statusCode, errorCode);
this.errorMessage = errorMessage;
}
public int getStatusCode() {
return this.statusCode;
}
public AzureServiceErrorCode getErrorCode() {
return this.errorCode;
}
public String getErrorMessage() {
return this.errorMessage;
}
private static String formatMessage(final AbfsHttpOperation abfsHttpOperation) {
return String.format(
"%1$s %2$s%nStatusCode=%3$s%nStatusDescription=%4$s%nErrorCode=%5$s%nErrorMessage=%6$s",
abfsHttpOperation.getMethod(),
abfsHttpOperation.getUrl().toString(),
abfsHttpOperation.getStatusCode(),
abfsHttpOperation.getStatusDescription(),
abfsHttpOperation.getStorageErrorCode(),
abfsHttpOperation.getStorageErrorMessage());
}
}

View File

@ -0,0 +1,56 @@
/**
* 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.contracts.exceptions;
import java.io.IOException;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
/**
* Base exception for any Azure Blob File System driver exceptions. All the exceptions must inherit this class.
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving
public abstract class AzureBlobFileSystemException extends IOException {
public AzureBlobFileSystemException(final String message) {
super(message);
}
public AzureBlobFileSystemException(final String message, final Exception innerException) {
super(message, innerException);
}
@Override
public String toString() {
if (this.getMessage() == null && this.getCause() == null) {
return "";
}
if (this.getCause() == null) {
return this.getMessage();
}
if (this.getMessage() == null) {
return this.getCause().toString();
}
return this.getMessage() + this.getCause().toString();
}
}

View File

@ -0,0 +1,32 @@
/**
* 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.contracts.exceptions;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
/**
* Thrown when a searched for element is not found
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving
public class ConfigurationPropertyNotFoundException extends AzureBlobFileSystemException {
public ConfigurationPropertyNotFoundException(String property) {
super("Configuration property " + property + "not found.");
}
}

View File

@ -0,0 +1,33 @@
/**
* 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.contracts.exceptions;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
/**
* Thrown when an unhandled exception is occurred during a file system operation.
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving
public final class FileSystemOperationUnhandledException extends AzureBlobFileSystemException {
public FileSystemOperationUnhandledException(Exception innerException) {
super("An unhandled file operation exception", innerException);
}
}

View File

@ -0,0 +1,40 @@
/**
* 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
* <p>
* http://www.apache.org/licenses/LICENSE-2.0
* <p>
* 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.contracts.exceptions;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.fs.azurebfs.contracts.services.AzureServiceErrorCode;
/**
* Exception to wrap invalid Azure service error responses.
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving
public class InvalidAbfsRestOperationException extends AbfsRestOperationException {
public InvalidAbfsRestOperationException(
final Exception innerException) {
super(
AzureServiceErrorCode.UNKNOWN.getStatusCode(),
AzureServiceErrorCode.UNKNOWN.getErrorCode(),
"InvalidAbfsRestOperationException",
innerException);
}
}

View File

@ -0,0 +1,37 @@
/**
* 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.contracts.exceptions;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
/**
* Thrown when a configuration value is invalid
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving
public class InvalidConfigurationValueException extends AzureBlobFileSystemException {
public InvalidConfigurationValueException(String configKey, Exception innerException) {
super("Invalid configuration value detected for " + configKey, innerException);
}
public InvalidConfigurationValueException(String configKey) {
super("Invalid configuration value detected for " + configKey);
}
}

View File

@ -0,0 +1,33 @@
/**
* 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.contracts.exceptions;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
/**
* Thrown when a file system property is invalid.
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving
public final class InvalidFileSystemPropertyException extends AzureBlobFileSystemException {
public InvalidFileSystemPropertyException(String property) {
super(String.format("%s is invalid.", property));
}
}

View File

@ -0,0 +1,33 @@
/**
* 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.contracts.exceptions;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
/**
* Thrown when URI authority is invalid.
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving
public final class InvalidUriAuthorityException extends AzureBlobFileSystemException {
public InvalidUriAuthorityException(String url) {
super(String.format("%s has invalid authority.", url));
}
}

View File

@ -0,0 +1,33 @@
/**
* 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.contracts.exceptions;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
/**
* Thrown when URI is invalid.
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving
public final class InvalidUriException extends AzureBlobFileSystemException {
public InvalidUriException(String url) {
super(String.format("%s is invalid.", url));
}
}

View File

@ -0,0 +1,36 @@
/**
* 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.contracts.exceptions;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.fs.azurebfs.services.AbfsServiceProviderImpl;
/**
* Thrown a service is either not configured to be injected or the service is not existing.
* For service registration
* @see AbfsServiceProviderImpl
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving
public final class ServiceResolutionException extends AzureBlobFileSystemException {
public ServiceResolutionException(String serviceName, Exception innerException) {
super(String.format("%s cannot be resolved.", serviceName), innerException);
}
}

View File

@ -0,0 +1,33 @@
/**
* 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.contracts.exceptions;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
/**
* Thrown when a timeout happens.
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving
public final class TimeoutException extends AzureBlobFileSystemException {
public TimeoutException(String message) {
super(message);
}
}

View File

@ -0,0 +1,22 @@
/*
* 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.
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving
package org.apache.hadoop.fs.azurebfs.contracts.exceptions;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;

View File

@ -0,0 +1,22 @@
/*
* 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.
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving
package org.apache.hadoop.fs.azurebfs.contracts;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;

View File

@ -0,0 +1,39 @@
/**
* 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.contracts.services;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystem;
import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException;
import org.apache.hadoop.fs.azurebfs.services.AbfsClient;
/**
* AbfsClient factory.
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving
public interface AbfsHttpClientFactory extends InjectableService {
/**
* Creates and configures an instance of new AbfsClient
* @return AbfsClient instance
*/
AbfsClient create(AzureBlobFileSystem fs) throws AzureBlobFileSystemException;
}

View File

@ -0,0 +1,162 @@
/**
* 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.contracts.services;
import java.io.InputStream;
import java.io.OutputStream;
import java.util.Hashtable;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystem;
import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException;
/**
* File System http service to provide network calls for file system operations.
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving
public interface AbfsHttpService extends InjectableService {
/**
* Gets filesystem properties on the Azure service.
* @param azureBlobFileSystem filesystem to get the properties.
* @return Hashtable<String, String> hash table containing all the filesystem properties.
*/
Hashtable<String, String> getFilesystemProperties(AzureBlobFileSystem azureBlobFileSystem) throws AzureBlobFileSystemException;
/**
* Sets filesystem properties on the Azure service.
* @param azureBlobFileSystem filesystem to get the properties.
* @param properties file system properties to set.
*/
void setFilesystemProperties(AzureBlobFileSystem azureBlobFileSystem, Hashtable<String, String> properties) throws
AzureBlobFileSystemException;
/**
* Gets path properties on the Azure service.
* @param azureBlobFileSystem filesystem to get the properties of the path.
* @param path path to get properties.
* @return Hashtable<String, String> hash table containing all the path properties.
*/
Hashtable<String, String> getPathProperties(AzureBlobFileSystem azureBlobFileSystem, Path path) throws AzureBlobFileSystemException;
/**
* Sets path properties on the Azure service.
* @param azureBlobFileSystem filesystem to get the properties of the path.
* @param path path to set properties.
* @param properties hash table containing all the path properties.
*/
void setPathProperties(AzureBlobFileSystem azureBlobFileSystem, Path path, Hashtable<String, String> properties) throws
AzureBlobFileSystemException;
/**
* Creates filesystem on the Azure service.
* @param azureBlobFileSystem filesystem to be created.
*/
void createFilesystem(AzureBlobFileSystem azureBlobFileSystem) throws AzureBlobFileSystemException;
/**
* Deletes filesystem on the Azure service.
* @param azureBlobFileSystem filesystem to be deleted.
*/
void deleteFilesystem(AzureBlobFileSystem azureBlobFileSystem) throws AzureBlobFileSystemException;
/**
* Creates a file on the Azure service.
* @param azureBlobFileSystem filesystem to create file or directory.
* @param path path of the file to be created.
* @param overwrite should overwrite.
* @return OutputStream stream to the file.
*/
OutputStream createFile(AzureBlobFileSystem azureBlobFileSystem, Path path, boolean overwrite) throws AzureBlobFileSystemException;
/**
* Creates a directory on the Azure service.
* @param azureBlobFileSystem filesystem to create file or directory.
* @param path path of the directory to be created.
* @return OutputStream stream to the file.
*/
Void createDirectory(AzureBlobFileSystem azureBlobFileSystem, Path path) throws AzureBlobFileSystemException;
/**
* Opens a file to read and returns the stream.
* @param azureBlobFileSystem filesystem to read a file from.
* @param path file path to read.
* @return InputStream a stream to the file to read.
*/
InputStream openFileForRead(AzureBlobFileSystem azureBlobFileSystem, Path path, FileSystem.Statistics statistics) throws AzureBlobFileSystemException;
/**
* Opens a file to write and returns the stream.
* @param azureBlobFileSystem filesystem to write a file to.
* @param path file path to write.
* @param overwrite should overwrite.
* @return OutputStream a stream to the file to write.
*/
OutputStream openFileForWrite(AzureBlobFileSystem azureBlobFileSystem, Path path, boolean overwrite) throws AzureBlobFileSystemException;
/**
* Renames a file or directory from source to destination.
* @param azureBlobFileSystem filesystem to rename a path.
* @param source source path.
* @param destination destination path.
*/
void rename(AzureBlobFileSystem azureBlobFileSystem, Path source, Path destination) throws AzureBlobFileSystemException;
/**
* Deletes a file or directory.
* @param azureBlobFileSystem filesystem to delete the path.
* @param path file path to be deleted.
* @param recursive true if path is a directory and recursive deletion is desired.
*/
void delete(AzureBlobFileSystem azureBlobFileSystem, Path path, boolean recursive) throws AzureBlobFileSystemException;
/**
* Gets path's status under the provided path on the Azure service.
* @param azureBlobFileSystem filesystem to perform the get file status operation.
* @param path path delimiter.
* @return FileStatus FileStatus of the path in the file system.
*/
FileStatus getFileStatus(AzureBlobFileSystem azureBlobFileSystem, Path path) throws AzureBlobFileSystemException;
/**
* Lists all the paths under the provided path on the Azure service.
* @param azureBlobFileSystem filesystem to perform the list operation.
* @param path path delimiter.
* @return FileStatus[] list of all paths in the file system.
*/
FileStatus[] listStatus(AzureBlobFileSystem azureBlobFileSystem, Path path) throws AzureBlobFileSystemException;
/**
* Closes the client to filesystem to Azure service.
* @param azureBlobFileSystem filesystem to perform the list operation.
*/
void closeFileSystem(AzureBlobFileSystem azureBlobFileSystem) throws AzureBlobFileSystemException;
/**
* Checks for the given path if it is marked as atomic rename directory or not.
* @param key
* @return True if the given path is listed under atomic rename property otherwise False.
*/
boolean isAtomicRenameKey(String key);
}

View File

@ -0,0 +1,40 @@
/**
* 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.contracts.services;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.fs.azurebfs.contracts.exceptions.ServiceResolutionException;
/**
* Dependency injected Azure Storage services provider interface.
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving
public interface AbfsServiceProvider {
/**
* Returns an instance of resolved injectable service by class name.
* The injectable service must be configured first to be resolvable.
* @param clazz the injectable service which is expected to be returned.
* @param <T> The type of injectable service.
* @return T instance
* @throws ServiceResolutionException if the service is not resolvable.
*/
<T extends InjectableService> T get(Class<T> clazz) throws ServiceResolutionException;
}

View File

@ -0,0 +1,112 @@
/**
* 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.contracts.services;
import java.net.HttpURLConnection;
import java.util.ArrayList;
import java.util.List;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
/**
* Azure service error codes.
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving
public enum AzureServiceErrorCode {
FILE_SYSTEM_ALREADY_EXISTS("FilesystemAlreadyExists", HttpURLConnection.HTTP_CONFLICT, null),
PATH_ALREADY_EXISTS("PathAlreadyExists", HttpURLConnection.HTTP_CONFLICT, null),
PATH_CONFLICT("PathConflict", HttpURLConnection.HTTP_CONFLICT, null),
FILE_SYSTEM_NOT_FOUND("FilesystemNotFound", HttpURLConnection.HTTP_NOT_FOUND, null),
PATH_NOT_FOUND("PathNotFound", HttpURLConnection.HTTP_NOT_FOUND, null),
PRE_CONDITION_FAILED("PreconditionFailed", HttpURLConnection.HTTP_PRECON_FAILED, null),
SOURCE_PATH_NOT_FOUND("SourcePathNotFound", HttpURLConnection.HTTP_NOT_FOUND, null),
INVALID_SOURCE_OR_DESTINATION_RESOURCE_TYPE("InvalidSourceOrDestinationResourceType", HttpURLConnection.HTTP_CONFLICT, null),
RENAME_DESTINATION_PARENT_PATH_NOT_FOUND("RenameDestinationParentPathNotFound", HttpURLConnection.HTTP_NOT_FOUND, null),
INVALID_RENAME_SOURCE_PATH("InvalidRenameSourcePath", HttpURLConnection.HTTP_CONFLICT, null),
INGRESS_OVER_ACCOUNT_LIMIT(null, HttpURLConnection.HTTP_UNAVAILABLE, "Ingress is over the account limit."),
EGRESS_OVER_ACCOUNT_LIMIT(null, HttpURLConnection.HTTP_UNAVAILABLE, "Egress is over the account limit."),
UNKNOWN(null, -1, null);
private final String errorCode;
private final int httpStatusCode;
private final String errorMessage;
AzureServiceErrorCode(String errorCode, int httpStatusCodes, String errorMessage) {
this.errorCode = errorCode;
this.httpStatusCode = httpStatusCodes;
this.errorMessage = errorMessage;
}
public int getStatusCode() {
return this.httpStatusCode;
}
public String getErrorCode() {
return this.errorCode;
}
public static List<AzureServiceErrorCode> getAzureServiceCode(int httpStatusCode) {
List<AzureServiceErrorCode> errorCodes = new ArrayList<>();
if (httpStatusCode == UNKNOWN.httpStatusCode) {
errorCodes.add(UNKNOWN);
return errorCodes;
}
for (AzureServiceErrorCode azureServiceErrorCode : AzureServiceErrorCode.values()) {
if (azureServiceErrorCode.httpStatusCode == httpStatusCode) {
errorCodes.add(azureServiceErrorCode);
}
}
return errorCodes;
}
public static AzureServiceErrorCode getAzureServiceCode(int httpStatusCode, String errorCode) {
if (errorCode == null || errorCode.isEmpty() || httpStatusCode == UNKNOWN.httpStatusCode) {
return UNKNOWN;
}
for (AzureServiceErrorCode azureServiceErrorCode : AzureServiceErrorCode.values()) {
if (errorCode.equalsIgnoreCase(azureServiceErrorCode.errorCode)
&& azureServiceErrorCode.httpStatusCode == httpStatusCode) {
return azureServiceErrorCode;
}
}
return UNKNOWN;
}
public static AzureServiceErrorCode getAzureServiceCode(int httpStatusCode, String errorCode, final String errorMessage) {
if (errorCode == null || errorCode.isEmpty() || httpStatusCode == UNKNOWN.httpStatusCode || errorMessage == null || errorMessage.isEmpty()) {
return UNKNOWN;
}
for (AzureServiceErrorCode azureServiceErrorCode : AzureServiceErrorCode.values()) {
if (azureServiceErrorCode.httpStatusCode == httpStatusCode
&& errorCode.equalsIgnoreCase(azureServiceErrorCode.errorCode)
&& errorMessage.equalsIgnoreCase(azureServiceErrorCode.errorMessage)
) {
return azureServiceErrorCode;
}
}
return UNKNOWN;
}
}

View File

@ -0,0 +1,143 @@
/**
* 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.contracts.services;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.azurebfs.contracts.exceptions.ConfigurationPropertyNotFoundException;
/**
* Configuration service collects required Azure Hadoop configurations and provides it to the consumers.
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving
public interface ConfigurationService extends InjectableService {
/**
* Checks if ABFS is running from Emulator;
* @return is emulator mode.
*/
boolean isEmulator();
/**
* Retrieves storage secure mode from Hadoop configuration;
* @return storage secure mode;
*/
boolean isSecureMode();
/**
* Retrieves storage account key for provided account name from Hadoop configuration.
* @param accountName the account name to retrieve the key.
* @return storage account key;
*/
String getStorageAccountKey(String accountName) throws ConfigurationPropertyNotFoundException;
/**
* Returns Hadoop configuration.
* @return Hadoop configuration.
*/
Configuration getConfiguration();
/**
* Retrieves configured write buffer size
* @return the size of the write buffer
*/
int getWriteBufferSize();
/**
* Retrieves configured read buffer size
* @return the size of the read buffer
*/
int getReadBufferSize();
/**
* Retrieves configured min backoff interval
* @return min backoff interval
*/
int getMinBackoffIntervalMilliseconds();
/**
* Retrieves configured max backoff interval
* @return max backoff interval
*/
int getMaxBackoffIntervalMilliseconds();
/**
* Retrieves configured backoff interval
* @return backoff interval
*/
int getBackoffIntervalMilliseconds();
/**
* Retrieves configured num of retries
* @return num of retries
*/
int getMaxIoRetries();
/**
* Retrieves configured azure block size
* @return azure block size
*/
long getAzureBlockSize();
/**
* Retrieves configured azure block location host
* @return azure block location host
*/
String getAzureBlockLocationHost();
/**
* Retrieves configured number of concurrent threads
* @return number of concurrent write threads
*/
int getMaxConcurrentWriteThreads();
/**
* Retrieves configured number of concurrent threads
* @return number of concurrent read threads
*/
int getMaxConcurrentReadThreads();
/**
* Retrieves configured boolean for tolerating out of band writes to files
* @return configured boolean for tolerating out of band writes to files
*/
boolean getTolerateOobAppends();
/**
* Retrieves the comma-separated list of directories to receive special treatment so that folder
* rename is made atomic. The default value for this setting is just '/hbase'.
* Example directories list : <value>/hbase,/data</value>
* @see <a href="https://hadoop.apache.org/docs/stable/hadoop-azure/index.html#Configuring_Credentials">AtomicRenameProperty</a>
* @return atomic rename directories
*/
String getAzureAtomicRenameDirs();
/**
* Retrieves configured boolean for creating remote file system during initialization
* @return configured boolean for creating remote file system during initialization
*/
boolean getCreateRemoteFileSystemDuringInitialization();
/**
* Retrieves configured value of read ahead queue
* @return depth of read ahead
*/
int getReadAheadQueueDepth();
}

View File

@ -0,0 +1,30 @@
/**
* 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.contracts.services;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
/**
* Marker interface for all the injectable services.
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving
public interface InjectableService {
}

View File

@ -0,0 +1,160 @@
/**
* 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
* <p>
* http://www.apache.org/licenses/LICENSE-2.0
* <p>
* 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.contracts.services;
import com.fasterxml.jackson.annotation.JsonProperty;
import org.apache.hadoop.classification.InterfaceStability;
/**
* The ListResultEntrySchema model.
*/
@InterfaceStability.Evolving
public class ListResultEntrySchema {
/**
* The name property.
*/
@JsonProperty(value = "name")
private String name;
/**
* The isDirectory property.
*/
@JsonProperty(value = "isDirectory")
private Boolean isDirectory;
/**
* The lastModified property.
*/
@JsonProperty(value = "lastModified")
private String lastModified;
/**
* The eTag property.
*/
@JsonProperty(value = "etag")
private String eTag;
/**
* The contentLength property.
*/
@JsonProperty(value = "contentLength")
private Long contentLength;
/**
* Get the name value.
*
* @return the name value
*/
public String name() {
return this.name;
}
/**
* Set the name value.
*
* @param name the name value to set
* @return the ListEntrySchema object itself.
*/
public ListResultEntrySchema withName(String name) {
this.name = name;
return this;
}
/**
* Get the isDirectory value.
*
* @return the isDirectory value
*/
public Boolean isDirectory() {
return this.isDirectory;
}
/**
* Set the isDirectory value.
*
* @param isDirectory the isDirectory value to set
* @return the ListEntrySchema object itself.
*/
public ListResultEntrySchema withIsDirectory(final Boolean isDirectory) {
this.isDirectory = isDirectory;
return this;
}
/**
* Get the lastModified value.
*
* @return the lastModified value
*/
public String lastModified() {
return this.lastModified;
}
/**
* Set the lastModified value.
*
* @param lastModified the lastModified value to set
* @return the ListEntrySchema object itself.
*/
public ListResultEntrySchema withLastModified(String lastModified) {
this.lastModified = lastModified;
return this;
}
/**
* Get the etag value.
*
* @return the etag value
*/
public String eTag() {
return this.eTag;
}
/**
* Set the eTag value.
*
* @param eTag the eTag value to set
* @return the ListEntrySchema object itself.
*/
public ListResultEntrySchema withETag(final String eTag) {
this.eTag = eTag;
return this;
}
/**
* Get the contentLength value.
*
* @return the contentLength value
*/
public Long contentLength() {
return this.contentLength;
}
/**
* Set the contentLength value.
*
* @param contentLength the contentLength value to set
* @return the ListEntrySchema object itself.
*/
public ListResultEntrySchema withContentLength(final Long contentLength) {
this.contentLength = contentLength;
return this;
}
}

View File

@ -0,0 +1,58 @@
/**
* 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
* <p>
* http://www.apache.org/licenses/LICENSE-2.0
* <p>
* 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.contracts.services;
import java.util.List;
import com.fasterxml.jackson.annotation.JsonProperty;
import org.apache.hadoop.classification.InterfaceStability;
/**
* The ListResultSchema model.
*/
@InterfaceStability.Evolving
public class ListResultSchema {
/**
* The paths property.
*/
@JsonProperty(value = "paths")
private List<ListResultEntrySchema> paths;
/**
* * Get the paths value.
*
* @return the paths value
*/
public List<ListResultEntrySchema> paths() {
return this.paths;
}
/**
* Set the paths value.
*
* @param paths the paths value to set
* @return the ListSchema object itself.
*/
public ListResultSchema withPaths(final List<ListResultEntrySchema> paths) {
this.paths = paths;
return this;
}
}

View File

@ -0,0 +1,29 @@
/**
* 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.contracts.services;
/**
* The ReadBufferStatus for Rest AbfsClient
*/
public enum ReadBufferStatus {
NOT_AVAILABLE, // buffers sitting in readaheadqueue have this stats
READING_IN_PROGRESS, // reading is in progress on this buffer. Buffer should be in inProgressList
AVAILABLE, // data is available in buffer. It should be in completedList
READ_FAILED // read completed, but failed.
}

View File

@ -0,0 +1,66 @@
/**
* 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.contracts.services;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException;
import org.apache.htrace.core.SpanId;
import org.apache.htrace.core.TraceScope;
/**
* Azure Blob File System tracing service.
*/
@InterfaceAudience.Private
@InterfaceStability.Evolving
public interface TracingService extends InjectableService {
/**
* Creates a {@link TraceScope} object with the provided description.
* @param description the trace description.
* @return created traceScope.
*/
TraceScope traceBegin(String description);
/**
* Creates a {@link TraceScope} object with the provided description.
* @param description the trace description.
* @param parentSpanId the span id of the parent trace scope.
* @return create traceScope
*/
TraceScope traceBegin(String description, SpanId parentSpanId);
/**
* Gets current thread latest generated traceScope id.
* @return current thread latest generated traceScope id.
*/
SpanId getCurrentTraceScopeSpanId();
/**
* Appends the provided exception to the trace scope.
* @param traceScope the scope which exception needs to be attached to.
* @param azureBlobFileSystemException the exception to be attached to the scope.
*/
void traceException(TraceScope traceScope, AzureBlobFileSystemException azureBlobFileSystemException);
/**
* Ends the provided traceScope.
* @param traceScope the scope that needs to be ended.
*/
void traceEnd(TraceScope traceScope);
}

View File

@ -0,0 +1,22 @@
/*
* 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.
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving
package org.apache.hadoop.fs.azurebfs.contracts.services;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;

View File

@ -0,0 +1,50 @@
/**
* 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.diagnostics;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.fs.azurebfs.contracts.diagnostics.ConfigurationValidator;
import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidConfigurationValueException;
import org.apache.commons.codec.binary.Base64;
/**
* String Base64 configuration value Validator
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving
public class Base64StringConfigurationBasicValidator extends ConfigurationBasicValidator<String> implements ConfigurationValidator{
public Base64StringConfigurationBasicValidator(final String configKey, final String defaultVal, final boolean throwIfInvalid){
super(configKey, defaultVal, throwIfInvalid);
}
public String validate(final String configValue) throws InvalidConfigurationValueException {
String result = super.validate((configValue));
if (result != null) {
return result;
}
if (!Base64.isBase64(configValue)) {
throw new InvalidConfigurationValueException(getConfigKey());
}
return configValue;
}
}

View File

@ -0,0 +1,50 @@
/**
* 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
* <p>
* http://www.apache.org/licenses/LICENSE-2.0
* <p>
* 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.diagnostics;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidConfigurationValueException;
/**
* Boolean configuration value validator
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving
public class BooleanConfigurationBasicValidator extends ConfigurationBasicValidator<Boolean> {
private static final String TRUE = "true";
private static final String FALSE = "false";
public BooleanConfigurationBasicValidator(final String configKey, final boolean defaultVal, final boolean throwIfInvalid) {
super(configKey, defaultVal, throwIfInvalid);
}
public Boolean validate(final String configValue) throws InvalidConfigurationValueException {
Boolean result = super.validate(configValue);
if (result != null) {
return result;
}
if (configValue.equalsIgnoreCase(TRUE) || configValue.equalsIgnoreCase(FALSE)) {
return Boolean.valueOf(configValue);
}
throw new InvalidConfigurationValueException(getConfigKey());
}
}

View File

@ -0,0 +1,67 @@
/**
* 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
* <p>
* http://www.apache.org/licenses/LICENSE-2.0
* <p>
* 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.diagnostics;
import org.apache.hadoop.fs.azurebfs.contracts.diagnostics.ConfigurationValidator;
import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidConfigurationValueException;
/**
* ConfigurationBasicValidator covers the base case of missing user defined configuration value
* @param <T> the type of the validated value
*/
abstract class ConfigurationBasicValidator<T> implements ConfigurationValidator {
private final T defaultVal;
private final String configKey;
private final boolean throwIfInvalid;
ConfigurationBasicValidator(final String configKey, final T defaultVal, final boolean throwIfInvalid) {
this.configKey = configKey;
this.defaultVal = defaultVal;
this.throwIfInvalid = throwIfInvalid;
}
/**
* 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
* @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
* @throws InvalidConfigurationValueException in case the configValue is null and required to be set
*/
public T validate(final String configValue) throws InvalidConfigurationValueException {
if (configValue == null) {
if (this.throwIfInvalid) {
throw new InvalidConfigurationValueException(this.configKey);
}
return this.defaultVal;
}
return null;
}
public T getDefaultVal() {
return this.defaultVal;
}
public String getConfigKey() {
return this.configKey;
}
public boolean getThrowIfInvalid() {
return this.throwIfInvalid;
}
}

View File

@ -0,0 +1,68 @@
/**
* 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
* <p>
* http://www.apache.org/licenses/LICENSE-2.0
* <p>
* 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.diagnostics;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.fs.azurebfs.contracts.diagnostics.ConfigurationValidator;
import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidConfigurationValueException;
/**
* Integer configuration value Validator
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving
public class IntegerConfigurationBasicValidator extends ConfigurationBasicValidator<Integer> implements ConfigurationValidator {
private final int min;
private final int max;
public IntegerConfigurationBasicValidator(final int min, final int max, final int defaultVal, final String configKey, final boolean throwIfInvalid) {
super(configKey, defaultVal, throwIfInvalid);
this.min = min;
this.max = max;
}
public Integer validate(final String configValue) throws InvalidConfigurationValueException {
Integer result = super.validate(configValue);
if (result != null) {
return result;
}
try {
result = Integer.parseInt(configValue);
// throw an exception if a 'within bounds' value is missing
if (getThrowIfInvalid() && (result < this.min || result > this.max)) {
throw new InvalidConfigurationValueException(getConfigKey());
}
// set the value to the nearest bound if it's out of bounds
if (result < this.min) {
return this.min;
}
if (result > this.max) {
return this.max;
}
} catch (NumberFormatException ex) {
throw new InvalidConfigurationValueException(getConfigKey(), ex);
}
return result;
}
}

View File

@ -0,0 +1,65 @@
/**
* 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
* <p>
* http://www.apache.org/licenses/LICENSE-2.0
* <p>
* 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.diagnostics;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.fs.azurebfs.contracts.diagnostics.ConfigurationValidator;
import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidConfigurationValueException;
/**
* Long configuration value Validator
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving
public class LongConfigurationBasicValidator extends ConfigurationBasicValidator<Long> implements ConfigurationValidator {
private final long min;
private final long max;
public LongConfigurationBasicValidator(final long min, final long max, final long defaultVal, final String configKey, final boolean throwIfInvalid) {
super(configKey, defaultVal, throwIfInvalid);
this.min = min;
this.max = max;
}
public Long validate(final String configValue) throws InvalidConfigurationValueException {
Long result = super.validate(configValue);
if (result != null) {
return result;
}
try {
result = Long.parseLong(configValue);
// throw an exception if a 'within bounds' value is missing
if (getThrowIfInvalid() && (result < this.min || result > this.max)) {
throw new InvalidConfigurationValueException(getConfigKey());
}
// set the value to the nearest bound if it's out of bounds
if (result < this.min) {
return this.min;
} else if (result > this.max) {
return this.max;
}
} catch (NumberFormatException ex) {
throw new InvalidConfigurationValueException(getConfigKey(), ex);
}
return result;
}
}

View File

@ -0,0 +1,45 @@
/**
* 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.diagnostics;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.fs.azurebfs.contracts.diagnostics.ConfigurationValidator;
import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidConfigurationValueException;
/**
* String configuration value Validator
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving
public class StringConfigurationBasicValidator extends ConfigurationBasicValidator<String> implements ConfigurationValidator{
public StringConfigurationBasicValidator(final String configKey, final String defaultVal, final boolean throwIfInvalid){
super(configKey, defaultVal, throwIfInvalid);
}
public String validate(final String configValue) throws InvalidConfigurationValueException {
String result = super.validate((configValue));
if (result != null) {
return result;
}
return configValue;
}
}

View File

@ -0,0 +1,22 @@
/*
* 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.
*/
@InterfaceAudience.Private
@InterfaceStability.Evolving
package org.apache.hadoop.fs.azurebfs.diagnostics;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;

View File

@ -0,0 +1,31 @@
<html>
<!--
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.
-->
<body>
<p>
A distributed implementation of {@link
org.apache.hadoop.fs.FileSystem} for reading and writing files on
<a href="http://store.azure.com">Azure Storage</a>.
This implementation stores files on Azure in their native form for
interoperability with other Azure tools.
</p>
</body>
</html>

View File

@ -0,0 +1,402 @@
/**
* 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
* <p>
* http://www.apache.org/licenses/LICENSE-2.0
* <p>
* 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.services;
import java.io.UnsupportedEncodingException;
import java.net.MalformedURLException;
import java.net.URL;
import java.net.URLEncoder;
import java.util.ArrayList;
import java.util.List;
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.contracts.services.ConfigurationService;
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.LoggerFactory;
/**
* AbfsClient
*/
public class AbfsClient {
public static final Logger LOG = LoggerFactory.getLogger(AbfsClient.class);
private final URL baseUrl;
private final SharedKeyCredentials sharedKeyCredentials;
private final String xMsVersion = "2018-03-28";
private final ExponentialRetryPolicy retryPolicy;
private final String filesystem;
private final ConfigurationService configurationService;
private final String userAgent;
public AbfsClient(final URL baseUrl, final SharedKeyCredentials sharedKeyCredentials,
final ConfigurationService configurationService,
final ExponentialRetryPolicy exponentialRetryPolicy) {
this.baseUrl = baseUrl;
this.sharedKeyCredentials = sharedKeyCredentials;
String baseUrlString = baseUrl.toString();
this.filesystem = baseUrlString.substring(baseUrlString.lastIndexOf(AbfsHttpConstants.FORWARD_SLASH) + 1);
this.configurationService = configurationService;
this.retryPolicy = exponentialRetryPolicy;
this.userAgent = initializeUserAgent();
}
public String getFileSystem() {
return filesystem;
}
ExponentialRetryPolicy getRetryPolicy() {
return retryPolicy;
}
SharedKeyCredentials getSharedKeyCredentials() {
return sharedKeyCredentials;
}
List<AbfsHttpHeader> createDefaultHeaders() {
final List<AbfsHttpHeader> requestHeaders = new ArrayList<AbfsHttpHeader>();
requestHeaders.add(new AbfsHttpHeader(HttpHeaderConfigurations.X_MS_VERSION, xMsVersion));
requestHeaders.add(new AbfsHttpHeader(HttpHeaderConfigurations.ACCEPT, AbfsHttpConstants.APPLICATION_JSON
+ AbfsHttpConstants.COMMA + AbfsHttpConstants.SINGLE_WHITE_SPACE + AbfsHttpConstants.APPLICATION_OCTET_STREAM));
requestHeaders.add(new AbfsHttpHeader(HttpHeaderConfigurations.ACCEPT_CHARSET,
AbfsHttpConstants.UTF_8));
requestHeaders.add(new AbfsHttpHeader(HttpHeaderConfigurations.CONTENT_TYPE, AbfsHttpConstants.EMPTY_STRING));
requestHeaders.add(new AbfsHttpHeader(HttpHeaderConfigurations.USER_AGENT, userAgent));
return requestHeaders;
}
AbfsUriQueryBuilder createDefaultUriQueryBuilder() {
final AbfsUriQueryBuilder abfsUriQueryBuilder = new AbfsUriQueryBuilder();
abfsUriQueryBuilder.addQuery(HttpQueryParams.QUERY_PARAM_TIMEOUT, AbfsHttpConstants.DEFAULT_TIMEOUT);
return abfsUriQueryBuilder;
}
public AbfsRestOperation createFilesystem() throws AzureBlobFileSystemException {
final List<AbfsHttpHeader> requestHeaders = createDefaultHeaders();
final AbfsUriQueryBuilder abfsUriQueryBuilder = new AbfsUriQueryBuilder();
abfsUriQueryBuilder.addQuery(HttpQueryParams.QUERY_PARAM_RESOURCE, AbfsHttpConstants.FILESYSTEM);
final URL url = createRequestUrl(abfsUriQueryBuilder.toString());
final AbfsRestOperation op = new AbfsRestOperation(
this,
AbfsHttpConstants.HTTP_METHOD_PUT,
url,
requestHeaders);
op.execute();
return op;
}
public AbfsRestOperation setFilesystemProperties(final String properties) throws AzureBlobFileSystemException {
final List<AbfsHttpHeader> requestHeaders = createDefaultHeaders();
// 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.
requestHeaders.add(new AbfsHttpHeader(HttpHeaderConfigurations.X_HTTP_METHOD_OVERRIDE,
AbfsHttpConstants.HTTP_METHOD_PATCH));
requestHeaders.add(new AbfsHttpHeader(HttpHeaderConfigurations.X_MS_PROPERTIES,
properties));
final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder();
abfsUriQueryBuilder.addQuery(HttpQueryParams.QUERY_PARAM_RESOURCE, AbfsHttpConstants.FILESYSTEM);
final URL url = createRequestUrl(abfsUriQueryBuilder.toString());
final AbfsRestOperation op = new AbfsRestOperation(
this,
AbfsHttpConstants.HTTP_METHOD_PUT,
url,
requestHeaders);
op.execute();
return op;
}
public AbfsRestOperation listPath(final String relativePath, final boolean recursive, final int listMaxResults,
final String continuation) throws AzureBlobFileSystemException {
final List<AbfsHttpHeader> requestHeaders = createDefaultHeaders();
final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder();
abfsUriQueryBuilder.addQuery(HttpQueryParams.QUERY_PARAM_RESOURCE, AbfsHttpConstants.FILESYSTEM);
abfsUriQueryBuilder.addQuery(HttpQueryParams.QUERY_PARAM_DIRECTORY, relativePath == null ? "" : urlEncode(relativePath));
abfsUriQueryBuilder.addQuery(HttpQueryParams.QUERY_PARAM_RECURSIVE, String.valueOf(recursive));
abfsUriQueryBuilder.addQuery(HttpQueryParams.QUERY_PARAM_CONTINUATION, continuation);
abfsUriQueryBuilder.addQuery(HttpQueryParams.QUERY_PARAM_MAXRESULTS, String.valueOf(listMaxResults));
final URL url = createRequestUrl(abfsUriQueryBuilder.toString());
final AbfsRestOperation op = new AbfsRestOperation(
this,
AbfsHttpConstants.HTTP_METHOD_GET,
url,
requestHeaders);
op.execute();
return op;
}
public AbfsRestOperation getFilesystemProperties() throws AzureBlobFileSystemException {
final List<AbfsHttpHeader> requestHeaders = createDefaultHeaders();
final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder();
abfsUriQueryBuilder.addQuery(HttpQueryParams.QUERY_PARAM_RESOURCE, AbfsHttpConstants.FILESYSTEM);
final URL url = createRequestUrl(abfsUriQueryBuilder.toString());
final AbfsRestOperation op = new AbfsRestOperation(
this,
AbfsHttpConstants.HTTP_METHOD_HEAD,
url,
requestHeaders);
op.execute();
return op;
}
public AbfsRestOperation deleteFilesystem() throws AzureBlobFileSystemException {
final List<AbfsHttpHeader> requestHeaders = createDefaultHeaders();
final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder();
abfsUriQueryBuilder.addQuery(HttpQueryParams.QUERY_PARAM_RESOURCE, AbfsHttpConstants.FILESYSTEM);
final URL url = createRequestUrl(abfsUriQueryBuilder.toString());
final AbfsRestOperation op = new AbfsRestOperation(
this,
AbfsHttpConstants.HTTP_METHOD_DELETE,
url,
requestHeaders);
op.execute();
return op;
}
public AbfsRestOperation createPath(final String path, final boolean isFile, final boolean overwrite)
throws AzureBlobFileSystemException {
final List<AbfsHttpHeader> requestHeaders = createDefaultHeaders();
if (!overwrite) {
requestHeaders.add(new AbfsHttpHeader(HttpHeaderConfigurations.IF_NONE_MATCH, "*"));
}
final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder();
abfsUriQueryBuilder.addQuery(HttpQueryParams.QUERY_PARAM_RESOURCE, isFile ? AbfsHttpConstants.FILE : AbfsHttpConstants.DIRECTORY);
final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString());
final AbfsRestOperation op = new AbfsRestOperation(
this,
AbfsHttpConstants.HTTP_METHOD_PUT,
url,
requestHeaders);
op.execute();
return op;
}
public AbfsRestOperation renamePath(final String source, final String destination, final String continuation)
throws AzureBlobFileSystemException {
final List<AbfsHttpHeader> requestHeaders = createDefaultHeaders();
final String encodedRenameSource = urlEncode(AbfsHttpConstants.FORWARD_SLASH + this.getFileSystem() + source);
requestHeaders.add(new AbfsHttpHeader(HttpHeaderConfigurations.X_MS_RENAME_SOURCE, encodedRenameSource));
requestHeaders.add(new AbfsHttpHeader(HttpHeaderConfigurations.IF_NONE_MATCH, AbfsHttpConstants.STAR));
final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder();
abfsUriQueryBuilder.addQuery(HttpQueryParams.QUERY_PARAM_CONTINUATION, continuation);
final URL url = createRequestUrl(destination, abfsUriQueryBuilder.toString());
final AbfsRestOperation op = new AbfsRestOperation(
this,
AbfsHttpConstants.HTTP_METHOD_PUT,
url,
requestHeaders);
op.execute();
return op;
}
public AbfsRestOperation append(final String path, final long position, final byte[] buffer, final int offset,
final int length) throws AzureBlobFileSystemException {
final List<AbfsHttpHeader> requestHeaders = createDefaultHeaders();
// 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.
requestHeaders.add(new AbfsHttpHeader(HttpHeaderConfigurations.X_HTTP_METHOD_OVERRIDE,
AbfsHttpConstants.HTTP_METHOD_PATCH));
final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder();
abfsUriQueryBuilder.addQuery(HttpQueryParams.QUERY_PARAM_ACTION, AbfsHttpConstants.APPEND_ACTION);
abfsUriQueryBuilder.addQuery(HttpQueryParams.QUERY_PARAM_POSITION, Long.toString(position));
final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString());
final AbfsRestOperation op = new AbfsRestOperation(
this,
AbfsHttpConstants.HTTP_METHOD_PUT,
url,
requestHeaders, buffer, offset, length);
op.execute();
return op;
}
public AbfsRestOperation flush(final String path, final long position, boolean retainUncommittedData) throws AzureBlobFileSystemException {
final List<AbfsHttpHeader> requestHeaders = createDefaultHeaders();
// 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.
requestHeaders.add(new AbfsHttpHeader(HttpHeaderConfigurations.X_HTTP_METHOD_OVERRIDE,
AbfsHttpConstants.HTTP_METHOD_PATCH));
final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder();
abfsUriQueryBuilder.addQuery(HttpQueryParams.QUERY_PARAM_ACTION, AbfsHttpConstants.FLUSH_ACTION);
abfsUriQueryBuilder.addQuery(HttpQueryParams.QUERY_PARAM_POSITION, Long.toString(position));
abfsUriQueryBuilder.addQuery(HttpQueryParams.QUERY_PARAM_RETAIN_UNCOMMITTED_DATA, String.valueOf(retainUncommittedData));
final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString());
final AbfsRestOperation op = new AbfsRestOperation(
this,
AbfsHttpConstants.HTTP_METHOD_PUT,
url,
requestHeaders);
op.execute();
return op;
}
public AbfsRestOperation setPathProperties(final String path, final String properties) throws AzureBlobFileSystemException {
final List<AbfsHttpHeader> requestHeaders = createDefaultHeaders();
// 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.
requestHeaders.add(new AbfsHttpHeader(HttpHeaderConfigurations.X_HTTP_METHOD_OVERRIDE,
AbfsHttpConstants.HTTP_METHOD_PATCH));
requestHeaders.add(new AbfsHttpHeader(HttpHeaderConfigurations.X_MS_PROPERTIES, properties));
final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder();
abfsUriQueryBuilder.addQuery(HttpQueryParams.QUERY_PARAM_ACTION, AbfsHttpConstants.SET_PROPERTIES_ACTION);
final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString());
final AbfsRestOperation op = new AbfsRestOperation(
this,
AbfsHttpConstants.HTTP_METHOD_PUT,
url,
requestHeaders);
op.execute();
return op;
}
public AbfsRestOperation getPathProperties(final String path) throws AzureBlobFileSystemException {
final List<AbfsHttpHeader> requestHeaders = createDefaultHeaders();
final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder();
final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString());
final AbfsRestOperation op = new AbfsRestOperation(
this,
AbfsHttpConstants.HTTP_METHOD_HEAD,
url,
requestHeaders);
op.execute();
return op;
}
public AbfsRestOperation read(final String path, final long position, final byte[] buffer, final int bufferOffset,
final int bufferLength, final String eTag) throws AzureBlobFileSystemException {
final List<AbfsHttpHeader> requestHeaders = createDefaultHeaders();
requestHeaders.add(new AbfsHttpHeader(HttpHeaderConfigurations.RANGE,
String.format("bytes=%d-%d", position, position + bufferLength - 1)));
requestHeaders.add(new AbfsHttpHeader(HttpHeaderConfigurations.IF_MATCH, eTag));
final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder();
final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString());
final AbfsRestOperation op = new AbfsRestOperation(
this,
AbfsHttpConstants.HTTP_METHOD_GET,
url,
requestHeaders,
buffer,
bufferOffset,
bufferLength);
op.execute();
return op;
}
public AbfsRestOperation deletePath(final String path, final boolean recursive, final String continuation)
throws AzureBlobFileSystemException {
final List<AbfsHttpHeader> requestHeaders = createDefaultHeaders();
final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder();
abfsUriQueryBuilder.addQuery(HttpQueryParams.QUERY_PARAM_RECURSIVE, String.valueOf(recursive));
abfsUriQueryBuilder.addQuery(HttpQueryParams.QUERY_PARAM_CONTINUATION, continuation);
final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString());
final AbfsRestOperation op = new AbfsRestOperation(
this,
AbfsHttpConstants.HTTP_METHOD_DELETE,
url,
requestHeaders);
op.execute();
return op;
}
private URL createRequestUrl(final String query) throws AzureBlobFileSystemException {
return createRequestUrl(AbfsHttpConstants.EMPTY_STRING, query);
}
private URL createRequestUrl(final String path, final String query)
throws AzureBlobFileSystemException {
final String base = baseUrl.toString();
String encodedPath = path;
try {
encodedPath = urlEncode(path);
} catch (AzureBlobFileSystemException ex) {
this.LOG.debug(
"Unexpected error.", ex);
}
final StringBuilder sb = new StringBuilder();
sb.append(base);
sb.append(encodedPath);
sb.append(query);
final URL url;
try {
url = new URL(sb.toString());
} catch (MalformedURLException ex) {
throw new InvalidUriException(sb.toString());
}
return url;
}
private static String urlEncode(final String value) throws AzureBlobFileSystemException {
String encodedString = null;
try {
encodedString = URLEncoder.encode(value, AbfsHttpConstants.UTF_8)
.replace(AbfsHttpConstants.PLUS, AbfsHttpConstants.PLUS_ENCODE)
.replace(AbfsHttpConstants.FORWARD_SLASH_ENCODE, AbfsHttpConstants.FORWARD_SLASH);
} catch (UnsupportedEncodingException ex) {
throw new InvalidUriException(value);
}
return encodedString;
}
private String initializeUserAgent() {
final String userAgentComment = String.format(Locale.ROOT,
"(JavaJRE %s; %s %s)",
System.getProperty(AbfsHttpConstants.JAVA_VERSION),
System.getProperty(AbfsHttpConstants.OS_NAME)
.replaceAll(AbfsHttpConstants.SINGLE_WHITE_SPACE, AbfsHttpConstants.EMPTY_STRING),
System.getProperty(AbfsHttpConstants.OS_VERSION));
return String.format(AbfsHttpConstants.CLIENT_VERSION + " %s", userAgentComment);
}
}

View File

@ -0,0 +1,116 @@
/**
* 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.services;
import java.net.MalformedURLException;
import java.net.URI;
import java.net.URL;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Preconditions;
import com.google.inject.Inject;
import com.google.inject.Singleton;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystem;
import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants;
import org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes;
import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException;
import org.apache.hadoop.fs.azurebfs.contracts.services.AbfsHttpClientFactory;
import org.apache.hadoop.fs.azurebfs.contracts.services.ConfigurationService;
import org.apache.http.client.utils.URIBuilder;
import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidUriAuthorityException;
import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidUriException;
@Singleton
@InterfaceAudience.Private
@InterfaceStability.Evolving
class AbfsHttpClientFactoryImpl implements AbfsHttpClientFactory {
private final ConfigurationService configurationService;
@Inject
AbfsHttpClientFactoryImpl(
final ConfigurationService configurationService) {
Preconditions.checkNotNull(configurationService, "configurationService");
this.configurationService = configurationService;
}
@VisibleForTesting
URIBuilder getURIBuilder(final String hostName, final FileSystem fs) {
final AzureBlobFileSystem abfs = (AzureBlobFileSystem) fs;
String scheme = FileSystemUriSchemes.HTTP_SCHEME;
if (abfs.isSecure()) {
scheme = FileSystemUriSchemes.HTTPS_SCHEME;
}
final URIBuilder uriBuilder = new URIBuilder();
uriBuilder.setScheme(scheme);
uriBuilder.setHost(hostName);
return uriBuilder;
}
public AbfsClient create(final AzureBlobFileSystem fs) throws AzureBlobFileSystemException {
final URI uri = fs.getUri();
final String authority = uri.getRawAuthority();
if (null == authority) {
throw new InvalidUriAuthorityException(uri.toString());
}
if (!authority.contains(AbfsHttpConstants.AZURE_DISTRIBUTED_FILE_SYSTEM_AUTHORITY_DELIMITER)) {
throw new InvalidUriAuthorityException(uri.toString());
}
final String[] authorityParts = authority.split(AbfsHttpConstants.AZURE_DISTRIBUTED_FILE_SYSTEM_AUTHORITY_DELIMITER, 2);
if (authorityParts.length < 2 || "".equals(authorityParts[0])) {
final String errMsg = String
.format("URI '%s' has a malformed authority, expected container name. "
+ "Authority takes the form "+ FileSystemUriSchemes.ABFS_SCHEME + "://[<container name>@]<account name>",
uri.toString());
throw new InvalidUriException(errMsg);
}
final String fileSystemName = authorityParts[0];
final String accountName = authorityParts[1];
final URIBuilder uriBuilder = getURIBuilder(accountName, fs);
final String url = uriBuilder.toString() + AbfsHttpConstants.FORWARD_SLASH + fileSystemName;
URL baseUrl;
try {
baseUrl = new URL(url);
} catch (MalformedURLException e) {
throw new InvalidUriException(String.format("URI '%s' is malformed", uri.toString()));
}
SharedKeyCredentials creds =
new SharedKeyCredentials(accountName.substring(0, accountName.indexOf(AbfsHttpConstants.DOT)),
this.configurationService.getStorageAccountKey(accountName));
return new AbfsClient(baseUrl, creds, configurationService, new ExponentialRetryPolicy());
}
}

View File

@ -0,0 +1,40 @@
/**
* 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
* <p>
* http://www.apache.org/licenses/LICENSE-2.0
* <p>
* 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.services;
/**
* The Http Request / Response Headers for Rest AbfsClient
*/
public class AbfsHttpHeader {
private final String name;
private final String value;
public AbfsHttpHeader(final String name, final String value) {
this.name = name;
this.value = value;
}
public String getName() {
return name;
}
public String getValue() {
return value;
}
}

View File

@ -0,0 +1,430 @@
/**
* 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.services;
import java.io.IOException;
import java.io.InputStream;
import java.io.OutputStream;
import java.net.HttpURLConnection;
import java.net.URL;
import java.util.List;
import java.util.UUID;
import com.fasterxml.jackson.core.JsonFactory;
import com.fasterxml.jackson.core.JsonParser;
import com.fasterxml.jackson.core.JsonToken;
import com.fasterxml.jackson.databind.ObjectMapper;
import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants;
import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations;
import org.apache.hadoop.fs.azurebfs.contracts.services.ListResultSchema;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* Represents an HTTP operation.
*/
public class AbfsHttpOperation {
private static final Logger LOG = LoggerFactory.getLogger(AbfsHttpOperation.class);
private static final int CONNECT_TIMEOUT = 30 * 1000;
private static final int READ_TIMEOUT = 30 * 1000;
private static final int CLEAN_UP_BUFFER_SIZE = 64 * 1024;
private static final int ONE_THOUSAND = 1000;
private static final int ONE_MILLION = ONE_THOUSAND * ONE_THOUSAND;
private final String method;
private final URL url;
private HttpURLConnection connection;
private int statusCode;
private String statusDescription;
private String storageErrorCode = "";
private String storageErrorMessage = "";
private String clientRequestId = "";
private String requestId = "";
private ListResultSchema listResultSchema = null;
// metrics
private int bytesSent;
private long bytesReceived;
// optional trace enabled metrics
private final boolean isTraceEnabled;
private long connectionTimeMs;
private long sendRequestTimeMs;
private long recvResponseTimeMs;
protected HttpURLConnection getConnection() {
return connection;
}
public String getMethod() {
return method;
}
public URL getUrl() {
return url;
}
public int getStatusCode() {
return statusCode;
}
public String getStatusDescription() {
return statusDescription;
}
public String getStorageErrorCode() {
return storageErrorCode;
}
public String getStorageErrorMessage() {
return storageErrorMessage;
}
public String getClientRequestId() {
return clientRequestId;
}
public String getRequestId() {
return requestId;
}
public int getBytesSent() {
return bytesSent;
}
public long getBytesReceived() {
return bytesReceived;
}
public ListResultSchema getListResultSchema() {
return listResultSchema;
}
public String getResponseHeader(String httpHeader) {
return connection.getHeaderField(httpHeader);
}
// Returns a trace message for the request
@Override
public String toString() {
final String urlStr = url.toString();
final StringBuilder sb = new StringBuilder();
sb.append(statusCode);
sb.append(",");
sb.append(storageErrorCode);
sb.append(",cid=");
sb.append(clientRequestId);
sb.append(",rid=");
sb.append(requestId);
if (isTraceEnabled) {
sb.append(",connMs=");
sb.append(connectionTimeMs);
sb.append(",sendMs=");
sb.append(sendRequestTimeMs);
sb.append(",recvMs=");
sb.append(recvResponseTimeMs);
}
sb.append(",sent=");
sb.append(bytesSent);
sb.append(",recv=");
sb.append(bytesReceived);
sb.append(",");
sb.append(method);
sb.append(",");
sb.append(urlStr);
return sb.toString();
}
/**
* Initializes a new HTTP request and opens the connection.
*
* @param url The full URL including query string parameters.
* @param method The HTTP method (PUT, PATCH, POST, GET, HEAD, or DELETE).
* @param requestHeaders The HTTP request headers.READ_TIMEOUT
*
* @throws IOException if an error occurs.
*/
public AbfsHttpOperation(final URL url, final String method, final List<AbfsHttpHeader> requestHeaders)
throws IOException {
this.isTraceEnabled = this.LOG.isTraceEnabled();
this.url = url;
this.method = method;
this.clientRequestId = UUID.randomUUID().toString();
this.connection = openConnection();
this.connection.setConnectTimeout(CONNECT_TIMEOUT);
this.connection.setReadTimeout(READ_TIMEOUT);
this.connection.setRequestMethod(method);
for (AbfsHttpHeader header : requestHeaders) {
this.connection.setRequestProperty(header.getName(), header.getValue());
}
this.connection.setRequestProperty(HttpHeaderConfigurations.X_MS_CLIENT_REQUEST_ID, clientRequestId);
}
/**
* Sends the HTTP request. Note that HttpUrlConnection requires that an
* empty buffer be sent in order to set the "Content-Length: 0" header, which
* is required by our endpoint.
*
* @param buffer the request entity body.
* @param offset an offset into the buffer where the data beings.
* @param length the length of the data in the buffer.
*
* @throws IOException if an error occurs.
*/
public void sendRequest(byte[] buffer, int offset, int length) throws IOException {
this.connection.setDoOutput(true);
this.connection.setFixedLengthStreamingMode(length);
if (buffer == null) {
// An empty buffer is sent to set the "Content-Length: 0" header, which
// is required by our endpoint.
buffer = new byte[]{};
offset = 0;
length = 0;
}
// send the request body
long startTime = 0;
if (this.isTraceEnabled) {
startTime = System.nanoTime();
}
try (OutputStream outputStream = this.connection.getOutputStream()) {
// update bytes sent before they are sent so we may observe
// attempted sends as well as successful sends via the
// accompanying statusCode
this.bytesSent = length;
outputStream.write(buffer, offset, length);
} finally {
if (this.isTraceEnabled) {
this.sendRequestTimeMs = elapsedTimeMs(startTime);
}
}
}
/**
* Gets and processes the HTTP response.
*
* @throws IOException if an error occurs.
*/
public void processResponse(final byte[] buffer, final int offset, final int length) throws IOException {
// get the response
long startTime = 0;
if (this.isTraceEnabled) {
startTime = System.nanoTime();
}
this.statusCode = this.connection.getResponseCode();
if (this.isTraceEnabled) {
this.recvResponseTimeMs = elapsedTimeMs(startTime);
}
this.statusDescription = this.connection.getResponseMessage();
this.requestId = this.connection.getHeaderField(HttpHeaderConfigurations.X_MS_REQUEST_ID);
if (this.requestId == null) {
this.requestId = AbfsHttpConstants.EMPTY_STRING;
}
if (AbfsHttpConstants.HTTP_METHOD_HEAD.equals(this.method)) {
// If it is HEAD, and it is ERROR
return;
}
if (this.isTraceEnabled) {
startTime = System.nanoTime();
}
if (statusCode >= HttpURLConnection.HTTP_BAD_REQUEST) {
processStorageErrorResponse();
if (this.isTraceEnabled) {
this.recvResponseTimeMs += elapsedTimeMs(startTime);
}
this.bytesReceived = this.connection.getHeaderFieldLong(HttpHeaderConfigurations.CONTENT_LENGTH, 0);
} else {
// consume the input stream to release resources
int totalBytesRead = 0;
try (InputStream stream = this.connection.getInputStream()) {
if (isNullInputStream(stream)) {
return;
}
boolean endOfStream = false;
// this is a list operation and need to retrieve the data
// need a better solution
if (AbfsHttpConstants.HTTP_METHOD_GET.equals(this.method) && buffer == null) {
parseListFilesResponse(stream);
} else {
if (buffer != null) {
while (totalBytesRead < length) {
int bytesRead = stream.read(buffer, offset + totalBytesRead, length - totalBytesRead);
if (bytesRead == -1) {
endOfStream = true;
break;
}
totalBytesRead += bytesRead;
}
}
if (!endOfStream && stream.read() != -1) {
// read and discard
int bytesRead = 0;
byte[] b = new byte[CLEAN_UP_BUFFER_SIZE];
while ((bytesRead = stream.read(b)) >= 0) {
totalBytesRead += bytesRead;
}
}
}
} catch (IOException ex) {
this.LOG.error("UnexpectedError: ", ex);
throw ex;
} finally {
if (this.isTraceEnabled) {
this.recvResponseTimeMs += elapsedTimeMs(startTime);
}
this.bytesReceived = totalBytesRead;
}
}
}
/**
* Open the HTTP connection.
*
* @throws IOException if an error occurs.
*/
private HttpURLConnection openConnection() throws IOException {
if (!isTraceEnabled) {
return (HttpURLConnection) url.openConnection();
}
long start = System.nanoTime();
try {
return (HttpURLConnection) url.openConnection();
} finally {
connectionTimeMs = elapsedTimeMs(start);
}
}
/**
* When the request fails, this function is used to parse the responseAbfsHttpClient.LOG.debug("ExpectedError: ", ex);
* and extract the storageErrorCode and storageErrorMessage. Any errors
* encountered while attempting to process the error response are logged,
* but otherwise ignored.
*
* For storage errors, the response body *usually* has the following format:
*
* {
* "error":
* {
* "code": "string",
* "message": "string"
* }
* }
*
*/
private void processStorageErrorResponse() {
try (InputStream stream = connection.getErrorStream()) {
if (stream == null) {
return;
}
JsonFactory jf = new JsonFactory();
try (JsonParser jp = jf.createParser(stream)) {
String fieldName, fieldValue;
jp.nextToken(); // START_OBJECT - {
jp.nextToken(); // FIELD_NAME - "error":
jp.nextToken(); // START_OBJECT - {
jp.nextToken();
while (jp.hasCurrentToken()) {
if (jp.getCurrentToken() == JsonToken.FIELD_NAME) {
fieldName = jp.getCurrentName();
jp.nextToken();
fieldValue = jp.getText();
switch (fieldName) {
case "code":
storageErrorCode = fieldValue;
break;
case "message":
storageErrorMessage = fieldValue;
break;
default:
break;
}
}
jp.nextToken();
}
}
} catch (IOException ex) {
// Ignore errors that occur while attempting to parse the storage
// error, since the response may have been handled by the HTTP driver
// or for other reasons have an unexpected
this.LOG.debug("ExpectedError: ", ex);
}
}
/**
* Returns the elapsed time in milliseconds.
*/
private long elapsedTimeMs(final long startTime) {
return (System.nanoTime() - startTime) / ONE_MILLION;
}
/**
* Parse the list file response
*
* @param stream InputStream contains the list results.
* @throws IOException
*/
private void parseListFilesResponse(final InputStream stream) throws IOException {
if (stream == null) {
return;
}
if (listResultSchema != null) {
// already parse the response
return;
}
try {
final ObjectMapper objectMapper = new ObjectMapper();
this.listResultSchema = objectMapper.readValue(stream, ListResultSchema.class);
} catch (IOException ex) {
this.LOG.error("Unable to deserialize list results", ex);
throw ex;
}
}
/**
* Check null stream, this is to pass findbugs's redundant check for NULL
* @param stream InputStream
*/
private boolean isNullInputStream(InputStream stream) {
return stream == null ? true : false;
}
}

View File

@ -0,0 +1,693 @@
/**
* 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.services;
import javax.xml.bind.DatatypeConverter;
import java.io.File;
import java.io.InputStream;
import java.io.OutputStream;
import java.net.URI;
import java.net.URISyntaxException;
import java.nio.ByteBuffer;
import java.nio.CharBuffer;
import java.nio.charset.CharacterCodingException;
import java.nio.charset.Charset;
import java.nio.charset.CharsetDecoder;
import java.nio.charset.CharsetEncoder;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Set;
import java.util.HashSet;
import java.util.Hashtable;
import java.util.Map;
import java.util.concurrent.ConcurrentHashMap;
import com.google.common.base.Preconditions;
import com.google.inject.Inject;
import com.google.inject.Singleton;
import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException;
import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidAbfsRestOperationException;
import org.joda.time.DateTime;
import org.joda.time.format.DateTimeFormat;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FSDataOutputStream;
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.TimeoutException;
import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidFileSystemPropertyException;
import org.apache.hadoop.fs.azurebfs.contracts.services.AbfsHttpService;
import org.apache.hadoop.fs.azurebfs.contracts.services.AbfsHttpClientFactory;
import org.apache.hadoop.fs.azurebfs.contracts.services.ConfigurationService;
import org.apache.hadoop.fs.azurebfs.contracts.services.TracingService;
import org.apache.hadoop.fs.azurebfs.contracts.services.ListResultEntrySchema;
import org.apache.hadoop.fs.azurebfs.contracts.services.ListResultSchema;
import org.apache.hadoop.fs.azurebfs.contracts.services.AzureServiceErrorCode;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystem;
import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations;
import org.apache.hadoop.fs.permission.FsAction;
import org.apache.hadoop.fs.permission.FsPermission;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import static org.apache.hadoop.util.Time.now;
@Singleton
@InterfaceAudience.Private
@InterfaceStability.Evolving
final class AbfsHttpServiceImpl implements AbfsHttpService {
public static final Logger LOG = LoggerFactory.getLogger(AbfsHttpService.class);
private static final String DATE_TIME_PATTERN = "E, dd MMM yyyy HH:mm:ss 'GMT'";
private static final String XMS_PROPERTIES_ENCODING = "ISO-8859-1";
private static final int LIST_MAX_RESULTS = 5000;
private static final int DELETE_DIRECTORY_TIMEOUT_MILISECONDS = 180000;
private static final int RENAME_TIMEOUT_MILISECONDS = 180000;
private final AbfsHttpClientFactory abfsHttpClientFactory;
private final ConcurrentHashMap<AzureBlobFileSystem, AbfsClient> clientCache;
private final ConfigurationService configurationService;
private final Set<String> azureAtomicRenameDirSet;
@Inject
AbfsHttpServiceImpl(
final ConfigurationService configurationService,
final AbfsHttpClientFactory abfsHttpClientFactory,
final TracingService tracingService) {
Preconditions.checkNotNull(abfsHttpClientFactory, "abfsHttpClientFactory");
Preconditions.checkNotNull(configurationService, "configurationService");
Preconditions.checkNotNull(tracingService, "tracingService");
this.configurationService = configurationService;
this.clientCache = new ConcurrentHashMap<>();
this.abfsHttpClientFactory = abfsHttpClientFactory;
this.azureAtomicRenameDirSet = new HashSet<>(Arrays.asList(configurationService.getAzureAtomicRenameDirs().split(AbfsHttpConstants.COMMA)));
}
@Override
public Hashtable<String, String> getFilesystemProperties(final AzureBlobFileSystem azureBlobFileSystem)
throws AzureBlobFileSystemException{
final AbfsClient client = this.getOrCreateClient(azureBlobFileSystem);
this.LOG.debug(
"getFilesystemProperties for filesystem: {}",
client.getFileSystem());
final Hashtable<String, String> parsedXmsProperties;
final AbfsRestOperation op = client.getFilesystemProperties();
final String xMsProperties = op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_PROPERTIES);
parsedXmsProperties = parseCommaSeparatedXmsProperties(xMsProperties);
return parsedXmsProperties;
}
@Override
public void setFilesystemProperties(final AzureBlobFileSystem azureBlobFileSystem, final Hashtable<String, String> properties) throws
AzureBlobFileSystemException {
if (properties == null || properties.size() == 0) {
return;
}
final AbfsClient client = this.getOrCreateClient(azureBlobFileSystem);
this.LOG.debug(
"setFilesystemProperties for filesystem: {} with properties: {}",
client.getFileSystem(),
properties);
final String commaSeparatedProperties;
try {
commaSeparatedProperties = convertXmsPropertiesToCommaSeparatedString(properties);
} catch (CharacterCodingException ex) {
throw new InvalidAbfsRestOperationException(ex);
}
client.setFilesystemProperties(commaSeparatedProperties);
}
@Override
public Hashtable<String, String> getPathProperties(final AzureBlobFileSystem azureBlobFileSystem, final Path path) throws
AzureBlobFileSystemException {
final AbfsClient client = this.getOrCreateClient(azureBlobFileSystem);
this.LOG.debug(
"getPathProperties for filesystem: {} path: {}",
client.getFileSystem(),
path.toString());
final Hashtable<String, String> parsedXmsProperties;
final AbfsRestOperation op = client.getPathProperties(AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path));
final String xMsProperties = op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_PROPERTIES);
parsedXmsProperties = parseCommaSeparatedXmsProperties(xMsProperties);
return parsedXmsProperties;
}
@Override
public void setPathProperties(final AzureBlobFileSystem azureBlobFileSystem, final Path path, final Hashtable<String,
String> properties) throws
AzureBlobFileSystemException {
final AbfsClient client = this.getOrCreateClient(azureBlobFileSystem);
this.LOG.debug(
"setFilesystemProperties for filesystem: {} path: {} with properties: {}",
client.getFileSystem(),
path.toString(),
properties);
final String commaSeparatedProperties;
try {
commaSeparatedProperties = convertXmsPropertiesToCommaSeparatedString(properties);
} catch (CharacterCodingException ex) {
throw new InvalidAbfsRestOperationException(ex);
}
client.setPathProperties("/" + getRelativePath(path), commaSeparatedProperties);
}
@Override
public void createFilesystem(final AzureBlobFileSystem azureBlobFileSystem) throws AzureBlobFileSystemException {
final AbfsClient client = this.getOrCreateClient(azureBlobFileSystem);
this.LOG.debug(
"createFilesystem for filesystem: {}",
client.getFileSystem());
client.createFilesystem();
}
@Override
public void deleteFilesystem(final AzureBlobFileSystem azureBlobFileSystem) throws AzureBlobFileSystemException {
final AbfsClient client = this.getOrCreateClient(azureBlobFileSystem);
this.LOG.debug(
"deleteFilesystem for filesystem: {}",
client.getFileSystem());
client.deleteFilesystem();
}
@Override
public OutputStream createFile(final AzureBlobFileSystem azureBlobFileSystem, final Path path, final boolean overwrite) throws
AzureBlobFileSystemException {
final AbfsClient client = this.getOrCreateClient(azureBlobFileSystem);
this.LOG.debug(
"createFile filesystem: {} path: {} overwrite: {}",
client.getFileSystem(),
path.toString(),
overwrite);
client.createPath(AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path), true, overwrite);
final OutputStream outputStream;
outputStream = new FSDataOutputStream(
new AbfsOutputStream(client, AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path), 0,
configurationService.getWriteBufferSize()), null);
return outputStream;
}
@Override
public Void createDirectory(final AzureBlobFileSystem azureBlobFileSystem, final Path path) throws AzureBlobFileSystemException {
final AbfsClient client = this.getOrCreateClient(azureBlobFileSystem);
this.LOG.debug(
"createDirectory filesystem: {} path: {} overwrite: {}",
client.getFileSystem(),
path.toString());
client.createPath("/" + getRelativePath(path), false, true);
return null;
}
@Override
public InputStream openFileForRead(final AzureBlobFileSystem azureBlobFileSystem, final Path path,
final FileSystem.Statistics statistics) throws AzureBlobFileSystemException {
final AbfsClient client = getOrCreateClient(azureBlobFileSystem);
this.LOG.debug(
"openFileForRead filesystem: {} path: {}",
client.getFileSystem(),
path.toString());
final AbfsRestOperation op = client.getPathProperties(AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path));
final String resourceType = op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_RESOURCE_TYPE);
final long contentLength = Long.parseLong(op.getResult().getResponseHeader(HttpHeaderConfigurations.CONTENT_LENGTH));
final String eTag = op.getResult().getResponseHeader(HttpHeaderConfigurations.ETAG);
if (parseIsDirectory(resourceType)) {
throw new AbfsRestOperationException(
AzureServiceErrorCode.PATH_NOT_FOUND.getStatusCode(),
AzureServiceErrorCode.PATH_NOT_FOUND.getErrorCode(),
"openFileForRead must be used with files and not directories",
null);
}
// Add statistics for InputStream
return new FSDataInputStream(
new AbfsInputStream(client, statistics, AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path), contentLength,
configurationService.getReadBufferSize(), configurationService.getReadAheadQueueDepth(), eTag));
}
@Override
public OutputStream openFileForWrite(final AzureBlobFileSystem azureBlobFileSystem, final Path path, final boolean overwrite) throws
AzureBlobFileSystemException {
final AbfsClient client = getOrCreateClient(azureBlobFileSystem);
this.LOG.debug(
"openFileForWrite filesystem: {} path: {} overwrite: {}",
client.getFileSystem(),
path.toString(),
overwrite);
final AbfsRestOperation op = client.getPathProperties(AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path));
final String resourceType = op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_RESOURCE_TYPE);
final Long contentLength = Long.valueOf(op.getResult().getResponseHeader(HttpHeaderConfigurations.CONTENT_LENGTH));
if (parseIsDirectory(resourceType)) {
throw new AbfsRestOperationException(
AzureServiceErrorCode.PATH_NOT_FOUND.getStatusCode(),
AzureServiceErrorCode.PATH_NOT_FOUND.getErrorCode(),
"openFileForRead must be used with files and not directories",
null);
}
final long offset = overwrite ? 0 : contentLength;
final OutputStream outputStream;
outputStream = new FSDataOutputStream(
new AbfsOutputStream(client, AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path),
offset, configurationService.getWriteBufferSize()), null);
return outputStream;
}
@Override
public void rename(final AzureBlobFileSystem azureBlobFileSystem, final Path source, final Path destination) throws
AzureBlobFileSystemException {
if (isAtomicRenameKey(source.getName())) {
this.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.");
}
final AbfsClient client = this.getOrCreateClient(azureBlobFileSystem);
this.LOG.debug(
"renameAsync filesystem: {} source: {} destination: {}",
client.getFileSystem(),
source.toString(),
destination.toString());
String continuation = null;
long deadline = now() + RENAME_TIMEOUT_MILISECONDS;
do {
if (now() > deadline) {
LOG.debug(
"Rename {} to {} timed out.",
source,
destination);
throw new TimeoutException("Rename timed out.");
}
AbfsRestOperation op = client.renamePath(AbfsHttpConstants.FORWARD_SLASH + getRelativePath(source),
AbfsHttpConstants.FORWARD_SLASH + getRelativePath(destination), continuation);
continuation = op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_CONTINUATION);
} while (continuation != null && !continuation.isEmpty());
}
@Override
public void delete(final AzureBlobFileSystem azureBlobFileSystem, final Path path, final boolean recursive) throws
AzureBlobFileSystemException {
final AbfsClient client = this.getOrCreateClient(azureBlobFileSystem);
this.LOG.debug(
"delete filesystem: {} path: {} recursive: {}",
client.getFileSystem(),
path.toString(),
String.valueOf(recursive));
String continuation = null;
long deadline = now() + DELETE_DIRECTORY_TIMEOUT_MILISECONDS;
do {
if (now() > deadline) {
this.LOG.debug(
"Delete directory {} timed out.", path);
throw new TimeoutException("Delete directory timed out.");
}
AbfsRestOperation op = client.deletePath(AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path), recursive, continuation);
continuation = op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_CONTINUATION);
} while (continuation != null && !continuation.isEmpty());
}
@Override
public FileStatus getFileStatus(final AzureBlobFileSystem azureBlobFileSystem, final Path path) throws AzureBlobFileSystemException {
final AbfsClient client = this.getOrCreateClient(azureBlobFileSystem);
this.LOG.debug(
"getFileStatus filesystem: {} path: {}",
client.getFileSystem(),
path.toString());
if (path.isRoot()) {
AbfsRestOperation op = client.getFilesystemProperties();
final long blockSize = configurationService.getAzureBlockSize();
final String eTag = op.getResult().getResponseHeader(HttpHeaderConfigurations.ETAG);
final String lastModified = op.getResult().getResponseHeader(HttpHeaderConfigurations.LAST_MODIFIED);
return new VersionedFileStatus(
azureBlobFileSystem.getOwnerUser(),
azureBlobFileSystem.getOwnerUserPrimaryGroup(),
0,
true,
1,
blockSize,
parseLastModifiedTime(lastModified).getMillis(),
path,
eTag);
} else {
AbfsRestOperation op = client.getPathProperties(AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path));
final long blockSize = configurationService.getAzureBlockSize();
final String eTag = op.getResult().getResponseHeader(HttpHeaderConfigurations.ETAG);
final String lastModified = op.getResult().getResponseHeader(HttpHeaderConfigurations.LAST_MODIFIED);
final String contentLength = op.getResult().getResponseHeader(HttpHeaderConfigurations.CONTENT_LENGTH);
final String resourceType = op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_RESOURCE_TYPE);
return new VersionedFileStatus(
azureBlobFileSystem.getOwnerUser(),
azureBlobFileSystem.getOwnerUserPrimaryGroup(),
parseContentLength(contentLength),
parseIsDirectory(resourceType),
1,
blockSize,
parseLastModifiedTime(lastModified).getMillis(),
path,
eTag);
}
}
@Override
public FileStatus[] listStatus(final AzureBlobFileSystem azureBlobFileSystem, final Path path) throws AzureBlobFileSystemException {
final AbfsClient client = this.getOrCreateClient(azureBlobFileSystem);
this.LOG.debug(
"listStatus filesystem: {} path: {}",
client.getFileSystem(),
path.toString());
String relativePath = path.isRoot() ? AbfsHttpConstants.EMPTY_STRING : getRelativePath(path);
String continuation = null;
ArrayList<FileStatus> fileStatuses = new ArrayList<>();
do {
AbfsRestOperation op = client.listPath(relativePath, false, LIST_MAX_RESULTS, continuation);
continuation = op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_CONTINUATION);
ListResultSchema retrievedSchema = op.getResult().getListResultSchema();
if (retrievedSchema == null) {
throw new AbfsRestOperationException(
AzureServiceErrorCode.PATH_NOT_FOUND.getStatusCode(),
AzureServiceErrorCode.PATH_NOT_FOUND.getErrorCode(),
"listStatusAsync path not found",
null, op.getResult());
}
long blockSize = configurationService.getAzureBlockSize();
for (ListResultEntrySchema entry : retrievedSchema.paths()) {
long lastModifiedMillis = 0;
long contentLength = entry.contentLength() == null ? 0 : entry.contentLength();
boolean isDirectory = entry.isDirectory() == null ? false : entry.isDirectory();
if (entry.lastModified() != null && !entry.lastModified().isEmpty()) {
final DateTime dateTime = DateTime.parse(
entry.lastModified(),
DateTimeFormat.forPattern(DATE_TIME_PATTERN).withZoneUTC());
lastModifiedMillis = dateTime.getMillis();
}
fileStatuses.add(
new VersionedFileStatus(
azureBlobFileSystem.getOwnerUser(),
azureBlobFileSystem.getOwnerUserPrimaryGroup(),
contentLength,
isDirectory,
1,
blockSize,
lastModifiedMillis,
azureBlobFileSystem.makeQualified(new Path(File.separator + entry.name())),
entry.eTag()));
}
} while (continuation != null && !continuation.isEmpty());
return fileStatuses.toArray(new FileStatus[0]);
}
@Override
public synchronized void closeFileSystem(final AzureBlobFileSystem azureBlobFileSystem) throws AzureBlobFileSystemException {
this.clientCache.remove(azureBlobFileSystem);
}
@Override
public boolean isAtomicRenameKey(String key) {
return isKeyForDirectorySet(key, azureAtomicRenameDirSet);
}
private String getRelativePath(final Path path) {
Preconditions.checkNotNull(path, "path");
final String relativePath = path.toUri().getPath();
if (relativePath.length() == 0) {
return relativePath;
}
if (relativePath.charAt(0) == Path.SEPARATOR_CHAR) {
if (relativePath.length() == 1) {
return AbfsHttpConstants.EMPTY_STRING;
}
return relativePath.substring(1);
}
return relativePath;
}
private synchronized AbfsClient getOrCreateClient(final AzureBlobFileSystem azureBlobFileSystem) throws
AzureBlobFileSystemException {
Preconditions.checkNotNull(azureBlobFileSystem, "azureBlobFileSystem");
AbfsClient client = this.clientCache.get(azureBlobFileSystem);
if (client != null) {
return client;
}
client = abfsHttpClientFactory.create(azureBlobFileSystem);
this.clientCache.put(
azureBlobFileSystem,
client);
return client;
}
private long parseContentLength(final String contentLength) {
if (contentLength == null) {
return -1;
}
return Long.parseLong(contentLength);
}
private boolean parseIsDirectory(final String resourceType) {
return resourceType == null ? false : resourceType.equalsIgnoreCase(AbfsHttpConstants.DIRECTORY);
}
private DateTime parseLastModifiedTime(final String lastModifiedTime) {
return DateTime.parse(
lastModifiedTime,
DateTimeFormat.forPattern(DATE_TIME_PATTERN).withZoneUTC());
}
private String convertXmsPropertiesToCommaSeparatedString(final Hashtable<String, String> properties) throws
CharacterCodingException {
StringBuilder commaSeparatedProperties = new StringBuilder();
final CharsetEncoder encoder = Charset.forName(XMS_PROPERTIES_ENCODING).newEncoder();
for (Map.Entry<String, String> propertyEntry : properties.entrySet()) {
String key = propertyEntry.getKey();
String value = propertyEntry.getValue();
Boolean canEncodeValue = encoder.canEncode(value);
if (!canEncodeValue) {
throw new CharacterCodingException();
}
String encodedPropertyValue = DatatypeConverter.printBase64Binary(encoder.encode(CharBuffer.wrap(value)).array());
commaSeparatedProperties.append(key)
.append(AbfsHttpConstants.EQUAL)
.append(encodedPropertyValue);
commaSeparatedProperties.append(AbfsHttpConstants.COMMA);
}
if (commaSeparatedProperties.length() != 0) {
commaSeparatedProperties.deleteCharAt(commaSeparatedProperties.length() - 1);
}
return commaSeparatedProperties.toString();
}
private Hashtable<String, String> parseCommaSeparatedXmsProperties(String xMsProperties) throws
InvalidFileSystemPropertyException, InvalidAbfsRestOperationException {
Hashtable<String, String> properties = new Hashtable<>();
final CharsetDecoder decoder = Charset.forName(XMS_PROPERTIES_ENCODING).newDecoder();
if (xMsProperties != null && !xMsProperties.isEmpty()) {
String[] userProperties = xMsProperties.split(AbfsHttpConstants.COMMA);
if (userProperties.length == 0) {
return properties;
}
for (String property : userProperties) {
if (property.isEmpty()) {
throw new InvalidFileSystemPropertyException(xMsProperties);
}
String[] nameValue = property.split(AbfsHttpConstants.EQUAL, 2);
if (nameValue.length != 2) {
throw new InvalidFileSystemPropertyException(xMsProperties);
}
byte[] decodedValue = DatatypeConverter.parseBase64Binary(nameValue[1]);
final String value;
try {
value = decoder.decode(ByteBuffer.wrap(decodedValue)).toString();
} catch (CharacterCodingException ex) {
throw new InvalidAbfsRestOperationException(ex);
}
properties.put(nameValue[0], value);
}
}
return properties;
}
private boolean isKeyForDirectorySet(String key, Set<String> dirSet) {
for (String dir : dirSet) {
if (dir.isEmpty() || key.startsWith(dir + AbfsHttpConstants.FORWARD_SLASH)) {
return true;
}
try {
URI uri = new URI(dir);
if (null == uri.getAuthority()) {
if (key.startsWith(dir + "/")){
return true;
}
}
} catch (URISyntaxException e) {
this.LOG.info("URI syntax error creating URI for {}", dir);
}
}
return false;
}
private static class VersionedFileStatus extends FileStatus {
private final String version;
VersionedFileStatus(
final String owner, final String group,
final long length, final boolean isdir, final int blockReplication,
final long blocksize, final long modificationTime, final Path path,
String version) {
super(length, isdir, blockReplication, blocksize, modificationTime, 0,
new FsPermission(FsAction.ALL, FsAction.ALL, FsAction.ALL),
owner,
group,
path);
this.version = version;
}
/** Compare if this object is equal to another object.
* @param obj the object to be compared.
* @return true if two file status has the same path name; false if not.
*/
@Override
public boolean equals(Object obj) {
if (obj == this) {
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
* the hash code of the path name.
*
* @return a hash code value for the path name and version
*/
@Override
public int hashCode() {
int hash = getPath().hashCode();
hash = 89 * hash + (this.version != null ? this.version.hashCode() : 0);
return hash;
}
/**
* Returns the version of this FileStatus
*
* @return a string value for the FileStatus version
*/
public String getVersion() {
return this.version;
}
}
}

View File

@ -0,0 +1,382 @@
/**
* 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.services;
import java.io.EOFException;
import java.io.IOException;
import com.google.common.base.Preconditions;
import org.apache.hadoop.fs.FSExceptionMessages;
import org.apache.hadoop.fs.FSInputStream;
import org.apache.hadoop.fs.FileSystem.Statistics;
import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException;
/**
* The AbfsInputStream for AbfsClient
*/
public class AbfsInputStream extends FSInputStream {
private final AbfsClient client;
private final Statistics statistics;
private final String path;
private final long contentLength;
private final int bufferSize; // default buffer size
private final int readAheadQueueDepth; // initialized in constructor
private final String eTag; // eTag of the path when InputStream are created
private final boolean tolerateOobAppends; // whether tolerate Oob Appends
private final boolean readAheadEnabled; // whether enable readAhead;
private byte[] buffer = null; // will be initialized on first use
private long fCursor = 0; // cursor of buffer within file - offset of next byte to read from remote server
private long fCursorAfterLastRead = -1;
private int bCursor = 0; // cursor of read within buffer - offset of next byte to be returned from buffer
private int limit = 0; // offset of next byte to be read into buffer from service (i.e., upper marker+1
// of valid bytes in buffer)
private boolean closed = false;
public AbfsInputStream(
final AbfsClient client,
final Statistics statistics,
final String path,
final long contentLength,
final int bufferSize,
final int readAheadQueueDepth,
final String eTag) {
super();
this.client = client;
this.statistics = statistics;
this.path = path;
this.contentLength = contentLength;
this.bufferSize = bufferSize;
this.readAheadQueueDepth = (readAheadQueueDepth >= 0) ? readAheadQueueDepth : 2 * Runtime.getRuntime().availableProcessors();
this.eTag = eTag;
this.tolerateOobAppends = false;
this.readAheadEnabled = true;
}
public String getPath() {
return path;
}
@Override
public int read() throws IOException {
byte[] b = new byte[1];
int numberOfBytesRead = read(b, 0, 1);
if (numberOfBytesRead < 0) {
return -1;
} else {
return (b[0] & 0xFF);
}
}
@Override
public synchronized int read(final byte[] b, final int off, final int len) throws IOException {
int currentOff = off;
int currentLen = len;
int lastReadBytes;
int totalReadBytes = 0;
do {
lastReadBytes = readOneBlock(b, currentOff, currentLen);
if (lastReadBytes > 0) {
currentOff += lastReadBytes;
currentLen -= lastReadBytes;
totalReadBytes += lastReadBytes;
}
if (currentLen <= 0 || currentLen > b.length - currentOff) {
break;
}
} while (lastReadBytes > 0);
return totalReadBytes > 0 ? totalReadBytes : lastReadBytes;
}
private int readOneBlock(final byte[] b, final int off, final int len) throws IOException {
if (closed) {
throw new IOException(FSExceptionMessages.STREAM_IS_CLOSED);
}
Preconditions.checkNotNull(b);
if (len == 0) {
return 0;
}
if (this.available() == 0) {
return -1;
}
if (off < 0 || len < 0 || len > b.length - off) {
throw new IndexOutOfBoundsException();
}
//If buffer is empty, then fill the buffer.
if (bCursor == limit) {
//If EOF, then return -1
if (fCursor >= contentLength) {
return -1;
}
long bytesRead = 0;
//reset buffer to initial state - i.e., throw away existing data
bCursor = 0;
limit = 0;
if (buffer == null) {
buffer = new byte[bufferSize];
}
// Enable readAhead when reading sequentially
if (-1 == fCursorAfterLastRead || fCursorAfterLastRead == fCursor || b.length >= bufferSize) {
bytesRead = readInternal(fCursor, buffer, 0, bufferSize, false);
} else {
bytesRead = readInternal(fCursor, buffer, 0, b.length, true);
}
if (bytesRead == -1) {
return -1;
}
limit += bytesRead;
fCursor += bytesRead;
fCursorAfterLastRead = fCursor;
}
//If there is anything in the buffer, then return lesser of (requested bytes) and (bytes in buffer)
//(bytes returned may be less than requested)
int bytesRemaining = limit - bCursor;
int bytesToRead = Math.min(len, bytesRemaining);
System.arraycopy(buffer, bCursor, b, off, bytesToRead);
bCursor += bytesToRead;
if (statistics != null) {
statistics.incrementBytesRead(bytesToRead);
}
return bytesToRead;
}
private int readInternal(final long position, final byte[] b, final int offset, final int length,
final boolean bypassReadAhead) throws IOException {
if (readAheadEnabled && !bypassReadAhead) {
// try reading from read-ahead
if (offset != 0) {
throw new IllegalArgumentException("readahead buffers cannot have non-zero buffer offsets");
}
int receivedBytes;
// queue read-aheads
int numReadAheads = this.readAheadQueueDepth;
long nextSize;
long nextOffset = position;
while (numReadAheads > 0 && nextOffset < contentLength) {
nextSize = Math.min((long) bufferSize, contentLength - nextOffset);
ReadBufferManager.getBufferManager().queueReadAhead(this, nextOffset, (int) nextSize);
nextOffset = nextOffset + nextSize;
numReadAheads--;
}
// try reading from buffers first
receivedBytes = ReadBufferManager.getBufferManager().getBlock(this, position, length, b);
if (receivedBytes > 0) {
return receivedBytes;
}
// got nothing from read-ahead, do our own read now
receivedBytes = readRemote(position, b, offset, length);
return receivedBytes;
} else {
return readRemote(position, b, offset, length);
}
}
int readRemote(long position, byte[] b, int offset, int length) throws IOException {
if (position < 0) {
throw new IllegalArgumentException("attempting to read from negative offset");
}
if (position >= contentLength) {
return -1; // Hadoop prefers -1 to EOFException
}
if (b == null) {
throw new IllegalArgumentException("null byte array passed in to read() method");
}
if (offset >= b.length) {
throw new IllegalArgumentException("offset greater than length of array");
}
if (length < 0) {
throw new IllegalArgumentException("requested read length is less than zero");
}
if (length > (b.length - offset)) {
throw new IllegalArgumentException("requested read length is more than will fit after requested offset in buffer");
}
final AbfsRestOperation op;
try {
op = client.read(path, position, b, offset, length, tolerateOobAppends ? "*" : eTag);
} catch (AzureBlobFileSystemException ex) {
throw new IOException(ex);
}
long bytesRead = op.getResult().getBytesReceived();
if (bytesRead > Integer.MAX_VALUE) {
throw new IOException("Unexpected Content-Length");
}
return (int) bytesRead;
}
/**
* Seek to given position in stream.
* @param n position to seek to
* @throws IOException if there is an error
* @throws EOFException if attempting to seek past end of file
*/
@Override
public synchronized void seek(long n) throws IOException {
if (closed) {
throw new IOException(FSExceptionMessages.STREAM_IS_CLOSED);
}
if (n < 0) {
throw new EOFException(FSExceptionMessages.NEGATIVE_SEEK);
}
if (n > contentLength) {
throw new EOFException(FSExceptionMessages.CANNOT_SEEK_PAST_EOF);
}
if (n>=fCursor-limit && n<=fCursor) { // within buffer
bCursor = (int) (n-(fCursor-limit));
return;
}
// next read will read from here
fCursor = n;
//invalidate buffer
limit = 0;
bCursor = 0;
}
@Override
public synchronized long skip(long n) throws IOException {
if (closed) {
throw new IOException(FSExceptionMessages.STREAM_IS_CLOSED);
}
long currentPos = getPos();
if (currentPos == contentLength) {
if (n > 0) {
throw new EOFException(FSExceptionMessages.CANNOT_SEEK_PAST_EOF);
}
}
long newPos = currentPos + n;
if (newPos < 0) {
newPos = 0;
n = newPos - currentPos;
}
if (newPos > contentLength) {
newPos = contentLength;
n = newPos - currentPos;
}
seek(newPos);
return n;
}
/**
* Return the size of the remaining available bytes
* if the size is less than or equal to {@link Integer#MAX_VALUE},
* otherwise, return {@link Integer#MAX_VALUE}.
*
* This is to match the behavior of DFSInputStream.available(),
* which some clients may rely on (HBase write-ahead log reading in
* particular).
*/
@Override
public synchronized int available() throws IOException {
if (closed) {
throw new IOException(
FSExceptionMessages.STREAM_IS_CLOSED);
}
final long remaining = this.contentLength - this.getPos();
return remaining <= Integer.MAX_VALUE
? (int) remaining : Integer.MAX_VALUE;
}
/**
* Returns the length of the file that this stream refers to. Note that the length returned is the length
* as of the time the Stream was opened. Specifically, if there have been subsequent appends to the file,
* they wont be reflected in the returned length.
*
* @return length of the file.
* @throws IOException if the stream is closed
*/
public long length() throws IOException {
if (closed) {
throw new IOException(FSExceptionMessages.STREAM_IS_CLOSED);
}
return contentLength;
}
/**
* Return the current offset from the start of the file
* @throws IOException throws {@link IOException} if there is an error
*/
@Override
public synchronized long getPos() throws IOException {
if (closed) {
throw new IOException(FSExceptionMessages.STREAM_IS_CLOSED);
}
return fCursor - limit + bCursor;
}
/**
* Seeks a different copy of the data. Returns true if
* found a new source, false otherwise.
* @throws IOException throws {@link IOException} if there is an error
*/
@Override
public boolean seekToNewSource(long l) throws IOException {
return false;
}
@Override
public synchronized void close() throws IOException {
closed = true;
buffer = null; // de-reference the buffer so it can be GC'ed sooner
}
/**
* Not supported by this stream. Throws {@link UnsupportedOperationException}
* @param readlimit ignored
*/
@Override
public synchronized void mark(int readlimit) {
throw new UnsupportedOperationException("mark()/reset() not supported on this stream");
}
/**
* Not supported by this stream. Throws {@link UnsupportedOperationException}
*/
@Override
public synchronized void reset() throws IOException {
throw new UnsupportedOperationException("mark()/reset() not supported on this stream");
}
/**
* gets whether mark and reset are supported by {@code ADLFileInputStream}. Always returns false.
*
* @return always {@code false}
*/
@Override
public boolean markSupported() {
return false;
}
}

View File

@ -0,0 +1,335 @@
/**
* 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.services;
import java.io.IOException;
import java.io.OutputStream;
import java.util.concurrent.ConcurrentLinkedDeque;
import java.util.concurrent.LinkedBlockingQueue;
import java.util.concurrent.ExecutorCompletionService;
import java.util.concurrent.ThreadPoolExecutor;
import java.util.concurrent.Callable;
import java.util.concurrent.Future;
import java.util.concurrent.TimeUnit;
import com.google.common.base.Preconditions;
import org.apache.hadoop.fs.FSExceptionMessages;
import org.apache.hadoop.fs.Syncable;
import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException;
/**
* The BlobFsOutputStream for Rest AbfsClient
*/
public class AbfsOutputStream extends OutputStream implements Syncable {
private final AbfsClient client;
private final String path;
private long position;
private boolean closed;
private volatile IOException lastError;
private long lastFlushOffset;
private long lastTotalAppendOffset = 0;
private final int bufferSize;
private byte[] buffer;
private int bufferIndex;
private final int maxConcurrentRequestCount;
private ConcurrentLinkedDeque<WriteOperation> writeOperations;
private final ThreadPoolExecutor threadExecutor;
private final ExecutorCompletionService<Void> completionService;
public AbfsOutputStream(
final AbfsClient client,
final String path,
final long position,
final int bufferSize) {
this.client = client;
this.path = path;
this.position = position;
this.closed = false;
this.lastError = null;
this.lastFlushOffset = 0;
this.bufferSize = bufferSize;
this.buffer = new byte[bufferSize];
this.bufferIndex = 0;
this.writeOperations = new ConcurrentLinkedDeque<>();
this.maxConcurrentRequestCount = 4 * Runtime.getRuntime().availableProcessors();
this.threadExecutor
= new ThreadPoolExecutor(maxConcurrentRequestCount,
maxConcurrentRequestCount,
10L,
TimeUnit.SECONDS,
new LinkedBlockingQueue());
this.completionService = new ExecutorCompletionService(this.threadExecutor);
}
/**
* Writes the specified byte to this output stream. The general contract for
* write is that one byte is written to the output stream. The byte to be
* written is the eight low-order bits of the argument b. The 24 high-order
* bits of b are ignored.
*
* @param byteVal the byteValue to write.
* @throws IOException if an I/O error occurs. In particular, an IOException may be
* thrown if the output stream has been closed.
*/
@Override
public void write(final int byteVal) throws IOException {
write(new byte[]{(byte) (byteVal & 0xFF)});
}
/**
* Writes length bytes from the specified byte array starting at off to
* this output stream.
*
* @param data the byte array to write.
* @param off the start off in the data.
* @param length the number of bytes to write.
* @throws IOException if an I/O error occurs. In particular, an IOException may be
* thrown if the output stream has been closed.
*/
@Override
public synchronized void write(final byte[] data, final int off, final int length)
throws IOException {
if (this.lastError != null) {
throw this.lastError;
}
Preconditions.checkArgument(data != null, "null data");
if (off < 0 || length < 0 || length > data.length - off) {
throw new IndexOutOfBoundsException();
}
int currentOffset = off;
int writableBytes = bufferSize - bufferIndex;
int numberOfBytesToWrite = length;
while (numberOfBytesToWrite > 0) {
if (writableBytes <= numberOfBytesToWrite) {
System.arraycopy(data, currentOffset, buffer, bufferIndex, writableBytes);
bufferIndex += writableBytes;
writeCurrentBufferToService();
currentOffset += writableBytes;
numberOfBytesToWrite = numberOfBytesToWrite - writableBytes;
} else {
System.arraycopy(data, currentOffset, buffer, bufferIndex, numberOfBytesToWrite);
bufferIndex += numberOfBytesToWrite;
numberOfBytesToWrite = 0;
}
writableBytes = bufferSize - bufferIndex;
}
}
/**
* 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
* service. Data is queued for writing and forced out to the service
* before the call returns.
*/
@Override
public void flush() throws IOException {
this.flushInternalAsync();
}
/** Similar to posix fsync, flush out the data in client's user buffer
* all the way to the disk device (but the disk may have it in its cache).
* @throws IOException if error occurs
*/
@Override
public void hsync() throws IOException {
this.flushInternal();
}
/** Flush out the data in client's user buffer. After the return of
* this call, new readers will see the data.
* @throws IOException if any error occurs
*/
@Override
public void hflush() throws IOException {
this.flushInternal();
}
/**
* Force all data in the output stream to be written to Azure storage.
* Wait to return until this is complete. Close the access to the stream and
* shutdown the upload thread pool.
* If the blob was created, its lease will be released.
* Any error encountered caught in threads and stored will be rethrown here
* after cleanup.
*/
@Override
public synchronized void close() throws IOException {
if (closed) {
return;
}
try {
this.flushInternal();
this.threadExecutor.shutdown();
} finally {
this.lastError = new IOException(FSExceptionMessages.STREAM_IS_CLOSED);
this.buffer = null;
this.bufferIndex = 0;
this.closed = true;
this.writeOperations.clear();
if (!this.threadExecutor.isShutdown()) {
this.threadExecutor.shutdownNow();
}
}
}
private synchronized void flushInternal() throws IOException {
if (this.lastError != null) {
throw this.lastError;
}
this.writeCurrentBufferToService();
this.flushWrittenBytesToService();
}
private synchronized void flushInternalAsync() throws IOException {
if (this.lastError != null) {
throw this.lastError;
}
this.writeCurrentBufferToService();
this.flushWrittenBytesToServiceAsync();
}
private synchronized void writeCurrentBufferToService() throws IOException {
if (bufferIndex == 0) {
return;
}
final byte[] bytes = this.buffer;
final int bytesLength = bufferIndex;
this.buffer = new byte[bufferSize];
this.bufferIndex = 0;
final long offset = this.position;
this.position += bytesLength;
if (this.threadExecutor.getQueue().size() >= maxConcurrentRequestCount * 2) {
this.waitForTaskToComplete();
}
final Future job = this.completionService.submit(new Callable<Void>() {
@Override
public Void call() throws Exception {
client.append(path, offset, bytes, 0,
bytesLength);
return null;
}
});
this.writeOperations.add(new WriteOperation(job, offset, bytesLength));
// Try to shrink the queue
shrinkWriteOperationQueue();
}
private synchronized void flushWrittenBytesToService() throws IOException {
for (WriteOperation writeOperation : this.writeOperations) {
try {
writeOperation.task.get();
} catch (Exception ex) {
if (AzureBlobFileSystemException.class.isInstance(ex.getCause())) {
ex = AzureBlobFileSystemException.class.cast(ex.getCause());
}
this.lastError = new IOException(ex);
throw this.lastError;
}
}
flushWrittenBytesToServiceInternal(this.position, false);
}
private synchronized void flushWrittenBytesToServiceAsync() throws IOException {
shrinkWriteOperationQueue();
if (this.lastTotalAppendOffset > this.lastFlushOffset) {
this.flushWrittenBytesToServiceInternal(this.lastTotalAppendOffset, true);
}
this.lastTotalAppendOffset = 0;
}
private synchronized void flushWrittenBytesToServiceInternal(final long offset, final boolean retainUncommitedData) throws IOException {
try {
client.flush(path, offset, retainUncommitedData);
} catch (AzureBlobFileSystemException ex) {
throw new IOException(ex);
}
this.lastFlushOffset = offset;
}
/**
* Try to remove the completed write operations from the beginning of write
* operation FIFO queue.
*/
private synchronized void shrinkWriteOperationQueue() throws IOException {
try {
while (this.writeOperations.peek() != null && this.writeOperations.peek().task.isDone()) {
this.writeOperations.peek().task.get();
this.lastTotalAppendOffset += this.writeOperations.peek().length;
this.writeOperations.remove();
}
} catch (Exception e) {
if (AzureBlobFileSystemException.class.isInstance(e.getCause())) {
this.lastError = IOException.class.cast(e.getCause());
} else {
this.lastError = new IOException(e);
}
throw this.lastError;
}
}
private void waitForTaskToComplete() throws IOException {
boolean completed;
for (completed = false; this.completionService.poll() != null; completed = true) {}
if (!completed) {
try {
this.completionService.take();
} catch (InterruptedException e) {
this.lastError = new IOException(e);
throw this.lastError;
}
}
}
private static class WriteOperation {
private final Future<Void> task;
private final long startOffset;
private final long length;
WriteOperation(final Future<Void> task, final long startOffset, final long length) {
Preconditions.checkNotNull(task, "task");
Preconditions.checkArgument(startOffset >= 0, "startOffset");
Preconditions.checkArgument(length >= 0, "length");
this.task = task;
this.startOffset = startOffset;
this.length = length;
}
}
}

View File

@ -0,0 +1,178 @@
/**
* 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.services;
import java.io.IOException;
import java.net.HttpURLConnection;
import java.net.URL;
import java.util.List;
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.InvalidAbfsRestOperationException;
import org.slf4j.Logger;
/**
* The AbfsRestOperation for Rest AbfsClient
*/
public class AbfsRestOperation {
// Blob FS client, which has the credentials, retry policy, and logs.
private final AbfsClient client;
// the HTTP method (PUT, PATCH, POST, GET, HEAD, or DELETE)
private final String method;
// full URL including query parameters
private final URL url;
// all the custom HTTP request headers provided by the caller
private final List<AbfsHttpHeader> requestHeaders;
// This is a simple operation class, where all the upload methods have a
// request body and all the download methods have a response body.
private final boolean hasRequestBody;
private final Logger logger;
// For uploads, this is the request entity body. For downloads,
// this will hold the response entity body.
private byte[] buffer;
private int bufferOffset;
private int bufferLength;
private AbfsHttpOperation result;
public AbfsHttpOperation getResult() {
return result;
}
/**
* Initializes a new REST operation.
*
* @param client The Blob FS client.
* @param method The HTTP method (PUT, PATCH, POST, GET, HEAD, or DELETE).
* @param url The full URL including query string parameters.
* @param requestHeaders The HTTP request headers.
*/
AbfsRestOperation(final AbfsClient client,
final String method,
final URL url,
final List<AbfsHttpHeader> requestHeaders) {
this.client = client;
this.method = method;
this.url = url;
this.requestHeaders = requestHeaders;
this.hasRequestBody = (AbfsHttpConstants.HTTP_METHOD_PUT.equals(method)
|| AbfsHttpConstants.HTTP_METHOD_PATCH.equals(method));
this.logger = client.LOG;
}
/**
* Initializes a new REST operation.
*
* @param client The Blob FS client.
* @param method The HTTP method (PUT, PATCH, POST, GET, HEAD, or DELETE).
* @param url The full URL including query string parameters.
* @param requestHeaders The HTTP request headers.
* @param buffer For uploads, this is the request entity body. For downloads,
* this will hold the response entity body.
* @param bufferOffset An offset into the buffer where the data beings.
* @param bufferLength The length of the data in the buffer.
*/
AbfsRestOperation(AbfsClient client,
String method,
URL url,
List<AbfsHttpHeader> requestHeaders,
byte[] buffer,
int bufferOffset,
int bufferLength) {
this(client, method, url, requestHeaders);
this.buffer = buffer;
this.bufferOffset = bufferOffset;
this.bufferLength = bufferLength;
}
/**
* Executes the REST operation with retry, by issuing one or more
* HTTP operations.
*/
void execute() throws AzureBlobFileSystemException {
int retryCount = 0;
while (!executeHttpOperation(retryCount++)) {
try {
Thread.sleep(client.getRetryPolicy().getRetryInterval(retryCount));
} catch (InterruptedException ex) {
Thread.currentThread().interrupt();
}
}
if (result.getStatusCode() > HttpURLConnection.HTTP_BAD_REQUEST) {
throw new AbfsRestOperationException(result.getStatusCode(), result.getStorageErrorCode(),
result.getStorageErrorMessage(), null, result);
}
}
/**
* Executes a single HTTP operation to complete the REST operation. If it
* fails, there may be a retry. The retryCount is incremented with each
* attempt.
*/
private boolean executeHttpOperation(final int retryCount) throws AzureBlobFileSystemException {
AbfsHttpOperation httpOperation = null;
try {
// initialize the HTTP request and open the connection
httpOperation = new AbfsHttpOperation(url, method, requestHeaders);
// sign the HTTP request
client.getSharedKeyCredentials().signRequest(
httpOperation.getConnection(),
hasRequestBody ? bufferLength : 0);
if (hasRequestBody) {
// HttpUrlConnection requires
httpOperation.sendRequest(buffer, bufferOffset, bufferLength);
}
httpOperation.processResponse(buffer, bufferOffset, bufferLength);
} catch (IOException ex) {
if (logger.isDebugEnabled()) {
if (httpOperation != null) {
logger.debug("HttpRequestFailure: " + httpOperation.toString(), ex);
} else {
logger.debug("HttpRequestFailure: " + method + "," + url, ex);
}
}
if (!client.getRetryPolicy().shouldRetry(retryCount, -1)) {
throw new InvalidAbfsRestOperationException(ex);
}
return false;
}
if (logger.isDebugEnabled()) {
logger.debug("HttpRequest: " + httpOperation.toString());
}
if (client.getRetryPolicy().shouldRetry(retryCount, httpOperation.getStatusCode())) {
return false;
}
result = httpOperation;
return true;
}
}

View File

@ -0,0 +1,81 @@
/**
* 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.services;
import java.util.HashMap;
import java.util.Map;
import com.google.inject.AbstractModule;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.azurebfs.contracts.services.AbfsHttpClientFactory;
import org.apache.hadoop.fs.azurebfs.contracts.services.AbfsHttpService;
import org.apache.hadoop.fs.azurebfs.contracts.services.ConfigurationService;
import org.apache.hadoop.fs.azurebfs.contracts.services.TracingService;
/**
* This class is responsible to configure all the services used by Azure Blob File System.
*/
@InterfaceAudience.Private
@InterfaceStability.Evolving
class AbfsServiceInjectorImpl extends AbstractModule {
private final Configuration configuration;
private final Map<Class, Class> providers;
private final Map<Class, Object> instances;
AbfsServiceInjectorImpl(Configuration configuration) {
this.providers = new HashMap<>();
this.instances = new HashMap<>();
this.configuration = configuration;
this.instances.put(Configuration.class, this.configuration);
this.providers.put(ConfigurationService.class, ConfigurationServiceImpl.class);
this.providers.put(AbfsHttpService.class, AbfsHttpServiceImpl.class);
this.providers.put(AbfsHttpClientFactory.class, AbfsHttpClientFactoryImpl.class);
this.providers.put(TracingService.class, TracingServiceImpl.class);
}
@Override
protected void configure() {
for (Map.Entry<Class, Object> entrySet : this.instances.entrySet()) {
bind(entrySet.getKey()).toInstance(entrySet.getValue());
}
for (Map.Entry<Class, Class> entrySet : this.providers.entrySet()) {
bind(entrySet.getKey()).to(entrySet.getValue());
}
}
protected Configuration getConfiguration() {
return this.configuration;
}
protected Map<Class, Class> getProviders() {
return this.providers;
}
protected Map<Class, Object> getInstances() {
return this.instances;
}
}

View File

@ -0,0 +1,96 @@
/**
* 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.services;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Preconditions;
import com.google.inject.Guice;
import com.google.inject.Injector;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.azurebfs.contracts.exceptions.ServiceResolutionException;
import org.apache.hadoop.fs.azurebfs.contracts.services.AbfsServiceProvider;
import org.apache.hadoop.fs.azurebfs.contracts.services.InjectableService;
/**
* Dependency injected Azure Storage services provider.
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving
public final class AbfsServiceProviderImpl implements AbfsServiceProvider {
private static AbfsServiceProviderImpl abfsServiceProvider;
private final Injector abfsServiceInjector;
private AbfsServiceProviderImpl(final Configuration configuration) {
this.abfsServiceInjector = Guice.createInjector(new AbfsServiceInjectorImpl(Preconditions.checkNotNull(configuration, "configuration")));
}
@VisibleForTesting
private AbfsServiceProviderImpl(final Injector abfsServiceInjector) {
Preconditions.checkNotNull(abfsServiceInjector, "abfsServiceInjector");
this.abfsServiceInjector = abfsServiceInjector;
}
/**
* Create an instance or returns existing instance of service provider.
* This method must be marked as synchronized to ensure thread-safety.
* @param configuration hadoop configuration.
* @return AbfsServiceProvider the service provider instance.
*/
public static synchronized AbfsServiceProvider create(final Configuration configuration) {
if (abfsServiceProvider == null) {
abfsServiceProvider = new AbfsServiceProviderImpl(configuration);
}
return abfsServiceProvider;
}
/**
* Returns current instance of service provider.
* @return AbfsServiceProvider the service provider instance.
*/
public static AbfsServiceProvider instance() {
return abfsServiceProvider;
}
@VisibleForTesting
static synchronized AbfsServiceProvider create(Injector serviceInjector) {
abfsServiceProvider = new AbfsServiceProviderImpl(serviceInjector);
return abfsServiceProvider;
}
/**
* Returns an instance of resolved injectable service by class name.
* The injectable service must be configured first to be resolvable.
* @param clazz the injectable service which is expected to be returned.
* @param <T> The type of injectable service.
* @return T instance
* @throws ServiceResolutionException if the service is not resolvable.
*/
@Override
public <T extends InjectableService> T get(final Class<T> clazz) throws ServiceResolutionException {
try {
return this.abfsServiceInjector.getInstance(clazz);
} catch (Exception ex) {
throw new ServiceResolutionException(clazz.getSimpleName(), ex);
}
}
}

View File

@ -0,0 +1,58 @@
/**
* 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
* <p>
* http://www.apache.org/licenses/LICENSE-2.0
* <p>
* 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.services;
import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants;
import java.util.HashMap;
import java.util.Map;
/**
* The UrlQueryBuilder for Rest AbfsClient
*/
public class AbfsUriQueryBuilder {
private Map<String, String> parameters;
public AbfsUriQueryBuilder() {
this.parameters = new HashMap<>();
}
public void addQuery(final String name, final String value) {
if (value != null && !value.isEmpty()) {
this.parameters.put(name, value);
}
}
@Override
public String toString() {
StringBuilder sb = new StringBuilder();
boolean first = true;
for (Map.Entry<String, String> entry : parameters.entrySet()) {
if (first) {
sb.append(AbfsHttpConstants.QUESTION_MARK);
first = false;
} else {
sb.append(AbfsHttpConstants.AND_MARK);
}
sb.append(entry.getKey()).append(AbfsHttpConstants.EQUAL).append(entry.getValue());
}
return sb.toString();
}
}

View File

@ -0,0 +1,317 @@
/**
* 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.services;
import java.lang.reflect.Field;
import java.util.Map;
import com.google.common.annotations.VisibleForTesting;
import com.google.inject.Inject;
import com.google.inject.Singleton;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys;
import org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations;
import org.apache.hadoop.fs.azurebfs.contracts.annotations.ConfigurationValidationAnnotations.IntegerConfigurationValidatorAnnotation;
import org.apache.hadoop.fs.azurebfs.contracts.annotations.ConfigurationValidationAnnotations.LongConfigurationValidatorAnnotation;
import org.apache.hadoop.fs.azurebfs.contracts.annotations.ConfigurationValidationAnnotations.StringConfigurationValidatorAnnotation;
import org.apache.hadoop.fs.azurebfs.contracts.annotations.ConfigurationValidationAnnotations.Base64StringConfigurationValidatorAnnotation;
import org.apache.hadoop.fs.azurebfs.contracts.annotations.ConfigurationValidationAnnotations.BooleanConfigurationValidatorAnnotation;
import org.apache.hadoop.fs.azurebfs.contracts.exceptions.ConfigurationPropertyNotFoundException;
import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidConfigurationValueException;
import org.apache.hadoop.fs.azurebfs.contracts.services.ConfigurationService;
import org.apache.hadoop.fs.azurebfs.diagnostics.Base64StringConfigurationBasicValidator;
import org.apache.hadoop.fs.azurebfs.diagnostics.BooleanConfigurationBasicValidator;
import org.apache.hadoop.fs.azurebfs.diagnostics.IntegerConfigurationBasicValidator;
import org.apache.hadoop.fs.azurebfs.diagnostics.LongConfigurationBasicValidator;
import org.apache.hadoop.fs.azurebfs.diagnostics.StringConfigurationBasicValidator;
@Singleton
@InterfaceAudience.Private
@InterfaceStability.Evolving
class ConfigurationServiceImpl implements ConfigurationService {
private final Configuration configuration;
private final boolean isSecure;
@IntegerConfigurationValidatorAnnotation(ConfigurationKey = ConfigurationKeys.AZURE_WRITE_BUFFER_SIZE,
MinValue = FileSystemConfigurations.MIN_BUFFER_SIZE,
MaxValue = FileSystemConfigurations.MAX_BUFFER_SIZE,
DefaultValue = FileSystemConfigurations.DEFAULT_WRITE_BUFFER_SIZE)
private int writeBufferSize;
@IntegerConfigurationValidatorAnnotation(ConfigurationKey = ConfigurationKeys.AZURE_READ_BUFFER_SIZE,
MinValue = FileSystemConfigurations.MIN_BUFFER_SIZE,
MaxValue = FileSystemConfigurations.MAX_BUFFER_SIZE,
DefaultValue = FileSystemConfigurations.DEFAULT_READ_BUFFER_SIZE)
private int readBufferSize;
@IntegerConfigurationValidatorAnnotation(ConfigurationKey = ConfigurationKeys.AZURE_MIN_BACKOFF_INTERVAL,
DefaultValue = FileSystemConfigurations.DEFAULT_MIN_BACKOFF_INTERVAL)
private int minBackoffInterval;
@IntegerConfigurationValidatorAnnotation(ConfigurationKey = ConfigurationKeys.AZURE_MAX_BACKOFF_INTERVAL,
DefaultValue = FileSystemConfigurations.DEFAULT_MAX_BACKOFF_INTERVAL)
private int maxBackoffInterval;
@IntegerConfigurationValidatorAnnotation(ConfigurationKey = ConfigurationKeys.AZURE_BACKOFF_INTERVAL,
DefaultValue = FileSystemConfigurations.DEFAULT_BACKOFF_INTERVAL)
private int backoffInterval;
@IntegerConfigurationValidatorAnnotation(ConfigurationKey = ConfigurationKeys.AZURE_MAX_IO_RETRIES,
MinValue = 0,
DefaultValue = FileSystemConfigurations.DEFAULT_MAX_RETRY_ATTEMPTS)
private int maxIoRetries;
@LongConfigurationValidatorAnnotation(ConfigurationKey = ConfigurationKeys.AZURE_BLOCK_SIZE_PROPERTY_NAME,
MinValue = 0,
MaxValue = FileSystemConfigurations.MAX_AZURE_BLOCK_SIZE,
DefaultValue = FileSystemConfigurations.MAX_AZURE_BLOCK_SIZE)
private long azureBlockSize;
@StringConfigurationValidatorAnnotation(ConfigurationKey = ConfigurationKeys.AZURE_BLOCK_LOCATION_HOST_PROPERTY_NAME,
DefaultValue = FileSystemConfigurations.AZURE_BLOCK_LOCATION_HOST_DEFAULT)
private String azureBlockLocationHost;
@IntegerConfigurationValidatorAnnotation(ConfigurationKey = ConfigurationKeys.AZURE_CONCURRENT_CONNECTION_VALUE_OUT,
MinValue = 1,
DefaultValue = FileSystemConfigurations.MAX_CONCURRENT_WRITE_THREADS)
private int maxConcurrentWriteThreads;
@IntegerConfigurationValidatorAnnotation(ConfigurationKey = ConfigurationKeys.AZURE_CONCURRENT_CONNECTION_VALUE_IN,
MinValue = 1,
DefaultValue = FileSystemConfigurations.MAX_CONCURRENT_READ_THREADS)
private int maxConcurrentReadThreads;
@BooleanConfigurationValidatorAnnotation(ConfigurationKey = ConfigurationKeys.AZURE_TOLERATE_CONCURRENT_APPEND,
DefaultValue = FileSystemConfigurations.DEFAULT_READ_TOLERATE_CONCURRENT_APPEND)
private boolean tolerateOobAppends;
@StringConfigurationValidatorAnnotation(ConfigurationKey = ConfigurationKeys.FS_AZURE_ATOMIC_RENAME_KEY,
DefaultValue = FileSystemConfigurations.DEFAULT_FS_AZURE_ATOMIC_RENAME_DIRECTORIES)
private String azureAtomicDirs;
@BooleanConfigurationValidatorAnnotation(ConfigurationKey = ConfigurationKeys.AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION,
DefaultValue = FileSystemConfigurations.DEFAULT_AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION)
private boolean createRemoteFileSystemDuringInitialization;
@IntegerConfigurationValidatorAnnotation(ConfigurationKey = ConfigurationKeys.FS_AZURE_READ_AHEAD_QUEUE_DEPTH,
DefaultValue = FileSystemConfigurations.DEFAULT_READ_AHEAD_QUEUE_DEPTH)
private int readAheadQueueDepth;
private Map<String, String> storageAccountKeys;
@Inject
ConfigurationServiceImpl(final Configuration configuration) throws IllegalAccessException, InvalidConfigurationValueException {
this.configuration = configuration;
this.isSecure = this.configuration.getBoolean(ConfigurationKeys.FS_AZURE_SECURE_MODE, false);
validateStorageAccountKeys();
Field[] fields = this.getClass().getDeclaredFields();
for (Field field : fields) {
field.setAccessible(true);
if (field.isAnnotationPresent(IntegerConfigurationValidatorAnnotation.class)) {
field.set(this, validateInt(field));
} else if (field.isAnnotationPresent(LongConfigurationValidatorAnnotation.class)) {
field.set(this, validateLong(field));
} else if (field.isAnnotationPresent(StringConfigurationValidatorAnnotation.class)) {
field.set(this, validateString(field));
} else if (field.isAnnotationPresent(Base64StringConfigurationValidatorAnnotation.class)) {
field.set(this, validateBase64String(field));
} else if (field.isAnnotationPresent(BooleanConfigurationValidatorAnnotation.class)) {
field.set(this, validateBoolean(field));
}
}
}
@Override
public boolean isEmulator() {
return this.getConfiguration().getBoolean(ConfigurationKeys.FS_AZURE_EMULATOR_ENABLED, false);
}
@Override
public boolean isSecureMode() {
return this.isSecure;
}
@Override
public String getStorageAccountKey(final String accountName) throws ConfigurationPropertyNotFoundException {
String accountKey = this.storageAccountKeys.get(ConfigurationKeys.FS_AZURE_ACCOUNT_KEY_PROPERTY_NAME + accountName);
if (accountKey == null) {
throw new ConfigurationPropertyNotFoundException(accountName);
}
return accountKey;
}
@Override
public Configuration getConfiguration() {
return this.configuration;
}
@Override
public int getWriteBufferSize() {
return this.writeBufferSize;
}
@Override
public int getReadBufferSize() {
return this.readBufferSize;
}
@Override
public int getMinBackoffIntervalMilliseconds() {
return this.minBackoffInterval;
}
@Override
public int getMaxBackoffIntervalMilliseconds() {
return this.maxBackoffInterval;
}
@Override
public int getBackoffIntervalMilliseconds() {
return this.backoffInterval;
}
@Override
public int getMaxIoRetries() {
return this.maxIoRetries;
}
@Override
public long getAzureBlockSize() {
return this.azureBlockSize;
}
@Override
public String getAzureBlockLocationHost() {
return this.azureBlockLocationHost;
}
@Override
public int getMaxConcurrentWriteThreads() {
return this.maxConcurrentWriteThreads;
}
@Override
public int getMaxConcurrentReadThreads() {
return this.maxConcurrentReadThreads;
}
@Override
public boolean getTolerateOobAppends() {
return this.tolerateOobAppends;
}
@Override
public String getAzureAtomicRenameDirs() {
return this.azureAtomicDirs;
}
@Override
public boolean getCreateRemoteFileSystemDuringInitialization() {
return this.createRemoteFileSystemDuringInitialization;
}
@Override
public int getReadAheadQueueDepth() {
return this.readAheadQueueDepth;
}
void validateStorageAccountKeys() throws InvalidConfigurationValueException {
Base64StringConfigurationBasicValidator validator = new Base64StringConfigurationBasicValidator(
ConfigurationKeys.FS_AZURE_ACCOUNT_KEY_PROPERTY_NAME, "", true);
this.storageAccountKeys = this.configuration.getValByRegex(ConfigurationKeys.FS_AZURE_ACCOUNT_KEY_PROPERTY_NAME_REGX);
for (Map.Entry<String, String> account : this.storageAccountKeys.entrySet()) {
validator.validate(account.getValue());
}
}
int validateInt(Field field) throws IllegalAccessException, InvalidConfigurationValueException {
IntegerConfigurationValidatorAnnotation validator = field.getAnnotation(IntegerConfigurationValidatorAnnotation.class);
String value = this.configuration.get(validator.ConfigurationKey());
// validate
return new IntegerConfigurationBasicValidator(
validator.MinValue(),
validator.MaxValue(),
validator.DefaultValue(),
validator.ConfigurationKey(),
validator.ThrowIfInvalid()).validate(value);
}
long validateLong(Field field) throws IllegalAccessException, InvalidConfigurationValueException {
LongConfigurationValidatorAnnotation validator = field.getAnnotation(LongConfigurationValidatorAnnotation.class);
String value = this.configuration.get(validator.ConfigurationKey());
// validate
return new LongConfigurationBasicValidator(
validator.MinValue(),
validator.MaxValue(),
validator.DefaultValue(),
validator.ConfigurationKey(),
validator.ThrowIfInvalid()).validate(value);
}
String validateString(Field field) throws IllegalAccessException, InvalidConfigurationValueException {
StringConfigurationValidatorAnnotation validator = field.getAnnotation(StringConfigurationValidatorAnnotation.class);
String value = this.configuration.get(validator.ConfigurationKey());
// validate
return new StringConfigurationBasicValidator(
validator.ConfigurationKey(),
validator.DefaultValue(),
validator.ThrowIfInvalid()).validate(value);
}
String validateBase64String(Field field) throws IllegalAccessException, InvalidConfigurationValueException {
Base64StringConfigurationValidatorAnnotation validator = field.getAnnotation((Base64StringConfigurationValidatorAnnotation.class));
String value = this.configuration.get(validator.ConfigurationKey());
// validate
return new Base64StringConfigurationBasicValidator(
validator.ConfigurationKey(),
validator.DefaultValue(),
validator.ThrowIfInvalid()).validate(value);
}
boolean validateBoolean(Field field) throws IllegalAccessException, InvalidConfigurationValueException {
BooleanConfigurationValidatorAnnotation validator = field.getAnnotation(BooleanConfigurationValidatorAnnotation.class);
String value = this.configuration.get(validator.ConfigurationKey());
// validate
return new BooleanConfigurationBasicValidator(
validator.ConfigurationKey(),
validator.DefaultValue(),
validator.ThrowIfInvalid()).validate(value);
}
@VisibleForTesting
void setReadBufferSize(int bufferSize) {
this.readBufferSize = bufferSize;
}
@VisibleForTesting
void setWriteBufferSize(int bufferSize) {
this.writeBufferSize = bufferSize;
}
}

View File

@ -0,0 +1,141 @@
/**
* 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.services;
import java.util.Random;
import java.net.HttpURLConnection;
class ExponentialRetryPolicy {
/**
* Represents the default number of retry attempts.
*/
private static final int DEFAULT_CLIENT_RETRY_COUNT = 30;
/**
* Represents the default amount of time used when calculating a random delta in the exponential
* delay between retries.
*/
private static final int DEFAULT_CLIENT_BACKOFF = 1000 * 3;
/**
* Represents the default maximum amount of time used when calculating the exponential
* delay between retries.
*/
private static final int DEFAULT_MAX_BACKOFF = 1000 * 30;
/**
*Represents the default minimum amount of time used when calculating the exponential
* delay between retries.
*/
private static final int DEFAULT_MIN_BACKOFF = 1000 * 3;
/**
* The minimum random ratio used for delay interval calculation.
*/
private static final double MIN_RANDOM_RATIO = 0.8;
/**
* The maximum random ratio used for delay interval calculation.
*/
private static final double MAX_RANDOM_RATIO = 1.2;
/**
* Holds the random number generator used to calculate randomized backoff intervals
*/
private final Random randRef = new Random();
/**
* The value that will be used to calculate a random delta in the exponential delay interval
*/
private final int deltaBackoff;
/**
* The maximum backoff time.
*/
private final int maxBackoff;
/**
* The minimum backoff time.
*/
private final int minBackoff;
/**
* The maximum number of retry attempts.
*/
private final int retryCount;
/**
* Initializes a new instance of the {@link ExponentialRetryPolicy} class.
*/
ExponentialRetryPolicy() {
this(DEFAULT_CLIENT_RETRY_COUNT, DEFAULT_MIN_BACKOFF, DEFAULT_MAX_BACKOFF, DEFAULT_CLIENT_BACKOFF);
}
/**
* Initializes a new instance of the {@link ExponentialRetryPolicy} class.
*
* @param retryCount The maximum number of retry attempts.
* @param minBackoff The minimum backoff time.
* @param maxBackoff The maximum backoff time.
* @param deltaBackoff The value that will be used to calculate a random delta in the exponential delay
* between retries.
*/
ExponentialRetryPolicy(final int retryCount, final int minBackoff, final int maxBackoff, final int deltaBackoff) {
this.retryCount = retryCount;
this.minBackoff = minBackoff;
this.maxBackoff = maxBackoff;
this.deltaBackoff = deltaBackoff;
}
/**
* Returns if a request should be retried based on the retry count, current response,
* and the current strategy.
*
* @param retryCount The current retry attempt count.
* @param statusCode The status code of the response, or -1 for socket error.
* @return true if the request should be retried; false otherwise.
*/
public boolean shouldRetry(final int retryCount, final int statusCode) {
return retryCount < this.retryCount
&& (statusCode == -1
|| statusCode == HttpURLConnection.HTTP_CLIENT_TIMEOUT
|| (statusCode >= HttpURLConnection.HTTP_INTERNAL_ERROR
&& statusCode != HttpURLConnection.HTTP_NOT_IMPLEMENTED
&& statusCode != HttpURLConnection.HTTP_VERSION));
}
/**
* Returns backoff interval between 80% and 120% of the desired backoff,
* multiply by 2^n-1 for exponential.
*
* @param retryCount The current retry attempt count.
* @return backoff Interval time
*/
public long getRetryInterval(final int retryCount) {
final long boundedRandDelta = (int) (this.deltaBackoff * MIN_RANDOM_RATIO)
+ this.randRef.nextInt((int) (this.deltaBackoff * MAX_RANDOM_RATIO)
- (int) (this.deltaBackoff * MIN_RANDOM_RATIO));
final double incrementDelta = (Math.pow(2, retryCount - 1)) * boundedRandDelta;
final long retryInterval = (int) Math.round(Math.min(this.minBackoff + incrementDelta, maxBackoff));
return retryInterval;
}
}

View File

@ -0,0 +1,74 @@
/**
* 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.services;
import java.io.IOException;
import com.google.common.base.Preconditions;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystem;
import org.apache.htrace.core.HTraceConfiguration;
import org.apache.htrace.core.Span;
import org.apache.htrace.core.SpanReceiver;
import org.apache.htrace.fasterxml.jackson.core.JsonProcessingException;
import org.apache.htrace.fasterxml.jackson.databind.ObjectMapper;
import org.apache.htrace.fasterxml.jackson.databind.ObjectWriter;
import org.apache.htrace.fasterxml.jackson.databind.SerializationFeature;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* LoggerSpanReceiver is a layer between HTrace and log4j only used for {@link org.apache.hadoop.fs.azurebfs.contracts.services.TracingService}
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving
public class LoggerSpanReceiver extends SpanReceiver {
private static final ObjectWriter JSON_WRITER =
new ObjectMapper()
.configure(SerializationFeature.INDENT_OUTPUT, true)
.configure(SerializationFeature.WRITE_BIGDECIMAL_AS_PLAIN, true)
.configure(SerializationFeature.WRITE_EMPTY_JSON_ARRAYS, false)
.configure(SerializationFeature.USE_EQUALITY_FOR_OBJECT_ID, false)
.writer();
public LoggerSpanReceiver(HTraceConfiguration hTraceConfiguration) {
Preconditions.checkNotNull(hTraceConfiguration, "hTraceConfiguration");
}
@Override
public void receiveSpan(final Span span) {
String jsonValue;
Logger logger = LoggerFactory.getLogger(AzureBlobFileSystem.class);
try {
jsonValue = JSON_WRITER.writeValueAsString(span);
logger.trace(jsonValue);
} catch (JsonProcessingException e) {
logger.error("Json processing error: " + e.getMessage());
}
}
@Override
public void close() throws IOException {
// No-Op
}
}

View File

@ -0,0 +1,139 @@
/**
* 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.services;
import org.apache.hadoop.fs.azurebfs.contracts.services.ReadBufferStatus;
import java.util.concurrent.CountDownLatch;
class ReadBuffer {
private AbfsInputStream stream;
private long offset; // offset within the file for the buffer
private int length; // actual length, set after the buffer is filles
private int requestedLength; // requested length of the read
private byte[] buffer; // the buffer itself
private int bufferindex = -1; // index in the buffers array in Buffer manager
private ReadBufferStatus status; // status of the buffer
private CountDownLatch latch = null; // signaled when the buffer is done reading, so any client
// waiting on this buffer gets unblocked
// fields to help with eviction logic
private long timeStamp = 0; // tick at which buffer became available to read
private boolean isFirstByteConsumed = false;
private boolean isLastByteConsumed = false;
private boolean isAnyByteConsumed = false;
public AbfsInputStream getStream() {
return stream;
}
public void setStream(AbfsInputStream stream) {
this.stream = stream;
}
public long getOffset() {
return offset;
}
public void setOffset(long offset) {
this.offset = offset;
}
public int getLength() {
return length;
}
public void setLength(int length) {
this.length = length;
}
public int getRequestedLength() {
return requestedLength;
}
public void setRequestedLength(int requestedLength) {
this.requestedLength = requestedLength;
}
public byte[] getBuffer() {
return buffer;
}
public void setBuffer(byte[] buffer) {
this.buffer = buffer;
}
public int getBufferindex() {
return bufferindex;
}
public void setBufferindex(int bufferindex) {
this.bufferindex = bufferindex;
}
public ReadBufferStatus getStatus() {
return status;
}
public void setStatus(ReadBufferStatus status) {
this.status = status;
}
public CountDownLatch getLatch() {
return latch;
}
public void setLatch(CountDownLatch latch) {
this.latch = latch;
}
public long getTimeStamp() {
return timeStamp;
}
public void setTimeStamp(long timeStamp) {
this.timeStamp = timeStamp;
}
public boolean isFirstByteConsumed() {
return isFirstByteConsumed;
}
public void setFirstByteConsumed(boolean isFirstByteConsumed) {
this.isFirstByteConsumed = isFirstByteConsumed;
}
public boolean isLastByteConsumed() {
return isLastByteConsumed;
}
public void setLastByteConsumed(boolean isLastByteConsumed) {
this.isLastByteConsumed = isLastByteConsumed;
}
public boolean isAnyByteConsumed() {
return isAnyByteConsumed;
}
public void setAnyByteConsumed(boolean isAnyByteConsumed) {
this.isAnyByteConsumed = isAnyByteConsumed;
}
}

View File

@ -0,0 +1,391 @@
/**
* 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
* <p>
* http://www.apache.org/licenses/LICENSE-2.0
* <p>
* 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.services;
import org.apache.hadoop.fs.azurebfs.contracts.services.ReadBufferStatus;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.util.Collection;
import java.util.LinkedList;
import java.util.Queue;
import java.util.Stack;
import java.util.concurrent.CountDownLatch;
/**
* The Read Buffer Manager for Rest AbfsClient
*/
final class ReadBufferManager {
private static final Logger LOGGER = LoggerFactory.getLogger(ReadBufferManager.class);
private static final int NUM_BUFFERS = 16;
private static final int BLOCK_SIZE = 4 * 1024 * 1024;
private static final int NUM_THREADS = 8;
private static final int THRESHOLD_AGE_MILLISECONDS = 3000; // have to see if 3 seconds is a good threshold
private Thread[] threads = new Thread[NUM_THREADS];
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 Queue<ReadBuffer> readAheadQueue = new LinkedList<ReadBuffer>(); // 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> completedReadList = new LinkedList<ReadBuffer>(); // buffers available for reading
private static final ReadBufferManager BUFFER_MANAGER; // singleton, initialized in static initialization block
static {
BUFFER_MANAGER = new ReadBufferManager();
BUFFER_MANAGER.init();
}
static ReadBufferManager getBufferManager() {
return BUFFER_MANAGER;
}
private void init() {
buffers = new byte[NUM_BUFFERS][];
for (int i = 0; i < NUM_BUFFERS; i++) {
buffers[i] = new byte[BLOCK_SIZE]; // same buffers are reused. The byte array never goes back to GC
freeList.add(i);
}
for (int i = 0; i < NUM_THREADS; i++) {
Thread t = new Thread(new ReadBufferWorker(i));
t.setDaemon(true);
threads[i] = t;
t.setName("ABFS-prefetch-" + i);
t.start();
}
ReadBufferWorker.UNLEASH_WORKERS.countDown();
}
// hide instance constructor
private ReadBufferManager() {
}
/*
*
* AbfsInputStream-facing methods
*
*/
/**
* {@link AbfsInputStream} calls this method to queue read-aheads
*
* @param stream The {@link AbfsInputStream} for which to do the read-ahead
* @param requestedOffset The offset in the file which shoukd be read
* @param requestedLength The length to read
*/
void queueReadAhead(final AbfsInputStream stream, final long requestedOffset, final int requestedLength) {
if (LOGGER.isTraceEnabled()) {
LOGGER.trace("Start Queueing readAhead for " + stream.getPath() + " offset " + requestedOffset
+ " length " + requestedLength);
}
ReadBuffer buffer;
synchronized (this) {
if (isAlreadyQueued(stream, requestedOffset)) {
return; // already queued, do not queue again
}
if (freeList.size() == 0 && !tryEvict()) {
return; // no buffers available, cannot queue anything
}
buffer = new ReadBuffer();
buffer.setStream(stream);
buffer.setOffset(requestedOffset);
buffer.setLength(0);
buffer.setRequestedLength(requestedLength);
buffer.setStatus(ReadBufferStatus.NOT_AVAILABLE);
buffer.setLatch(new CountDownLatch(1));
Integer bufferIndex = freeList.pop(); // will return a value, since we have checked size > 0 already
buffer.setBuffer(buffers[bufferIndex]);
buffer.setBufferindex(bufferIndex);
readAheadQueue.add(buffer);
notifyAll();
}
if (LOGGER.isTraceEnabled()) {
LOGGER.trace("Done q-ing readAhead for file " + stream.getPath() + " offset " + requestedOffset
+ " buffer idx " + buffer.getBufferindex());
}
}
/**
* {@link AbfsInputStream} calls this method read any bytes already available in a buffer (thereby saving a
* remote read). This returns the bytes if the data already exists in buffer. If there is a buffer that is reading
* the requested offset, then this method blocks until that read completes. If the data is queued in a read-ahead
* but not picked up by a worker thread yet, then it cancels that read-ahead and reports cache miss. This is because
* depending on worker thread availability, the read-ahead may take a while - the calling thread can do it's own
* read to get the data faster (copmared to the read waiting in queue for an indeterminate amount of time).
*
* @param stream the file to read bytes for
* @param position the offset in the file to do a read for
* @param length the length to read
* @param buffer the buffer to read data into. Note that the buffer will be written into from offset 0.
* @return the number of bytes read
*/
int getBlock(final AbfsInputStream stream, final long position, final int length, final byte[] buffer) {
// not synchronized, so have to be careful with locking
if (LOGGER.isTraceEnabled()) {
LOGGER.trace("getBlock for file " + stream.getPath() + " position " + position + " thread " + Thread.currentThread().getName());
}
waitForProcess(stream, position);
int bytesRead = 0;
synchronized (this) {
bytesRead = getBlockFromCompletedQueue(stream, position, length, buffer);
}
if (bytesRead > 0) {
if (LOGGER.isTraceEnabled()) {
LOGGER.trace("Done read from Cache for " + stream.getPath() + " position " + position + " length " + bytesRead);
}
return bytesRead;
}
// otherwise, just say we got nothing - calling thread can do it's own read
return 0;
}
/*
*
* Internal methods
*
*/
private void waitForProcess(final AbfsInputStream stream, final long position) {
ReadBuffer readBuf;
synchronized (this) {
clearFromReadAheadQueue(stream, position);
readBuf = getFromList(inProgressList, stream, position);
}
if (readBuf != null) { // if in in-progress queue, then block for it
try {
if (LOGGER.isTraceEnabled()) {
LOGGER.trace("got a relevant read buffer for file " + stream.getPath() + " offset " + readBuf.getOffset()
+ " buffer idx " + readBuf.getBufferindex());
}
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
// is done processing it (in doneReading). There, the latch is set after removing the buffer from
// inProgressList. So this latch is safe to be outside the synchronized block.
// Putting it in synchronized would result in a deadlock, since this thread would be holding the lock
// while waiting, so no one will be able to change any state. If this becomes more complex in the future,
// then the latch cane be removed and replaced with wait/notify whenever inProgressList is touched.
} catch (InterruptedException ex) {
Thread.currentThread().interrupt();
}
if (LOGGER.isTraceEnabled()) {
LOGGER.trace("latch done for file " + stream.getPath() + " buffer idx " + readBuf.getBufferindex()
+ " length " + readBuf.getLength());
}
}
}
/**
* If any buffer in the completedlist can be reclaimed then reclaim it and return the buffer to free list.
* The objective is to find just one buffer - there is no advantage to evicting more than one.
*
* @return whether the eviction succeeeded - i.e., were we able to free up one buffer
*/
private synchronized boolean tryEvict() {
ReadBuffer nodeToEvict = null;
if (completedReadList.size() <= 0) {
return false; // there are no evict-able buffers
}
// first, try buffers where all bytes have been consumed (approximated as first and last bytes consumed)
for (ReadBuffer buf : completedReadList) {
if (buf.isFirstByteConsumed() && buf.isLastByteConsumed()) {
nodeToEvict = buf;
break;
}
}
if (nodeToEvict != null) {
return evict(nodeToEvict);
}
// next, try buffers where any bytes have been consumed (may be a bad idea? have to experiment and see)
for (ReadBuffer buf : completedReadList) {
if (buf.isAnyByteConsumed()) {
nodeToEvict = buf;
break;
}
}
if (nodeToEvict != null) {
return evict(nodeToEvict);
}
// next, try any old nodes that have not been consumed
long earliestBirthday = Long.MAX_VALUE;
for (ReadBuffer buf : completedReadList) {
if (buf.getTimeStamp() < earliestBirthday) {
nodeToEvict = buf;
earliestBirthday = buf.getTimeStamp();
}
}
if ((currentTimeMillis() - earliestBirthday > THRESHOLD_AGE_MILLISECONDS) && (nodeToEvict != null)) {
return evict(nodeToEvict);
}
// nothing can be evicted
return false;
}
private boolean evict(final ReadBuffer buf) {
freeList.push(buf.getBufferindex());
completedReadList.remove(buf);
if (LOGGER.isTraceEnabled()) {
LOGGER.trace("Evicting buffer idx " + buf.getBufferindex() + "; was used for file " + buf.getStream().getPath()
+ " offset " + buf.getOffset() + " length " + buf.getLength());
}
return true;
}
private boolean isAlreadyQueued(final AbfsInputStream stream, final long requestedOffset) {
// returns true if any part of the buffer is already queued
return (isInList(readAheadQueue, stream, requestedOffset)
|| isInList(inProgressList, stream, requestedOffset)
|| isInList(completedReadList, stream, requestedOffset));
}
private boolean isInList(final Collection<ReadBuffer> list, final AbfsInputStream stream, final long requestedOffset) {
return (getFromList(list, stream, requestedOffset) != null);
}
private ReadBuffer getFromList(final Collection<ReadBuffer> list, final AbfsInputStream stream, final long requestedOffset) {
for (ReadBuffer buffer : list) {
if (buffer.getStream() == stream) {
if (buffer.getStatus() == ReadBufferStatus.AVAILABLE
&& requestedOffset >= buffer.getOffset()
&& requestedOffset < buffer.getOffset() + buffer.getLength()) {
return buffer;
} else if (requestedOffset >= buffer.getOffset()
&& requestedOffset < buffer.getOffset() + buffer.getRequestedLength()) {
return buffer;
}
}
}
return null;
}
private void clearFromReadAheadQueue(final AbfsInputStream stream, final long requestedOffset) {
ReadBuffer buffer = getFromList(readAheadQueue, stream, requestedOffset);
if (buffer != null) {
readAheadQueue.remove(buffer);
notifyAll(); // lock is held in calling method
freeList.push(buffer.getBufferindex());
}
}
private int getBlockFromCompletedQueue(final AbfsInputStream stream, final long position, final int length,
final byte[] buffer) {
ReadBuffer buf = getFromList(completedReadList, stream, position);
if (buf == null || position >= buf.getOffset() + buf.getLength()) {
return 0;
}
int cursor = (int) (position - buf.getOffset());
int availableLengthInBuffer = buf.getLength() - cursor;
int lengthToCopy = Math.min(length, availableLengthInBuffer);
System.arraycopy(buf.getBuffer(), cursor, buffer, 0, lengthToCopy);
if (cursor == 0) {
buf.setFirstByteConsumed(true);
}
if (cursor + lengthToCopy == buf.getLength()) {
buf.setLastByteConsumed(true);
}
buf.setAnyByteConsumed(true);
return lengthToCopy;
}
/*
*
* ReadBufferWorker-thread-facing methods
*
*/
/**
* ReadBufferWorker thread calls this to get the next buffer that it should work on.
*
* @return {@link ReadBuffer}
* @throws InterruptedException if thread is interrupted
*/
ReadBuffer getNextBlockToRead() throws InterruptedException {
ReadBuffer buffer = null;
synchronized (this) {
//buffer = readAheadQueue.take(); // blocking method
while (readAheadQueue.size() == 0) {
wait();
}
buffer = readAheadQueue.remove();
notifyAll();
if (buffer == null) {
return null; // should never happen
}
buffer.setStatus(ReadBufferStatus.READING_IN_PROGRESS);
inProgressList.add(buffer);
}
if (LOGGER.isTraceEnabled()) {
LOGGER.trace("ReadBufferWorker picked file " + buffer.getStream().getPath() + " for offset " + buffer.getOffset());
}
return buffer;
}
/**
* ReadBufferWorker thread calls this method to post completion
*
* @param buffer the buffer whose read was completed
* @param result the {@link ReadBufferStatus} after the read operation in the worker thread
* @param bytesActuallyRead the number of bytes that the worker thread was actually able to read
*/
void doneReading(final ReadBuffer buffer, final ReadBufferStatus result, final int bytesActuallyRead) {
if (LOGGER.isTraceEnabled()) {
LOGGER.trace("ReadBufferWorker completed file " + buffer.getStream().getPath() + " for offset " + buffer.getOffset()
+ " bytes " + bytesActuallyRead);
}
synchronized (this) {
inProgressList.remove(buffer);
if (result == ReadBufferStatus.AVAILABLE && bytesActuallyRead > 0) {
buffer.setStatus(ReadBufferStatus.AVAILABLE);
buffer.setTimeStamp(currentTimeMillis());
buffer.setLength(bytesActuallyRead);
completedReadList.add(buffer);
} else {
freeList.push(buffer.getBufferindex());
// buffer should go out of scope after the end of the calling method in ReadBufferWorker, and eligible for GC
}
}
//outside the synchronized, since anyone receiving a wake-up from the latch must see safe-published results
buffer.getLatch().countDown(); // wake up waiting threads (if any)
}
/**
* 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),
* making it unsuitable for measuring time intervals. nanotime is strictly monotonically increasing,
* so it is much more suitable to measuring intervals.
*
* @return current time in milliseconds
*/
private long currentTimeMillis() {
return System.nanoTime() / 1000 / 1000;
}
}

View File

@ -0,0 +1,72 @@
/**
* 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.services;
import org.apache.hadoop.fs.azurebfs.contracts.services.ReadBufferStatus;
import java.util.concurrent.CountDownLatch;
class ReadBufferWorker implements Runnable {
protected static final CountDownLatch UNLEASH_WORKERS = new CountDownLatch(1);
private int id;
ReadBufferWorker(final int id) {
this.id = id;
}
/**
* return the ID of ReadBufferWorker.
*/
public int getId() {
return this.id;
}
/**
* Waits until a buffer becomes available in ReadAheadQueue.
* Once a buffer becomes available, reads the file specified in it and then posts results back to buffer manager.
* Rinse and repeat. Forever.
*/
public void run() {
try {
UNLEASH_WORKERS.await();
} catch (InterruptedException ex) {
Thread.currentThread().interrupt();
}
ReadBufferManager bufferManager = ReadBufferManager.getBufferManager();
ReadBuffer buffer;
while (true) {
try {
buffer = bufferManager.getNextBlockToRead(); // blocks, until a buffer is available for this thread
} catch (InterruptedException ex) {
Thread.currentThread().interrupt();
return;
}
if (buffer != null) {
try {
// do the actual read, from the file.
int bytesRead = buffer.getStream().readRemote(buffer.getOffset(), buffer.getBuffer(), 0, buffer.getRequestedLength());
bufferManager.doneReading(buffer, ReadBufferStatus.AVAILABLE, bytesRead); // post result back to ReadBufferManager
} catch (Exception ex) {
bufferManager.doneReading(buffer, ReadBufferStatus.READ_FAILED, 0);
}
}
}
}
}

View File

@ -0,0 +1,507 @@
/**
* 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
* <p>
* http://www.apache.org/licenses/LICENSE-2.0
* <p>
* 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.services;
import javax.crypto.Mac;
import javax.crypto.spec.SecretKeySpec;
import java.io.UnsupportedEncodingException;
import java.net.HttpURLConnection;
import java.net.URLDecoder;
import java.text.DateFormat;
import java.text.SimpleDateFormat;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.Date;
import java.util.HashMap;
import java.util.List;
import java.util.Locale;
import java.util.Map;
import java.util.Map.Entry;
import java.util.TimeZone;
import java.util.regex.Matcher;
import java.util.regex.Pattern;
import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants;
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
* account.
*/
public class SharedKeyCredentials {
private static final int EXPECTED_BLOB_QUEUE_CANONICALIZED_STRING_LENGTH = 300;
private static final Pattern CRLF = Pattern.compile("\r\n", Pattern.LITERAL);
private static final String HMAC_SHA256 = "HmacSHA256";
private static final Base64 BASE_64 = new Base64();
/**
* Stores a reference to the RFC1123 date/time pattern.
*/
private static final String RFC1123_PATTERN = "EEE, dd MMM yyyy HH:mm:ss z";
private String accountName;
private byte[] accountKey;
private Mac hmacSha256;
public SharedKeyCredentials(final String accountName,
final String accountKey) {
if (accountName == null || accountName.isEmpty()) {
throw new IllegalArgumentException("Invalid account name.");
}
if (accountKey == null || accountKey.isEmpty()) {
throw new IllegalArgumentException("Invalid account key.");
}
this.accountName = accountName;
this.accountKey = BASE_64.decode(accountKey);
initializeMac();
}
public void signRequest(HttpURLConnection connection, final long contentLength) throws UnsupportedEncodingException {
connection.setRequestProperty(HttpHeaderConfigurations.X_MS_DATE, getGMTTime());
final String stringToSign = canonicalize(connection, accountName, contentLength);
final String computedBase64Signature = computeHmac256(stringToSign);
connection.setRequestProperty(HttpHeaderConfigurations.AUTHORIZATION,
String.format("%s %s:%s", "SharedKey", accountName, computedBase64Signature));
}
private String computeHmac256(final String stringToSign) {
byte[] utf8Bytes = null;
try {
utf8Bytes = stringToSign.getBytes(AbfsHttpConstants.UTF_8);
} catch (final UnsupportedEncodingException e) {
throw new IllegalArgumentException(e);
}
byte[] hmac;
synchronized (this) {
hmac = hmacSha256.doFinal(utf8Bytes);
}
return new String(BASE_64.encode(hmac), Charsets.UTF_8);
}
/**
* Add x-ms- prefixed headers in a fixed order.
*
* @param conn the HttpURLConnection for the operation
* @param canonicalizedString the canonicalized string to add the canonicalized headerst to.
*/
private static void addCanonicalizedHeaders(final HttpURLConnection conn, final StringBuilder canonicalizedString) {
// Look for header names that start with
// HeaderNames.PrefixForStorageHeader
// Then sort them in case-insensitive manner.
final Map<String, List<String>> headers = conn.getRequestProperties();
final ArrayList<String> httpStorageHeaderNameArray = new ArrayList<String>();
for (final String key : headers.keySet()) {
if (key.toLowerCase(Locale.ROOT).startsWith(AbfsHttpConstants.HTTP_HEADER_PREFIX)) {
httpStorageHeaderNameArray.add(key.toLowerCase(Locale.ROOT));
}
}
Collections.sort(httpStorageHeaderNameArray);
// Now go through each header's values in the sorted order and append
// them to the canonicalized string.
for (final String key : httpStorageHeaderNameArray) {
final StringBuilder canonicalizedElement = new StringBuilder(key);
String delimiter = ":";
final ArrayList<String> values = getHeaderValues(headers, key);
boolean appendCanonicalizedElement = false;
// Go through values, unfold them, and then append them to the
// canonicalized element string.
for (final String value : values) {
if (value != null) {
appendCanonicalizedElement = true;
}
// Unfolding is simply removal of CRLF.
final String unfoldedValue = CRLF.matcher(value)
.replaceAll(Matcher.quoteReplacement(""));
// Append it to the canonicalized element string.
canonicalizedElement.append(delimiter);
canonicalizedElement.append(unfoldedValue);
delimiter = ",";
}
// Now, add this canonicalized element to the canonicalized header
// string.
if (appendCanonicalizedElement) {
appendCanonicalizedElement(canonicalizedString, canonicalizedElement.toString());
}
}
}
/**
* Initialie the HmacSha256 associated with the account key.
*/
private void initializeMac() {
// Initializes the HMAC-SHA256 Mac and SecretKey.
try {
hmacSha256 = Mac.getInstance(HMAC_SHA256);
hmacSha256.init(new SecretKeySpec(accountKey, HMAC_SHA256));
} catch (final Exception e) {
throw new IllegalArgumentException(e);
}
}
/**
* Append a string to a string builder with a newline constant
*
* @param builder the StringBuilder object
* @param element the string to append.
*/
private static void appendCanonicalizedElement(final StringBuilder builder, final String element) {
builder.append("\n");
builder.append(element);
}
/**
* Constructs a canonicalized string from the request's headers that will be used to construct the signature string
* for signing a Blob or Queue service request under the Shared Key Full authentication scheme.
*
* @param address the request URI
* @param accountName the account name associated with the request
* @param method the verb to be used for the HTTP request.
* @param contentType the content type of the HTTP request.
* @param contentLength the length of the content written to the outputstream in bytes, -1 if unknown
* @param date the date/time specification for the HTTP request
* @param conn the HttpURLConnection for the operation.
* @return A canonicalized string.
*/
private static String canonicalizeHttpRequest(final java.net.URL address, final String accountName,
final String method, final String contentType, final long contentLength, final String date,
final HttpURLConnection conn) throws UnsupportedEncodingException {
// The first element should be the Method of the request.
// I.e. GET, POST, PUT, or HEAD.
final StringBuilder canonicalizedString = new StringBuilder(EXPECTED_BLOB_QUEUE_CANONICALIZED_STRING_LENGTH);
canonicalizedString.append(conn.getRequestMethod());
// The next elements are
// If any element is missing it may be empty.
appendCanonicalizedElement(canonicalizedString,
getHeaderValue(conn, HttpHeaderConfigurations.CONTENT_ENCODING, AbfsHttpConstants.EMPTY_STRING));
appendCanonicalizedElement(canonicalizedString,
getHeaderValue(conn, HttpHeaderConfigurations.CONTENT_LANGUAGE, AbfsHttpConstants.EMPTY_STRING));
appendCanonicalizedElement(canonicalizedString,
contentLength <= 0 ? "" : String.valueOf(contentLength));
appendCanonicalizedElement(canonicalizedString,
getHeaderValue(conn, HttpHeaderConfigurations.CONTENT_MD5, AbfsHttpConstants.EMPTY_STRING));
appendCanonicalizedElement(canonicalizedString, contentType != null ? contentType : AbfsHttpConstants.EMPTY_STRING);
final String dateString = getHeaderValue(conn, HttpHeaderConfigurations.X_MS_DATE, AbfsHttpConstants.EMPTY_STRING);
// If x-ms-date header exists, Date should be empty string
appendCanonicalizedElement(canonicalizedString, dateString.equals(AbfsHttpConstants.EMPTY_STRING) ? date
: "");
appendCanonicalizedElement(canonicalizedString,
getHeaderValue(conn, HttpHeaderConfigurations.IF_MODIFIED_SINCE, AbfsHttpConstants.EMPTY_STRING));
appendCanonicalizedElement(canonicalizedString,
getHeaderValue(conn, HttpHeaderConfigurations.IF_MATCH, AbfsHttpConstants.EMPTY_STRING));
appendCanonicalizedElement(canonicalizedString,
getHeaderValue(conn, HttpHeaderConfigurations.IF_NONE_MATCH, AbfsHttpConstants.EMPTY_STRING));
appendCanonicalizedElement(canonicalizedString,
getHeaderValue(conn, HttpHeaderConfigurations.IF_UNMODIFIED_SINCE, AbfsHttpConstants.EMPTY_STRING));
appendCanonicalizedElement(canonicalizedString,
getHeaderValue(conn, HttpHeaderConfigurations.RANGE, AbfsHttpConstants.EMPTY_STRING));
addCanonicalizedHeaders(conn, canonicalizedString);
appendCanonicalizedElement(canonicalizedString, getCanonicalizedResource(address, accountName));
return canonicalizedString.toString();
}
/**
* Gets the canonicalized resource string for a Blob or Queue service request under the Shared Key Lite
* authentication scheme.
*
* @param address the resource URI.
* @param accountName the account name for the request.
* @return the canonicalized resource string.
*/
private static String getCanonicalizedResource(final java.net.URL address, final String accountName) throws UnsupportedEncodingException {
// Resource path
final StringBuilder resourcepath = new StringBuilder(AbfsHttpConstants.FORWARD_SLASH);
resourcepath.append(accountName);
// Note that AbsolutePath starts with a '/'.
resourcepath.append(address.getPath());
final StringBuilder canonicalizedResource = new StringBuilder(resourcepath.toString());
// query parameters
if (address.getQuery() == null || !address.getQuery().contains(AbfsHttpConstants.EQUAL)) {
//no query params.
return canonicalizedResource.toString();
}
final Map<String, String[]> queryVariables = parseQueryString(address.getQuery());
final Map<String, String> lowercasedKeyNameValue = new HashMap<String, String>();
for (final Entry<String, String[]> entry : queryVariables.entrySet()) {
// sort the value and organize it as comma separated values
final List<String> sortedValues = Arrays.asList(entry.getValue());
Collections.sort(sortedValues);
final StringBuilder stringValue = new StringBuilder();
for (final String value : sortedValues) {
if (stringValue.length() > 0) {
stringValue.append(AbfsHttpConstants.COMMA);
}
stringValue.append(value);
}
// key turns out to be null for ?a&b&c&d
lowercasedKeyNameValue.put((entry.getKey()) == null ? null
: entry.getKey().toLowerCase(Locale.ROOT), stringValue.toString());
}
final ArrayList<String> sortedKeys = new ArrayList<String>(lowercasedKeyNameValue.keySet());
Collections.sort(sortedKeys);
for (final String key : sortedKeys) {
final StringBuilder queryParamString = new StringBuilder();
queryParamString.append(key);
queryParamString.append(":");
queryParamString.append(lowercasedKeyNameValue.get(key));
appendCanonicalizedElement(canonicalizedResource, queryParamString.toString());
}
return canonicalizedResource.toString();
}
/**
* 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 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
* if the header is not found.
*/
private static ArrayList<String> getHeaderValues(final Map<String, List<String>> headers, final String headerName) {
final ArrayList<String> arrayOfValues = new ArrayList<String>();
List<String> values = null;
for (final Entry<String, List<String>> entry : headers.entrySet()) {
if (entry.getKey().toLowerCase(Locale.ROOT).equals(headerName)) {
values = entry.getValue();
break;
}
}
if (values != null) {
for (final String value : values) {
// canonicalization formula requires the string to be left
// trimmed.
arrayOfValues.add(trimStart(value));
}
}
return arrayOfValues;
}
/**
* Parses a query string into a one to many hashmap.
*
* @param parseString the string to parse
* @return a HashMap<String, String[]> of the key values.
*/
private static HashMap<String, String[]> parseQueryString(String parseString) throws UnsupportedEncodingException {
final HashMap<String, String[]> retVals = new HashMap<String, String[]>();
if (parseString == null || parseString.isEmpty()) {
return retVals;
}
// 1. Remove ? if present
final int queryDex = parseString.indexOf(AbfsHttpConstants.QUESTION_MARK);
if (queryDex >= 0 && parseString.length() > 0) {
parseString = parseString.substring(queryDex + 1);
}
// 2. split name value pairs by splitting on the 'c&' character
final String[] valuePairs = parseString.contains(AbfsHttpConstants.AND_MARK)
? parseString.split(AbfsHttpConstants.AND_MARK)
: parseString.split(AbfsHttpConstants.SEMICOLON);
// 3. for each field value pair parse into appropriate map entries
for (int m = 0; m < valuePairs.length; m++) {
final int equalDex = valuePairs[m].indexOf(AbfsHttpConstants.EQUAL);
if (equalDex < 0 || equalDex == valuePairs[m].length() - 1) {
continue;
}
String key = valuePairs[m].substring(0, equalDex);
String value = valuePairs[m].substring(equalDex + 1);
key = safeDecode(key);
value = safeDecode(value);
// 3.1 add to map
String[] values = retVals.get(key);
if (values == null) {
values = new String[]{value};
if (!value.equals("")) {
retVals.put(key, values);
}
}
}
return retVals;
}
/**
* Performs safe decoding of the specified string, taking care to preserve each <code>+</code> character, rather
* than replacing it with a space character.
*
* @param stringToDecode A <code>String</code> that represents the string to decode.
* @return A <code>String</code> that represents the decoded string.
* <p>
* If a storage service error occurred.
*/
private static String safeDecode(final String stringToDecode) throws UnsupportedEncodingException {
if (stringToDecode == null) {
return null;
}
if (stringToDecode.length() == 0) {
return "";
}
if (stringToDecode.contains(AbfsHttpConstants.PLUS)) {
final StringBuilder outBuilder = new StringBuilder();
int startDex = 0;
for (int m = 0; m < stringToDecode.length(); m++) {
if (stringToDecode.charAt(m) == '+') {
if (m > startDex) {
outBuilder.append(URLDecoder.decode(stringToDecode.substring(startDex, m),
AbfsHttpConstants.UTF_8));
}
outBuilder.append(AbfsHttpConstants.PLUS);
startDex = m + 1;
}
}
if (startDex != stringToDecode.length()) {
outBuilder.append(URLDecoder.decode(stringToDecode.substring(startDex, stringToDecode.length()),
AbfsHttpConstants.UTF_8));
}
return outBuilder.toString();
} else {
return URLDecoder.decode(stringToDecode, AbfsHttpConstants.UTF_8);
}
}
private static String trimStart(final String value) {
int spaceDex = 0;
while (spaceDex < value.length() && value.charAt(spaceDex) == ' ') {
spaceDex++;
}
return value.substring(spaceDex);
}
private static String getHeaderValue(final HttpURLConnection conn, final String headerName, final String defaultValue) {
final String headerValue = conn.getRequestProperty(headerName);
return headerValue == null ? defaultValue : headerValue;
}
/**
* Constructs a canonicalized string for signing a request.
*
* @param conn the HttpURLConnection to canonicalize
* @param accountName the account name associated with the request
* @param contentLength the length of the content written to the outputstream in bytes,
* -1 if unknown
* @return a canonicalized string.
*/
private String canonicalize(final HttpURLConnection conn,
final String accountName,
final Long contentLength) throws UnsupportedEncodingException {
if (contentLength < -1) {
throw new IllegalArgumentException(
"The Content-Length header must be greater than or equal to -1.");
}
String contentType = getHeaderValue(conn, HttpHeaderConfigurations.CONTENT_TYPE, "");
return canonicalizeHttpRequest(conn.getURL(), accountName,
conn.getRequestMethod(), contentType, contentLength, null, conn);
}
/**
* Thread local for storing GMT date format.
*/
private static ThreadLocal<DateFormat> rfc1123GmtDateTimeFormatter
= new ThreadLocal<DateFormat>() {
@Override
protected DateFormat initialValue() {
final DateFormat formatter = new SimpleDateFormat(RFC1123_PATTERN, Locale.ROOT);
formatter.setTimeZone(GMT_ZONE);
return formatter;
}
};
public static final TimeZone GMT_ZONE = TimeZone.getTimeZone(AbfsHttpConstants.GMT_TIMEZONE);
/**
* Returns the current GMT date/time String using the RFC1123 pattern.
*
* @return A <code>String</code> that represents the current GMT date/time using the RFC1123 pattern.
*/
static String getGMTTime() {
return getGMTTime(new Date());
}
/**
* Returns the GTM date/time String for the specified value using the RFC1123 pattern.
*
* @param date
* A <code>Date</code> object that represents the date to convert to GMT date/time in the RFC1123
* pattern.
*
* @return A <code>String</code> that represents the GMT date/time for the specified value using the RFC1123
* pattern.
*/
static String getGMTTime(final Date date) {
return rfc1123GmtDateTimeFormatter.get().format(date);
}
}

View File

@ -0,0 +1,134 @@
/**
* 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.services;
import java.io.PrintWriter;
import java.io.StringWriter;
import java.util.Objects;
import com.google.common.base.Preconditions;
import com.google.inject.Inject;
import com.google.inject.Singleton;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException;
import org.apache.hadoop.fs.azurebfs.contracts.services.TracingService;
import org.apache.htrace.core.HTraceConfiguration;
import org.apache.htrace.core.Sampler;
import org.apache.htrace.core.SpanId;
import org.apache.htrace.core.TraceScope;
import org.apache.htrace.core.Tracer;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@Singleton
@InterfaceAudience.Private
@InterfaceStability.Evolving
final class TracingServiceImpl implements TracingService {
private static final Logger LOG = LoggerFactory.getLogger(TracingService.class);
private final Tracer tracer;
private final ThreadLocal<SpanId> currentScopeId;
@Inject
TracingServiceImpl(
final Configuration configuration) {
Preconditions.checkNotNull(configuration, "configuration");
this.currentScopeId = new ThreadLocal<>();
this.tracer = new Tracer.Builder(TracingService.class.getSimpleName()).
conf(new HTraceConfiguration() {
@Override
public String get(String key) {
if (Objects.equals(key, Tracer.SPAN_RECEIVER_CLASSES_KEY)) {
return LoggerSpanReceiver.class.getName();
}
return null;
}
@Override
public String get(String key, String defaultValue) {
String value = get(key);
if (value != null) {
return value;
}
return defaultValue;
}
}).
build();
this.tracer.addSampler(Sampler.ALWAYS);
}
@Override
public TraceScope traceBegin(String description) {
if (this.LOG.isTraceEnabled()) {
TraceScope traceScope = this.tracer.newScope(description);
this.currentScopeId.set(traceScope.getSpanId());
return traceScope;
}
return null;
}
@Override
public TraceScope traceBegin(String description, SpanId parentSpanId) {
if (this.LOG.isTraceEnabled()) {
TraceScope traceScope = this.tracer.newScope(description, parentSpanId);
this.currentScopeId.set(traceScope.getSpanId());
return traceScope;
}
return null;
}
@Override
public void traceException(TraceScope traceScope, AzureBlobFileSystemException azureBlobFileSystemException) {
if (this.LOG.isTraceEnabled()) {
Preconditions.checkNotNull(traceScope, "traceScope");
Preconditions.checkNotNull(azureBlobFileSystemException, "azureBlobFileSystemException");
StringWriter stringWriter = new StringWriter();
PrintWriter printWriter = new PrintWriter(stringWriter);
azureBlobFileSystemException.printStackTrace(printWriter);
printWriter.flush();
traceScope.addKVAnnotation("Exception", stringWriter.toString());
}
}
@Override
public SpanId getCurrentTraceScopeSpanId() {
return this.currentScopeId.get();
}
@Override
public void traceEnd(TraceScope traceScope) {
if (this.LOG.isTraceEnabled()) {
Preconditions.checkNotNull(traceScope, "traceScope");
SpanId[] parents = traceScope.getSpan().getParents();
this.currentScopeId.set(parents != null && parents.length > 0 ? parents[parents.length - 1] : null);
traceScope.close();
}
}
}

View File

@ -0,0 +1,22 @@
/*
* 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.
*/
@InterfaceAudience.Private
@InterfaceStability.Evolving
package org.apache.hadoop.fs.azurebfs.services;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;

View File

@ -0,0 +1,77 @@
/**
* 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.utils;
import java.util.regex.Pattern;
/**
* Utility class to help with Abfs url transformation to blob urls.
*/
public final class UriUtils {
private static final String ABFS_URI_REGEX = "[^.]+\\.dfs\\.(preprod\\.){0,1}core\\.windows\\.net";
private static final Pattern ABFS_URI_PATTERN = Pattern.compile(ABFS_URI_REGEX);
/**
* Checks whether a string includes abfs url.
* @param string the string to check.
* @return true if string has abfs url.
*/
public static boolean containsAbfsUrl(final String string) {
if (string == null || string.isEmpty()) {
return false;
}
return ABFS_URI_PATTERN.matcher(string).matches();
}
/**
* Extracts the raw account name from account name.
* @param accountName to extract the raw account name.
* @return extracted raw account name.
*/
public static String extractRawAccountFromAccountName(final String accountName) {
if (accountName == null || accountName.isEmpty()) {
return null;
}
if (!containsAbfsUrl(accountName)) {
return null;
}
String[] splitByDot = accountName.split("\\.");
if (splitByDot.length == 0) {
return null;
}
return splitByDot[0];
}
/**
* Generate unique test path for multiple user tests.
*
* @return root test path
*/
public static String generateUniqueTestPath() {
String testUniqueForkId = System.getProperty("test.unique.fork.id");
return testUniqueForkId == null ? "/test" : "/" + testUniqueForkId + "/test";
}
private UriUtils() {
}
}

View File

@ -0,0 +1,22 @@
/*
* 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.
*/
@InterfaceAudience.Private
@InterfaceStability.Evolving
package org.apache.hadoop.fs.azurebfs.utils;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;

View File

@ -0,0 +1,225 @@
/**
* 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.apache.hadoop.fs.azurebfs.services.AbfsServiceProviderImpl;
import org.junit.After;
import org.junit.Assert;
import org.junit.Before;
import org.mockito.internal.util.MockUtil;
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 org.apache.hadoop.fs.azurebfs.contracts.services.AbfsHttpClientFactory;
import org.apache.hadoop.fs.azurebfs.contracts.services.AbfsHttpService;
import org.apache.hadoop.fs.azurebfs.contracts.services.ConfigurationService;
import org.apache.hadoop.fs.azurebfs.services.MockAbfsHttpClientFactoryImpl;
import org.apache.hadoop.fs.azurebfs.services.MockAbfsServiceInjectorImpl;
import org.apache.hadoop.fs.azurebfs.services.MockServiceProviderImpl;
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 MockAbfsServiceInjectorImpl mockServiceInjector;
private final boolean isEmulator;
private NativeAzureFileSystem wasb;
private String abfsScheme;
private Configuration configuration;
private String fileSystemName;
private String accountName;
private String testUrl;
public DependencyInjectedTest(final boolean secure) {
this(secure ? FileSystemUriSchemes.ABFS_SECURE_SCHEME : FileSystemUriSchemes.ABFS_SCHEME);
}
public MockAbfsServiceInjectorImpl getMockServiceInjector() {
return this.mockServiceInjector;
}
protected DependencyInjectedTest() {
this(FileSystemUriSchemes.ABFS_SCHEME);
}
private DependencyInjectedTest(final String scheme) {
abfsScheme = scheme;
fileSystemName = 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.mockServiceInjector = new MockAbfsServiceInjectorImpl(configuration);
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 {
if (this.isEmulator) {
this.mockServiceInjector.replaceProvider(AbfsHttpClientFactory.class, MockAbfsHttpClientFactoryImpl.class);
}
MockServiceProviderImpl.create(this.mockServiceInjector);
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 AbfsHttpService abfsHttpService = AbfsServiceProviderImpl.instance().get(AbfsHttpService.class);
abfsHttpService.deleteFilesystem(fs);
if (!(new MockUtil().isMock(abfsHttpService))) {
AbfsRestOperationException ex = intercept(
AbfsRestOperationException.class,
new Callable<Void>() {
@Override
public Void call() throws Exception {
abfsHttpService.getFilesystemProperties(fs);
return null;
}
});
assertEquals(FILE_SYSTEM_NOT_FOUND.getStatusCode(), ex.getStatusCode());
}
}
public AzureBlobFileSystem getFileSystem() throws Exception {
final Configuration configuration = AbfsServiceProviderImpl.instance().get(ConfigurationService.class).getConfiguration();
final AzureBlobFileSystem fs = (AzureBlobFileSystem) FileSystem.get(configuration);
return fs;
}
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

@ -0,0 +1,78 @@
/**
* 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 java.util.Random;
import org.junit.Test;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.Path;
import static org.junit.Assert.assertEquals;
/**
* Test append operations.
*/
public class ITestAzureBlobFileSystemAppend extends DependencyInjectedTest {
private static final Path TEST_FILE_PATH = new Path("testfile");
private static final Path TEST_FOLDER_PATH = new Path("testFolder");
public ITestAzureBlobFileSystemAppend() {
super();
}
@Test(expected = FileNotFoundException.class)
public void testAppendDirShouldFail() throws Exception {
final AzureBlobFileSystem fs = this.getFileSystem();
final Path filePath = TEST_FILE_PATH;
fs.mkdirs(filePath);
fs.append(filePath, 0);
}
@Test
public void testAppendWithLength0() throws Exception {
final AzureBlobFileSystem fs = this.getFileSystem();
FSDataOutputStream stream = fs.create(TEST_FILE_PATH);
final byte[] b = new byte[1024];
new Random().nextBytes(b);
stream.write(b, 1000, 0);
assertEquals(0, stream.getPos());
}
@Test(expected = FileNotFoundException.class)
public void testAppendFileAfterDelete() throws Exception {
final AzureBlobFileSystem fs = this.getFileSystem();
final Path filePath = TEST_FILE_PATH;
fs.create(filePath);
fs.delete(filePath, false);
fs.append(filePath);
}
@Test(expected = FileNotFoundException.class)
public void testAppendDirectory() throws Exception {
final AzureBlobFileSystem fs = this.getFileSystem();
final Path folderPath = TEST_FOLDER_PATH;
fs.mkdirs(folderPath);
fs.append(folderPath);
}
}

View File

@ -0,0 +1,83 @@
/**
* 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 com.microsoft.azure.storage.CloudStorageAccount;
import com.microsoft.azure.storage.blob.CloudBlobClient;
import com.microsoft.azure.storage.blob.CloudBlobContainer;
import com.microsoft.azure.storage.blob.CloudBlockBlob;
import org.junit.Test;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.Path;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue;
/**
* Test AzureBlobFileSystem back compatibility with WASB.
*/
public class ITestAzureBlobFileSystemBackCompat extends DependencyInjectedTest {
public ITestAzureBlobFileSystemBackCompat() {
super();
}
@Test
public void testBlobBackCompat() throws Exception {
final AzureBlobFileSystem fs = this.getFileSystem();
String storageConnectionString = getBlobConnectionString();
CloudStorageAccount storageAccount = CloudStorageAccount.parse(storageConnectionString);
CloudBlobClient blobClient = storageAccount.createCloudBlobClient();
CloudBlobContainer container = blobClient.getContainerReference(this.getFileSystemName());
container.createIfNotExists();
CloudBlockBlob blockBlob = container.getBlockBlobReference("test/10/10/10");
blockBlob.uploadText("");
blockBlob = container.getBlockBlobReference("test/10/123/3/2/1/3");
blockBlob.uploadText("");
FileStatus[] fileStatuses = fs.listStatus(new Path("/test/10/"));
assertEquals(fileStatuses.length, 2);
assertEquals(fileStatuses[0].getPath().getName(), "10");
assertTrue(fileStatuses[0].isDirectory());
assertEquals(fileStatuses[0].getLen(), 0);
assertEquals(fileStatuses[1].getPath().getName(), "123");
assertTrue(fileStatuses[1].isDirectory());
assertEquals(fileStatuses[1].getLen(), 0);
}
private String getBlobConnectionString() {
String connectionString;
if (isEmulator()) {
connectionString = "DefaultEndpointsProtocol=http;BlobEndpoint=http://"
+ this.getHostName() + ":8880/" + this.getAccountName().split("\\.") [0]
+ ";AccountName=" + this.getAccountName().split("\\.")[0]
+ ";AccountKey=" + this.getAccountKey();
}
else {
connectionString = "DefaultEndpointsProtocol=http;BlobEndpoint=http://"
+ this.getAccountName().replaceFirst("\\.dfs\\.", ".blob.")
+ ";AccountName=" + this.getAccountName().split("\\.")[0]
+ ";AccountKey=" + this.getAccountKey();
}
return connectionString;
}
}

View File

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

View File

@ -0,0 +1,110 @@
/**
* 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 java.util.EnumSet;
import org.junit.Test;
import org.apache.hadoop.fs.CreateFlag;
import org.apache.hadoop.fs.FileAlreadyExistsException;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.permission.FsPermission;
import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertTrue;
/**
* Test create operation.
*/
public class ITestAzureBlobFileSystemCreate extends DependencyInjectedTest {
private static final Path TEST_FILE_PATH = new Path("testfile");
private static final Path TEST_FOLDER_PATH = new Path("testFolder");
private static final String TEST_CHILD_FILE = "childFile";
public ITestAzureBlobFileSystemCreate() {
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
public void testEnsureFileCreated() throws Exception {
final AzureBlobFileSystem fs = this.getFileSystem();
fs.create(TEST_FILE_PATH);
FileStatus fileStatus = fs.getFileStatus(TEST_FILE_PATH);
assertNotNull(fileStatus);
}
@Test
@SuppressWarnings("deprecation")
public void testCreateNonRecursive() throws Exception {
final AzureBlobFileSystem fs = this.getFileSystem();
Path testFile = new Path(TEST_FOLDER_PATH, TEST_CHILD_FILE);
try {
fs.createNonRecursive(testFile, true, 1024, (short) 1, 1024, null);
assertTrue("Should've thrown", false);
} catch (FileNotFoundException e) {
}
fs.mkdirs(TEST_FOLDER_PATH);
fs.createNonRecursive(testFile, true, 1024, (short) 1, 1024, null)
.close();
assertTrue(fs.exists(testFile));
}
@Test
@SuppressWarnings("deprecation")
public void testCreateNonRecursive1() throws Exception {
final AzureBlobFileSystem fs = this.getFileSystem();
Path testFile = new Path(TEST_FOLDER_PATH, TEST_CHILD_FILE);
try {
fs.createNonRecursive(testFile, FsPermission.getDefault(), EnumSet.of(CreateFlag.CREATE, CreateFlag.OVERWRITE), 1024, (short) 1, 1024, null);
assertTrue("Should've thrown", false);
} catch (FileNotFoundException e) {
}
fs.mkdirs(TEST_FOLDER_PATH);
fs.createNonRecursive(testFile, true, 1024, (short) 1, 1024, null)
.close();
assertTrue(fs.exists(testFile));
}
@Test
@SuppressWarnings("deprecation")
public void testCreateNonRecursive2() throws Exception {
final AzureBlobFileSystem fs = this.getFileSystem();
Path testFile = new Path(TEST_FOLDER_PATH, TEST_CHILD_FILE);
try {
fs.createNonRecursive(testFile, FsPermission.getDefault(), false, 1024, (short) 1, 1024, null);
assertTrue("Should've thrown", false);
} catch (FileNotFoundException e) {
}
fs.mkdirs(TEST_FOLDER_PATH);
fs.createNonRecursive(testFile, true, 1024, (short) 1, 1024, null)
.close();
assertTrue(fs.exists(testFile));
}
}

View File

@ -0,0 +1,119 @@
/**
* 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 java.util.ArrayList;
import java.util.List;
import java.util.concurrent.Callable;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.Future;
import org.junit.Test;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.Path;
import static org.junit.Assert.assertEquals;
/**
* Test delete operation.
*/
public class ITestAzureBlobFileSystemDelete extends DependencyInjectedTest {
public ITestAzureBlobFileSystemDelete() {
super();
}
@Test
public void testDeleteRoot() throws Exception {
final AzureBlobFileSystem fs = this.getFileSystem();
fs.mkdirs(new Path("/testFolder0"));
fs.mkdirs(new Path("/testFolder1"));
fs.mkdirs(new Path("/testFolder2"));
fs.create(new Path("/testFolder1/testfile"));
fs.create(new Path("/testFolder1/testfile2"));
fs.create(new Path("/testFolder1/testfile3"));
FileStatus[] ls = fs.listStatus(new Path("/"));
assertEquals(4, ls.length); // and user dir
fs.delete(new Path("/"), true);
ls = fs.listStatus(new Path("/"));
assertEquals(0, ls.length);
}
@Test(expected = FileNotFoundException.class)
public void testOpenFileAfterDelete() throws Exception {
final AzureBlobFileSystem fs = this.getFileSystem();
fs.create(new Path("/testFile"));
fs.delete(new Path("/testFile"), false);
fs.open(new Path("/testFile"));
}
@Test(expected = FileNotFoundException.class)
public void testEnsureFileIsDeleted() throws Exception {
final AzureBlobFileSystem fs = this.getFileSystem();
fs.create(new Path("testfile"));
fs.delete(new Path("testfile"), false);
fs.getFileStatus(new Path("testfile"));
}
@Test(expected = FileNotFoundException.class)
public void testDeleteDirectory() throws Exception {
final AzureBlobFileSystem fs = this.getFileSystem();
fs.mkdirs(new Path("testfile"));
fs.mkdirs(new Path("testfile/test1"));
fs.mkdirs(new Path("testfile/test1/test2"));
fs.delete(new Path("testfile"), true);
fs.getFileStatus(new Path("testfile"));
}
@Test(expected = FileNotFoundException.class)
public void testDeleteFirstLevelDirectory() throws Exception {
final AzureBlobFileSystem fs = this.getFileSystem();
final List<Future> tasks = new ArrayList<>();
ExecutorService es = Executors.newFixedThreadPool(10);
for (int i = 0; i < 1000; i++) {
final Path fileName = new Path("/test/" + i);
Callable<Void> callable = new Callable<Void>() {
@Override
public Void call() throws Exception {
fs.create(fileName);
return null;
}
};
tasks.add(es.submit(callable));
}
for (Future<Void> task : tasks) {
task.get();
}
es.shutdownNow();
fs.delete(new Path("/test"), true);
fs.getFileStatus(new Path("/test"));
}
}

View File

@ -0,0 +1,150 @@
/**
* 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.util.Arrays;
import java.util.Random;
import org.apache.hadoop.fs.azurebfs.services.AbfsServiceProviderImpl;
import org.junit.Test;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.azurebfs.contracts.services.ConfigurationService;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertNotEquals;
import static org.junit.Assert.assertArrayEquals;
/**
* Test end to end between ABFS client and ABFS server.
*/
public class ITestAzureBlobFileSystemE2E extends DependencyInjectedTest {
private static final Path TEST_FILE = new Path("testfile");
private static final int TEST_BYTE = 100;
private static final int TEST_OFFSET = 100;
private static final int TEST_DEFAULT_BUFFER_SIZE = 4 * 1024 * 1024;
private static final int TEST_DEFAULT_READ_BUFFER_SIZE = 1023900;
public ITestAzureBlobFileSystemE2E() {
super();
Configuration configuration = this.getConfiguration();
configuration.set(ConfigurationKeys.FS_AZURE_READ_AHEAD_QUEUE_DEPTH, "0");
this.getMockServiceInjector().replaceInstance(Configuration.class, configuration);
}
@Test
public void testWriteOneByteToFile() throws Exception {
final AzureBlobFileSystem fs = this.getFileSystem();
FSDataOutputStream stream = fs.create(TEST_FILE);
stream.write(TEST_BYTE);
stream.close();
FileStatus fileStatus = fs.getFileStatus(TEST_FILE);
assertEquals(1, fileStatus.getLen());
}
@Test
public void testReadWriteBytesToFile() throws Exception {
final AzureBlobFileSystem fs = this.getFileSystem();
testWriteOneByteToFile();
FSDataInputStream inputStream = fs.open(TEST_FILE, TEST_DEFAULT_BUFFER_SIZE);
int i = inputStream.read();
inputStream.close();
assertEquals(TEST_BYTE, i);
}
@Test (expected = IOException.class)
public void testOOBWrites() throws Exception {
final AzureBlobFileSystem fs = this.getFileSystem();
int readBufferSize = AbfsServiceProviderImpl.instance().get(ConfigurationService.class).getReadBufferSize();
fs.create(TEST_FILE);
FSDataOutputStream writeStream = fs.create(TEST_FILE);
byte[] bytesToRead = new byte[readBufferSize];
final byte[] b = new byte[2 * readBufferSize];
new Random().nextBytes(b);
writeStream.write(b);
writeStream.flush();
writeStream.close();
FSDataInputStream readStream = fs.open(TEST_FILE);
readStream.read(bytesToRead, 0, readBufferSize);
writeStream = fs.create(TEST_FILE);
writeStream.write(b);
writeStream.flush();
writeStream.close();
readStream.read(bytesToRead, 0, readBufferSize);
readStream.close();
}
@Test
public void testWriteWithBufferOffset() throws Exception {
final AzureBlobFileSystem fs = this.getFileSystem();
final FSDataOutputStream stream = fs.create(TEST_FILE);
final byte[] b = new byte[1024 * 1000];
new Random().nextBytes(b);
stream.write(b, TEST_OFFSET, b.length - TEST_OFFSET);
stream.close();
final byte[] r = new byte[TEST_DEFAULT_READ_BUFFER_SIZE];
FSDataInputStream inputStream = fs.open(TEST_FILE, TEST_DEFAULT_BUFFER_SIZE);
int result = inputStream.read(r);
assertNotEquals(-1, result);
assertArrayEquals(r, Arrays.copyOfRange(b, TEST_OFFSET, b.length));
inputStream.close();
}
@Test
public void testReadWriteHeavyBytesToFileWithSmallerChunks() throws Exception {
final AzureBlobFileSystem fs = this.getFileSystem();
final FSDataOutputStream stream = fs.create(TEST_FILE);
final byte[] writeBuffer = new byte[5 * 1000 * 1024];
new Random().nextBytes(writeBuffer);
stream.write(writeBuffer);
stream.close();
final byte[] readBuffer = new byte[5 * 1000 * 1024];
FSDataInputStream inputStream = fs.open(TEST_FILE, TEST_DEFAULT_BUFFER_SIZE);
int offset = 0;
while (inputStream.read(readBuffer, offset, TEST_OFFSET) > 0) {
offset += TEST_OFFSET;
}
assertArrayEquals(readBuffer, writeBuffer);
inputStream.close();
}
}

View File

@ -0,0 +1,133 @@
/**
* 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.ArrayList;
import java.util.List;
import java.util.Random;
import java.util.concurrent.Callable;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.Future;
import org.junit.Assert;
import org.junit.Test;
import org.apache.hadoop.fs.FSDataInputStream;
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 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.
*/
public class ITestAzureBlobFileSystemE2EScale extends DependencyInjectedTest {
private static final int TEN = 10;
private static final int ONE_THOUSAND = 1000;
private static final int BASE_SIZE = 1024;
private static final int ONE_MB = 1024 * 1024;
private static final int DEFAULT_WRITE_TIMES = 100;
private static final Path TEST_FILE = new Path("testfile");
public ITestAzureBlobFileSystemE2EScale() {
super();
}
@Test
public void testWriteHeavyBytesToFile() throws Exception {
final AzureBlobFileSystem fs = this.getFileSystem();
final FSDataOutputStream stream = fs.create(TEST_FILE);
ExecutorService es = Executors.newFixedThreadPool(TEN);
int testWriteBufferSize = 2 * TEN * ONE_THOUSAND * BASE_SIZE;
final byte[] b = new byte[testWriteBufferSize];
new Random().nextBytes(b);
List<Future<Void>> tasks = new ArrayList<>();
for (int i = 0; i < DEFAULT_WRITE_TIMES; i++) {
Callable<Void> callable = new Callable<Void>() {
@Override
public Void call() throws Exception {
stream.write(b);
return null;
}
};
tasks.add(es.submit(callable));
}
for (Future<Void> task : tasks) {
task.get();
}
tasks.clear();
stream.close();
es.shutdownNow();
FileStatus fileStatus = fs.getFileStatus(TEST_FILE);
assertEquals(testWriteBufferSize * DEFAULT_WRITE_TIMES, 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
public void testReadWriteHeavyBytesToFileWithStatistics() throws Exception {
final AzureBlobFileSystem fs = this.getFileSystem();
final FSDataOutputStream stream = fs.create(TEST_FILE);
final FileSystem.Statistics abfsStatistics = fs.getFsStatistics();
abfsStatistics.reset();
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);
inputStream.read(r);
inputStream.close();
Assert.assertEquals(r.length, abfsStatistics.getBytesRead());
Assert.assertEquals(b.length, abfsStatistics.getBytesWritten());
}
}

View File

@ -0,0 +1,64 @@
/**
* 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.junit.Test;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.permission.FsAction;
import org.apache.hadoop.fs.permission.FsPermission;
import static org.junit.Assert.assertEquals;
/**
* Test FileStatus.
*/
public class ITestAzureBlobFileSystemFileStatus extends DependencyInjectedTest {
private static final Path TEST_FILE = new Path("testFile");
private static final Path TEST_FOLDER = new Path("testDir");
public ITestAzureBlobFileSystemFileStatus() {
super();
}
@Test
public void testEnsureStatusWorksForRoot() throws Exception {
final AzureBlobFileSystem fs = this.getFileSystem();
fs.getFileStatus(new Path("/"));
fs.listStatus(new Path("/"));
}
@Test
public void testFileStatusPermissionsAndOwnerAndGroup() throws Exception {
final AzureBlobFileSystem fs = this.getFileSystem();
fs.create(TEST_FILE);
fs.mkdirs(TEST_FOLDER);
FileStatus fileStatus = fs.getFileStatus(TEST_FILE);
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

@ -0,0 +1,204 @@
/**
* 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.ArrayList;
import java.util.List;
import java.util.Random;
import java.util.concurrent.Callable;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.Future;
import org.junit.Test;
import org.apache.hadoop.fs.FSDataInputStream;
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 static org.junit.Assert.assertArrayEquals;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertNotEquals;
/**
* Test flush operation.
*/
public class ITestAzureBlobFileSystemFlush extends DependencyInjectedTest {
private static final int BASE_SIZE = 1024;
private static final int ONE_THOUSAND = 1000;
private static final int TEST_BUFFER_SIZE = 5 * ONE_THOUSAND * BASE_SIZE;
private static final int ONE_MB = 1024 * 1024;
private static final int FLUSH_TIMES = 200;
private static final int THREAD_SLEEP_TIME = 6000;
private static final Path TEST_FILE_PATH = new Path("/testfile");
public ITestAzureBlobFileSystemFlush() {
super();
}
@Test
public void testAbfsOutputStreamAsyncFlushWithRetainUncommitedData() throws Exception {
final AzureBlobFileSystem fs = this.getFileSystem();
final FSDataOutputStream stream = fs.create(TEST_FILE_PATH);
final byte[] b = new byte[TEST_BUFFER_SIZE];
new Random().nextBytes(b);
for (int i = 0; i < 2; i++) {
stream.write(b);
for (int j = 0; j < FLUSH_TIMES; j++) {
stream.flush();
Thread.sleep(10);
}
}
stream.close();
final byte[] r = new byte[TEST_BUFFER_SIZE];
FSDataInputStream inputStream = fs.open(TEST_FILE_PATH, 4 * ONE_MB);
while (inputStream.available() != 0) {
int result = inputStream.read(r);
assertNotEquals(-1, result);
assertArrayEquals(r, b);
}
inputStream.close();
}
@Test
public void testAbfsOutputStreamSyncFlush() throws Exception {
final AzureBlobFileSystem fs = this.getFileSystem();
final FSDataOutputStream stream = fs.create(TEST_FILE_PATH);
final byte[] b = new byte[TEST_BUFFER_SIZE];
new Random().nextBytes(b);
stream.write(b);
for (int i = 0; i < FLUSH_TIMES; i++) {
stream.hsync();
stream.hflush();
Thread.sleep(10);
}
stream.close();
final byte[] r = new byte[TEST_BUFFER_SIZE];
FSDataInputStream inputStream = fs.open(TEST_FILE_PATH, 4 * ONE_MB);
int result = inputStream.read(r);
assertNotEquals(-1, result);
assertArrayEquals(r, b);
inputStream.close();
}
@Test
public void testWriteHeavyBytesToFileSyncFlush() throws Exception {
final AzureBlobFileSystem fs = this.getFileSystem();
final FSDataOutputStream stream = fs.create(TEST_FILE_PATH);
final FileSystem.Statistics abfsStatistics = fs.getFsStatistics();
abfsStatistics.reset();
ExecutorService es = Executors.newFixedThreadPool(10);
final byte[] b = new byte[TEST_BUFFER_SIZE];
new Random().nextBytes(b);
List<Future<Void>> tasks = new ArrayList<>();
for (int i = 0; i < FLUSH_TIMES; i++) {
Callable<Void> callable = new Callable<Void>() {
@Override
public Void call() throws Exception {
stream.write(b);
return null;
}
};
tasks.add(es.submit(callable));
}
boolean shouldStop = false;
while (!shouldStop) {
shouldStop = true;
for (Future<Void> task : tasks) {
if (!task.isDone()) {
stream.hsync();
shouldStop = false;
Thread.sleep(THREAD_SLEEP_TIME);
}
}
}
tasks.clear();
stream.close();
es.shutdownNow();
FileStatus fileStatus = fs.getFileStatus(TEST_FILE_PATH);
assertEquals((long) TEST_BUFFER_SIZE * FLUSH_TIMES, fileStatus.getLen());
assertEquals((long) TEST_BUFFER_SIZE * FLUSH_TIMES, abfsStatistics.getBytesWritten());
}
@Test
public void testWriteHeavyBytesToFileAsyncFlush() throws Exception {
final AzureBlobFileSystem fs = this.getFileSystem();
fs.create(TEST_FILE_PATH);
final FSDataOutputStream stream = fs.create(TEST_FILE_PATH);
ExecutorService es = Executors.newFixedThreadPool(10);
final byte[] b = new byte[TEST_BUFFER_SIZE];
new Random().nextBytes(b);
List<Future<Void>> tasks = new ArrayList<>();
for (int i = 0; i < FLUSH_TIMES; i++) {
Callable<Void> callable = new Callable<Void>() {
@Override
public Void call() throws Exception {
stream.write(b);
return null;
}
};
tasks.add(es.submit(callable));
}
boolean shouldStop = false;
while (!shouldStop) {
shouldStop = true;
for (Future<Void> task : tasks) {
if (!task.isDone()) {
stream.flush();
shouldStop = false;
}
}
}
Thread.sleep(THREAD_SLEEP_TIME);
tasks.clear();
stream.close();
es.shutdownNow();
FileStatus fileStatus = fs.getFileStatus(TEST_FILE_PATH);
assertEquals((long) TEST_BUFFER_SIZE * FLUSH_TIMES, fileStatus.getLen());
}
}

View File

@ -0,0 +1,50 @@
/**
* 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.azurebfs.constants.ConfigurationKeys;
/**
* Test filesystem initialization and creation.
*/
public class ITestAzureBlobFileSystemInitAndCreate extends DependencyInjectedTest {
public ITestAzureBlobFileSystemInitAndCreate() {
super();
this.getConfiguration().unset(ConfigurationKeys.AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION);
}
@Override
public void initialize() {
}
@Override
public void testCleanup() {
}
@Test (expected = FileNotFoundException.class)
public void ensureFilesystemWillNotBeCreatedIfCreationConfigIsNotSet() throws Exception {
super.initialize();
this.getFileSystem();
}
}

View File

@ -0,0 +1,132 @@
/**
* 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 java.util.ArrayList;
import java.util.List;
import java.util.concurrent.Callable;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.Future;
import org.junit.Assert;
import org.junit.Test;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.Path;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.assertFalse;
/**
* Test listStatus operation.
*/
public class ITestAzureBlobFileSystemListStatus extends DependencyInjectedTest {
private static final int TEST_FILES_NUMBER = 6000;
public ITestAzureBlobFileSystemListStatus() {
super();
}
@Test
public void testListPath() throws Exception {
final AzureBlobFileSystem fs = this.getFileSystem();
final List<Future> tasks = new ArrayList<>();
ExecutorService es = Executors.newFixedThreadPool(10);
for (int i = 0; i < TEST_FILES_NUMBER; i++) {
final Path fileName = new Path("/test" + i);
Callable<Void> callable = new Callable<Void>() {
@Override
public Void call() throws Exception {
fs.create(fileName);
return null;
}
};
tasks.add(es.submit(callable));
}
for (Future<Void> task : tasks) {
task.get();
}
es.shutdownNow();
FileStatus[] files = fs.listStatus(new Path("/"));
Assert.assertEquals(files.length, TEST_FILES_NUMBER + 1 /* user directory */);
}
@Test
public void testListFileVsListDir() throws Exception {
final AzureBlobFileSystem fs = this.getFileSystem();
fs.create(new Path("/testFile"));
FileStatus[] testFiles = fs.listStatus(new Path("/testFile"));
Assert.assertEquals(testFiles.length, 1);
Assert.assertFalse(testFiles[0].isDirectory());
}
@Test
public void testListFileVsListDir2() throws Exception {
final AzureBlobFileSystem fs = this.getFileSystem();
fs.mkdirs(new Path("/testFolder"));
fs.mkdirs(new Path("/testFolder/testFolder2"));
fs.mkdirs(new Path("/testFolder/testFolder2/testFolder3"));
fs.create(new Path("/testFolder/testFolder2/testFolder3/testFile"));
FileStatus[] testFiles = fs.listStatus(new Path("/testFolder/testFolder2/testFolder3/testFile"));
Assert.assertEquals(testFiles.length, 1);
Assert.assertEquals(testFiles[0].getPath(), new Path(this.getTestUrl(),
"/testFolder/testFolder2/testFolder3/testFile"));
Assert.assertFalse(testFiles[0].isDirectory());
}
@Test(expected = FileNotFoundException.class)
public void testListNonExistentDir() throws Exception {
final AzureBlobFileSystem fs = this.getFileSystem();
fs.listStatus(new Path("/testFile/"));
}
@Test
public void testListFiles() throws Exception {
final AzureBlobFileSystem fs = this.getFileSystem();
fs.mkdirs(new Path("/test"));
FileStatus[] fileStatuses = fs.listStatus(new Path("/"));
assertEquals(fileStatuses.length, 2);
fs.mkdirs(new Path("/test/sub"));
fileStatuses = fs.listStatus(new Path("/test"));
assertEquals(fileStatuses.length, 1);
assertEquals(fileStatuses[0].getPath().getName(), "sub");
assertTrue(fileStatuses[0].isDirectory());
assertEquals(fileStatuses[0].getLen(), 0);
fs.create(new Path("/test/f"));
fileStatuses = fs.listStatus(new Path("/test"));
assertEquals(fileStatuses.length, 2);
assertEquals(fileStatuses[0].getPath().getName(), "f");
assertFalse(fileStatuses[0].isDirectory());
assertEquals(fileStatuses[0].getLen(), 0);
assertEquals(fileStatuses[1].getPath().getName(), "sub");
assertTrue(fileStatuses[1].isDirectory());
assertEquals(fileStatuses[1].getLen(), 0);
}
}

View File

@ -0,0 +1,88 @@
/**
* 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.concurrent.Callable;
import org.junit.Test;
import org.apache.hadoop.fs.FileAlreadyExistsException;
import org.apache.hadoop.fs.Path;
import static org.apache.hadoop.test.LambdaTestUtils.intercept;
import static org.junit.Assert.assertTrue;
/**
* Test mkdir operation.
*/
public class ITestAzureBlobFileSystemMkDir extends DependencyInjectedTest {
public ITestAzureBlobFileSystemMkDir() {
super();
}
@Test
public void testCreateDirWithExistingDir() throws Exception {
final AzureBlobFileSystem fs = this.getFileSystem();
assertTrue(fs.mkdirs(new Path("testFolder")));
assertTrue(fs.mkdirs(new Path("testFolder")));
}
@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
public void testCreateRoot() throws Exception {
final AzureBlobFileSystem fs = this.getFileSystem();
assertTrue(fs.mkdirs(new Path("/")));
}
}

View File

@ -0,0 +1,41 @@
/**
* 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

@ -0,0 +1,582 @@
/**
* 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.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.IOException;
import java.util.Random;
import java.util.concurrent.Callable;
import static org.apache.hadoop.test.LambdaTestUtils.intercept;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertArrayEquals;
/**
* Test random read operation.
*/
public class ITestAzureBlobFileSystemRandomRead extends DependencyInjectedTest {
private static final int KILOBYTE = 1024;
private static final int MEGABYTE = KILOBYTE * KILOBYTE;
private static final long TEST_FILE_SIZE = 8 * MEGABYTE;
private static final int MAX_ELAPSEDTIMEMS = 20;
private static final int SEQUENTIAL_READ_BUFFER_SIZE = 16 * KILOBYTE;
private static final int CREATE_BUFFER_SIZE = 26 * KILOBYTE;
private static final int SEEK_POSITION_ONE = 2* KILOBYTE;
private static final int SEEK_POSITION_TWO = 5 * KILOBYTE;
private static final int SEEK_POSITION_THREE = 10 * KILOBYTE;
private static final int SEEK_POSITION_FOUR = 4100 * KILOBYTE;
private static final Path TEST_FILE_PATH = new Path(
"/TestRandomRead.txt");
private static final String WASB = "WASB";
private static final String ABFS = "ABFS";
private static long testFileLength = 0;
public ITestAzureBlobFileSystemRandomRead() throws Exception {
super();
}
@Test
public void testBasicRead() throws Exception {
assumeHugeFileExists();
try (FSDataInputStream inputStream = this.getFileSystem().open(TEST_FILE_PATH)) {
byte[] buffer = new byte[3 * MEGABYTE];
// forward seek and read a kilobyte into first kilobyte of bufferV2
inputStream.seek(5 * MEGABYTE);
int numBytesRead = inputStream.read(buffer, 0, KILOBYTE);
assertEquals(KILOBYTE, numBytesRead);
int len = MEGABYTE;
int offset = buffer.length - len;
// reverse seek and read a megabyte into last megabyte of bufferV1
inputStream.seek(3 * MEGABYTE);
numBytesRead = inputStream.read(buffer, offset, len);
assertEquals(len, numBytesRead);
}
}
/**
* Validates the implementation of random read in ABFS
* @throws IOException
*/
@Test
public void testRandomRead() throws Exception {
assumeHugeFileExists();
try (
FSDataInputStream inputStreamV1
= this.getFileSystem().open(TEST_FILE_PATH);
FSDataInputStream inputStreamV2
= this.getWasbFileSystem().open(TEST_FILE_PATH);
) {
final int bufferSize = 4 * KILOBYTE;
byte[] bufferV1 = new byte[bufferSize];
byte[] bufferV2 = new byte[bufferV1.length];
verifyConsistentReads(inputStreamV1, inputStreamV2, bufferV1, bufferV2);
inputStreamV1.seek(0);
inputStreamV2.seek(0);
verifyConsistentReads(inputStreamV1, inputStreamV2, bufferV1, bufferV2);
verifyConsistentReads(inputStreamV1, inputStreamV2, bufferV1, bufferV2);
inputStreamV1.seek(SEEK_POSITION_ONE);
inputStreamV2.seek(SEEK_POSITION_ONE);
inputStreamV1.seek(0);
inputStreamV2.seek(0);
verifyConsistentReads(inputStreamV1, inputStreamV2, bufferV1, bufferV2);
verifyConsistentReads(inputStreamV1, inputStreamV2, bufferV1, bufferV2);
verifyConsistentReads(inputStreamV1, inputStreamV2, bufferV1, bufferV2);
inputStreamV1.seek(SEEK_POSITION_TWO);
inputStreamV2.seek(SEEK_POSITION_TWO);
verifyConsistentReads(inputStreamV1, inputStreamV2, bufferV1, bufferV2);
inputStreamV1.seek(SEEK_POSITION_THREE);
inputStreamV2.seek(SEEK_POSITION_THREE);
verifyConsistentReads(inputStreamV1, inputStreamV2, bufferV1, bufferV2);
verifyConsistentReads(inputStreamV1, inputStreamV2, bufferV1, bufferV2);
inputStreamV1.seek(SEEK_POSITION_FOUR);
inputStreamV2.seek(SEEK_POSITION_FOUR);
verifyConsistentReads(inputStreamV1, inputStreamV2, bufferV1, bufferV2);
}
}
/**
* Validates the implementation of Seekable.seekToNewSource
* @throws IOException
*/
@Test
public void testSeekToNewSource() throws Exception {
assumeHugeFileExists();
try (FSDataInputStream inputStream = this.getFileSystem().open(TEST_FILE_PATH)) {
assertFalse(inputStream.seekToNewSource(0));
}
}
/**
* Validates the implementation of InputStream.skip and ensures there is no
* network I/O for AbfsInputStream
* @throws Exception
*/
@Test
public void testSkipBounds() throws Exception {
assumeHugeFileExists();
try (FSDataInputStream inputStream = this.getFileSystem().open(TEST_FILE_PATH)) {
ContractTestUtils.NanoTimer timer = new ContractTestUtils.NanoTimer();
long skipped = inputStream.skip(-1);
assertEquals(0, skipped);
skipped = inputStream.skip(0);
assertEquals(0, skipped);
assertTrue(testFileLength > 0);
skipped = inputStream.skip(testFileLength);
assertEquals(testFileLength, skipped);
intercept(EOFException.class,
new Callable<Long>() {
@Override
public Long call() throws Exception {
return inputStream.skip(1);
}
}
);
long elapsedTimeMs = timer.elapsedTimeMs();
assertTrue(
String.format(
"There should not be any network I/O (elapsedTimeMs=%1$d).",
elapsedTimeMs),
elapsedTimeMs < MAX_ELAPSEDTIMEMS);
}
}
/**
* Validates the implementation of Seekable.seek and ensures there is no
* network I/O for forward seek.
* @throws Exception
*/
@Test
public void testValidateSeekBounds() throws Exception {
assumeHugeFileExists();
try (FSDataInputStream inputStream = this.getFileSystem().open(TEST_FILE_PATH)) {
ContractTestUtils.NanoTimer timer = new ContractTestUtils.NanoTimer();
inputStream.seek(0);
assertEquals(0, inputStream.getPos());
intercept(EOFException.class,
FSExceptionMessages.NEGATIVE_SEEK,
new Callable<FSDataInputStream>() {
@Override
public FSDataInputStream call() throws Exception {
inputStream.seek(-1);
return inputStream;
}
}
);
assertTrue("Test file length only " + testFileLength, testFileLength > 0);
inputStream.seek(testFileLength);
assertEquals(testFileLength, inputStream.getPos());
intercept(EOFException.class,
FSExceptionMessages.CANNOT_SEEK_PAST_EOF,
new Callable<FSDataInputStream>() {
@Override
public FSDataInputStream call() throws Exception {
inputStream.seek(testFileLength + 1);
return inputStream;
}
}
);
long elapsedTimeMs = timer.elapsedTimeMs();
assertTrue(
String.format(
"There should not be any network I/O (elapsedTimeMs=%1$d).",
elapsedTimeMs),
elapsedTimeMs < MAX_ELAPSEDTIMEMS);
}
}
/**
* Validates the implementation of Seekable.seek, Seekable.getPos,
* and InputStream.available.
* @throws Exception
*/
@Test
public void testSeekAndAvailableAndPosition() throws Exception {
assumeHugeFileExists();
try (FSDataInputStream inputStream = this.getFileSystem().open(TEST_FILE_PATH)) {
byte[] expected1 = {(byte) 'a', (byte) 'b', (byte) 'c'};
byte[] expected2 = {(byte) 'd', (byte) 'e', (byte) 'f'};
byte[] expected3 = {(byte) 'b', (byte) 'c', (byte) 'd'};
byte[] expected4 = {(byte) 'g', (byte) 'h', (byte) 'i'};
byte[] buffer = new byte[3];
int bytesRead = inputStream.read(buffer);
assertEquals(buffer.length, bytesRead);
assertArrayEquals(expected1, buffer);
assertEquals(buffer.length, inputStream.getPos());
assertEquals(testFileLength - inputStream.getPos(),
inputStream.available());
bytesRead = inputStream.read(buffer);
assertEquals(buffer.length, bytesRead);
assertArrayEquals(expected2, buffer);
assertEquals(2 * buffer.length, inputStream.getPos());
assertEquals(testFileLength - inputStream.getPos(),
inputStream.available());
// reverse seek
int seekPos = 0;
inputStream.seek(seekPos);
bytesRead = inputStream.read(buffer);
assertEquals(buffer.length, bytesRead);
assertArrayEquals(expected1, buffer);
assertEquals(buffer.length + seekPos, inputStream.getPos());
assertEquals(testFileLength - inputStream.getPos(),
inputStream.available());
// reverse seek
seekPos = 1;
inputStream.seek(seekPos);
bytesRead = inputStream.read(buffer);
assertEquals(buffer.length, bytesRead);
assertArrayEquals(expected3, buffer);
assertEquals(buffer.length + seekPos, inputStream.getPos());
assertEquals(testFileLength - inputStream.getPos(),
inputStream.available());
// forward seek
seekPos = 6;
inputStream.seek(seekPos);
bytesRead = inputStream.read(buffer);
assertEquals(buffer.length, bytesRead);
assertArrayEquals(expected4, buffer);
assertEquals(buffer.length + seekPos, inputStream.getPos());
assertEquals(testFileLength - inputStream.getPos(),
inputStream.available());
}
}
/**
* Validates the implementation of InputStream.skip, Seekable.getPos,
* and InputStream.available.
* @throws IOException
*/
@Test
public void testSkipAndAvailableAndPosition() throws Exception {
assumeHugeFileExists();
try (FSDataInputStream inputStream = this.getFileSystem().open(TEST_FILE_PATH)) {
byte[] expected1 = {(byte) 'a', (byte) 'b', (byte) 'c'};
byte[] expected2 = {(byte) 'd', (byte) 'e', (byte) 'f'};
byte[] expected3 = {(byte) 'b', (byte) 'c', (byte) 'd'};
byte[] expected4 = {(byte) 'g', (byte) 'h', (byte) 'i'};
assertEquals(testFileLength, inputStream.available());
assertEquals(0, inputStream.getPos());
int n = 3;
long skipped = inputStream.skip(n);
assertEquals(skipped, inputStream.getPos());
assertEquals(testFileLength - inputStream.getPos(),
inputStream.available());
assertEquals(skipped, n);
byte[] buffer = new byte[3];
int bytesRead = inputStream.read(buffer);
assertEquals(buffer.length, bytesRead);
assertArrayEquals(expected2, buffer);
assertEquals(buffer.length + skipped, inputStream.getPos());
assertEquals(testFileLength - inputStream.getPos(),
inputStream.available());
// does skip still work after seek?
int seekPos = 1;
inputStream.seek(seekPos);
bytesRead = inputStream.read(buffer);
assertEquals(buffer.length, bytesRead);
assertArrayEquals(expected3, buffer);
assertEquals(buffer.length + seekPos, inputStream.getPos());
assertEquals(testFileLength - inputStream.getPos(),
inputStream.available());
long currentPosition = inputStream.getPos();
n = 2;
skipped = inputStream.skip(n);
assertEquals(currentPosition + skipped, inputStream.getPos());
assertEquals(testFileLength - inputStream.getPos(),
inputStream.available());
assertEquals(skipped, n);
bytesRead = inputStream.read(buffer);
assertEquals(buffer.length, bytesRead);
assertArrayEquals(expected4, buffer);
assertEquals(buffer.length + skipped + currentPosition,
inputStream.getPos());
assertEquals(testFileLength - inputStream.getPos(),
inputStream.available());
}
}
/**
* Ensures parity in the performance of sequential read after reverse seek for
* abfs of the AbfsInputStream.
* @throws IOException
*/
@Test
public void testSequentialReadAfterReverseSeekPerformance()
throws Exception {
assumeHugeFileExists();
final int maxAttempts = 10;
final double maxAcceptableRatio = 1.01;
double beforeSeekElapsedMs = 0, afterSeekElapsedMs = 0;
double ratio = Double.MAX_VALUE;
for (int i = 0; i < maxAttempts && ratio >= maxAcceptableRatio; i++) {
beforeSeekElapsedMs = sequentialRead(ABFS,
this.getFileSystem(), false);
afterSeekElapsedMs = sequentialRead(ABFS,
this.getFileSystem(), true);
ratio = afterSeekElapsedMs / beforeSeekElapsedMs;
System.out.println((String.format(
"beforeSeekElapsedMs=%1$d, afterSeekElapsedMs=%2$d, ratio=%3$.2f",
(long) beforeSeekElapsedMs,
(long) afterSeekElapsedMs,
ratio)));
}
assertTrue(String.format(
"Performance of ABFS stream after reverse seek is not acceptable:"
+ " beforeSeekElapsedMs=%1$d, afterSeekElapsedMs=%2$d,"
+ " ratio=%3$.2f",
(long) beforeSeekElapsedMs,
(long) afterSeekElapsedMs,
ratio),
ratio < maxAcceptableRatio);
}
@Test
public void testRandomReadPerformance() throws Exception {
createTestFile();
assumeHugeFileExists();
final AzureBlobFileSystem abFs = this.getFileSystem();
final NativeAzureFileSystem wasbFs = this.getWasbFileSystem();
final int maxAttempts = 10;
final double maxAcceptableRatio = 1.025;
double v1ElapsedMs = 0, v2ElapsedMs = 0;
double ratio = Double.MAX_VALUE;
for (int i = 0; i < maxAttempts && ratio >= maxAcceptableRatio; i++) {
v1ElapsedMs = randomRead(1, wasbFs);
v2ElapsedMs = randomRead(2, abFs);
ratio = v2ElapsedMs / v1ElapsedMs;
System.out.println(String.format(
"v1ElapsedMs=%1$d, v2ElapsedMs=%2$d, ratio=%3$.2f",
(long) v1ElapsedMs,
(long) v2ElapsedMs,
ratio));
}
assertTrue(String.format(
"Performance of version 2 is not acceptable: v1ElapsedMs=%1$d,"
+ " v2ElapsedMs=%2$d, ratio=%3$.2f",
(long) v1ElapsedMs,
(long) v2ElapsedMs,
ratio),
ratio < maxAcceptableRatio);
}
private long sequentialRead(String version,
FileSystem fs,
boolean afterReverseSeek) throws IOException {
byte[] buffer = new byte[SEQUENTIAL_READ_BUFFER_SIZE];
long totalBytesRead = 0;
long bytesRead = 0;
try(FSDataInputStream inputStream = fs.open(TEST_FILE_PATH)) {
if (afterReverseSeek) {
while (bytesRead > 0 && totalBytesRead < 4 * MEGABYTE) {
bytesRead = inputStream.read(buffer);
totalBytesRead += bytesRead;
}
totalBytesRead = 0;
inputStream.seek(0);
}
ContractTestUtils.NanoTimer timer = new ContractTestUtils.NanoTimer();
while ((bytesRead = inputStream.read(buffer)) > 0) {
totalBytesRead += bytesRead;
}
long elapsedTimeMs = timer.elapsedTimeMs();
System.out.println(String.format(
"v%1$s: bytesRead=%2$d, elapsedMs=%3$d, Mbps=%4$.2f,"
+ " afterReverseSeek=%5$s",
version,
totalBytesRead,
elapsedTimeMs,
toMbps(totalBytesRead, elapsedTimeMs),
afterReverseSeek));
assertEquals(testFileLength, totalBytesRead);
inputStream.close();
return elapsedTimeMs;
}
}
private long randomRead(int version, FileSystem fs) throws Exception {
assumeHugeFileExists();
final long minBytesToRead = 2 * MEGABYTE;
Random random = new Random();
byte[] buffer = new byte[8 * KILOBYTE];
long totalBytesRead = 0;
long bytesRead = 0;
try(FSDataInputStream inputStream = fs.open(TEST_FILE_PATH)) {
ContractTestUtils.NanoTimer timer = new ContractTestUtils.NanoTimer();
do {
bytesRead = inputStream.read(buffer);
totalBytesRead += bytesRead;
inputStream.seek(random.nextInt(
(int) (TEST_FILE_SIZE - buffer.length)));
} while (bytesRead > 0 && totalBytesRead < minBytesToRead);
long elapsedTimeMs = timer.elapsedTimeMs();
inputStream.close();
System.out.println(String.format(
"v%1$d: totalBytesRead=%2$d, elapsedTimeMs=%3$d, Mbps=%4$.2f",
version,
totalBytesRead,
elapsedTimeMs,
toMbps(totalBytesRead, elapsedTimeMs)));
assertTrue(minBytesToRead <= totalBytesRead);
return elapsedTimeMs;
}
}
/**
* Calculate megabits per second from the specified values for bytes and
* milliseconds.
* @param bytes The number of bytes.
* @param milliseconds The number of milliseconds.
* @return The number of megabits per second.
*/
private static double toMbps(long bytes, long milliseconds) {
return bytes / 1000.0 * 8 / milliseconds;
}
private void createTestFile() throws Exception {
FileSystem fs = this.getWasbFileSystem();
if (fs.exists(TEST_FILE_PATH)) {
FileStatus status = fs.getFileStatus(TEST_FILE_PATH);
if (status.getLen() >= TEST_FILE_SIZE) {
return;
}
}
byte[] buffer = new byte[CREATE_BUFFER_SIZE];
char character = 'a';
for (int i = 0; i < buffer.length; i++) {
buffer[i] = (byte) character;
character = (character == 'z') ? 'a' : (char) ((int) character + 1);
}
System.out.println(("Creating test file {} of size: {} " + TEST_FILE_PATH
+ TEST_FILE_SIZE));
ContractTestUtils.NanoTimer timer = new ContractTestUtils.NanoTimer();
try(FSDataOutputStream outputStream = fs.create(TEST_FILE_PATH)) {
int bytesWritten = 0;
while (bytesWritten < TEST_FILE_SIZE) {
outputStream.write(buffer);
bytesWritten += buffer.length;
}
System.out.println("Closing stream {}" + outputStream);
ContractTestUtils.NanoTimer closeTimer
= new ContractTestUtils.NanoTimer();
outputStream.close();
closeTimer.end("time to close() output stream");
}
timer.end("time to write %d KB", TEST_FILE_SIZE / 1024);
testFileLength = fs.getFileStatus(TEST_FILE_PATH).getLen();
}
private void assumeHugeFileExists() throws Exception{
createTestFile();
FileSystem fs = this.getFileSystem();
ContractTestUtils.assertPathExists(this.getFileSystem(), "huge file not created", TEST_FILE_PATH);
FileStatus status = fs.getFileStatus(TEST_FILE_PATH);
ContractTestUtils.assertIsFile(TEST_FILE_PATH, status);
assertTrue("File " + TEST_FILE_PATH + " is empty", status.getLen() > 0);
}
private void verifyConsistentReads(FSDataInputStream inputStreamV1,
FSDataInputStream inputStreamV2,
byte[] bufferV1,
byte[] bufferV2) throws IOException {
int size = bufferV1.length;
final int numBytesReadV1 = inputStreamV1.read(bufferV1, 0, size);
assertEquals("Bytes read from wasb stream", size, numBytesReadV1);
final int numBytesReadV2 = inputStreamV2.read(bufferV2, 0, size);
assertEquals("Bytes read from abfs stream", size, numBytesReadV2);
assertArrayEquals("Mismatch in read data", bufferV1, bufferV2);
}
}

View File

@ -0,0 +1,152 @@
/**
* 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 java.util.ArrayList;
import java.util.List;
import java.util.concurrent.Callable;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.Future;
import org.junit.Assert;
import org.junit.Test;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.Path;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertNotNull;
/**
* Test rename operation.
*/
public class ITestAzureBlobFileSystemRename extends DependencyInjectedTest {
public ITestAzureBlobFileSystemRename() {
super();
}
@Test(expected = FileNotFoundException.class)
public void testEnsureFileIsRenamed() throws Exception {
final AzureBlobFileSystem fs = this.getFileSystem();
fs.create(new Path("testfile"));
fs.rename(new Path("testfile"), new Path("testfile2"));
FileStatus fileStatus = fs.getFileStatus(new Path("testfile2"));
assertNotNull(fileStatus);
fs.getFileStatus(new Path("testfile"));
}
@Test
public void testRenameFile() throws Exception {
final AzureBlobFileSystem fs = this.getFileSystem();
fs.mkdirs(new Path("/testSrc"));
fs.create(new Path("/testSrc/file1"));
fs.rename(new Path("/testSrc"), new Path("/testDst"));
FileStatus[] fileStatus = fs.listStatus(new Path("/testDst"));
assertNotNull(fileStatus);
}
@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 {
final AzureBlobFileSystem fs = this.getFileSystem();
fs.mkdirs(new Path("testDir"));
fs.mkdirs(new Path("testDir/test1"));
fs.mkdirs(new Path("testDir/test1/test2"));
fs.mkdirs(new Path("testDir/test1/test2/test3"));
Assert.assertTrue(fs.rename(new Path("testDir/test1"), new Path("testDir/test10")));
fs.getFileStatus(new Path("testDir/test1"));
}
@Test(expected = FileNotFoundException.class)
public void testRenameFirstLevelDirectory() throws Exception {
final AzureBlobFileSystem fs = this.getFileSystem();
final List<Future> tasks = new ArrayList<>();
ExecutorService es = Executors.newFixedThreadPool(10);
for (int i = 0; i < 1000; i++) {
final Path fileName = new Path("/test/" + i);
Callable<Void> callable = new Callable<Void>() {
@Override
public Void call() throws Exception {
fs.create(fileName);
return null;
}
};
tasks.add(es.submit(callable));
}
for (Future<Void> task : tasks) {
task.get();
}
es.shutdownNow();
fs.rename(new Path("/test"), new Path("/renamedDir"));
FileStatus[] files = fs.listStatus(new Path("/renamedDir"));
Assert.assertEquals(files.length, 1000);
fs.getFileStatus(new Path("/test"));
}
@Test
public void testRenameRoot() throws Exception {
final AzureBlobFileSystem fs = this.getFileSystem();
boolean renamed = fs.rename(new Path("/"), new Path("/ddd"));
assertFalse(renamed);
renamed = fs.rename(new Path(fs.getUri().toString() + "/"), new Path(fs.getUri().toString() + "/s"));
assertFalse(renamed);
}
}

View File

@ -0,0 +1,78 @@
/**
* 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 org.junit.Assert;
import org.junit.Test;
import org.mockito.Mockito;
import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes;
import org.apache.hadoop.fs.azurebfs.contracts.services.AbfsHttpService;
import org.apache.hadoop.fs.azurebfs.services.AbfsServiceProviderImpl;
import static org.mockito.Matchers.anyObject;
import static org.mockito.Mockito.doReturn;
/**
* Test AzureBlobFileSystem initialization.
*/
public class ITestFileSystemInitialization extends DependencyInjectedTest {
public ITestFileSystemInitialization() {
super();
this.getMockServiceInjector().removeProvider(AbfsHttpService.class);
this.getMockServiceInjector().replaceInstance(AbfsHttpService.class, Mockito.mock(AbfsHttpService.class));
}
@Test
public void ensureAzureBlobFileSystemIsInitialized() throws Exception {
doReturn(new FileStatus(0, true, 0, 0, 0, new Path("/blah")))
.when(AbfsServiceProviderImpl.instance().get(AbfsHttpService.class))
.getFileStatus((AzureBlobFileSystem) anyObject(), (Path) anyObject());
final FileSystem fs = FileSystem.get(this.getConfiguration());
final String accountName = this.getAccountName();
final String filesystem = this.getFileSystemName();
Assert.assertEquals(fs.getUri(), new URI(FileSystemUriSchemes.ABFS_SCHEME, filesystem + "@" + accountName, null, null, null));
Assert.assertNotNull(fs.getWorkingDirectory());
}
@Test
public void ensureSecureAzureBlobFileSystemIsInitialized() throws Exception {
doReturn(new FileStatus(0, true, 0, 0, 0, new Path("/blah")))
.when(AbfsServiceProviderImpl.instance().get(AbfsHttpService.class))
.getFileStatus((AzureBlobFileSystem) anyObject(), (Path) anyObject());
final String accountName = this.getAccountName();
final String filesystem = this.getFileSystemName();
final URI defaultUri = new URI(FileSystemUriSchemes.ABFS_SECURE_SCHEME, filesystem + "@" + accountName, null, null, null);
this.getConfiguration().set(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY, defaultUri.toString());
final FileSystem fs = FileSystem.get(this.getConfiguration());
Assert.assertEquals(fs.getUri(), new URI(FileSystemUriSchemes.ABFS_SECURE_SCHEME, filesystem + "@" + accountName, null, null, null));
Assert.assertNotNull(fs.getWorkingDirectory());
}
}

View File

@ -0,0 +1,82 @@
/**
* 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 org.apache.hadoop.fs.azurebfs.services.AbfsServiceProviderImpl;
import org.junit.Assert;
import org.junit.Test;
import org.mockito.Mockito;
import org.apache.hadoop.fs.AbstractFileSystem;
import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
import org.apache.hadoop.fs.FileContext;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes;
import org.apache.hadoop.fs.azurebfs.contracts.services.AbfsHttpService;
import static org.mockito.Matchers.anyObject;
import static org.mockito.Mockito.doReturn;
/**
* Test AzureBlobFileSystem registration.
*/
public class ITestFileSystemRegistration extends DependencyInjectedTest {
public ITestFileSystemRegistration() throws Exception {
super();
this.getMockServiceInjector().removeProvider(AbfsHttpService.class);
this.getMockServiceInjector().replaceInstance(AbfsHttpService.class, Mockito.mock(AbfsHttpService.class));
}
@Test
public void ensureAzureBlobFileSystemIsDefaultFileSystem() throws Exception {
doReturn(new FileStatus(0, true, 0, 0, 0, new Path("/blah")))
.when(AbfsServiceProviderImpl.instance().get(AbfsHttpService.class))
.getFileStatus((AzureBlobFileSystem) anyObject(), (Path) anyObject());
FileSystem fs = FileSystem.get(this.getConfiguration());
Assert.assertTrue(fs instanceof AzureBlobFileSystem);
AbstractFileSystem afs = FileContext.getFileContext(this.getConfiguration()).getDefaultFileSystem();
Assert.assertTrue(afs instanceof Abfs);
}
@Test
public void ensureSecureAzureBlobFileSystemIsDefaultFileSystem() throws Exception {
doReturn(new FileStatus(0, true, 0, 0, 0, new Path("/blah")))
.when(AbfsServiceProviderImpl.instance().get(AbfsHttpService.class))
.getFileStatus((AzureBlobFileSystem) anyObject(), (Path) anyObject());
final String accountName = this.getAccountName();
final String filesystem = this.getFileSystemName();
final URI defaultUri = new URI(FileSystemUriSchemes.ABFS_SECURE_SCHEME, filesystem + "@" + accountName, null, null, null);
this.getConfiguration().set(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY, defaultUri.toString());
FileSystem fs = FileSystem.get(this.getConfiguration());
Assert.assertTrue(fs instanceof SecureAzureBlobFileSystem);
AbstractFileSystem afs = FileContext.getFileContext(this.getConfiguration()).getDefaultFileSystem();
Assert.assertTrue(afs instanceof Abfss);
}
}

View File

@ -0,0 +1,202 @@
/**
* 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.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.InputStreamReader;
import static junit.framework.TestCase.assertEquals;
import static junit.framework.TestCase.assertFalse;
import static junit.framework.TestCase.assertTrue;
/**
* Test compatibility between ABFS client and WASB client.
*/
public class ITestWasbAbfsCompatibility extends DependencyInjectedTest {
private static final String WASB_TEST_CONTEXT = "wasb test file";
private static final String ABFS_TEST_CONTEXT = "abfs test file";
private static final String TEST_CONTEXT = "THIS IS FOR TEST";
public ITestWasbAbfsCompatibility() throws Exception {
super();
Assume.assumeFalse(this.isEmulator());
}
@Test
public void testListFileStatus() throws Exception {
// crate file using abfs
AzureBlobFileSystem fs = this.getFileSystem();
NativeAzureFileSystem wasb = this.getWasbFileSystem();
Path path1 = new Path("/testfiles/~12/!008/3/abFsTestfile");
FSDataOutputStream abfsStream = fs.create(path1, true);
abfsStream.write(ABFS_TEST_CONTEXT.getBytes());
abfsStream.flush();
abfsStream.hsync();
abfsStream.close();
// create file using wasb
Path path2 = new Path("/testfiles/~12/!008/3/nativeFsTestfile");
System.out.println(wasb.getUri());
FSDataOutputStream nativeFsStream = wasb.create(path2, true);
nativeFsStream.write(WASB_TEST_CONTEXT.getBytes());
nativeFsStream.flush();
nativeFsStream.hsync();
nativeFsStream.close();
// list file using abfs and wasb
FileStatus[] abfsFileStatus = fs.listStatus(new Path("/testfiles/~12/!008/3/"));
FileStatus[] nativeFsFileStatus = wasb.listStatus(new Path("/testfiles/~12/!008/3/"));
assertEquals(2, abfsFileStatus.length);
assertEquals(2, nativeFsFileStatus.length);
}
@Test
public void testReadFile() throws Exception {
boolean[] createFileWithAbfs = new boolean[]{false, true, false, true};
boolean[] readFileWithAbfs = new boolean[]{false, true, true, false};
AzureBlobFileSystem abfs = this.getFileSystem();
NativeAzureFileSystem wasb = this.getWasbFileSystem();
FileSystem fs;
BufferedReader br = null;
for (int i = 0; i< 4; i++) {
try {
Path path = new Path("/testfiles/~12/!008/testfile" + i);
if (createFileWithAbfs[i]) {
fs = abfs;
} else {
fs = wasb;
}
// Write
FSDataOutputStream nativeFsStream = fs.create(path, true);
nativeFsStream.write(TEST_CONTEXT.getBytes());
nativeFsStream.flush();
nativeFsStream.hsync();
nativeFsStream.close();
// Check file status
assertEquals(true, fs.exists(path));
assertEquals(false, fs.getFileStatus(path).isDirectory());
// Read
if (readFileWithAbfs[i]) {
fs = abfs;
} else {
fs = wasb;
}
FSDataInputStream inputStream = fs.open(path);
br = new BufferedReader(new InputStreamReader(fs.open(path)));
String line = br.readLine();
assertEquals(TEST_CONTEXT, line);
// Remove file
fs.delete(path, true);
assertFalse(fs.exists(path));
} catch (Exception e) {
e.printStackTrace();
} finally {
if (br != null) {
br.close();
}
}
}
}
@Test
public void testDir() throws Exception {
boolean[] createDirWithAbfs = new boolean[]{false, true, false, true};
boolean[] readDirWithAbfs = new boolean[]{false, true, true, false};
AzureBlobFileSystem abfs = this.getFileSystem();
NativeAzureFileSystem wasb = this.getWasbFileSystem();
FileSystem fs;
for (int i = 0; i < 4; i++) {
Path path = new Path("/testDir/t" + i);
//create
if (createDirWithAbfs[i]) {
fs = abfs;
} else {
fs = wasb;
}
assertTrue(fs.mkdirs(path));
//check
assertTrue(fs.exists(path));
//read
if (readDirWithAbfs[i]) {
fs = abfs;
} else {
fs = wasb;
}
assertTrue(fs.exists(path));
FileStatus dirStatus = fs.getFileStatus(path);
assertTrue(dirStatus.isDirectory());
fs.delete(path, true);
assertFalse(fs.exists(path));
}
}
@Test
public void testUrlConversion(){
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";
Assert.assertEquals(abfsUrl, wasbUrlToAbfsUrl(wabsUrl));
Assert.assertEquals(wabsUrl, abfsUrlToWasbUrl(abfsUrl));
}
@Test
public void testSetWorkingDirectory() throws Exception {
//create folders
AzureBlobFileSystem abfs = this.getFileSystem();
NativeAzureFileSystem wasb = this.getWasbFileSystem();
assertTrue(abfs.mkdirs(new Path("/d1/d2/d3/d4")));
//set working directory to path1
Path path1 = new Path("/d1/d2");
wasb.setWorkingDirectory(path1);
abfs.setWorkingDirectory(path1);
assertEquals(path1, wasb.getWorkingDirectory());
assertEquals(path1, abfs.getWorkingDirectory());
//set working directory to path2
Path path2 = new Path("d3/d4");
wasb.setWorkingDirectory(path2);
abfs.setWorkingDirectory(path2);
Path path3 = new Path("/d1/d2/d3/d4");
assertEquals(path3, wasb.getWorkingDirectory());
assertEquals(path3, abfs.getWorkingDirectory());
}
}

View File

@ -0,0 +1,37 @@
/**
* 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.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.
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving
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_KEY_PREFIX = "fs.azure.test.account.key.";
public static final String FS_AZURE_TEST_HOST_NAME = "fs.azure.test.host.name";
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";
private TestConfigurationKeys() {}
}

View File

@ -0,0 +1,22 @@
/*
* 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.
*/
@InterfaceAudience.Private
@InterfaceStability.Evolving
package org.apache.hadoop.fs.azurebfs.constants;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;

View File

@ -0,0 +1,63 @@
/**
* 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.contract;
import java.net.URI;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
import org.apache.hadoop.fs.azurebfs.DependencyInjectedTest;
import org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes;
import org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys;
/**
* Dependency inject for ABFS contract tests.
*/
public class DependencyInjectedContractTest extends DependencyInjectedTest {
private final URI testUri;
public DependencyInjectedContractTest(final boolean secure) throws Exception {
this(secure, true);
}
public DependencyInjectedContractTest(final boolean secure, final boolean useExistedFileSystem) throws Exception{
super(secure);
if (useExistedFileSystem) {
Configuration configuration = getConfiguration();
String testUrl = configuration.get(TestConfigurationKeys.FS_AZURE_CONTRACT_TEST_URI);
if (secure) {
testUrl = testUrl.replaceFirst(FileSystemUriSchemes.ABFS_SCHEME, FileSystemUriSchemes.ABFS_SECURE_SCHEME);
}
updateTestUrl(testUrl);
this.testUri = new URI(testUrl);
//Get container for contract tests
configuration.set(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY, this.testUri.toString());
String[] splitAuthority = this.testUri.getAuthority().split("\\@");
updateFileSystemName(splitAuthority[0]);
} else {
this.testUri = new URI(super.getTestUrl());
}
}
public Configuration getConfiguration() {
return super.getConfiguration();
}
}

View File

@ -0,0 +1,54 @@
/**
* 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.contract;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes;
import org.apache.hadoop.fs.azurebfs.utils.UriUtils;
import org.apache.hadoop.fs.contract.AbstractBondedFSContract;
/**
* Azure BlobFileSystem Contract. Test paths are created using any maven fork
* identifier, if defined. This guarantees paths unique to tests
* running in parallel.
*/
public class ITestAbfsFileSystemContract extends AbstractBondedFSContract {
public static final String CONTRACT_XML = "abfs.xml";
private final boolean isSecure;
protected ITestAbfsFileSystemContract(final Configuration conf, boolean secure) {
super(conf);
//insert the base features
addConfResource(CONTRACT_XML);
this.isSecure = secure;
}
@Override
public String getScheme() {
return isSecure ? FileSystemUriSchemes.ABFS_SECURE_SCHEME : FileSystemUriSchemes.ABFS_SCHEME;
}
@Override
public Path getTestPath() {
Path path = new Path(UriUtils.generateUniqueTestPath());
return path;
}
}

View File

@ -0,0 +1,70 @@
/**
* 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.contract;
import java.util.Arrays;
import org.junit.runner.RunWith;
import org.junit.runners.Parameterized;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.contract.AbstractContractAppendTest;
import org.apache.hadoop.fs.contract.AbstractFSContract;
import static org.apache.hadoop.fs.contract.ContractTestUtils.skip;
/**
* Contract test for open operation.
*/
@RunWith(Parameterized.class)
public class ITestAbfsFileSystemContractAppend extends AbstractContractAppendTest {
@Parameterized.Parameters(name = "SecureMode={0}")
public static Iterable<Object[]> secure() {
return Arrays.asList(new Object[][] { {true}, {false} });
}
private final boolean isSecure;
private final DependencyInjectedContractTest dependencyInjectedContractTest;
public ITestAbfsFileSystemContractAppend(final boolean secure) throws Exception {
this.isSecure = secure;
dependencyInjectedContractTest = new DependencyInjectedContractTest(this.isSecure);
}
@Override
public void setup() throws Exception {
dependencyInjectedContractTest.initialize();
super.setup();
}
@Override
protected Configuration createConfiguration() {
return this.dependencyInjectedContractTest.getConfiguration();
}
@Override
protected AbstractFSContract createContract(final Configuration conf) {
return new ITestAbfsFileSystemContract(conf, this.isSecure);
}
@Override
public void testRenameFileBeingAppended() throws Throwable {
skip("Skipping as renaming an opened file is not supported");
}
}

View File

@ -0,0 +1,62 @@
/**
* 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.contract;
import java.util.Arrays;
import org.junit.runner.RunWith;
import org.junit.runners.Parameterized;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.contract.AbstractContractConcatTest;
import org.apache.hadoop.fs.contract.AbstractFSContract;
/**
* Contract test for concat operation.
*/
@RunWith(Parameterized.class)
public class ITestAbfsFileSystemContractConcat extends AbstractContractConcatTest{
@Parameterized.Parameters(name = "SecureMode={0}")
public static Iterable<Object[]> secure() {
return Arrays.asList(new Object[][] { {true}, {false} });
}
private final boolean isSecure;
private final DependencyInjectedContractTest dependencyInjectedContractTest;
public ITestAbfsFileSystemContractConcat(final boolean secure) throws Exception {
this.isSecure = secure;
dependencyInjectedContractTest = new DependencyInjectedContractTest(isSecure);
}
@Override
public void setup() throws Exception {
dependencyInjectedContractTest.initialize();
super.setup();
}
@Override
protected Configuration createConfiguration() {
return this.dependencyInjectedContractTest.getConfiguration();
}
@Override
protected AbstractFSContract createContract(final Configuration conf) {
return new ITestAbfsFileSystemContract(conf, this.isSecure);
}
}

Some files were not shown because too many files have changed in this diff Show More