HADOOP-17290. ABFS: Add Identifiers to Client Request Header (#2520)
Contributed by Sumangala Patki.
This commit is contained in:
parent
ea259f236c
commit
35570e414a
|
@ -61,6 +61,7 @@ import org.apache.hadoop.fs.azurebfs.services.AuthType;
|
|||
import org.apache.hadoop.fs.azurebfs.services.ExponentialRetryPolicy;
|
||||
import org.apache.hadoop.fs.azurebfs.services.KeyProvider;
|
||||
import org.apache.hadoop.fs.azurebfs.services.SimpleKeyProvider;
|
||||
import org.apache.hadoop.fs.azurebfs.utils.TracingHeaderFormat;
|
||||
import org.apache.hadoop.security.ssl.DelegatingSSLSocketFactory;
|
||||
import org.apache.hadoop.security.ProviderUtils;
|
||||
import org.apache.hadoop.util.ReflectionUtils;
|
||||
|
@ -68,6 +69,7 @@ import org.apache.hadoop.util.ReflectionUtils;
|
|||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.EMPTY_STRING;
|
||||
import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.*;
|
||||
import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.*;
|
||||
|
||||
|
@ -264,6 +266,10 @@ public class AbfsConfiguration{
|
|||
DefaultValue = DEFAULT_VALUE_UNKNOWN)
|
||||
private String clusterType;
|
||||
|
||||
@StringConfigurationValidatorAnnotation(ConfigurationKey = FS_AZURE_CLIENT_CORRELATIONID,
|
||||
DefaultValue = EMPTY_STRING)
|
||||
private String clientCorrelationId;
|
||||
|
||||
@BooleanConfigurationValidatorAnnotation(ConfigurationKey = FS_AZURE_ENABLE_DELEGATION_TOKEN,
|
||||
DefaultValue = DEFAULT_ENABLE_DELEGATION_TOKEN)
|
||||
private boolean enableDelegationToken;
|
||||
|
@ -332,6 +338,14 @@ public class AbfsConfiguration{
|
|||
return accountName;
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets client correlation ID provided in config.
|
||||
* @return Client Correlation ID config
|
||||
*/
|
||||
public String getClientCorrelationId() {
|
||||
return clientCorrelationId;
|
||||
}
|
||||
|
||||
/**
|
||||
* Appends an account name to a configuration key yielding the
|
||||
* account-specific form.
|
||||
|
@ -722,6 +736,14 @@ public class AbfsConfiguration{
|
|||
return getEnum(FS_AZURE_SSL_CHANNEL_MODE_KEY, DEFAULT_FS_AZURE_SSL_CHANNEL_MODE);
|
||||
}
|
||||
|
||||
/**
|
||||
* Enum config to allow user to pick format of x-ms-client-request-id header
|
||||
* @return tracingContextFormat config if valid, else default ALL_ID_FORMAT
|
||||
*/
|
||||
public TracingHeaderFormat getTracingHeaderFormat() {
|
||||
return getEnum(FS_AZURE_TRACINGHEADER_FORMAT, TracingHeaderFormat.ALL_ID_FORMAT);
|
||||
}
|
||||
|
||||
public AuthType getAuthType(String accountName) {
|
||||
return getEnum(FS_AZURE_ACCOUNT_AUTH_TYPE_PROPERTY_NAME, AuthType.SharedKey);
|
||||
}
|
||||
|
|
|
@ -34,6 +34,7 @@ import java.util.Collections;
|
|||
import java.util.EnumSet;
|
||||
import java.util.Map;
|
||||
import java.util.Optional;
|
||||
import java.util.UUID;
|
||||
import java.util.concurrent.Callable;
|
||||
import java.util.concurrent.CompletableFuture;
|
||||
import java.util.concurrent.ExecutionException;
|
||||
|
@ -68,6 +69,7 @@ import org.apache.hadoop.fs.XAttrSetFlag;
|
|||
import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants;
|
||||
import org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations;
|
||||
import org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes;
|
||||
import org.apache.hadoop.fs.azurebfs.constants.FSOperationType;
|
||||
import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException;
|
||||
import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException;
|
||||
import org.apache.hadoop.fs.azurebfs.contracts.exceptions.FileSystemOperationUnhandledException;
|
||||
|
@ -77,6 +79,9 @@ import org.apache.hadoop.fs.azurebfs.contracts.exceptions.SASTokenProviderExcept
|
|||
import org.apache.hadoop.fs.azurebfs.contracts.services.AzureServiceErrorCode;
|
||||
import org.apache.hadoop.fs.azurebfs.security.AbfsDelegationTokenManager;
|
||||
import org.apache.hadoop.fs.azurebfs.services.AbfsCounters;
|
||||
import org.apache.hadoop.fs.azurebfs.utils.Listener;
|
||||
import org.apache.hadoop.fs.azurebfs.utils.TracingContext;
|
||||
import org.apache.hadoop.fs.azurebfs.utils.TracingHeaderFormat;
|
||||
import org.apache.hadoop.fs.impl.AbstractFSBuilderImpl;
|
||||
import org.apache.hadoop.fs.impl.OpenFileParameters;
|
||||
import org.apache.hadoop.fs.permission.AclEntry;
|
||||
|
@ -111,10 +116,14 @@ public class AzureBlobFileSystem extends FileSystem
|
|||
private Path workingDir;
|
||||
private AzureBlobFileSystemStore abfsStore;
|
||||
private boolean isClosed;
|
||||
private final String fileSystemId = UUID.randomUUID().toString();
|
||||
|
||||
private boolean delegationTokenEnabled = false;
|
||||
private AbfsDelegationTokenManager delegationTokenManager;
|
||||
private AbfsCounters abfsCounters;
|
||||
private String clientCorrelationId;
|
||||
private TracingHeaderFormat tracingHeaderFormat;
|
||||
private Listener listener;
|
||||
|
||||
@Override
|
||||
public void initialize(URI uri, Configuration configuration)
|
||||
|
@ -131,13 +140,19 @@ public class AzureBlobFileSystem extends FileSystem
|
|||
configuration, abfsCounters);
|
||||
LOG.trace("AzureBlobFileSystemStore init complete");
|
||||
|
||||
final AbfsConfiguration abfsConfiguration = abfsStore.getAbfsConfiguration();
|
||||
final AbfsConfiguration abfsConfiguration = abfsStore
|
||||
.getAbfsConfiguration();
|
||||
clientCorrelationId = TracingContext.validateClientCorrelationID(
|
||||
abfsConfiguration.getClientCorrelationId());
|
||||
tracingHeaderFormat = abfsConfiguration.getTracingHeaderFormat();
|
||||
this.setWorkingDirectory(this.getHomeDirectory());
|
||||
|
||||
if (abfsConfiguration.getCreateRemoteFileSystemDuringInitialization()) {
|
||||
if (this.tryGetFileStatus(new Path(AbfsHttpConstants.ROOT_PATH)) == null) {
|
||||
TracingContext tracingContext = new TracingContext(clientCorrelationId,
|
||||
fileSystemId, FSOperationType.CREATE_FILESYSTEM, tracingHeaderFormat, listener);
|
||||
if (this.tryGetFileStatus(new Path(AbfsHttpConstants.ROOT_PATH), tracingContext) == null) {
|
||||
try {
|
||||
this.createFileSystem();
|
||||
this.createFileSystem(tracingContext);
|
||||
} catch (AzureBlobFileSystemException ex) {
|
||||
checkException(null, ex, AzureServiceErrorCode.FILE_SYSTEM_ALREADY_EXISTS);
|
||||
}
|
||||
|
@ -181,6 +196,10 @@ public class AzureBlobFileSystem extends FileSystem
|
|||
return this.uri;
|
||||
}
|
||||
|
||||
public void registerListener(Listener listener1) {
|
||||
listener = listener1;
|
||||
}
|
||||
|
||||
@Override
|
||||
public FSDataInputStream open(final Path path, final int bufferSize) throws IOException {
|
||||
LOG.debug("AzureBlobFileSystem.open path: {} bufferSize: {}", path, bufferSize);
|
||||
|
@ -194,8 +213,11 @@ public class AzureBlobFileSystem extends FileSystem
|
|||
Path qualifiedPath = makeQualified(path);
|
||||
|
||||
try {
|
||||
TracingContext tracingContext = new TracingContext(clientCorrelationId,
|
||||
fileSystemId, FSOperationType.OPEN, tracingHeaderFormat,
|
||||
listener);
|
||||
InputStream inputStream = abfsStore.openFileForRead(qualifiedPath,
|
||||
options, statistics);
|
||||
options, statistics, tracingContext);
|
||||
return new FSDataInputStream(inputStream);
|
||||
} catch(AzureBlobFileSystemException ex) {
|
||||
checkException(path, ex);
|
||||
|
@ -231,8 +253,11 @@ public class AzureBlobFileSystem extends FileSystem
|
|||
Path qualifiedPath = makeQualified(f);
|
||||
|
||||
try {
|
||||
TracingContext tracingContext = new TracingContext(clientCorrelationId,
|
||||
fileSystemId, FSOperationType.CREATE, overwrite, tracingHeaderFormat, listener);
|
||||
OutputStream outputStream = abfsStore.createFile(qualifiedPath, statistics, overwrite,
|
||||
permission == null ? FsPermission.getFileDefault() : permission, FsPermission.getUMask(getConf()));
|
||||
permission == null ? FsPermission.getFileDefault() : permission,
|
||||
FsPermission.getUMask(getConf()), tracingContext);
|
||||
statIncrement(FILES_CREATED);
|
||||
return new FSDataOutputStream(outputStream, statistics);
|
||||
} catch(AzureBlobFileSystemException ex) {
|
||||
|
@ -249,7 +274,10 @@ public class AzureBlobFileSystem extends FileSystem
|
|||
|
||||
statIncrement(CALL_CREATE_NON_RECURSIVE);
|
||||
final Path parent = f.getParent();
|
||||
final FileStatus parentFileStatus = tryGetFileStatus(parent);
|
||||
TracingContext tracingContext = new TracingContext(clientCorrelationId,
|
||||
fileSystemId, FSOperationType.CREATE_NON_RECURSIVE, tracingHeaderFormat,
|
||||
listener);
|
||||
final FileStatus parentFileStatus = tryGetFileStatus(parent, tracingContext);
|
||||
|
||||
if (parentFileStatus == null) {
|
||||
throw new FileNotFoundException("Cannot create file "
|
||||
|
@ -295,7 +323,11 @@ public class AzureBlobFileSystem extends FileSystem
|
|||
Path qualifiedPath = makeQualified(f);
|
||||
|
||||
try {
|
||||
OutputStream outputStream = abfsStore.openFileForWrite(qualifiedPath, statistics, false);
|
||||
TracingContext tracingContext = new TracingContext(clientCorrelationId,
|
||||
fileSystemId, FSOperationType.APPEND, tracingHeaderFormat,
|
||||
listener);
|
||||
OutputStream outputStream = abfsStore
|
||||
.openFileForWrite(qualifiedPath, statistics, false, tracingContext);
|
||||
return new FSDataOutputStream(outputStream, statistics);
|
||||
} catch(AzureBlobFileSystemException ex) {
|
||||
checkException(f, ex);
|
||||
|
@ -316,9 +348,12 @@ public class AzureBlobFileSystem extends FileSystem
|
|||
Path qualifiedSrcPath = makeQualified(src);
|
||||
Path qualifiedDstPath = makeQualified(dst);
|
||||
|
||||
TracingContext tracingContext = new TracingContext(clientCorrelationId,
|
||||
fileSystemId, FSOperationType.RENAME, true, tracingHeaderFormat,
|
||||
listener);
|
||||
// rename under same folder;
|
||||
if(makeQualified(parentFolder).equals(qualifiedDstPath)) {
|
||||
return tryGetFileStatus(qualifiedSrcPath) != null;
|
||||
return tryGetFileStatus(qualifiedSrcPath, tracingContext) != null;
|
||||
}
|
||||
|
||||
FileStatus dstFileStatus = null;
|
||||
|
@ -327,7 +362,7 @@ public class AzureBlobFileSystem extends FileSystem
|
|||
// - if it doesn't exist, return false
|
||||
// - if it is file, return true
|
||||
// - if it is dir, return false.
|
||||
dstFileStatus = tryGetFileStatus(qualifiedDstPath);
|
||||
dstFileStatus = tryGetFileStatus(qualifiedDstPath, tracingContext);
|
||||
if (dstFileStatus == null) {
|
||||
return false;
|
||||
}
|
||||
|
@ -335,8 +370,8 @@ public class AzureBlobFileSystem extends FileSystem
|
|||
}
|
||||
|
||||
// Non-HNS account need to check dst status on driver side.
|
||||
if (!abfsStore.getIsNamespaceEnabled() && dstFileStatus == null) {
|
||||
dstFileStatus = tryGetFileStatus(qualifiedDstPath);
|
||||
if (!abfsStore.getIsNamespaceEnabled(tracingContext) && dstFileStatus == null) {
|
||||
dstFileStatus = tryGetFileStatus(qualifiedDstPath, tracingContext);
|
||||
}
|
||||
|
||||
try {
|
||||
|
@ -352,7 +387,7 @@ public class AzureBlobFileSystem extends FileSystem
|
|||
|
||||
qualifiedDstPath = makeQualified(adjustedDst);
|
||||
|
||||
abfsStore.rename(qualifiedSrcPath, qualifiedDstPath);
|
||||
abfsStore.rename(qualifiedSrcPath, qualifiedDstPath, tracingContext);
|
||||
return true;
|
||||
} catch(AzureBlobFileSystemException ex) {
|
||||
LOG.debug("Rename operation failed. ", ex);
|
||||
|
@ -386,7 +421,10 @@ public class AzureBlobFileSystem extends FileSystem
|
|||
}
|
||||
|
||||
try {
|
||||
abfsStore.delete(qualifiedPath, recursive);
|
||||
TracingContext tracingContext = new TracingContext(clientCorrelationId,
|
||||
fileSystemId, FSOperationType.DELETE, tracingHeaderFormat,
|
||||
listener);
|
||||
abfsStore.delete(qualifiedPath, recursive, tracingContext);
|
||||
return true;
|
||||
} catch (AzureBlobFileSystemException ex) {
|
||||
checkException(f, ex, AzureServiceErrorCode.PATH_NOT_FOUND);
|
||||
|
@ -403,7 +441,10 @@ public class AzureBlobFileSystem extends FileSystem
|
|||
Path qualifiedPath = makeQualified(f);
|
||||
|
||||
try {
|
||||
FileStatus[] result = abfsStore.listStatus(qualifiedPath);
|
||||
TracingContext tracingContext = new TracingContext(clientCorrelationId,
|
||||
fileSystemId, FSOperationType.LISTSTATUS, true, tracingHeaderFormat,
|
||||
listener);
|
||||
FileStatus[] result = abfsStore.listStatus(qualifiedPath, tracingContext);
|
||||
return result;
|
||||
} catch (AzureBlobFileSystemException ex) {
|
||||
checkException(f, ex);
|
||||
|
@ -472,8 +513,12 @@ public class AzureBlobFileSystem extends FileSystem
|
|||
Path qualifiedPath = makeQualified(f);
|
||||
|
||||
try {
|
||||
abfsStore.createDirectory(qualifiedPath, permission == null ? FsPermission.getDirDefault() : permission,
|
||||
FsPermission.getUMask(getConf()));
|
||||
TracingContext tracingContext = new TracingContext(clientCorrelationId,
|
||||
fileSystemId, FSOperationType.MKDIR, false, tracingHeaderFormat,
|
||||
listener);
|
||||
abfsStore.createDirectory(qualifiedPath,
|
||||
permission == null ? FsPermission.getDirDefault() : permission,
|
||||
FsPermission.getUMask(getConf()), tracingContext);
|
||||
statIncrement(DIRECTORIES_CREATED);
|
||||
return true;
|
||||
} catch (AzureBlobFileSystemException ex) {
|
||||
|
@ -505,14 +550,22 @@ public class AzureBlobFileSystem extends FileSystem
|
|||
|
||||
@Override
|
||||
public FileStatus getFileStatus(final Path f) throws IOException {
|
||||
LOG.debug("AzureBlobFileSystem.getFileStatus path: {}", f);
|
||||
TracingContext tracingContext = new TracingContext(clientCorrelationId,
|
||||
fileSystemId, FSOperationType.GET_FILESTATUS, tracingHeaderFormat,
|
||||
listener);
|
||||
return getFileStatus(f, tracingContext);
|
||||
}
|
||||
|
||||
private FileStatus getFileStatus(final Path path,
|
||||
TracingContext tracingContext) throws IOException {
|
||||
LOG.debug("AzureBlobFileSystem.getFileStatus path: {}", path);
|
||||
statIncrement(CALL_GET_FILE_STATUS);
|
||||
Path qualifiedPath = makeQualified(f);
|
||||
Path qualifiedPath = makeQualified(path);
|
||||
|
||||
try {
|
||||
return abfsStore.getFileStatus(qualifiedPath);
|
||||
return abfsStore.getFileStatus(qualifiedPath, tracingContext);
|
||||
} catch(AzureBlobFileSystemException ex) {
|
||||
checkException(f, ex);
|
||||
checkException(path, ex);
|
||||
return null;
|
||||
}
|
||||
}
|
||||
|
@ -531,7 +584,10 @@ public class AzureBlobFileSystem extends FileSystem
|
|||
|
||||
try (DurationInfo ignored = new DurationInfo(LOG, false, "Break lease for %s",
|
||||
qualifiedPath)) {
|
||||
abfsStore.breakLease(qualifiedPath);
|
||||
TracingContext tracingContext = new TracingContext(clientCorrelationId,
|
||||
fileSystemId, FSOperationType.BREAK_LEASE, tracingHeaderFormat,
|
||||
listener);
|
||||
abfsStore.breakLease(qualifiedPath, tracingContext);
|
||||
} catch(AzureBlobFileSystemException ex) {
|
||||
checkException(f, ex);
|
||||
}
|
||||
|
@ -704,7 +760,11 @@ public class AzureBlobFileSystem extends FileSystem
|
|||
throws IOException {
|
||||
LOG.debug(
|
||||
"AzureBlobFileSystem.setOwner path: {}", path);
|
||||
if (!getIsNamespaceEnabled()) {
|
||||
TracingContext tracingContext = new TracingContext(clientCorrelationId,
|
||||
fileSystemId, FSOperationType.SET_OWNER, true, tracingHeaderFormat,
|
||||
listener);
|
||||
|
||||
if (!getIsNamespaceEnabled(tracingContext)) {
|
||||
super.setOwner(path, owner, group);
|
||||
return;
|
||||
}
|
||||
|
@ -718,7 +778,8 @@ public class AzureBlobFileSystem extends FileSystem
|
|||
try {
|
||||
abfsStore.setOwner(qualifiedPath,
|
||||
owner,
|
||||
group);
|
||||
group,
|
||||
tracingContext);
|
||||
} catch (AzureBlobFileSystemException ex) {
|
||||
checkException(path, ex);
|
||||
}
|
||||
|
@ -746,14 +807,18 @@ public class AzureBlobFileSystem extends FileSystem
|
|||
Path qualifiedPath = makeQualified(path);
|
||||
|
||||
try {
|
||||
Hashtable<String, String> properties = abfsStore.getPathStatus(qualifiedPath);
|
||||
TracingContext tracingContext = new TracingContext(clientCorrelationId,
|
||||
fileSystemId, FSOperationType.SET_ATTR, true, tracingHeaderFormat,
|
||||
listener);
|
||||
Hashtable<String, String> properties = abfsStore
|
||||
.getPathStatus(qualifiedPath, tracingContext);
|
||||
String xAttrName = ensureValidAttributeName(name);
|
||||
boolean xAttrExists = properties.containsKey(xAttrName);
|
||||
XAttrSetFlag.validate(name, xAttrExists, flag);
|
||||
|
||||
String xAttrValue = abfsStore.decodeAttribute(value);
|
||||
properties.put(xAttrName, xAttrValue);
|
||||
abfsStore.setPathProperties(qualifiedPath, properties);
|
||||
abfsStore.setPathProperties(qualifiedPath, properties, tracingContext);
|
||||
} catch (AzureBlobFileSystemException ex) {
|
||||
checkException(path, ex);
|
||||
}
|
||||
|
@ -782,7 +847,11 @@ public class AzureBlobFileSystem extends FileSystem
|
|||
|
||||
byte[] value = null;
|
||||
try {
|
||||
Hashtable<String, String> properties = abfsStore.getPathStatus(qualifiedPath);
|
||||
TracingContext tracingContext = new TracingContext(clientCorrelationId,
|
||||
fileSystemId, FSOperationType.GET_ATTR, true, tracingHeaderFormat,
|
||||
listener);
|
||||
Hashtable<String, String> properties = abfsStore
|
||||
.getPathStatus(qualifiedPath, tracingContext);
|
||||
String xAttrName = ensureValidAttributeName(name);
|
||||
if (properties.containsKey(xAttrName)) {
|
||||
String xAttrValue = properties.get(xAttrName);
|
||||
|
@ -809,7 +878,10 @@ public class AzureBlobFileSystem extends FileSystem
|
|||
public void setPermission(final Path path, final FsPermission permission)
|
||||
throws IOException {
|
||||
LOG.debug("AzureBlobFileSystem.setPermission path: {}", path);
|
||||
if (!getIsNamespaceEnabled()) {
|
||||
TracingContext tracingContext = new TracingContext(clientCorrelationId,
|
||||
fileSystemId, FSOperationType.SET_PERMISSION, true, tracingHeaderFormat, listener);
|
||||
|
||||
if (!getIsNamespaceEnabled(tracingContext)) {
|
||||
super.setPermission(path, permission);
|
||||
return;
|
||||
}
|
||||
|
@ -821,8 +893,7 @@ public class AzureBlobFileSystem extends FileSystem
|
|||
Path qualifiedPath = makeQualified(path);
|
||||
|
||||
try {
|
||||
abfsStore.setPermission(qualifiedPath,
|
||||
permission);
|
||||
abfsStore.setPermission(qualifiedPath, permission, tracingContext);
|
||||
} catch (AzureBlobFileSystemException ex) {
|
||||
checkException(path, ex);
|
||||
}
|
||||
|
@ -842,8 +913,11 @@ public class AzureBlobFileSystem extends FileSystem
|
|||
public void modifyAclEntries(final Path path, final List<AclEntry> aclSpec)
|
||||
throws IOException {
|
||||
LOG.debug("AzureBlobFileSystem.modifyAclEntries path: {}", path);
|
||||
TracingContext tracingContext = new TracingContext(clientCorrelationId,
|
||||
fileSystemId, FSOperationType.MODIFY_ACL, true, tracingHeaderFormat,
|
||||
listener);
|
||||
|
||||
if (!getIsNamespaceEnabled()) {
|
||||
if (!getIsNamespaceEnabled(tracingContext)) {
|
||||
throw new UnsupportedOperationException(
|
||||
"modifyAclEntries is only supported by storage accounts with the "
|
||||
+ "hierarchical namespace enabled.");
|
||||
|
@ -856,8 +930,7 @@ public class AzureBlobFileSystem extends FileSystem
|
|||
Path qualifiedPath = makeQualified(path);
|
||||
|
||||
try {
|
||||
abfsStore.modifyAclEntries(qualifiedPath,
|
||||
aclSpec);
|
||||
abfsStore.modifyAclEntries(qualifiedPath, aclSpec, tracingContext);
|
||||
} catch (AzureBlobFileSystemException ex) {
|
||||
checkException(path, ex);
|
||||
}
|
||||
|
@ -875,8 +948,11 @@ public class AzureBlobFileSystem extends FileSystem
|
|||
public void removeAclEntries(final Path path, final List<AclEntry> aclSpec)
|
||||
throws IOException {
|
||||
LOG.debug("AzureBlobFileSystem.removeAclEntries path: {}", path);
|
||||
TracingContext tracingContext = new TracingContext(clientCorrelationId,
|
||||
fileSystemId, FSOperationType.REMOVE_ACL_ENTRIES, true,
|
||||
tracingHeaderFormat, listener);
|
||||
|
||||
if (!getIsNamespaceEnabled()) {
|
||||
if (!getIsNamespaceEnabled(tracingContext)) {
|
||||
throw new UnsupportedOperationException(
|
||||
"removeAclEntries is only supported by storage accounts with the "
|
||||
+ "hierarchical namespace enabled.");
|
||||
|
@ -889,7 +965,7 @@ public class AzureBlobFileSystem extends FileSystem
|
|||
Path qualifiedPath = makeQualified(path);
|
||||
|
||||
try {
|
||||
abfsStore.removeAclEntries(qualifiedPath, aclSpec);
|
||||
abfsStore.removeAclEntries(qualifiedPath, aclSpec, tracingContext);
|
||||
} catch (AzureBlobFileSystemException ex) {
|
||||
checkException(path, ex);
|
||||
}
|
||||
|
@ -904,8 +980,11 @@ public class AzureBlobFileSystem extends FileSystem
|
|||
@Override
|
||||
public void removeDefaultAcl(final Path path) throws IOException {
|
||||
LOG.debug("AzureBlobFileSystem.removeDefaultAcl path: {}", path);
|
||||
TracingContext tracingContext = new TracingContext(clientCorrelationId,
|
||||
fileSystemId, FSOperationType.REMOVE_DEFAULT_ACL, true,
|
||||
tracingHeaderFormat, listener);
|
||||
|
||||
if (!getIsNamespaceEnabled()) {
|
||||
if (!getIsNamespaceEnabled(tracingContext)) {
|
||||
throw new UnsupportedOperationException(
|
||||
"removeDefaultAcl is only supported by storage accounts with the "
|
||||
+ "hierarchical namespace enabled.");
|
||||
|
@ -914,7 +993,7 @@ public class AzureBlobFileSystem extends FileSystem
|
|||
Path qualifiedPath = makeQualified(path);
|
||||
|
||||
try {
|
||||
abfsStore.removeDefaultAcl(qualifiedPath);
|
||||
abfsStore.removeDefaultAcl(qualifiedPath, tracingContext);
|
||||
} catch (AzureBlobFileSystemException ex) {
|
||||
checkException(path, ex);
|
||||
}
|
||||
|
@ -931,8 +1010,11 @@ public class AzureBlobFileSystem extends FileSystem
|
|||
@Override
|
||||
public void removeAcl(final Path path) throws IOException {
|
||||
LOG.debug("AzureBlobFileSystem.removeAcl path: {}", path);
|
||||
TracingContext tracingContext = new TracingContext(clientCorrelationId,
|
||||
fileSystemId, FSOperationType.REMOVE_ACL, true, tracingHeaderFormat,
|
||||
listener);
|
||||
|
||||
if (!getIsNamespaceEnabled()) {
|
||||
if (!getIsNamespaceEnabled(tracingContext)) {
|
||||
throw new UnsupportedOperationException(
|
||||
"removeAcl is only supported by storage accounts with the "
|
||||
+ "hierarchical namespace enabled.");
|
||||
|
@ -941,7 +1023,7 @@ public class AzureBlobFileSystem extends FileSystem
|
|||
Path qualifiedPath = makeQualified(path);
|
||||
|
||||
try {
|
||||
abfsStore.removeAcl(qualifiedPath);
|
||||
abfsStore.removeAcl(qualifiedPath, tracingContext);
|
||||
} catch (AzureBlobFileSystemException ex) {
|
||||
checkException(path, ex);
|
||||
}
|
||||
|
@ -961,8 +1043,11 @@ public class AzureBlobFileSystem extends FileSystem
|
|||
public void setAcl(final Path path, final List<AclEntry> aclSpec)
|
||||
throws IOException {
|
||||
LOG.debug("AzureBlobFileSystem.setAcl path: {}", path);
|
||||
TracingContext tracingContext = new TracingContext(clientCorrelationId,
|
||||
fileSystemId, FSOperationType.SET_ACL, true, tracingHeaderFormat,
|
||||
listener);
|
||||
|
||||
if (!getIsNamespaceEnabled()) {
|
||||
if (!getIsNamespaceEnabled(tracingContext)) {
|
||||
throw new UnsupportedOperationException(
|
||||
"setAcl is only supported by storage accounts with the hierarchical "
|
||||
+ "namespace enabled.");
|
||||
|
@ -975,7 +1060,7 @@ public class AzureBlobFileSystem extends FileSystem
|
|||
Path qualifiedPath = makeQualified(path);
|
||||
|
||||
try {
|
||||
abfsStore.setAcl(qualifiedPath, aclSpec);
|
||||
abfsStore.setAcl(qualifiedPath, aclSpec, tracingContext);
|
||||
} catch (AzureBlobFileSystemException ex) {
|
||||
checkException(path, ex);
|
||||
}
|
||||
|
@ -991,8 +1076,10 @@ public class AzureBlobFileSystem extends FileSystem
|
|||
@Override
|
||||
public AclStatus getAclStatus(final Path path) throws IOException {
|
||||
LOG.debug("AzureBlobFileSystem.getAclStatus path: {}", path);
|
||||
TracingContext tracingContext = new TracingContext(clientCorrelationId,
|
||||
fileSystemId, FSOperationType.GET_ACL_STATUS, true, tracingHeaderFormat, listener);
|
||||
|
||||
if (!getIsNamespaceEnabled()) {
|
||||
if (!getIsNamespaceEnabled(tracingContext)) {
|
||||
throw new UnsupportedOperationException(
|
||||
"getAclStatus is only supported by storage account with the "
|
||||
+ "hierarchical namespace enabled.");
|
||||
|
@ -1001,7 +1088,7 @@ public class AzureBlobFileSystem extends FileSystem
|
|||
Path qualifiedPath = makeQualified(path);
|
||||
|
||||
try {
|
||||
return abfsStore.getAclStatus(qualifiedPath);
|
||||
return abfsStore.getAclStatus(qualifiedPath, tracingContext);
|
||||
} catch (AzureBlobFileSystemException ex) {
|
||||
checkException(path, ex);
|
||||
return null;
|
||||
|
@ -1025,7 +1112,10 @@ public class AzureBlobFileSystem extends FileSystem
|
|||
LOG.debug("AzureBlobFileSystem.access path : {}, mode : {}", path, mode);
|
||||
Path qualifiedPath = makeQualified(path);
|
||||
try {
|
||||
this.abfsStore.access(qualifiedPath, mode);
|
||||
TracingContext tracingContext = new TracingContext(clientCorrelationId,
|
||||
fileSystemId, FSOperationType.ACCESS, tracingHeaderFormat,
|
||||
listener);
|
||||
this.abfsStore.access(qualifiedPath, mode, tracingContext);
|
||||
} catch (AzureBlobFileSystemException ex) {
|
||||
checkCheckAccessException(path, ex);
|
||||
}
|
||||
|
@ -1049,17 +1139,20 @@ public class AzureBlobFileSystem extends FileSystem
|
|||
throws IOException {
|
||||
LOG.debug("AzureBlobFileSystem.listStatusIterator path : {}", path);
|
||||
if (abfsStore.getAbfsConfiguration().enableAbfsListIterator()) {
|
||||
TracingContext tracingContext = new TracingContext(clientCorrelationId,
|
||||
fileSystemId, FSOperationType.LISTSTATUS, true, tracingHeaderFormat, listener);
|
||||
AbfsListStatusRemoteIterator abfsLsItr =
|
||||
new AbfsListStatusRemoteIterator(getFileStatus(path), abfsStore);
|
||||
new AbfsListStatusRemoteIterator(getFileStatus(path, tracingContext), abfsStore,
|
||||
tracingContext);
|
||||
return RemoteIterators.typeCastingRemoteIterator(abfsLsItr);
|
||||
} else {
|
||||
return super.listStatusIterator(path);
|
||||
}
|
||||
}
|
||||
|
||||
private FileStatus tryGetFileStatus(final Path f) {
|
||||
private FileStatus tryGetFileStatus(final Path f, TracingContext tracingContext) {
|
||||
try {
|
||||
return getFileStatus(f);
|
||||
return getFileStatus(f, tracingContext);
|
||||
} catch (IOException ex) {
|
||||
LOG.debug("File not found {}", f);
|
||||
statIncrement(ERROR_IGNORED);
|
||||
|
@ -1071,7 +1164,9 @@ public class AzureBlobFileSystem extends FileSystem
|
|||
LOG.debug(
|
||||
"AzureBlobFileSystem.fileSystemExists uri: {}", uri);
|
||||
try {
|
||||
abfsStore.getFilesystemProperties();
|
||||
TracingContext tracingContext = new TracingContext(clientCorrelationId,
|
||||
fileSystemId, FSOperationType.TEST_OP, tracingHeaderFormat, listener);
|
||||
abfsStore.getFilesystemProperties(tracingContext);
|
||||
} catch (AzureBlobFileSystemException ex) {
|
||||
try {
|
||||
checkException(null, ex);
|
||||
|
@ -1086,11 +1181,11 @@ public class AzureBlobFileSystem extends FileSystem
|
|||
return true;
|
||||
}
|
||||
|
||||
private void createFileSystem() throws IOException {
|
||||
private void createFileSystem(TracingContext tracingContext) throws IOException {
|
||||
LOG.debug(
|
||||
"AzureBlobFileSystem.createFileSystem uri: {}", uri);
|
||||
try {
|
||||
abfsStore.createFilesystem();
|
||||
abfsStore.createFilesystem(tracingContext);
|
||||
} catch (AzureBlobFileSystemException ex) {
|
||||
checkException(null, ex);
|
||||
}
|
||||
|
@ -1283,6 +1378,11 @@ public class AzureBlobFileSystem extends FileSystem
|
|||
return this.statistics;
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
void setListenerOperation(FSOperationType operation) {
|
||||
listener.setOperation(operation);
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
static class FileSystemOperation<T> {
|
||||
private final T result;
|
||||
|
@ -1318,8 +1418,9 @@ public class AzureBlobFileSystem extends FileSystem
|
|||
}
|
||||
|
||||
@VisibleForTesting
|
||||
boolean getIsNamespaceEnabled() throws AzureBlobFileSystemException {
|
||||
return abfsStore.getIsNamespaceEnabled();
|
||||
boolean getIsNamespaceEnabled(TracingContext tracingContext)
|
||||
throws AzureBlobFileSystemException {
|
||||
return abfsStore.getIsNamespaceEnabled(tracingContext);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -1333,6 +1434,16 @@ public class AzureBlobFileSystem extends FileSystem
|
|||
return abfsCounters.toMap();
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
String getFileSystemId() {
|
||||
return fileSystemId;
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
String getClientCorrelationId() {
|
||||
return clientCorrelationId;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean hasPathCapability(final Path path, final String capability)
|
||||
throws IOException {
|
||||
|
@ -1343,7 +1454,10 @@ public class AzureBlobFileSystem extends FileSystem
|
|||
case CommonPathCapabilities.FS_APPEND:
|
||||
return true;
|
||||
case CommonPathCapabilities.FS_ACLS:
|
||||
return getIsNamespaceEnabled();
|
||||
return getIsNamespaceEnabled(
|
||||
new TracingContext(clientCorrelationId, fileSystemId,
|
||||
FSOperationType.HAS_PATH_CAPABILITY, tracingHeaderFormat,
|
||||
listener));
|
||||
default:
|
||||
return super.hasPathCapability(p, capability);
|
||||
}
|
||||
|
|
|
@ -113,6 +113,7 @@ import org.apache.hadoop.fs.azurebfs.services.ListingSupport;
|
|||
import org.apache.hadoop.fs.azurebfs.utils.Base64;
|
||||
import org.apache.hadoop.fs.azurebfs.utils.CRC64;
|
||||
import org.apache.hadoop.fs.azurebfs.utils.DateTimeUtils;
|
||||
import org.apache.hadoop.fs.azurebfs.utils.TracingContext;
|
||||
import org.apache.hadoop.fs.azurebfs.utils.UriUtils;
|
||||
import org.apache.hadoop.fs.permission.AclEntry;
|
||||
import org.apache.hadoop.fs.permission.AclStatus;
|
||||
|
@ -310,7 +311,8 @@ public class AzureBlobFileSystemStore implements Closeable, ListingSupport {
|
|||
return authorityParts;
|
||||
}
|
||||
|
||||
public boolean getIsNamespaceEnabled() throws AzureBlobFileSystemException {
|
||||
public boolean getIsNamespaceEnabled(TracingContext tracingContext)
|
||||
throws AzureBlobFileSystemException {
|
||||
try {
|
||||
return this.isNamespaceEnabled.toBoolean();
|
||||
} catch (TrileanConversionException e) {
|
||||
|
@ -321,7 +323,8 @@ public class AzureBlobFileSystemStore implements Closeable, ListingSupport {
|
|||
LOG.debug("Get root ACL status");
|
||||
try (AbfsPerfInfo perfInfo = startTracking("getIsNamespaceEnabled",
|
||||
"getAclStatus")) {
|
||||
AbfsRestOperation op = client.getAclStatus(AbfsHttpConstants.ROOT_PATH);
|
||||
AbfsRestOperation op = client
|
||||
.getAclStatus(AbfsHttpConstants.ROOT_PATH, tracingContext);
|
||||
perfInfo.registerResult(op.getResult());
|
||||
isNamespaceEnabled = Trilean.getTrilean(true);
|
||||
perfInfo.registerSuccess(true);
|
||||
|
@ -374,7 +377,8 @@ public class AzureBlobFileSystemStore implements Closeable, ListingSupport {
|
|||
return this.abfsConfiguration;
|
||||
}
|
||||
|
||||
public Hashtable<String, String> getFilesystemProperties() throws AzureBlobFileSystemException {
|
||||
public Hashtable<String, String> getFilesystemProperties(
|
||||
TracingContext tracingContext) throws AzureBlobFileSystemException {
|
||||
try (AbfsPerfInfo perfInfo = startTracking("getFilesystemProperties",
|
||||
"getFilesystemProperties")) {
|
||||
LOG.debug("getFilesystemProperties for filesystem: {}",
|
||||
|
@ -382,7 +386,8 @@ public class AzureBlobFileSystemStore implements Closeable, ListingSupport {
|
|||
|
||||
final Hashtable<String, String> parsedXmsProperties;
|
||||
|
||||
final AbfsRestOperation op = client.getFilesystemProperties();
|
||||
final AbfsRestOperation op = client
|
||||
.getFilesystemProperties(tracingContext);
|
||||
perfInfo.registerResult(op.getResult());
|
||||
|
||||
final String xMsProperties = op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_PROPERTIES);
|
||||
|
@ -394,7 +399,8 @@ public class AzureBlobFileSystemStore implements Closeable, ListingSupport {
|
|||
}
|
||||
}
|
||||
|
||||
public void setFilesystemProperties(final Hashtable<String, String> properties)
|
||||
public void setFilesystemProperties(
|
||||
final Hashtable<String, String> properties, TracingContext tracingContext)
|
||||
throws AzureBlobFileSystemException {
|
||||
if (properties == null || properties.isEmpty()) {
|
||||
LOG.trace("setFilesystemProperties no properties present");
|
||||
|
@ -414,19 +420,22 @@ public class AzureBlobFileSystemStore implements Closeable, ListingSupport {
|
|||
throw new InvalidAbfsRestOperationException(ex);
|
||||
}
|
||||
|
||||
final AbfsRestOperation op = client.setFilesystemProperties(commaSeparatedProperties);
|
||||
final AbfsRestOperation op = client
|
||||
.setFilesystemProperties(commaSeparatedProperties, tracingContext);
|
||||
perfInfo.registerResult(op.getResult()).registerSuccess(true);
|
||||
}
|
||||
}
|
||||
|
||||
public Hashtable<String, String> getPathStatus(final Path path) throws AzureBlobFileSystemException {
|
||||
public Hashtable<String, String> getPathStatus(final Path path,
|
||||
TracingContext tracingContext) throws AzureBlobFileSystemException {
|
||||
try (AbfsPerfInfo perfInfo = startTracking("getPathStatus", "getPathStatus")){
|
||||
LOG.debug("getPathStatus for filesystem: {} path: {}",
|
||||
client.getFileSystem(),
|
||||
path);
|
||||
|
||||
final Hashtable<String, String> parsedXmsProperties;
|
||||
final AbfsRestOperation op = client.getPathStatus(getRelativePath(path), true);
|
||||
final AbfsRestOperation op = client
|
||||
.getPathStatus(getRelativePath(path), true, tracingContext);
|
||||
perfInfo.registerResult(op.getResult());
|
||||
|
||||
final String xMsProperties = op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_PROPERTIES);
|
||||
|
@ -439,7 +448,9 @@ public class AzureBlobFileSystemStore implements Closeable, ListingSupport {
|
|||
}
|
||||
}
|
||||
|
||||
public void setPathProperties(final Path path, final Hashtable<String, String> properties) throws AzureBlobFileSystemException {
|
||||
public void setPathProperties(final Path path,
|
||||
final Hashtable<String, String> properties, TracingContext tracingContext)
|
||||
throws AzureBlobFileSystemException {
|
||||
try (AbfsPerfInfo perfInfo = startTracking("setPathProperties", "setPathProperties")){
|
||||
LOG.debug("setFilesystemProperties for filesystem: {} path: {} with properties: {}",
|
||||
client.getFileSystem(),
|
||||
|
@ -452,37 +463,41 @@ public class AzureBlobFileSystemStore implements Closeable, ListingSupport {
|
|||
} catch (CharacterCodingException ex) {
|
||||
throw new InvalidAbfsRestOperationException(ex);
|
||||
}
|
||||
final AbfsRestOperation op = client.setPathProperties(getRelativePath(path), commaSeparatedProperties);
|
||||
final AbfsRestOperation op = client
|
||||
.setPathProperties(getRelativePath(path), commaSeparatedProperties,
|
||||
tracingContext);
|
||||
perfInfo.registerResult(op.getResult()).registerSuccess(true);
|
||||
}
|
||||
}
|
||||
|
||||
public void createFilesystem() throws AzureBlobFileSystemException {
|
||||
public void createFilesystem(TracingContext tracingContext)
|
||||
throws AzureBlobFileSystemException {
|
||||
try (AbfsPerfInfo perfInfo = startTracking("createFilesystem", "createFilesystem")){
|
||||
LOG.debug("createFilesystem for filesystem: {}",
|
||||
client.getFileSystem());
|
||||
|
||||
final AbfsRestOperation op = client.createFilesystem();
|
||||
final AbfsRestOperation op = client.createFilesystem(tracingContext);
|
||||
perfInfo.registerResult(op.getResult()).registerSuccess(true);
|
||||
}
|
||||
}
|
||||
|
||||
public void deleteFilesystem() throws AzureBlobFileSystemException {
|
||||
public void deleteFilesystem(TracingContext tracingContext)
|
||||
throws AzureBlobFileSystemException {
|
||||
try (AbfsPerfInfo perfInfo = startTracking("deleteFilesystem", "deleteFilesystem")) {
|
||||
LOG.debug("deleteFilesystem for filesystem: {}",
|
||||
client.getFileSystem());
|
||||
|
||||
final AbfsRestOperation op = client.deleteFilesystem();
|
||||
final AbfsRestOperation op = client.deleteFilesystem(tracingContext);
|
||||
perfInfo.registerResult(op.getResult()).registerSuccess(true);
|
||||
}
|
||||
}
|
||||
|
||||
public OutputStream createFile(final Path path,
|
||||
final FileSystem.Statistics statistics,
|
||||
final boolean overwrite, final FsPermission permission,
|
||||
final FsPermission umask) throws AzureBlobFileSystemException {
|
||||
final FileSystem.Statistics statistics, final boolean overwrite,
|
||||
final FsPermission permission, final FsPermission umask,
|
||||
TracingContext tracingContext) throws AzureBlobFileSystemException {
|
||||
try (AbfsPerfInfo perfInfo = startTracking("createFile", "createPath")) {
|
||||
boolean isNamespaceEnabled = getIsNamespaceEnabled();
|
||||
boolean isNamespaceEnabled = getIsNamespaceEnabled(tracingContext);
|
||||
LOG.debug("createFile filesystem: {} path: {} overwrite: {} permission: {} umask: {} isNamespaceEnabled: {}",
|
||||
client.getFileSystem(),
|
||||
path,
|
||||
|
@ -512,7 +527,8 @@ public class AzureBlobFileSystemStore implements Closeable, ListingSupport {
|
|||
statistics,
|
||||
isNamespaceEnabled ? getOctalNotation(permission) : null,
|
||||
isNamespaceEnabled ? getOctalNotation(umask) : null,
|
||||
isAppendBlob
|
||||
isAppendBlob,
|
||||
tracingContext
|
||||
);
|
||||
|
||||
} else {
|
||||
|
@ -521,18 +537,21 @@ public class AzureBlobFileSystemStore implements Closeable, ListingSupport {
|
|||
isNamespaceEnabled ? getOctalNotation(permission) : null,
|
||||
isNamespaceEnabled ? getOctalNotation(umask) : null,
|
||||
isAppendBlob,
|
||||
null);
|
||||
null,
|
||||
tracingContext);
|
||||
|
||||
}
|
||||
perfInfo.registerResult(op.getResult()).registerSuccess(true);
|
||||
|
||||
AbfsLease lease = maybeCreateLease(relativePath);
|
||||
AbfsLease lease = maybeCreateLease(relativePath, tracingContext);
|
||||
|
||||
return new AbfsOutputStream(
|
||||
client,
|
||||
statistics,
|
||||
relativePath,
|
||||
0,
|
||||
populateAbfsOutputStreamContext(isAppendBlob, lease));
|
||||
populateAbfsOutputStreamContext(isAppendBlob, lease),
|
||||
tracingContext);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -551,20 +570,22 @@ public class AzureBlobFileSystemStore implements Closeable, ListingSupport {
|
|||
final FileSystem.Statistics statistics,
|
||||
final String permission,
|
||||
final String umask,
|
||||
final boolean isAppendBlob) throws AzureBlobFileSystemException {
|
||||
final boolean isAppendBlob,
|
||||
TracingContext tracingContext) throws AzureBlobFileSystemException {
|
||||
AbfsRestOperation op;
|
||||
|
||||
try {
|
||||
// Trigger a create with overwrite=false first so that eTag fetch can be
|
||||
// avoided for cases when no pre-existing file is present (major portion
|
||||
// of create file traffic falls into the case of no pre-existing file).
|
||||
op = client.createPath(relativePath, true,
|
||||
false, permission, umask, isAppendBlob, null);
|
||||
op = client.createPath(relativePath, true, false, permission, umask,
|
||||
isAppendBlob, null, tracingContext);
|
||||
|
||||
} catch (AbfsRestOperationException e) {
|
||||
if (e.getStatusCode() == HttpURLConnection.HTTP_CONFLICT) {
|
||||
// File pre-exists, fetch eTag
|
||||
try {
|
||||
op = client.getPathStatus(relativePath, false);
|
||||
op = client.getPathStatus(relativePath, false, tracingContext);
|
||||
} catch (AbfsRestOperationException ex) {
|
||||
if (ex.getStatusCode() == HttpURLConnection.HTTP_NOT_FOUND) {
|
||||
// Is a parallel access case, as file which was found to be
|
||||
|
@ -582,8 +603,8 @@ public class AzureBlobFileSystemStore implements Closeable, ListingSupport {
|
|||
|
||||
try {
|
||||
// overwrite only if eTag matches with the file properties fetched befpre
|
||||
op = client.createPath(relativePath, true,
|
||||
true, permission, umask, isAppendBlob, eTag);
|
||||
op = client.createPath(relativePath, true, true, permission, umask,
|
||||
isAppendBlob, eTag, tracingContext);
|
||||
} catch (AbfsRestOperationException ex) {
|
||||
if (ex.getStatusCode() == HttpURLConnection.HTTP_PRECON_FAILED) {
|
||||
// Is a parallel access case, as file with eTag was just queried
|
||||
|
@ -623,10 +644,11 @@ public class AzureBlobFileSystemStore implements Closeable, ListingSupport {
|
|||
.build();
|
||||
}
|
||||
|
||||
public void createDirectory(final Path path, final FsPermission permission, final FsPermission umask)
|
||||
public void createDirectory(final Path path, final FsPermission permission,
|
||||
final FsPermission umask, TracingContext tracingContext)
|
||||
throws AzureBlobFileSystemException {
|
||||
try (AbfsPerfInfo perfInfo = startTracking("createDirectory", "createPath")) {
|
||||
boolean isNamespaceEnabled = getIsNamespaceEnabled();
|
||||
boolean isNamespaceEnabled = getIsNamespaceEnabled(tracingContext);
|
||||
LOG.debug("createDirectory filesystem: {} path: {} permission: {} umask: {} isNamespaceEnabled: {}",
|
||||
client.getFileSystem(),
|
||||
path,
|
||||
|
@ -639,20 +661,21 @@ public class AzureBlobFileSystemStore implements Closeable, ListingSupport {
|
|||
final AbfsRestOperation op = client.createPath(getRelativePath(path),
|
||||
false, overwrite,
|
||||
isNamespaceEnabled ? getOctalNotation(permission) : null,
|
||||
isNamespaceEnabled ? getOctalNotation(umask) : null, false, null);
|
||||
isNamespaceEnabled ? getOctalNotation(umask) : null, false, null,
|
||||
tracingContext);
|
||||
perfInfo.registerResult(op.getResult()).registerSuccess(true);
|
||||
}
|
||||
}
|
||||
|
||||
public AbfsInputStream openFileForRead(final Path path,
|
||||
final FileSystem.Statistics statistics)
|
||||
final FileSystem.Statistics statistics, TracingContext tracingContext)
|
||||
throws AzureBlobFileSystemException {
|
||||
return openFileForRead(path, Optional.empty(), statistics);
|
||||
return openFileForRead(path, Optional.empty(), statistics, tracingContext);
|
||||
}
|
||||
|
||||
public AbfsInputStream openFileForRead(final Path path,
|
||||
final Optional<Configuration> options,
|
||||
final FileSystem.Statistics statistics)
|
||||
final FileSystem.Statistics statistics, TracingContext tracingContext)
|
||||
throws AzureBlobFileSystemException {
|
||||
try (AbfsPerfInfo perfInfo = startTracking("openFileForRead", "getPathStatus")) {
|
||||
LOG.debug("openFileForRead filesystem: {} path: {}",
|
||||
|
@ -661,7 +684,8 @@ public class AzureBlobFileSystemStore implements Closeable, ListingSupport {
|
|||
|
||||
String relativePath = getRelativePath(path);
|
||||
|
||||
final AbfsRestOperation op = client.getPathStatus(relativePath, false);
|
||||
final AbfsRestOperation op = client
|
||||
.getPathStatus(relativePath, false, tracingContext);
|
||||
perfInfo.registerResult(op.getResult());
|
||||
|
||||
final String resourceType = op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_RESOURCE_TYPE);
|
||||
|
@ -682,7 +706,7 @@ public class AzureBlobFileSystemStore implements Closeable, ListingSupport {
|
|||
return new AbfsInputStream(client, statistics,
|
||||
relativePath, contentLength,
|
||||
populateAbfsInputStreamContext(options),
|
||||
eTag);
|
||||
eTag, tracingContext);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -705,8 +729,9 @@ public class AzureBlobFileSystemStore implements Closeable, ListingSupport {
|
|||
.build();
|
||||
}
|
||||
|
||||
public OutputStream openFileForWrite(final Path path, final FileSystem.Statistics statistics, final boolean overwrite) throws
|
||||
AzureBlobFileSystemException {
|
||||
public OutputStream openFileForWrite(final Path path,
|
||||
final FileSystem.Statistics statistics, final boolean overwrite,
|
||||
TracingContext tracingContext) throws AzureBlobFileSystemException {
|
||||
try (AbfsPerfInfo perfInfo = startTracking("openFileForWrite", "getPathStatus")) {
|
||||
LOG.debug("openFileForWrite filesystem: {} path: {} overwrite: {}",
|
||||
client.getFileSystem(),
|
||||
|
@ -715,7 +740,8 @@ public class AzureBlobFileSystemStore implements Closeable, ListingSupport {
|
|||
|
||||
String relativePath = getRelativePath(path);
|
||||
|
||||
final AbfsRestOperation op = client.getPathStatus(relativePath, false);
|
||||
final AbfsRestOperation op = client
|
||||
.getPathStatus(relativePath, false, tracingContext);
|
||||
perfInfo.registerResult(op.getResult());
|
||||
|
||||
final String resourceType = op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_RESOURCE_TYPE);
|
||||
|
@ -738,14 +764,15 @@ public class AzureBlobFileSystemStore implements Closeable, ListingSupport {
|
|||
isAppendBlob = true;
|
||||
}
|
||||
|
||||
AbfsLease lease = maybeCreateLease(relativePath);
|
||||
AbfsLease lease = maybeCreateLease(relativePath, tracingContext);
|
||||
|
||||
return new AbfsOutputStream(
|
||||
client,
|
||||
statistics,
|
||||
relativePath,
|
||||
offset,
|
||||
populateAbfsOutputStreamContext(isAppendBlob, lease));
|
||||
populateAbfsOutputStreamContext(isAppendBlob, lease),
|
||||
tracingContext);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -753,15 +780,16 @@ public class AzureBlobFileSystemStore implements Closeable, ListingSupport {
|
|||
* Break any current lease on an ABFS file.
|
||||
*
|
||||
* @param path file name
|
||||
* @param tracingContext TracingContext instance to track correlation IDs
|
||||
* @throws AzureBlobFileSystemException on any exception while breaking the lease
|
||||
*/
|
||||
public void breakLease(final Path path) throws AzureBlobFileSystemException {
|
||||
public void breakLease(final Path path, final TracingContext tracingContext) throws AzureBlobFileSystemException {
|
||||
LOG.debug("lease path: {}", path);
|
||||
|
||||
client.breakLease(getRelativePath(path));
|
||||
client.breakLease(getRelativePath(path), tracingContext);
|
||||
}
|
||||
|
||||
public void rename(final Path source, final Path destination) throws
|
||||
public void rename(final Path source, final Path destination, TracingContext tracingContext) throws
|
||||
AzureBlobFileSystemException {
|
||||
final Instant startAggregate = abfsPerfTracker.getLatencyInstant();
|
||||
long countAggregate = 0;
|
||||
|
@ -784,8 +812,9 @@ public class AzureBlobFileSystemStore implements Closeable, ListingSupport {
|
|||
|
||||
do {
|
||||
try (AbfsPerfInfo perfInfo = startTracking("rename", "renamePath")) {
|
||||
AbfsRestOperation op = client.renamePath(sourceRelativePath,
|
||||
destinationRelativePath, continuation);
|
||||
AbfsRestOperation op = client
|
||||
.renamePath(sourceRelativePath, destinationRelativePath,
|
||||
continuation, tracingContext);
|
||||
perfInfo.registerResult(op.getResult());
|
||||
continuation = op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_CONTINUATION);
|
||||
perfInfo.registerSuccess(true);
|
||||
|
@ -799,8 +828,8 @@ public class AzureBlobFileSystemStore implements Closeable, ListingSupport {
|
|||
} while (shouldContinue);
|
||||
}
|
||||
|
||||
public void delete(final Path path, final boolean recursive)
|
||||
throws AzureBlobFileSystemException {
|
||||
public void delete(final Path path, final boolean recursive,
|
||||
TracingContext tracingContext) throws AzureBlobFileSystemException {
|
||||
final Instant startAggregate = abfsPerfTracker.getLatencyInstant();
|
||||
long countAggregate = 0;
|
||||
boolean shouldContinue = true;
|
||||
|
@ -816,8 +845,8 @@ public class AzureBlobFileSystemStore implements Closeable, ListingSupport {
|
|||
|
||||
do {
|
||||
try (AbfsPerfInfo perfInfo = startTracking("delete", "deletePath")) {
|
||||
AbfsRestOperation op = client.deletePath(
|
||||
relativePath, recursive, continuation);
|
||||
AbfsRestOperation op = client
|
||||
.deletePath(relativePath, recursive, continuation, tracingContext);
|
||||
perfInfo.registerResult(op.getResult());
|
||||
continuation = op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_CONTINUATION);
|
||||
perfInfo.registerSuccess(true);
|
||||
|
@ -831,9 +860,10 @@ public class AzureBlobFileSystemStore implements Closeable, ListingSupport {
|
|||
} while (shouldContinue);
|
||||
}
|
||||
|
||||
public FileStatus getFileStatus(final Path path) throws IOException {
|
||||
public FileStatus getFileStatus(final Path path,
|
||||
TracingContext tracingContext) throws IOException {
|
||||
try (AbfsPerfInfo perfInfo = startTracking("getFileStatus", "undetermined")) {
|
||||
boolean isNamespaceEnabled = getIsNamespaceEnabled();
|
||||
boolean isNamespaceEnabled = getIsNamespaceEnabled(tracingContext);
|
||||
LOG.debug("getFileStatus filesystem: {} path: {} isNamespaceEnabled: {}",
|
||||
client.getFileSystem(),
|
||||
path,
|
||||
|
@ -843,14 +873,14 @@ public class AzureBlobFileSystemStore implements Closeable, ListingSupport {
|
|||
if (path.isRoot()) {
|
||||
if (isNamespaceEnabled) {
|
||||
perfInfo.registerCallee("getAclStatus");
|
||||
op = client.getAclStatus(getRelativePath(path));
|
||||
op = client.getAclStatus(getRelativePath(path), tracingContext);
|
||||
} else {
|
||||
perfInfo.registerCallee("getFilesystemProperties");
|
||||
op = client.getFilesystemProperties();
|
||||
op = client.getFilesystemProperties(tracingContext);
|
||||
}
|
||||
} else {
|
||||
perfInfo.registerCallee("getPathStatus");
|
||||
op = client.getPathStatus(getRelativePath(path), false);
|
||||
op = client.getPathStatus(getRelativePath(path), false, tracingContext);
|
||||
}
|
||||
|
||||
perfInfo.registerResult(op.getResult());
|
||||
|
@ -902,11 +932,12 @@ public class AzureBlobFileSystemStore implements Closeable, ListingSupport {
|
|||
|
||||
/**
|
||||
* @param path The list path.
|
||||
* @param tracingContext Tracks identifiers for request header
|
||||
* @return the entries in the path.
|
||||
* */
|
||||
@Override
|
||||
public FileStatus[] listStatus(final Path path) throws IOException {
|
||||
return listStatus(path, null);
|
||||
public FileStatus[] listStatus(final Path path, TracingContext tracingContext) throws IOException {
|
||||
return listStatus(path, null, tracingContext);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -917,21 +948,21 @@ public class AzureBlobFileSystemStore implements Closeable, ListingSupport {
|
|||
* Notice that if startFrom is a non-existent entry name, then the list response contains
|
||||
* all entries after this non-existent entry in lexical order:
|
||||
* listStatus(Path("/folder"), "cfile") will return "/folder/hfile" and "/folder/ifile".
|
||||
*
|
||||
* @param tracingContext Tracks identifiers for request header
|
||||
* @return the entries in the path start from "startFrom" in lexical order.
|
||||
* */
|
||||
@InterfaceStability.Unstable
|
||||
@Override
|
||||
public FileStatus[] listStatus(final Path path, final String startFrom) throws IOException {
|
||||
public FileStatus[] listStatus(final Path path, final String startFrom, TracingContext tracingContext) throws IOException {
|
||||
List<FileStatus> fileStatuses = new ArrayList<>();
|
||||
listStatus(path, startFrom, fileStatuses, true, null);
|
||||
listStatus(path, startFrom, fileStatuses, true, null, tracingContext);
|
||||
return fileStatuses.toArray(new FileStatus[fileStatuses.size()]);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String listStatus(final Path path, final String startFrom,
|
||||
List<FileStatus> fileStatuses, final boolean fetchAll,
|
||||
String continuation) throws IOException {
|
||||
String continuation, TracingContext tracingContext) throws IOException {
|
||||
final Instant startAggregate = abfsPerfTracker.getLatencyInstant();
|
||||
long countAggregate = 0;
|
||||
boolean shouldContinue = true;
|
||||
|
@ -946,7 +977,7 @@ public class AzureBlobFileSystemStore implements Closeable, ListingSupport {
|
|||
if (continuation == null || continuation.isEmpty()) {
|
||||
// generate continuation token if a valid startFrom is provided.
|
||||
if (startFrom != null && !startFrom.isEmpty()) {
|
||||
continuation = getIsNamespaceEnabled()
|
||||
continuation = getIsNamespaceEnabled(tracingContext)
|
||||
? generateContinuationTokenForXns(startFrom)
|
||||
: generateContinuationTokenForNonXns(relativePath, startFrom);
|
||||
}
|
||||
|
@ -955,7 +986,8 @@ public class AzureBlobFileSystemStore implements Closeable, ListingSupport {
|
|||
do {
|
||||
try (AbfsPerfInfo perfInfo = startTracking("listStatus", "listPath")) {
|
||||
AbfsRestOperation op = client.listPath(relativePath, false,
|
||||
abfsConfiguration.getListMaxResults(), continuation);
|
||||
abfsConfiguration.getListMaxResults(), continuation,
|
||||
tracingContext);
|
||||
perfInfo.registerResult(op.getResult());
|
||||
continuation = op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_CONTINUATION);
|
||||
ListResultSchema retrievedSchema = op.getResult().getListResultSchema();
|
||||
|
@ -1073,9 +1105,9 @@ public class AzureBlobFileSystemStore implements Closeable, ListingSupport {
|
|||
return encodedTokenBuilder.toString();
|
||||
}
|
||||
|
||||
public void setOwner(final Path path, final String owner, final String group) throws
|
||||
AzureBlobFileSystemException {
|
||||
if (!getIsNamespaceEnabled()) {
|
||||
public void setOwner(final Path path, final String owner, final String group,
|
||||
TracingContext tracingContext) throws AzureBlobFileSystemException {
|
||||
if (!getIsNamespaceEnabled(tracingContext)) {
|
||||
throw new UnsupportedOperationException(
|
||||
"This operation is only valid for storage accounts with the hierarchical namespace enabled.");
|
||||
}
|
||||
|
@ -1094,15 +1126,16 @@ public class AzureBlobFileSystemStore implements Closeable, ListingSupport {
|
|||
|
||||
final AbfsRestOperation op = client.setOwner(getRelativePath(path),
|
||||
transformedOwner,
|
||||
transformedGroup);
|
||||
transformedGroup,
|
||||
tracingContext);
|
||||
|
||||
perfInfo.registerResult(op.getResult()).registerSuccess(true);
|
||||
}
|
||||
}
|
||||
|
||||
public void setPermission(final Path path, final FsPermission permission) throws
|
||||
AzureBlobFileSystemException {
|
||||
if (!getIsNamespaceEnabled()) {
|
||||
public void setPermission(final Path path, final FsPermission permission,
|
||||
TracingContext tracingContext) throws AzureBlobFileSystemException {
|
||||
if (!getIsNamespaceEnabled(tracingContext)) {
|
||||
throw new UnsupportedOperationException(
|
||||
"This operation is only valid for storage accounts with the hierarchical namespace enabled.");
|
||||
}
|
||||
|
@ -1116,15 +1149,16 @@ public class AzureBlobFileSystemStore implements Closeable, ListingSupport {
|
|||
permission);
|
||||
|
||||
final AbfsRestOperation op = client.setPermission(getRelativePath(path),
|
||||
String.format(AbfsHttpConstants.PERMISSION_FORMAT, permission.toOctal()));
|
||||
String.format(AbfsHttpConstants.PERMISSION_FORMAT,
|
||||
permission.toOctal()), tracingContext);
|
||||
|
||||
perfInfo.registerResult(op.getResult()).registerSuccess(true);
|
||||
}
|
||||
}
|
||||
|
||||
public void modifyAclEntries(final Path path, final List<AclEntry> aclSpec) throws
|
||||
AzureBlobFileSystemException {
|
||||
if (!getIsNamespaceEnabled()) {
|
||||
public void modifyAclEntries(final Path path, final List<AclEntry> aclSpec,
|
||||
TracingContext tracingContext) throws AzureBlobFileSystemException {
|
||||
if (!getIsNamespaceEnabled(tracingContext)) {
|
||||
throw new UnsupportedOperationException(
|
||||
"This operation is only valid for storage accounts with the hierarchical namespace enabled.");
|
||||
}
|
||||
|
@ -1143,7 +1177,8 @@ public class AzureBlobFileSystemStore implements Closeable, ListingSupport {
|
|||
|
||||
String relativePath = getRelativePath(path);
|
||||
|
||||
final AbfsRestOperation op = client.getAclStatus(relativePath, useUpn);
|
||||
final AbfsRestOperation op = client
|
||||
.getAclStatus(relativePath, useUpn, tracingContext);
|
||||
perfInfoGet.registerResult(op.getResult());
|
||||
final String eTag = op.getResult().getResponseHeader(HttpHeaderConfigurations.ETAG);
|
||||
|
||||
|
@ -1154,9 +1189,9 @@ public class AzureBlobFileSystemStore implements Closeable, ListingSupport {
|
|||
perfInfoGet.registerSuccess(true).finishTracking();
|
||||
|
||||
try (AbfsPerfInfo perfInfoSet = startTracking("modifyAclEntries", "setAcl")) {
|
||||
final AbfsRestOperation setAclOp
|
||||
= client.setAcl(relativePath,
|
||||
AbfsAclHelper.serializeAclSpec(aclEntries), eTag);
|
||||
final AbfsRestOperation setAclOp = client
|
||||
.setAcl(relativePath, AbfsAclHelper.serializeAclSpec(aclEntries),
|
||||
eTag, tracingContext);
|
||||
perfInfoSet.registerResult(setAclOp.getResult())
|
||||
.registerSuccess(true)
|
||||
.registerAggregates(perfInfoGet.getTrackingStart(), GET_SET_AGGREGATE_COUNT);
|
||||
|
@ -1164,8 +1199,9 @@ public class AzureBlobFileSystemStore implements Closeable, ListingSupport {
|
|||
}
|
||||
}
|
||||
|
||||
public void removeAclEntries(final Path path, final List<AclEntry> aclSpec) throws AzureBlobFileSystemException {
|
||||
if (!getIsNamespaceEnabled()) {
|
||||
public void removeAclEntries(final Path path, final List<AclEntry> aclSpec,
|
||||
TracingContext tracingContext) throws AzureBlobFileSystemException {
|
||||
if (!getIsNamespaceEnabled(tracingContext)) {
|
||||
throw new UnsupportedOperationException(
|
||||
"This operation is only valid for storage accounts with the hierarchical namespace enabled.");
|
||||
}
|
||||
|
@ -1184,7 +1220,8 @@ public class AzureBlobFileSystemStore implements Closeable, ListingSupport {
|
|||
|
||||
String relativePath = getRelativePath(path);
|
||||
|
||||
final AbfsRestOperation op = client.getAclStatus(relativePath, isUpnFormat);
|
||||
final AbfsRestOperation op = client
|
||||
.getAclStatus(relativePath, isUpnFormat, tracingContext);
|
||||
perfInfoGet.registerResult(op.getResult());
|
||||
final String eTag = op.getResult().getResponseHeader(HttpHeaderConfigurations.ETAG);
|
||||
|
||||
|
@ -1195,9 +1232,9 @@ public class AzureBlobFileSystemStore implements Closeable, ListingSupport {
|
|||
perfInfoGet.registerSuccess(true).finishTracking();
|
||||
|
||||
try (AbfsPerfInfo perfInfoSet = startTracking("removeAclEntries", "setAcl")) {
|
||||
final AbfsRestOperation setAclOp =
|
||||
client.setAcl(relativePath,
|
||||
AbfsAclHelper.serializeAclSpec(aclEntries), eTag);
|
||||
final AbfsRestOperation setAclOp = client
|
||||
.setAcl(relativePath, AbfsAclHelper.serializeAclSpec(aclEntries),
|
||||
eTag, tracingContext);
|
||||
perfInfoSet.registerResult(setAclOp.getResult())
|
||||
.registerSuccess(true)
|
||||
.registerAggregates(perfInfoGet.getTrackingStart(), GET_SET_AGGREGATE_COUNT);
|
||||
|
@ -1205,8 +1242,9 @@ public class AzureBlobFileSystemStore implements Closeable, ListingSupport {
|
|||
}
|
||||
}
|
||||
|
||||
public void removeDefaultAcl(final Path path) throws AzureBlobFileSystemException {
|
||||
if (!getIsNamespaceEnabled()) {
|
||||
public void removeDefaultAcl(final Path path, TracingContext tracingContext)
|
||||
throws AzureBlobFileSystemException {
|
||||
if (!getIsNamespaceEnabled(tracingContext)) {
|
||||
throw new UnsupportedOperationException(
|
||||
"This operation is only valid for storage accounts with the hierarchical namespace enabled.");
|
||||
}
|
||||
|
@ -1220,7 +1258,8 @@ public class AzureBlobFileSystemStore implements Closeable, ListingSupport {
|
|||
|
||||
String relativePath = getRelativePath(path);
|
||||
|
||||
final AbfsRestOperation op = client.getAclStatus(relativePath);
|
||||
final AbfsRestOperation op = client
|
||||
.getAclStatus(relativePath, tracingContext);
|
||||
perfInfoGet.registerResult(op.getResult());
|
||||
final String eTag = op.getResult().getResponseHeader(HttpHeaderConfigurations.ETAG);
|
||||
final Map<String, String> aclEntries = AbfsAclHelper.deserializeAclSpec(op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_ACL));
|
||||
|
@ -1237,9 +1276,9 @@ public class AzureBlobFileSystemStore implements Closeable, ListingSupport {
|
|||
perfInfoGet.registerSuccess(true).finishTracking();
|
||||
|
||||
try (AbfsPerfInfo perfInfoSet = startTracking("removeDefaultAcl", "setAcl")) {
|
||||
final AbfsRestOperation setAclOp =
|
||||
client.setAcl(relativePath,
|
||||
AbfsAclHelper.serializeAclSpec(aclEntries), eTag);
|
||||
final AbfsRestOperation setAclOp = client
|
||||
.setAcl(relativePath, AbfsAclHelper.serializeAclSpec(aclEntries),
|
||||
eTag, tracingContext);
|
||||
perfInfoSet.registerResult(setAclOp.getResult())
|
||||
.registerSuccess(true)
|
||||
.registerAggregates(perfInfoGet.getTrackingStart(), GET_SET_AGGREGATE_COUNT);
|
||||
|
@ -1247,8 +1286,9 @@ public class AzureBlobFileSystemStore implements Closeable, ListingSupport {
|
|||
}
|
||||
}
|
||||
|
||||
public void removeAcl(final Path path) throws AzureBlobFileSystemException {
|
||||
if (!getIsNamespaceEnabled()) {
|
||||
public void removeAcl(final Path path, TracingContext tracingContext)
|
||||
throws AzureBlobFileSystemException {
|
||||
if (!getIsNamespaceEnabled(tracingContext)) {
|
||||
throw new UnsupportedOperationException(
|
||||
"This operation is only valid for storage accounts with the hierarchical namespace enabled.");
|
||||
}
|
||||
|
@ -1262,7 +1302,8 @@ public class AzureBlobFileSystemStore implements Closeable, ListingSupport {
|
|||
|
||||
String relativePath = getRelativePath(path);
|
||||
|
||||
final AbfsRestOperation op = client.getAclStatus(relativePath);
|
||||
final AbfsRestOperation op = client
|
||||
.getAclStatus(relativePath, tracingContext);
|
||||
perfInfoGet.registerResult(op.getResult());
|
||||
final String eTag = op.getResult().getResponseHeader(HttpHeaderConfigurations.ETAG);
|
||||
|
||||
|
@ -1276,9 +1317,9 @@ public class AzureBlobFileSystemStore implements Closeable, ListingSupport {
|
|||
perfInfoGet.registerSuccess(true).finishTracking();
|
||||
|
||||
try (AbfsPerfInfo perfInfoSet = startTracking("removeAcl", "setAcl")) {
|
||||
final AbfsRestOperation setAclOp =
|
||||
client.setAcl(relativePath,
|
||||
AbfsAclHelper.serializeAclSpec(newAclEntries), eTag);
|
||||
final AbfsRestOperation setAclOp = client
|
||||
.setAcl(relativePath, AbfsAclHelper.serializeAclSpec(newAclEntries),
|
||||
eTag, tracingContext);
|
||||
perfInfoSet.registerResult(setAclOp.getResult())
|
||||
.registerSuccess(true)
|
||||
.registerAggregates(perfInfoGet.getTrackingStart(), GET_SET_AGGREGATE_COUNT);
|
||||
|
@ -1286,8 +1327,9 @@ public class AzureBlobFileSystemStore implements Closeable, ListingSupport {
|
|||
}
|
||||
}
|
||||
|
||||
public void setAcl(final Path path, final List<AclEntry> aclSpec) throws AzureBlobFileSystemException {
|
||||
if (!getIsNamespaceEnabled()) {
|
||||
public void setAcl(final Path path, final List<AclEntry> aclSpec,
|
||||
TracingContext tracingContext) throws AzureBlobFileSystemException {
|
||||
if (!getIsNamespaceEnabled(tracingContext)) {
|
||||
throw new UnsupportedOperationException(
|
||||
"This operation is only valid for storage accounts with the hierarchical namespace enabled.");
|
||||
}
|
||||
|
@ -1306,7 +1348,8 @@ public class AzureBlobFileSystemStore implements Closeable, ListingSupport {
|
|||
|
||||
String relativePath = getRelativePath(path);
|
||||
|
||||
final AbfsRestOperation op = client.getAclStatus(relativePath, isUpnFormat);
|
||||
final AbfsRestOperation op = client
|
||||
.getAclStatus(relativePath, isUpnFormat, tracingContext);
|
||||
perfInfoGet.registerResult(op.getResult());
|
||||
final String eTag = op.getResult().getResponseHeader(HttpHeaderConfigurations.ETAG);
|
||||
|
||||
|
@ -1319,7 +1362,7 @@ public class AzureBlobFileSystemStore implements Closeable, ListingSupport {
|
|||
try (AbfsPerfInfo perfInfoSet = startTracking("setAcl", "setAcl")) {
|
||||
final AbfsRestOperation setAclOp =
|
||||
client.setAcl(relativePath,
|
||||
AbfsAclHelper.serializeAclSpec(aclEntries), eTag);
|
||||
AbfsAclHelper.serializeAclSpec(aclEntries), eTag, tracingContext);
|
||||
perfInfoSet.registerResult(setAclOp.getResult())
|
||||
.registerSuccess(true)
|
||||
.registerAggregates(perfInfoGet.getTrackingStart(), GET_SET_AGGREGATE_COUNT);
|
||||
|
@ -1327,8 +1370,9 @@ public class AzureBlobFileSystemStore implements Closeable, ListingSupport {
|
|||
}
|
||||
}
|
||||
|
||||
public AclStatus getAclStatus(final Path path) throws IOException {
|
||||
if (!getIsNamespaceEnabled()) {
|
||||
public AclStatus getAclStatus(final Path path, TracingContext tracingContext)
|
||||
throws IOException {
|
||||
if (!getIsNamespaceEnabled(tracingContext)) {
|
||||
throw new UnsupportedOperationException(
|
||||
"This operation is only valid for storage accounts with the hierarchical namespace enabled.");
|
||||
}
|
||||
|
@ -1340,7 +1384,8 @@ public class AzureBlobFileSystemStore implements Closeable, ListingSupport {
|
|||
client.getFileSystem(),
|
||||
path);
|
||||
|
||||
AbfsRestOperation op = client.getAclStatus(getRelativePath(path));
|
||||
AbfsRestOperation op = client
|
||||
.getAclStatus(getRelativePath(path), tracingContext);
|
||||
AbfsHttpOperation result = op.getResult();
|
||||
perfInfo.registerResult(result);
|
||||
|
||||
|
@ -1373,19 +1418,19 @@ public class AzureBlobFileSystemStore implements Closeable, ListingSupport {
|
|||
}
|
||||
}
|
||||
|
||||
public void access(final Path path, final FsAction mode)
|
||||
throws AzureBlobFileSystemException {
|
||||
public void access(final Path path, final FsAction mode,
|
||||
TracingContext tracingContext) throws AzureBlobFileSystemException {
|
||||
LOG.debug("access for filesystem: {}, path: {}, mode: {}",
|
||||
this.client.getFileSystem(), path, mode);
|
||||
if (!this.abfsConfiguration.isCheckAccessEnabled()
|
||||
|| !getIsNamespaceEnabled()) {
|
||||
|| !getIsNamespaceEnabled(tracingContext)) {
|
||||
LOG.debug("Returning; either check access is not enabled or the account"
|
||||
+ " used is not namespace enabled");
|
||||
return;
|
||||
}
|
||||
try (AbfsPerfInfo perfInfo = startTracking("access", "checkAccess")) {
|
||||
final AbfsRestOperation op = this.client
|
||||
.checkAccess(getRelativePath(path), mode.SYMBOL);
|
||||
.checkAccess(getRelativePath(path), mode.SYMBOL, tracingContext);
|
||||
perfInfo.registerResult(op.getResult()).registerSuccess(true);
|
||||
}
|
||||
}
|
||||
|
@ -1698,13 +1743,13 @@ public class AzureBlobFileSystemStore implements Closeable, ListingSupport {
|
|||
this.azureInfiniteLeaseDirSet.remove("");
|
||||
}
|
||||
|
||||
private AbfsLease maybeCreateLease(String relativePath)
|
||||
private AbfsLease maybeCreateLease(String relativePath, TracingContext tracingContext)
|
||||
throws AzureBlobFileSystemException {
|
||||
boolean enableInfiniteLease = isInfiniteLeaseKey(relativePath);
|
||||
if (!enableInfiniteLease) {
|
||||
return null;
|
||||
}
|
||||
AbfsLease lease = new AbfsLease(client, relativePath);
|
||||
AbfsLease lease = new AbfsLease(client, relativePath, tracingContext);
|
||||
leaseRefs.put(lease, null);
|
||||
return lease;
|
||||
}
|
||||
|
|
|
@ -109,6 +109,12 @@ public final class ConfigurationKeys {
|
|||
* Default value of this config is true. **/
|
||||
public static final String FS_AZURE_DISABLE_OUTPUTSTREAM_FLUSH = "fs.azure.disable.outputstream.flush";
|
||||
public static final String FS_AZURE_USER_AGENT_PREFIX_KEY = "fs.azure.user.agent.prefix";
|
||||
/**
|
||||
* The client correlation ID provided over config that will be added to
|
||||
* x-ms-client-request-Id header. Defaults to empty string if the length and
|
||||
* character constraints are not satisfied. **/
|
||||
public static final String FS_AZURE_CLIENT_CORRELATIONID = "fs.azure.client.correlationid";
|
||||
public static final String FS_AZURE_TRACINGHEADER_FORMAT = "fs.azure.tracingheader.format";
|
||||
public static final String FS_AZURE_CLUSTER_NAME = "fs.azure.cluster.name";
|
||||
public static final String FS_AZURE_CLUSTER_TYPE = "fs.azure.cluster.type";
|
||||
public static final String FS_AZURE_SSL_CHANNEL_MODE_KEY = "fs.azure.ssl.channel.mode";
|
||||
|
|
|
@ -0,0 +1,60 @@
|
|||
/**
|
||||
* 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;
|
||||
|
||||
public enum FSOperationType {
|
||||
ACCESS("AS"),
|
||||
APPEND("AP"),
|
||||
BREAK_LEASE("BL"),
|
||||
CREATE("CR"),
|
||||
CREATE_FILESYSTEM("CF"),
|
||||
CREATE_NON_RECURSIVE("CN"),
|
||||
DELETE("DL"),
|
||||
GET_ACL_STATUS("GA"),
|
||||
GET_ATTR("GR"),
|
||||
GET_FILESTATUS("GF"),
|
||||
LISTSTATUS("LS"),
|
||||
MKDIR("MK"),
|
||||
MODIFY_ACL("MA"),
|
||||
OPEN("OP"),
|
||||
HAS_PATH_CAPABILITY("PC"),
|
||||
SET_PERMISSION("SP"),
|
||||
READ("RE"),
|
||||
RELEASE_LEASE("RL"),
|
||||
REMOVE_ACL("RA"),
|
||||
REMOVE_ACL_ENTRIES("RT"),
|
||||
REMOVE_DEFAULT_ACL("RD"),
|
||||
RENAME("RN"),
|
||||
SET_ATTR("SR"),
|
||||
SET_OWNER("SO"),
|
||||
SET_ACL("SA"),
|
||||
TEST_OP("TS"),
|
||||
WRITE("WR");
|
||||
|
||||
private final String opCode;
|
||||
|
||||
FSOperationType(String opCode) {
|
||||
this.opCode = opCode;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return opCode;
|
||||
}
|
||||
}
|
|
@ -111,6 +111,7 @@ public final class FileSystemConfigurations {
|
|||
public static final boolean DEFAULT_DELETE_CONSIDERED_IDEMPOTENT = true;
|
||||
public static final int DEFAULT_CLOCK_SKEW_WITH_SERVER_IN_MS = 5 * 60 * 1000; // 5 mins
|
||||
|
||||
public static final int STREAM_ID_LEN = 12;
|
||||
public static final boolean DEFAULT_ENABLE_ABFS_LIST_ITERATOR = true;
|
||||
|
||||
private FileSystemConfigurations() {}
|
||||
|
|
|
@ -63,6 +63,7 @@ import org.apache.hadoop.fs.azurebfs.AbfsConfiguration;
|
|||
import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters;
|
||||
import org.apache.hadoop.fs.azurebfs.oauth2.AccessTokenProvider;
|
||||
import org.apache.hadoop.fs.azurebfs.utils.DateTimeUtils;
|
||||
import org.apache.hadoop.fs.azurebfs.utils.TracingContext;
|
||||
import org.apache.hadoop.io.IOUtils;
|
||||
import org.apache.hadoop.security.ssl.DelegatingSSLSocketFactory;
|
||||
import org.apache.hadoop.util.concurrent.HadoopExecutors;
|
||||
|
@ -237,7 +238,7 @@ public class AbfsClient implements Closeable {
|
|||
return abfsUriQueryBuilder;
|
||||
}
|
||||
|
||||
public AbfsRestOperation createFilesystem() throws AzureBlobFileSystemException {
|
||||
public AbfsRestOperation createFilesystem(TracingContext tracingContext) throws AzureBlobFileSystemException {
|
||||
final List<AbfsHttpHeader> requestHeaders = createDefaultHeaders();
|
||||
|
||||
final AbfsUriQueryBuilder abfsUriQueryBuilder = new AbfsUriQueryBuilder();
|
||||
|
@ -250,11 +251,11 @@ public class AbfsClient implements Closeable {
|
|||
HTTP_METHOD_PUT,
|
||||
url,
|
||||
requestHeaders);
|
||||
op.execute();
|
||||
op.execute(tracingContext);
|
||||
return op;
|
||||
}
|
||||
|
||||
public AbfsRestOperation setFilesystemProperties(final String properties) throws AzureBlobFileSystemException {
|
||||
public AbfsRestOperation setFilesystemProperties(final String properties, TracingContext tracingContext) 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.
|
||||
|
@ -274,12 +275,13 @@ public class AbfsClient implements Closeable {
|
|||
HTTP_METHOD_PUT,
|
||||
url,
|
||||
requestHeaders);
|
||||
op.execute();
|
||||
op.execute(tracingContext);
|
||||
return op;
|
||||
}
|
||||
|
||||
public AbfsRestOperation listPath(final String relativePath, final boolean recursive, final int listMaxResults,
|
||||
final String continuation) throws AzureBlobFileSystemException {
|
||||
final String continuation, TracingContext tracingContext)
|
||||
throws AzureBlobFileSystemException {
|
||||
final List<AbfsHttpHeader> requestHeaders = createDefaultHeaders();
|
||||
|
||||
final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder();
|
||||
|
@ -298,11 +300,11 @@ public class AbfsClient implements Closeable {
|
|||
HTTP_METHOD_GET,
|
||||
url,
|
||||
requestHeaders);
|
||||
op.execute();
|
||||
op.execute(tracingContext);
|
||||
return op;
|
||||
}
|
||||
|
||||
public AbfsRestOperation getFilesystemProperties() throws AzureBlobFileSystemException {
|
||||
public AbfsRestOperation getFilesystemProperties(TracingContext tracingContext) throws AzureBlobFileSystemException {
|
||||
final List<AbfsHttpHeader> requestHeaders = createDefaultHeaders();
|
||||
|
||||
final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder();
|
||||
|
@ -315,11 +317,11 @@ public class AbfsClient implements Closeable {
|
|||
HTTP_METHOD_HEAD,
|
||||
url,
|
||||
requestHeaders);
|
||||
op.execute();
|
||||
op.execute(tracingContext);
|
||||
return op;
|
||||
}
|
||||
|
||||
public AbfsRestOperation deleteFilesystem() throws AzureBlobFileSystemException {
|
||||
public AbfsRestOperation deleteFilesystem(TracingContext tracingContext) throws AzureBlobFileSystemException {
|
||||
final List<AbfsHttpHeader> requestHeaders = createDefaultHeaders();
|
||||
|
||||
final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder();
|
||||
|
@ -332,13 +334,14 @@ public class AbfsClient implements Closeable {
|
|||
HTTP_METHOD_DELETE,
|
||||
url,
|
||||
requestHeaders);
|
||||
op.execute();
|
||||
op.execute(tracingContext);
|
||||
return op;
|
||||
}
|
||||
|
||||
public AbfsRestOperation createPath(final String path, final boolean isFile, final boolean overwrite,
|
||||
final String permission, final String umask,
|
||||
final boolean isAppendBlob, final String eTag) throws AzureBlobFileSystemException {
|
||||
final boolean isAppendBlob, final String eTag,
|
||||
TracingContext tracingContext) throws AzureBlobFileSystemException {
|
||||
final List<AbfsHttpHeader> requestHeaders = createDefaultHeaders();
|
||||
if (isFile) {
|
||||
addCustomerProvidedKeyHeaders(requestHeaders);
|
||||
|
@ -378,7 +381,7 @@ public class AbfsClient implements Closeable {
|
|||
url,
|
||||
requestHeaders);
|
||||
try {
|
||||
op.execute();
|
||||
op.execute(tracingContext);
|
||||
} catch (AzureBlobFileSystemException ex) {
|
||||
if (!isFile && op.getResult().getStatusCode() == HttpURLConnection.HTTP_CONFLICT) {
|
||||
String existingResource =
|
||||
|
@ -392,7 +395,7 @@ public class AbfsClient implements Closeable {
|
|||
return op;
|
||||
}
|
||||
|
||||
public AbfsRestOperation acquireLease(final String path, int duration) throws AzureBlobFileSystemException {
|
||||
public AbfsRestOperation acquireLease(final String path, int duration, TracingContext tracingContext) throws AzureBlobFileSystemException {
|
||||
final List<AbfsHttpHeader> requestHeaders = createDefaultHeaders();
|
||||
|
||||
requestHeaders.add(new AbfsHttpHeader(X_MS_LEASE_ACTION, ACQUIRE_LEASE_ACTION));
|
||||
|
@ -408,11 +411,12 @@ public class AbfsClient implements Closeable {
|
|||
HTTP_METHOD_POST,
|
||||
url,
|
||||
requestHeaders);
|
||||
op.execute();
|
||||
op.execute(tracingContext);
|
||||
return op;
|
||||
}
|
||||
|
||||
public AbfsRestOperation renewLease(final String path, final String leaseId) throws AzureBlobFileSystemException {
|
||||
public AbfsRestOperation renewLease(final String path, final String leaseId,
|
||||
TracingContext tracingContext) throws AzureBlobFileSystemException {
|
||||
final List<AbfsHttpHeader> requestHeaders = createDefaultHeaders();
|
||||
|
||||
requestHeaders.add(new AbfsHttpHeader(X_MS_LEASE_ACTION, RENEW_LEASE_ACTION));
|
||||
|
@ -427,11 +431,12 @@ public class AbfsClient implements Closeable {
|
|||
HTTP_METHOD_POST,
|
||||
url,
|
||||
requestHeaders);
|
||||
op.execute();
|
||||
op.execute(tracingContext);
|
||||
return op;
|
||||
}
|
||||
|
||||
public AbfsRestOperation releaseLease(final String path, final String leaseId) throws AzureBlobFileSystemException {
|
||||
public AbfsRestOperation releaseLease(final String path,
|
||||
final String leaseId, TracingContext tracingContext) throws AzureBlobFileSystemException {
|
||||
final List<AbfsHttpHeader> requestHeaders = createDefaultHeaders();
|
||||
|
||||
requestHeaders.add(new AbfsHttpHeader(X_MS_LEASE_ACTION, RELEASE_LEASE_ACTION));
|
||||
|
@ -446,11 +451,12 @@ public class AbfsClient implements Closeable {
|
|||
HTTP_METHOD_POST,
|
||||
url,
|
||||
requestHeaders);
|
||||
op.execute();
|
||||
op.execute(tracingContext);
|
||||
return op;
|
||||
}
|
||||
|
||||
public AbfsRestOperation breakLease(final String path) throws AzureBlobFileSystemException {
|
||||
public AbfsRestOperation breakLease(final String path,
|
||||
TracingContext tracingContext) throws AzureBlobFileSystemException {
|
||||
final List<AbfsHttpHeader> requestHeaders = createDefaultHeaders();
|
||||
|
||||
requestHeaders.add(new AbfsHttpHeader(X_MS_LEASE_ACTION, BREAK_LEASE_ACTION));
|
||||
|
@ -465,12 +471,13 @@ public class AbfsClient implements Closeable {
|
|||
HTTP_METHOD_POST,
|
||||
url,
|
||||
requestHeaders);
|
||||
op.execute();
|
||||
op.execute(tracingContext);
|
||||
return op;
|
||||
}
|
||||
|
||||
public AbfsRestOperation renamePath(String source, final String destination, final String continuation)
|
||||
throws AzureBlobFileSystemException {
|
||||
public AbfsRestOperation renamePath(String source, final String destination,
|
||||
final String continuation, TracingContext tracingContext)
|
||||
throws AzureBlobFileSystemException {
|
||||
final List<AbfsHttpHeader> requestHeaders = createDefaultHeaders();
|
||||
|
||||
String encodedRenameSource = urlEncode(FORWARD_SLASH + this.getFileSystem() + source);
|
||||
|
@ -497,10 +504,10 @@ public class AbfsClient implements Closeable {
|
|||
requestHeaders);
|
||||
Instant renameRequestStartTime = Instant.now();
|
||||
try {
|
||||
op.execute();
|
||||
op.execute(tracingContext);
|
||||
} catch (AzureBlobFileSystemException e) {
|
||||
final AbfsRestOperation idempotencyOp = renameIdempotencyCheckOp(
|
||||
renameRequestStartTime, op, destination);
|
||||
renameRequestStartTime, op, destination, tracingContext);
|
||||
if (idempotencyOp.getResult().getStatusCode()
|
||||
== op.getResult().getStatusCode()) {
|
||||
// idempotency did not return different result
|
||||
|
@ -525,13 +532,15 @@ public class AbfsClient implements Closeable {
|
|||
* @param renameRequestStartTime startTime for the rename request
|
||||
* @param op Rename request REST operation response
|
||||
* @param destination rename destination path
|
||||
* @param tracingContext Tracks identifiers for request header
|
||||
* @return REST operation response post idempotency check
|
||||
* @throws AzureBlobFileSystemException if GetFileStatus hits any exception
|
||||
*/
|
||||
public AbfsRestOperation renameIdempotencyCheckOp(
|
||||
final Instant renameRequestStartTime,
|
||||
final AbfsRestOperation op,
|
||||
final String destination) throws AzureBlobFileSystemException {
|
||||
final String destination,
|
||||
TracingContext tracingContext) throws AzureBlobFileSystemException {
|
||||
if ((op.isARetriedRequest())
|
||||
&& (op.getResult().getStatusCode() == HttpURLConnection.HTTP_NOT_FOUND)) {
|
||||
// Server has returned HTTP 404, which means rename source no longer
|
||||
|
@ -540,7 +549,7 @@ public class AbfsClient implements Closeable {
|
|||
|
||||
try {
|
||||
final AbfsRestOperation destStatusOp = getPathStatus(destination,
|
||||
false);
|
||||
false, tracingContext);
|
||||
if (destStatusOp.getResult().getStatusCode()
|
||||
== HttpURLConnection.HTTP_OK) {
|
||||
String lmt = destStatusOp.getResult().getResponseHeader(
|
||||
|
@ -561,7 +570,7 @@ public class AbfsClient implements Closeable {
|
|||
}
|
||||
|
||||
public AbfsRestOperation append(final String path, final byte[] buffer,
|
||||
AppendRequestParameters reqParams, final String cachedSasToken)
|
||||
AppendRequestParameters reqParams, final String cachedSasToken, TracingContext tracingContext)
|
||||
throws AzureBlobFileSystemException {
|
||||
final List<AbfsHttpHeader> requestHeaders = createDefaultHeaders();
|
||||
addCustomerProvidedKeyHeaders(requestHeaders);
|
||||
|
@ -601,11 +610,11 @@ public class AbfsClient implements Closeable {
|
|||
reqParams.getLength(),
|
||||
sasTokenForReuse);
|
||||
try {
|
||||
op.execute();
|
||||
op.execute(tracingContext);
|
||||
} catch (AzureBlobFileSystemException e) {
|
||||
if (reqParams.isAppendBlob()
|
||||
&& appendSuccessCheckOp(op, path,
|
||||
(reqParams.getPosition() + reqParams.getLength()))) {
|
||||
(reqParams.getPosition() + reqParams.getLength()), tracingContext)) {
|
||||
final AbfsRestOperation successOp = new AbfsRestOperation(
|
||||
AbfsRestOperationType.Append,
|
||||
this,
|
||||
|
@ -631,10 +640,10 @@ public class AbfsClient implements Closeable {
|
|||
// Hence, we pass/succeed the appendblob append call
|
||||
// in case we are doing a retry after checking the length of the file
|
||||
public boolean appendSuccessCheckOp(AbfsRestOperation op, final String path,
|
||||
final long length) throws AzureBlobFileSystemException {
|
||||
final long length, TracingContext tracingContext) throws AzureBlobFileSystemException {
|
||||
if ((op.isARetriedRequest())
|
||||
&& (op.getResult().getStatusCode() == HttpURLConnection.HTTP_BAD_REQUEST)) {
|
||||
final AbfsRestOperation destStatusOp = getPathStatus(path, false);
|
||||
final AbfsRestOperation destStatusOp = getPathStatus(path, false, tracingContext);
|
||||
if (destStatusOp.getResult().getStatusCode() == HttpURLConnection.HTTP_OK) {
|
||||
String fileLength = destStatusOp.getResult().getResponseHeader(
|
||||
HttpHeaderConfigurations.CONTENT_LENGTH);
|
||||
|
@ -647,9 +656,10 @@ public class AbfsClient implements Closeable {
|
|||
return false;
|
||||
}
|
||||
|
||||
public AbfsRestOperation flush(final String path, final long position, boolean retainUncommittedData,
|
||||
boolean isClose, final String cachedSasToken, final String leaseId)
|
||||
throws AzureBlobFileSystemException {
|
||||
public AbfsRestOperation flush(final String path, final long position,
|
||||
boolean retainUncommittedData, boolean isClose,
|
||||
final String cachedSasToken, final String leaseId,
|
||||
TracingContext tracingContext) throws AzureBlobFileSystemException {
|
||||
final List<AbfsHttpHeader> requestHeaders = createDefaultHeaders();
|
||||
addCustomerProvidedKeyHeaders(requestHeaders);
|
||||
// JDK7 does not support PATCH, so to workaround the issue we will use
|
||||
|
@ -676,11 +686,12 @@ public class AbfsClient implements Closeable {
|
|||
HTTP_METHOD_PUT,
|
||||
url,
|
||||
requestHeaders, sasTokenForReuse);
|
||||
op.execute();
|
||||
op.execute(tracingContext);
|
||||
return op;
|
||||
}
|
||||
|
||||
public AbfsRestOperation setPathProperties(final String path, final String properties)
|
||||
public AbfsRestOperation setPathProperties(final String path, final String properties,
|
||||
TracingContext tracingContext)
|
||||
throws AzureBlobFileSystemException {
|
||||
final List<AbfsHttpHeader> requestHeaders = createDefaultHeaders();
|
||||
addCustomerProvidedKeyHeaders(requestHeaders);
|
||||
|
@ -702,11 +713,12 @@ public class AbfsClient implements Closeable {
|
|||
HTTP_METHOD_PUT,
|
||||
url,
|
||||
requestHeaders);
|
||||
op.execute();
|
||||
op.execute(tracingContext);
|
||||
return op;
|
||||
}
|
||||
|
||||
public AbfsRestOperation getPathStatus(final String path, final boolean includeProperties) throws AzureBlobFileSystemException {
|
||||
public AbfsRestOperation getPathStatus(final String path, final boolean includeProperties,
|
||||
TracingContext tracingContext) throws AzureBlobFileSystemException {
|
||||
final List<AbfsHttpHeader> requestHeaders = createDefaultHeaders();
|
||||
|
||||
final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder();
|
||||
|
@ -730,12 +742,13 @@ public class AbfsClient implements Closeable {
|
|||
HTTP_METHOD_HEAD,
|
||||
url,
|
||||
requestHeaders);
|
||||
op.execute();
|
||||
op.execute(tracingContext);
|
||||
return op;
|
||||
}
|
||||
|
||||
public AbfsRestOperation read(final String path, final long position, final byte[] buffer, final int bufferOffset,
|
||||
final int bufferLength, final String eTag, String cachedSasToken) throws AzureBlobFileSystemException {
|
||||
final int bufferLength, final String eTag, String cachedSasToken,
|
||||
TracingContext tracingContext) throws AzureBlobFileSystemException {
|
||||
final List<AbfsHttpHeader> requestHeaders = createDefaultHeaders();
|
||||
addCustomerProvidedKeyHeaders(requestHeaders);
|
||||
requestHeaders.add(new AbfsHttpHeader(RANGE,
|
||||
|
@ -748,7 +761,6 @@ public class AbfsClient implements Closeable {
|
|||
abfsUriQueryBuilder, cachedSasToken);
|
||||
|
||||
final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString());
|
||||
|
||||
final AbfsRestOperation op = new AbfsRestOperation(
|
||||
AbfsRestOperationType.ReadFile,
|
||||
this,
|
||||
|
@ -758,12 +770,13 @@ public class AbfsClient implements Closeable {
|
|||
buffer,
|
||||
bufferOffset,
|
||||
bufferLength, sasTokenForReuse);
|
||||
op.execute();
|
||||
op.execute(tracingContext);
|
||||
|
||||
return op;
|
||||
}
|
||||
|
||||
public AbfsRestOperation deletePath(final String path, final boolean recursive, final String continuation)
|
||||
public AbfsRestOperation deletePath(final String path, final boolean recursive, final String continuation,
|
||||
TracingContext tracingContext)
|
||||
throws AzureBlobFileSystemException {
|
||||
final List<AbfsHttpHeader> requestHeaders = createDefaultHeaders();
|
||||
|
||||
|
@ -781,7 +794,7 @@ public class AbfsClient implements Closeable {
|
|||
url,
|
||||
requestHeaders);
|
||||
try {
|
||||
op.execute();
|
||||
op.execute(tracingContext);
|
||||
} catch (AzureBlobFileSystemException e) {
|
||||
final AbfsRestOperation idempotencyOp = deleteIdempotencyCheckOp(op);
|
||||
if (idempotencyOp.getResult().getStatusCode()
|
||||
|
@ -832,7 +845,8 @@ public class AbfsClient implements Closeable {
|
|||
return op;
|
||||
}
|
||||
|
||||
public AbfsRestOperation setOwner(final String path, final String owner, final String group)
|
||||
public AbfsRestOperation setOwner(final String path, final String owner, final String group,
|
||||
TracingContext tracingContext)
|
||||
throws AzureBlobFileSystemException {
|
||||
final List<AbfsHttpHeader> requestHeaders = createDefaultHeaders();
|
||||
// JDK7 does not support PATCH, so to workaround the issue we will use
|
||||
|
@ -858,11 +872,12 @@ public class AbfsClient implements Closeable {
|
|||
AbfsHttpConstants.HTTP_METHOD_PUT,
|
||||
url,
|
||||
requestHeaders);
|
||||
op.execute();
|
||||
op.execute(tracingContext);
|
||||
return op;
|
||||
}
|
||||
|
||||
public AbfsRestOperation setPermission(final String path, final String permission)
|
||||
public AbfsRestOperation setPermission(final String path, final String permission,
|
||||
TracingContext tracingContext)
|
||||
throws AzureBlobFileSystemException {
|
||||
final List<AbfsHttpHeader> requestHeaders = createDefaultHeaders();
|
||||
// JDK7 does not support PATCH, so to workaround the issue we will use
|
||||
|
@ -883,15 +898,17 @@ public class AbfsClient implements Closeable {
|
|||
AbfsHttpConstants.HTTP_METHOD_PUT,
|
||||
url,
|
||||
requestHeaders);
|
||||
op.execute();
|
||||
op.execute(tracingContext);
|
||||
return op;
|
||||
}
|
||||
|
||||
public AbfsRestOperation setAcl(final String path, final String aclSpecString) throws AzureBlobFileSystemException {
|
||||
return setAcl(path, aclSpecString, AbfsHttpConstants.EMPTY_STRING);
|
||||
public AbfsRestOperation setAcl(final String path, final String aclSpecString,
|
||||
TracingContext tracingContext) throws AzureBlobFileSystemException {
|
||||
return setAcl(path, aclSpecString, AbfsHttpConstants.EMPTY_STRING, tracingContext);
|
||||
}
|
||||
|
||||
public AbfsRestOperation setAcl(final String path, final String aclSpecString, final String eTag)
|
||||
public AbfsRestOperation setAcl(final String path, final String aclSpecString, final String eTag,
|
||||
TracingContext tracingContext)
|
||||
throws AzureBlobFileSystemException {
|
||||
final List<AbfsHttpHeader> requestHeaders = createDefaultHeaders();
|
||||
// JDK7 does not support PATCH, so to workaround the issue we will use
|
||||
|
@ -916,15 +933,17 @@ public class AbfsClient implements Closeable {
|
|||
AbfsHttpConstants.HTTP_METHOD_PUT,
|
||||
url,
|
||||
requestHeaders);
|
||||
op.execute();
|
||||
op.execute(tracingContext);
|
||||
return op;
|
||||
}
|
||||
|
||||
public AbfsRestOperation getAclStatus(final String path) throws AzureBlobFileSystemException {
|
||||
return getAclStatus(path, abfsConfiguration.isUpnUsed());
|
||||
public AbfsRestOperation getAclStatus(final String path, TracingContext tracingContext)
|
||||
throws AzureBlobFileSystemException {
|
||||
return getAclStatus(path, abfsConfiguration.isUpnUsed(), tracingContext);
|
||||
}
|
||||
|
||||
public AbfsRestOperation getAclStatus(final String path, final boolean useUPN) throws AzureBlobFileSystemException {
|
||||
public AbfsRestOperation getAclStatus(final String path, final boolean useUPN,
|
||||
TracingContext tracingContext) throws AzureBlobFileSystemException {
|
||||
final List<AbfsHttpHeader> requestHeaders = createDefaultHeaders();
|
||||
|
||||
final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder();
|
||||
|
@ -939,7 +958,7 @@ public class AbfsClient implements Closeable {
|
|||
AbfsHttpConstants.HTTP_METHOD_HEAD,
|
||||
url,
|
||||
requestHeaders);
|
||||
op.execute();
|
||||
op.execute(tracingContext);
|
||||
return op;
|
||||
}
|
||||
|
||||
|
@ -949,10 +968,11 @@ public class AbfsClient implements Closeable {
|
|||
*
|
||||
* @param path Path for which access check needs to be performed
|
||||
* @param rwx The permission to be checked on the path
|
||||
* @param tracingContext Tracks identifiers for request header
|
||||
* @return The {@link AbfsRestOperation} object for the operation
|
||||
* @throws AzureBlobFileSystemException in case of bad requests
|
||||
*/
|
||||
public AbfsRestOperation checkAccess(String path, String rwx)
|
||||
public AbfsRestOperation checkAccess(String path, String rwx, TracingContext tracingContext)
|
||||
throws AzureBlobFileSystemException {
|
||||
AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder();
|
||||
abfsUriQueryBuilder.addQuery(QUERY_PARAM_ACTION, CHECK_ACCESS);
|
||||
|
@ -962,7 +982,7 @@ public class AbfsClient implements Closeable {
|
|||
AbfsRestOperation op = new AbfsRestOperation(
|
||||
AbfsRestOperationType.CheckAccess, this,
|
||||
AbfsHttpConstants.HTTP_METHOD_HEAD, url, createDefaultHeaders());
|
||||
op.execute();
|
||||
op.execute(tracingContext);
|
||||
return op;
|
||||
}
|
||||
|
||||
|
|
|
@ -26,7 +26,6 @@ import java.net.HttpURLConnection;
|
|||
import java.net.URL;
|
||||
import java.net.URLEncoder;
|
||||
import java.util.List;
|
||||
import java.util.UUID;
|
||||
|
||||
import javax.net.ssl.HttpsURLConnection;
|
||||
import javax.net.ssl.SSLSocketFactory;
|
||||
|
@ -71,7 +70,6 @@ public class AbfsHttpOperation implements AbfsPerfLoggable {
|
|||
private String statusDescription;
|
||||
private String storageErrorCode = "";
|
||||
private String storageErrorMessage = "";
|
||||
private String clientRequestId = "";
|
||||
private String requestId = "";
|
||||
private String expectedAppendPos = "";
|
||||
private ListResultSchema listResultSchema = null;
|
||||
|
@ -139,7 +137,8 @@ public class AbfsHttpOperation implements AbfsPerfLoggable {
|
|||
}
|
||||
|
||||
public String getClientRequestId() {
|
||||
return clientRequestId;
|
||||
return this.connection
|
||||
.getRequestProperty(HttpHeaderConfigurations.X_MS_CLIENT_REQUEST_ID);
|
||||
}
|
||||
|
||||
public String getExpectedAppendPos() {
|
||||
|
@ -176,7 +175,7 @@ public class AbfsHttpOperation implements AbfsPerfLoggable {
|
|||
sb.append(",");
|
||||
sb.append(expectedAppendPos);
|
||||
sb.append(",cid=");
|
||||
sb.append(clientRequestId);
|
||||
sb.append(getClientRequestId());
|
||||
sb.append(",rid=");
|
||||
sb.append(requestId);
|
||||
if (isTraceEnabled) {
|
||||
|
@ -207,7 +206,7 @@ public class AbfsHttpOperation implements AbfsPerfLoggable {
|
|||
.append(" e=")
|
||||
.append(storageErrorCode)
|
||||
.append(" ci=")
|
||||
.append(clientRequestId)
|
||||
.append(getClientRequestId())
|
||||
.append(" ri=")
|
||||
.append(requestId);
|
||||
|
||||
|
@ -246,7 +245,6 @@ public class AbfsHttpOperation implements AbfsPerfLoggable {
|
|||
this.isTraceEnabled = LOG.isTraceEnabled();
|
||||
this.url = url;
|
||||
this.method = method;
|
||||
this.clientRequestId = UUID.randomUUID().toString();
|
||||
|
||||
this.connection = openConnection();
|
||||
if (this.connection instanceof HttpsURLConnection) {
|
||||
|
@ -265,8 +263,6 @@ public class AbfsHttpOperation implements AbfsPerfLoggable {
|
|||
for (AbfsHttpHeader header : requestHeaders) {
|
||||
this.connection.setRequestProperty(header.getName(), header.getValue());
|
||||
}
|
||||
|
||||
this.connection.setRequestProperty(HttpHeaderConfigurations.X_MS_CLIENT_REQUEST_ID, clientRequestId);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -404,6 +400,9 @@ public class AbfsHttpOperation implements AbfsPerfLoggable {
|
|||
}
|
||||
}
|
||||
|
||||
public void setRequestProperty(String key, String value) {
|
||||
this.connection.setRequestProperty(key, value);
|
||||
}
|
||||
|
||||
/**
|
||||
* Open the HTTP connection.
|
||||
|
|
|
@ -22,9 +22,11 @@ import java.io.EOFException;
|
|||
import java.io.FileNotFoundException;
|
||||
import java.io.IOException;
|
||||
import java.net.HttpURLConnection;
|
||||
import java.util.UUID;
|
||||
|
||||
import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
|
||||
import org.apache.commons.lang3.StringUtils;
|
||||
import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting;
|
||||
import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
|
||||
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
@ -34,9 +36,12 @@ import org.apache.hadoop.fs.FSExceptionMessages;
|
|||
import org.apache.hadoop.fs.FSInputStream;
|
||||
import org.apache.hadoop.fs.FileSystem.Statistics;
|
||||
import org.apache.hadoop.fs.StreamCapabilities;
|
||||
import org.apache.hadoop.fs.azurebfs.constants.FSOperationType;
|
||||
import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException;
|
||||
import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException;
|
||||
import org.apache.hadoop.fs.azurebfs.utils.CachedSASToken;
|
||||
import org.apache.hadoop.fs.azurebfs.utils.Listener;
|
||||
import org.apache.hadoop.fs.azurebfs.utils.TracingContext;
|
||||
import org.apache.hadoop.fs.statistics.IOStatistics;
|
||||
import org.apache.hadoop.fs.statistics.IOStatisticsSource;
|
||||
|
||||
|
@ -44,6 +49,7 @@ import static java.lang.Math.max;
|
|||
import static java.lang.Math.min;
|
||||
|
||||
import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.ONE_KB;
|
||||
import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.STREAM_ID_LEN;
|
||||
import static org.apache.hadoop.util.StringUtils.toLowerCase;
|
||||
|
||||
/**
|
||||
|
@ -66,6 +72,7 @@ public class AbfsInputStream extends FSInputStream implements CanUnbuffer,
|
|||
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 final String inputStreamId;
|
||||
private final boolean alwaysReadBufferSize;
|
||||
/*
|
||||
* By default the pread API will do a seek + read as in FSInputStream.
|
||||
|
@ -87,6 +94,7 @@ public class AbfsInputStream extends FSInputStream implements CanUnbuffer,
|
|||
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;
|
||||
private TracingContext tracingContext;
|
||||
|
||||
// Optimisations modify the pointer fields.
|
||||
// For better resilience the following fields are used to save the
|
||||
|
@ -100,6 +108,7 @@ public class AbfsInputStream extends FSInputStream implements CanUnbuffer,
|
|||
private final AbfsInputStreamStatistics streamStatistics;
|
||||
private long bytesFromReadAhead; // bytes read from readAhead; for testing
|
||||
private long bytesFromRemoteRead; // bytes read remotely; for testing
|
||||
private Listener listener;
|
||||
|
||||
private final AbfsInputStreamContext context;
|
||||
private IOStatistics ioStatistics;
|
||||
|
@ -110,7 +119,8 @@ public class AbfsInputStream extends FSInputStream implements CanUnbuffer,
|
|||
final String path,
|
||||
final long contentLength,
|
||||
final AbfsInputStreamContext abfsInputStreamContext,
|
||||
final String eTag) {
|
||||
final String eTag,
|
||||
TracingContext tracingContext) {
|
||||
this.client = client;
|
||||
this.statistics = statistics;
|
||||
this.path = path;
|
||||
|
@ -127,6 +137,10 @@ public class AbfsInputStream extends FSInputStream implements CanUnbuffer,
|
|||
this.cachedSasToken = new CachedSASToken(
|
||||
abfsInputStreamContext.getSasTokenRenewPeriodForStreamsInSeconds());
|
||||
this.streamStatistics = abfsInputStreamContext.getStreamStatistics();
|
||||
this.inputStreamId = createInputStreamId();
|
||||
this.tracingContext = new TracingContext(tracingContext);
|
||||
this.tracingContext.setOperation(FSOperationType.READ);
|
||||
this.tracingContext.setStreamID(inputStreamId);
|
||||
this.context = abfsInputStreamContext;
|
||||
readAheadBlockSize = abfsInputStreamContext.getReadAheadBlockSize();
|
||||
|
||||
|
@ -142,6 +156,10 @@ public class AbfsInputStream extends FSInputStream implements CanUnbuffer,
|
|||
return path;
|
||||
}
|
||||
|
||||
private String createInputStreamId() {
|
||||
return StringUtils.right(UUID.randomUUID().toString(), STREAM_ID_LEN);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int read(long position, byte[] buffer, int offset, int length)
|
||||
throws IOException {
|
||||
|
@ -167,7 +185,7 @@ public class AbfsInputStream extends FSInputStream implements CanUnbuffer,
|
|||
if (streamStatistics != null) {
|
||||
streamStatistics.readOperationStarted();
|
||||
}
|
||||
int bytesRead = readRemote(position, buffer, offset, length);
|
||||
int bytesRead = readRemote(position, buffer, offset, length, tracingContext);
|
||||
if (statistics != null) {
|
||||
statistics.incrementBytesRead(bytesRead);
|
||||
}
|
||||
|
@ -428,10 +446,13 @@ public class AbfsInputStream extends FSInputStream implements CanUnbuffer,
|
|||
// of readAhead Block size
|
||||
long nextSize = min((long) bufferSize, contentLength - nextOffset);
|
||||
LOG.debug("read ahead enabled issuing readheads num = {}", numReadAheads);
|
||||
TracingContext readAheadTracingContext = new TracingContext(tracingContext);
|
||||
readAheadTracingContext.setPrimaryRequestID();
|
||||
while (numReadAheads > 0 && nextOffset < contentLength) {
|
||||
LOG.debug("issuing read ahead requestedOffset = {} requested size {}",
|
||||
nextOffset, nextSize);
|
||||
ReadBufferManager.getBufferManager().queueReadAhead(this, nextOffset, (int) nextSize);
|
||||
ReadBufferManager.getBufferManager().queueReadAhead(this, nextOffset, (int) nextSize,
|
||||
new TracingContext(readAheadTracingContext));
|
||||
nextOffset = nextOffset + nextSize;
|
||||
numReadAheads--;
|
||||
// From next round onwards should be of readahead block size.
|
||||
|
@ -451,15 +472,15 @@ public class AbfsInputStream extends FSInputStream implements CanUnbuffer,
|
|||
}
|
||||
|
||||
// got nothing from read-ahead, do our own read now
|
||||
receivedBytes = readRemote(position, b, offset, length);
|
||||
receivedBytes = readRemote(position, b, offset, length, new TracingContext(tracingContext));
|
||||
return receivedBytes;
|
||||
} else {
|
||||
LOG.debug("read ahead disabled, reading remote");
|
||||
return readRemote(position, b, offset, length);
|
||||
return readRemote(position, b, offset, length, new TracingContext(tracingContext));
|
||||
}
|
||||
}
|
||||
|
||||
int readRemote(long position, byte[] b, int offset, int length) throws IOException {
|
||||
int readRemote(long position, byte[] b, int offset, int length, TracingContext tracingContext) throws IOException {
|
||||
if (position < 0) {
|
||||
throw new IllegalArgumentException("attempting to read from negative offset");
|
||||
}
|
||||
|
@ -483,7 +504,7 @@ public class AbfsInputStream extends FSInputStream implements CanUnbuffer,
|
|||
try (AbfsPerfInfo perfInfo = new AbfsPerfInfo(tracker, "readRemote", "read")) {
|
||||
LOG.trace("Trigger client.read for path={} position={} offset={} length={}", path, position, offset, length);
|
||||
op = client.read(path, position, b, offset, length,
|
||||
tolerateOobAppends ? "*" : eTag, cachedSasToken.get());
|
||||
tolerateOobAppends ? "*" : eTag, cachedSasToken.get(), tracingContext);
|
||||
cachedSasToken.update(op.getSasToken());
|
||||
if (streamStatistics != null) {
|
||||
streamStatistics.remoteReadOperation();
|
||||
|
@ -633,6 +654,10 @@ public class AbfsInputStream extends FSInputStream implements CanUnbuffer,
|
|||
return fCursor - limit + bCursor;
|
||||
}
|
||||
|
||||
public TracingContext getTracingContext() {
|
||||
return tracingContext;
|
||||
}
|
||||
|
||||
/**
|
||||
* Seeks a different copy of the data. Returns true if
|
||||
* found a new source, false otherwise.
|
||||
|
@ -701,6 +726,11 @@ public class AbfsInputStream extends FSInputStream implements CanUnbuffer,
|
|||
this.cachedSasToken = cachedSasToken;
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
public String getStreamID() {
|
||||
return inputStreamId;
|
||||
}
|
||||
|
||||
/**
|
||||
* Getter for AbfsInputStreamStatistics.
|
||||
*
|
||||
|
@ -711,6 +741,12 @@ public class AbfsInputStream extends FSInputStream implements CanUnbuffer,
|
|||
return streamStatistics;
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
public void registerListener(Listener listener1) {
|
||||
listener = listener1;
|
||||
tracingContext.setListener(listener);
|
||||
}
|
||||
|
||||
/**
|
||||
* Getter for bytes read from readAhead buffer that fills asynchronously.
|
||||
*
|
||||
|
|
|
@ -30,7 +30,9 @@ import org.slf4j.Logger;
|
|||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations;
|
||||
import org.apache.hadoop.fs.azurebfs.constants.FSOperationType;
|
||||
import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException;
|
||||
import org.apache.hadoop.fs.azurebfs.utils.TracingContext;
|
||||
import org.apache.hadoop.io.retry.RetryPolicies;
|
||||
import org.apache.hadoop.io.retry.RetryPolicy;
|
||||
|
||||
|
@ -60,6 +62,7 @@ public final class AbfsLease {
|
|||
|
||||
private final AbfsClient client;
|
||||
private final String path;
|
||||
private final TracingContext tracingContext;
|
||||
|
||||
// Lease status variables
|
||||
private volatile boolean leaseFreed;
|
||||
|
@ -78,16 +81,18 @@ public final class AbfsLease {
|
|||
}
|
||||
}
|
||||
|
||||
public AbfsLease(AbfsClient client, String path) throws AzureBlobFileSystemException {
|
||||
this(client, path, DEFAULT_LEASE_ACQUIRE_MAX_RETRIES, DEFAULT_LEASE_ACQUIRE_RETRY_INTERVAL);
|
||||
public AbfsLease(AbfsClient client, String path, TracingContext tracingContext) throws AzureBlobFileSystemException {
|
||||
this(client, path, DEFAULT_LEASE_ACQUIRE_MAX_RETRIES,
|
||||
DEFAULT_LEASE_ACQUIRE_RETRY_INTERVAL, tracingContext);
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
public AbfsLease(AbfsClient client, String path, int acquireMaxRetries,
|
||||
int acquireRetryInterval) throws AzureBlobFileSystemException {
|
||||
int acquireRetryInterval, TracingContext tracingContext) throws AzureBlobFileSystemException {
|
||||
this.leaseFreed = false;
|
||||
this.client = client;
|
||||
this.path = path;
|
||||
this.tracingContext = tracingContext;
|
||||
|
||||
if (client.getNumLeaseThreads() < 1) {
|
||||
throw new LeaseException(ERR_NO_LEASE_THREADS);
|
||||
|
@ -96,7 +101,8 @@ public final class AbfsLease {
|
|||
// Try to get the lease a specified number of times, else throw an error
|
||||
RetryPolicy retryPolicy = RetryPolicies.retryUpToMaximumCountWithFixedSleep(
|
||||
acquireMaxRetries, acquireRetryInterval, TimeUnit.SECONDS);
|
||||
acquireLease(retryPolicy, 0, acquireRetryInterval, 0);
|
||||
acquireLease(retryPolicy, 0, acquireRetryInterval, 0,
|
||||
new TracingContext(tracingContext));
|
||||
|
||||
while (leaseID == null && exception == null) {
|
||||
try {
|
||||
|
@ -114,13 +120,15 @@ public final class AbfsLease {
|
|||
LOG.debug("Acquired lease {} on {}", leaseID, path);
|
||||
}
|
||||
|
||||
private void acquireLease(RetryPolicy retryPolicy, int numRetries, int retryInterval, long delay)
|
||||
private void acquireLease(RetryPolicy retryPolicy, int numRetries,
|
||||
int retryInterval, long delay, TracingContext tracingContext)
|
||||
throws LeaseException {
|
||||
LOG.debug("Attempting to acquire lease on {}, retry {}", path, numRetries);
|
||||
if (future != null && !future.isDone()) {
|
||||
throw new LeaseException(ERR_LEASE_FUTURE_EXISTS);
|
||||
}
|
||||
future = client.schedule(() -> client.acquireLease(path, INFINITE_LEASE_DURATION),
|
||||
future = client.schedule(() -> client.acquireLease(path,
|
||||
INFINITE_LEASE_DURATION, tracingContext),
|
||||
delay, TimeUnit.SECONDS);
|
||||
client.addCallback(future, new FutureCallback<AbfsRestOperation>() {
|
||||
@Override
|
||||
|
@ -136,7 +144,8 @@ public final class AbfsLease {
|
|||
== retryPolicy.shouldRetry(null, numRetries, 0, true).action) {
|
||||
LOG.debug("Failed to acquire lease on {}, retrying: {}", path, throwable);
|
||||
acquireRetryCount++;
|
||||
acquireLease(retryPolicy, numRetries + 1, retryInterval, retryInterval);
|
||||
acquireLease(retryPolicy, numRetries + 1, retryInterval,
|
||||
retryInterval, tracingContext);
|
||||
} else {
|
||||
exception = throwable;
|
||||
}
|
||||
|
@ -161,7 +170,9 @@ public final class AbfsLease {
|
|||
if (future != null && !future.isDone()) {
|
||||
future.cancel(true);
|
||||
}
|
||||
client.releaseLease(path, leaseID);
|
||||
TracingContext tracingContext = new TracingContext(this.tracingContext);
|
||||
tracingContext.setOperation(FSOperationType.RELEASE_LEASE);
|
||||
client.releaseLease(path, leaseID, tracingContext);
|
||||
} catch (IOException e) {
|
||||
LOG.warn("Exception when trying to release lease {} on {}. Lease will need to be broken: {}",
|
||||
leaseID, path, e.getMessage());
|
||||
|
@ -185,4 +196,9 @@ public final class AbfsLease {
|
|||
public int getAcquireRetryCount() {
|
||||
return acquireRetryCount;
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
public TracingContext getTracingContext() {
|
||||
return tracingContext;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -34,6 +34,7 @@ import org.slf4j.LoggerFactory;
|
|||
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.RemoteIterator;
|
||||
import org.apache.hadoop.fs.azurebfs.utils.TracingContext;
|
||||
|
||||
public class AbfsListStatusRemoteIterator
|
||||
implements RemoteIterator<FileStatus> {
|
||||
|
@ -48,6 +49,7 @@ public class AbfsListStatusRemoteIterator
|
|||
private final FileStatus fileStatus;
|
||||
private final ListingSupport listingSupport;
|
||||
private final ArrayBlockingQueue<Object> iteratorsQueue;
|
||||
private final TracingContext tracingContext;
|
||||
|
||||
private volatile boolean isAsyncInProgress = false;
|
||||
private boolean isIterationComplete = false;
|
||||
|
@ -55,9 +57,10 @@ public class AbfsListStatusRemoteIterator
|
|||
private Iterator<FileStatus> currIterator;
|
||||
|
||||
public AbfsListStatusRemoteIterator(final FileStatus fileStatus,
|
||||
final ListingSupport listingSupport) {
|
||||
final ListingSupport listingSupport, TracingContext tracingContext) {
|
||||
this.fileStatus = fileStatus;
|
||||
this.listingSupport = listingSupport;
|
||||
this.tracingContext = tracingContext;
|
||||
iteratorsQueue = new ArrayBlockingQueue<>(MAX_QUEUE_SIZE);
|
||||
currIterator = Collections.emptyIterator();
|
||||
fetchBatchesAsync();
|
||||
|
@ -145,7 +148,7 @@ public class AbfsListStatusRemoteIterator
|
|||
List<FileStatus> fileStatuses = new ArrayList<>();
|
||||
continuation = listingSupport
|
||||
.listStatus(fileStatus.getPath(), null, fileStatuses, FETCH_ALL_FALSE,
|
||||
continuation);
|
||||
continuation, tracingContext);
|
||||
if (!fileStatuses.isEmpty()) {
|
||||
iteratorsQueue.put(fileStatuses.iterator());
|
||||
}
|
||||
|
|
|
@ -30,6 +30,7 @@ import java.util.concurrent.ExecutorCompletionService;
|
|||
import java.util.concurrent.ThreadPoolExecutor;
|
||||
import java.util.concurrent.Future;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.UUID;
|
||||
|
||||
import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting;
|
||||
import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
|
||||
|
@ -37,11 +38,15 @@ import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
|
|||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import org.apache.commons.lang3.StringUtils;
|
||||
import org.apache.hadoop.fs.PathIOException;
|
||||
import org.apache.hadoop.fs.azurebfs.constants.FSOperationType;
|
||||
import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException;
|
||||
import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException;
|
||||
import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters;
|
||||
import org.apache.hadoop.fs.azurebfs.utils.CachedSASToken;
|
||||
import org.apache.hadoop.fs.azurebfs.utils.Listener;
|
||||
import org.apache.hadoop.fs.azurebfs.utils.TracingContext;
|
||||
import org.apache.hadoop.fs.statistics.DurationTracker;
|
||||
import org.apache.hadoop.fs.statistics.IOStatistics;
|
||||
import org.apache.hadoop.fs.statistics.IOStatisticsSource;
|
||||
|
@ -51,6 +56,7 @@ import org.apache.hadoop.fs.FSExceptionMessages;
|
|||
import org.apache.hadoop.fs.StreamCapabilities;
|
||||
import org.apache.hadoop.fs.Syncable;
|
||||
|
||||
import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.STREAM_ID_LEN;
|
||||
import static org.apache.hadoop.fs.azurebfs.services.AbfsErrors.ERR_WRITE_WITHOUT_LEASE;
|
||||
import static org.apache.hadoop.fs.impl.StoreImplementationUtils.isProbeForSyncable;
|
||||
import static org.apache.hadoop.io.IOUtils.wrapException;
|
||||
|
@ -90,6 +96,9 @@ public class AbfsOutputStream extends OutputStream implements Syncable,
|
|||
|
||||
// SAS tokens can be re-used until they expire
|
||||
private CachedSASToken cachedSasToken;
|
||||
private final String outputStreamId;
|
||||
private final TracingContext tracingContext;
|
||||
private Listener listener;
|
||||
|
||||
private AbfsLease lease;
|
||||
private String leaseId;
|
||||
|
@ -115,7 +124,8 @@ public class AbfsOutputStream extends OutputStream implements Syncable,
|
|||
final Statistics statistics,
|
||||
final String path,
|
||||
final long position,
|
||||
AbfsOutputStreamContext abfsOutputStreamContext) {
|
||||
AbfsOutputStreamContext abfsOutputStreamContext,
|
||||
TracingContext tracingContext) {
|
||||
this.client = client;
|
||||
this.statistics = statistics;
|
||||
this.path = path;
|
||||
|
@ -160,6 +170,14 @@ public class AbfsOutputStream extends OutputStream implements Syncable,
|
|||
if (outputStreamStatistics != null) {
|
||||
this.ioStatistics = outputStreamStatistics.getIOStatistics();
|
||||
}
|
||||
this.outputStreamId = createOutputStreamId();
|
||||
this.tracingContext = new TracingContext(tracingContext);
|
||||
this.tracingContext.setStreamID(outputStreamId);
|
||||
this.tracingContext.setOperation(FSOperationType.WRITE);
|
||||
}
|
||||
|
||||
private String createOutputStreamId() {
|
||||
return StringUtils.right(UUID.randomUUID().toString(), STREAM_ID_LEN);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -292,6 +310,15 @@ public class AbfsOutputStream extends OutputStream implements Syncable,
|
|||
}
|
||||
}
|
||||
|
||||
public String getStreamID() {
|
||||
return outputStreamId;
|
||||
}
|
||||
|
||||
public void registerListener(Listener listener1) {
|
||||
listener = listener1;
|
||||
tracingContext.setListener(listener);
|
||||
}
|
||||
|
||||
/**
|
||||
* 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
|
||||
|
@ -385,7 +412,9 @@ public class AbfsOutputStream extends OutputStream implements Syncable,
|
|||
"writeCurrentBufferToService", "append")) {
|
||||
AppendRequestParameters reqParams = new AppendRequestParameters(offset, 0,
|
||||
bytesLength, APPEND_MODE, true, leaseId);
|
||||
AbfsRestOperation op = client.append(path, bytes, reqParams, cachedSasToken.get());
|
||||
AbfsRestOperation op = client
|
||||
.append(path, bytes, reqParams, cachedSasToken.get(),
|
||||
new TracingContext(tracingContext));
|
||||
cachedSasToken.update(op.getSasToken());
|
||||
if (outputStreamStatistics != null) {
|
||||
outputStreamStatistics.uploadSuccessful(bytesLength);
|
||||
|
@ -444,10 +473,8 @@ public class AbfsOutputStream extends OutputStream implements Syncable,
|
|||
waitForTaskToComplete();
|
||||
}
|
||||
}
|
||||
final Future<Void> job =
|
||||
completionService.submit(() -> {
|
||||
AbfsPerfTracker tracker =
|
||||
client.getAbfsPerfTracker();
|
||||
final Future<Void> job = completionService.submit(() -> {
|
||||
AbfsPerfTracker tracker = client.getAbfsPerfTracker();
|
||||
try (AbfsPerfInfo perfInfo = new AbfsPerfInfo(tracker,
|
||||
"writeCurrentBufferToService", "append")) {
|
||||
AppendRequestParameters.Mode
|
||||
|
@ -460,7 +487,7 @@ public class AbfsOutputStream extends OutputStream implements Syncable,
|
|||
AppendRequestParameters reqParams = new AppendRequestParameters(
|
||||
offset, 0, bytesLength, mode, false, leaseId);
|
||||
AbfsRestOperation op = client.append(path, bytes, reqParams,
|
||||
cachedSasToken.get());
|
||||
cachedSasToken.get(), new TracingContext(tracingContext));
|
||||
cachedSasToken.update(op.getSasToken());
|
||||
perfInfo.registerResult(op.getResult());
|
||||
byteBufferPool.putBuffer(ByteBuffer.wrap(bytes));
|
||||
|
@ -527,7 +554,7 @@ public class AbfsOutputStream extends OutputStream implements Syncable,
|
|||
try (AbfsPerfInfo perfInfo = new AbfsPerfInfo(tracker,
|
||||
"flushWrittenBytesToServiceInternal", "flush")) {
|
||||
AbfsRestOperation op = client.flush(path, offset, retainUncommitedData, isClose,
|
||||
cachedSasToken.get(), leaseId);
|
||||
cachedSasToken.get(), leaseId, new TracingContext(tracingContext));
|
||||
cachedSasToken.update(op.getSasToken());
|
||||
perfInfo.registerResult(op.getResult()).registerSuccess(true);
|
||||
} catch (AzureBlobFileSystemException ex) {
|
||||
|
|
|
@ -34,6 +34,7 @@ import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationExcep
|
|||
import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException;
|
||||
import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidAbfsRestOperationException;
|
||||
import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations;
|
||||
import org.apache.hadoop.fs.azurebfs.utils.TracingContext;
|
||||
import org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding;
|
||||
|
||||
/**
|
||||
|
@ -171,14 +172,15 @@ public class AbfsRestOperation {
|
|||
/**
|
||||
* Execute a AbfsRestOperation. Track the Duration of a request if
|
||||
* abfsCounters isn't null.
|
||||
*
|
||||
* @param tracingContext TracingContext instance to track correlation IDs
|
||||
*/
|
||||
public void execute() throws AzureBlobFileSystemException {
|
||||
public void execute(TracingContext tracingContext)
|
||||
throws AzureBlobFileSystemException {
|
||||
|
||||
try {
|
||||
IOStatisticsBinding.trackDurationOfInvocation(abfsCounters,
|
||||
AbfsStatistic.getStatNameFromHttpCall(method),
|
||||
() -> completeExecute());
|
||||
() -> completeExecute(tracingContext));
|
||||
} catch (AzureBlobFileSystemException aze) {
|
||||
throw aze;
|
||||
} catch (IOException e) {
|
||||
|
@ -190,8 +192,10 @@ public class AbfsRestOperation {
|
|||
/**
|
||||
* Executes the REST operation with retry, by issuing one or more
|
||||
* HTTP operations.
|
||||
* @param tracingContext TracingContext instance to track correlation IDs
|
||||
*/
|
||||
private void completeExecute() throws AzureBlobFileSystemException {
|
||||
private void completeExecute(TracingContext tracingContext)
|
||||
throws AzureBlobFileSystemException {
|
||||
// see if we have latency reports from the previous requests
|
||||
String latencyHeader = this.client.getAbfsPerfTracker().getClientLatency();
|
||||
if (latencyHeader != null && !latencyHeader.isEmpty()) {
|
||||
|
@ -202,9 +206,10 @@ public class AbfsRestOperation {
|
|||
|
||||
retryCount = 0;
|
||||
LOG.debug("First execution of REST operation - {}", operationType);
|
||||
while (!executeHttpOperation(retryCount)) {
|
||||
while (!executeHttpOperation(retryCount, tracingContext)) {
|
||||
try {
|
||||
++retryCount;
|
||||
tracingContext.setRetryCount(retryCount);
|
||||
LOG.debug("Retrying REST operation {}. RetryCount = {}",
|
||||
operationType, retryCount);
|
||||
Thread.sleep(client.getRetryPolicy().getRetryInterval(retryCount));
|
||||
|
@ -226,12 +231,14 @@ public class AbfsRestOperation {
|
|||
* fails, there may be a retry. The retryCount is incremented with each
|
||||
* attempt.
|
||||
*/
|
||||
private boolean executeHttpOperation(final int retryCount) throws AzureBlobFileSystemException {
|
||||
private boolean executeHttpOperation(final int retryCount,
|
||||
TracingContext tracingContext) throws AzureBlobFileSystemException {
|
||||
AbfsHttpOperation httpOperation = null;
|
||||
try {
|
||||
// initialize the HTTP request and open the connection
|
||||
httpOperation = new AbfsHttpOperation(url, method, requestHeaders);
|
||||
incrementCounter(AbfsStatistic.CONNECTIONS_MADE, 1);
|
||||
tracingContext.constructHeader(httpOperation);
|
||||
|
||||
switch(client.getAuthType()) {
|
||||
case Custom:
|
||||
|
|
|
@ -25,6 +25,7 @@ import org.apache.hadoop.classification.InterfaceAudience;
|
|||
import org.apache.hadoop.classification.InterfaceStability;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.fs.azurebfs.utils.TracingContext;
|
||||
|
||||
@InterfaceAudience.Private
|
||||
@InterfaceStability.Unstable
|
||||
|
@ -32,10 +33,11 @@ public interface ListingSupport {
|
|||
|
||||
/**
|
||||
* @param path The list path.
|
||||
* @param tracingContext TracingContext instance to track identifiers
|
||||
* @return the entries in the path.
|
||||
* @throws IOException in case of error
|
||||
*/
|
||||
FileStatus[] listStatus(Path path) throws IOException;
|
||||
FileStatus[] listStatus(Path path, TracingContext tracingContext) throws IOException;
|
||||
|
||||
/**
|
||||
* @param path Path the list path.
|
||||
|
@ -49,10 +51,11 @@ public interface ListingSupport {
|
|||
* non-existent entry in lexical order: listStatus
|
||||
* (Path("/folder"), "cfile") will return
|
||||
* "/folder/hfile" and "/folder/ifile".
|
||||
* @param tracingContext TracingContext instance to track identifiers
|
||||
* @return the entries in the path start from "startFrom" in lexical order.
|
||||
* @throws IOException in case of error
|
||||
*/
|
||||
FileStatus[] listStatus(Path path, String startFrom) throws IOException;
|
||||
FileStatus[] listStatus(Path path, String startFrom, TracingContext tracingContext) throws IOException;
|
||||
|
||||
/**
|
||||
* @param path The list path
|
||||
|
@ -71,9 +74,10 @@ public interface ListingSupport {
|
|||
* filled with just one page os results or the entire
|
||||
* result.
|
||||
* @param continuation Contiuation token. null means start rom the begining.
|
||||
* @param tracingContext TracingContext instance to track identifiers
|
||||
* @return Continuation tokem
|
||||
* @throws IOException in case of error
|
||||
*/
|
||||
String listStatus(Path path, String startFrom, List<FileStatus> fileStatuses,
|
||||
boolean fetchAll, String continuation) throws IOException;
|
||||
boolean fetchAll, String continuation, TracingContext tracingContext) throws IOException;
|
||||
}
|
||||
|
|
|
@ -22,6 +22,7 @@ import java.io.IOException;
|
|||
import java.util.concurrent.CountDownLatch;
|
||||
|
||||
import org.apache.hadoop.fs.azurebfs.contracts.services.ReadBufferStatus;
|
||||
import org.apache.hadoop.fs.azurebfs.utils.TracingContext;
|
||||
|
||||
import static org.apache.hadoop.fs.azurebfs.contracts.services.ReadBufferStatus.READ_FAILED;
|
||||
|
||||
|
@ -36,6 +37,7 @@ class ReadBuffer {
|
|||
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
|
||||
private TracingContext tracingContext;
|
||||
|
||||
// fields to help with eviction logic
|
||||
private long timeStamp = 0; // tick at which buffer became available to read
|
||||
|
@ -53,6 +55,14 @@ class ReadBuffer {
|
|||
this.stream = stream;
|
||||
}
|
||||
|
||||
public void setTracingContext(TracingContext tracingContext) {
|
||||
this.tracingContext = tracingContext;
|
||||
}
|
||||
|
||||
public TracingContext getTracingContext() {
|
||||
return tracingContext;
|
||||
}
|
||||
|
||||
public long getOffset() {
|
||||
return offset;
|
||||
}
|
||||
|
|
|
@ -30,6 +30,7 @@ import java.util.Stack;
|
|||
import java.util.concurrent.CountDownLatch;
|
||||
import java.util.concurrent.locks.ReentrantLock;
|
||||
|
||||
import org.apache.hadoop.fs.azurebfs.utils.TracingContext;
|
||||
import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting;
|
||||
|
||||
/**
|
||||
|
@ -115,7 +116,8 @@ final class ReadBufferManager {
|
|||
* @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) {
|
||||
void queueReadAhead(final AbfsInputStream stream, final long requestedOffset, final int requestedLength,
|
||||
TracingContext tracingContext) {
|
||||
if (LOGGER.isTraceEnabled()) {
|
||||
LOGGER.trace("Start Queueing readAhead for {} offset {} length {}",
|
||||
stream.getPath(), requestedOffset, requestedLength);
|
||||
|
@ -136,6 +138,7 @@ final class ReadBufferManager {
|
|||
buffer.setRequestedLength(requestedLength);
|
||||
buffer.setStatus(ReadBufferStatus.NOT_AVAILABLE);
|
||||
buffer.setLatch(new CountDownLatch(1));
|
||||
buffer.setTracingContext(tracingContext);
|
||||
|
||||
Integer bufferIndex = freeList.pop(); // will return a value, since we have checked size > 0 already
|
||||
|
||||
|
@ -304,6 +307,7 @@ final class ReadBufferManager {
|
|||
}
|
||||
|
||||
completedReadList.remove(buf);
|
||||
buf.setTracingContext(null);
|
||||
if (LOGGER.isTraceEnabled()) {
|
||||
LOGGER.trace("Evicting buffer idx {}; was used for file {} offset {} length {}",
|
||||
buf.getBufferindex(), buf.getStream().getPath(), buf.getOffset(), buf.getLength());
|
||||
|
|
|
@ -69,7 +69,8 @@ class ReadBufferWorker implements Runnable {
|
|||
// If AbfsInputStream was created with bigger buffer size than
|
||||
// read-ahead buffer size, make sure a valid length is passed
|
||||
// for remote read
|
||||
Math.min(buffer.getRequestedLength(), buffer.getBuffer().length));
|
||||
Math.min(buffer.getRequestedLength(), buffer.getBuffer().length),
|
||||
buffer.getTracingContext());
|
||||
|
||||
bufferManager.doneReading(buffer, ReadBufferStatus.AVAILABLE, bytesRead); // post result back to ReadBufferManager
|
||||
} catch (Exception ex) {
|
||||
|
|
|
@ -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.utils;
|
||||
|
||||
import org.apache.hadoop.fs.azurebfs.constants.FSOperationType;
|
||||
|
||||
/**
|
||||
* Interface for testing identifiers tracked via TracingContext
|
||||
* Implemented in TracingHeaderValidator
|
||||
*/
|
||||
|
||||
public interface Listener {
|
||||
void callTracingHeaderValidator(String header, TracingHeaderFormat format);
|
||||
void updatePrimaryRequestID(String primaryRequestID);
|
||||
Listener getClone();
|
||||
void setOperation(FSOperationType operation);
|
||||
}
|
|
@ -0,0 +1,185 @@
|
|||
/**
|
||||
* 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.utils;
|
||||
|
||||
import java.util.UUID;
|
||||
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import org.apache.hadoop.fs.azurebfs.constants.FSOperationType;
|
||||
import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations;
|
||||
import org.apache.hadoop.fs.azurebfs.services.AbfsClient;
|
||||
import org.apache.hadoop.fs.azurebfs.services.AbfsHttpOperation;
|
||||
|
||||
import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.EMPTY_STRING;
|
||||
|
||||
/**
|
||||
* The TracingContext class to correlate Store requests using unique
|
||||
* identifiers and resources common to requests (e.g. filesystem, stream)
|
||||
*
|
||||
* Implementing new HDFS method:
|
||||
* Create TracingContext instance in method of outer layer of
|
||||
* ABFS driver (AzureBlobFileSystem/AbfsInputStream/AbfsOutputStream), to be
|
||||
* passed through ABFS layers up to AbfsRestOperation.
|
||||
*
|
||||
* Add new operations to HdfsOperationConstants file.
|
||||
*
|
||||
* PrimaryRequestId can be enabled for individual Hadoop API that invoke
|
||||
* multiple Store calls.
|
||||
*
|
||||
* Testing:
|
||||
* Pass an instance of TracingHeaderValidator to registerListener() of ABFS
|
||||
* filesystem/stream class before calling the API in tests.
|
||||
*/
|
||||
|
||||
public class TracingContext {
|
||||
private final String clientCorrelationID; // passed over config by client
|
||||
private final String fileSystemID; // GUID for fileSystem instance
|
||||
private String clientRequestId = EMPTY_STRING; // GUID per http request
|
||||
//Optional, non-empty for methods that trigger two or more Store calls
|
||||
private String primaryRequestId;
|
||||
private String streamID; // appears per stream instance (read/write ops)
|
||||
private int retryCount; // retry number as recorded by AbfsRestOperation
|
||||
private FSOperationType opType; // two-lettered code representing Hadoop op
|
||||
private final TracingHeaderFormat format; // header ID display options
|
||||
private Listener listener = null; // null except when testing
|
||||
//final concatenated ID list set into x-ms-client-request-id header
|
||||
private String header = EMPTY_STRING;
|
||||
|
||||
private static final Logger LOG = LoggerFactory.getLogger(AbfsClient.class);
|
||||
public static final int MAX_CLIENT_CORRELATION_ID_LENGTH = 72;
|
||||
public static final String CLIENT_CORRELATION_ID_PATTERN = "[a-zA-Z0-9-]*";
|
||||
|
||||
/**
|
||||
* Initialize TracingContext
|
||||
* @param clientCorrelationID Provided over config by client
|
||||
* @param fileSystemID Unique guid for AzureBlobFileSystem instance
|
||||
* @param opType Code indicating the high-level Hadoop operation that
|
||||
* triggered the current Store request
|
||||
* @param tracingHeaderFormat Format of IDs to be printed in header and logs
|
||||
* @param listener Holds instance of TracingHeaderValidator during testing,
|
||||
* null otherwise
|
||||
*/
|
||||
public TracingContext(String clientCorrelationID, String fileSystemID,
|
||||
FSOperationType opType, TracingHeaderFormat tracingHeaderFormat,
|
||||
Listener listener) {
|
||||
this.fileSystemID = fileSystemID;
|
||||
this.opType = opType;
|
||||
this.clientCorrelationID = clientCorrelationID;
|
||||
streamID = EMPTY_STRING;
|
||||
retryCount = 0;
|
||||
primaryRequestId = EMPTY_STRING;
|
||||
format = tracingHeaderFormat;
|
||||
this.listener = listener;
|
||||
}
|
||||
|
||||
public TracingContext(String clientCorrelationID, String fileSystemID,
|
||||
FSOperationType opType, boolean needsPrimaryReqId,
|
||||
TracingHeaderFormat tracingHeaderFormat, Listener listener) {
|
||||
this(clientCorrelationID, fileSystemID, opType, tracingHeaderFormat,
|
||||
listener);
|
||||
primaryRequestId = needsPrimaryReqId ? UUID.randomUUID().toString() : "";
|
||||
if (listener != null) {
|
||||
listener.updatePrimaryRequestID(primaryRequestId);
|
||||
}
|
||||
}
|
||||
|
||||
public TracingContext(TracingContext originalTracingContext) {
|
||||
this.fileSystemID = originalTracingContext.fileSystemID;
|
||||
this.streamID = originalTracingContext.streamID;
|
||||
this.clientCorrelationID = originalTracingContext.clientCorrelationID;
|
||||
this.opType = originalTracingContext.opType;
|
||||
this.retryCount = 0;
|
||||
this.primaryRequestId = originalTracingContext.primaryRequestId;
|
||||
this.format = originalTracingContext.format;
|
||||
if (originalTracingContext.listener != null) {
|
||||
this.listener = originalTracingContext.listener.getClone();
|
||||
}
|
||||
}
|
||||
|
||||
public static String validateClientCorrelationID(String clientCorrelationID) {
|
||||
if ((clientCorrelationID.length() > MAX_CLIENT_CORRELATION_ID_LENGTH)
|
||||
|| (!clientCorrelationID.matches(CLIENT_CORRELATION_ID_PATTERN))) {
|
||||
LOG.debug(
|
||||
"Invalid config provided; correlation id not included in header.");
|
||||
return EMPTY_STRING;
|
||||
}
|
||||
return clientCorrelationID;
|
||||
}
|
||||
|
||||
public void setPrimaryRequestID() {
|
||||
primaryRequestId = UUID.randomUUID().toString();
|
||||
if (listener != null) {
|
||||
listener.updatePrimaryRequestID(primaryRequestId);
|
||||
}
|
||||
}
|
||||
|
||||
public void setStreamID(String stream) {
|
||||
streamID = stream;
|
||||
}
|
||||
|
||||
public void setOperation(FSOperationType operation) {
|
||||
this.opType = operation;
|
||||
}
|
||||
|
||||
public void setRetryCount(int retryCount) {
|
||||
this.retryCount = retryCount;
|
||||
}
|
||||
|
||||
public void setListener(Listener listener) {
|
||||
this.listener = listener;
|
||||
}
|
||||
|
||||
/**
|
||||
* Concatenate all identifiers separated by (:) into a string and set into
|
||||
* X_MS_CLIENT_REQUEST_ID header of the http operation
|
||||
* @param httpOperation AbfsHttpOperation instance to set header into
|
||||
* connection
|
||||
*/
|
||||
public void constructHeader(AbfsHttpOperation httpOperation) {
|
||||
clientRequestId = UUID.randomUUID().toString();
|
||||
switch (format) {
|
||||
case ALL_ID_FORMAT: // Optional IDs (e.g. streamId) may be empty
|
||||
header =
|
||||
clientCorrelationID + ":" + clientRequestId + ":" + fileSystemID + ":"
|
||||
+ primaryRequestId + ":" + streamID + ":" + opType + ":"
|
||||
+ retryCount;
|
||||
break;
|
||||
case TWO_ID_FORMAT:
|
||||
header = clientCorrelationID + ":" + clientRequestId;
|
||||
break;
|
||||
default:
|
||||
header = clientRequestId; //case SINGLE_ID_FORMAT
|
||||
}
|
||||
if (listener != null) { //for testing
|
||||
listener.callTracingHeaderValidator(header, format);
|
||||
}
|
||||
httpOperation.setRequestProperty(HttpHeaderConfigurations.X_MS_CLIENT_REQUEST_ID, header);
|
||||
}
|
||||
|
||||
/**
|
||||
* Return header representing the request associated with the tracingContext
|
||||
* @return Header string set into X_MS_CLIENT_REQUEST_ID
|
||||
*/
|
||||
public String getHeader() {
|
||||
return header;
|
||||
}
|
||||
|
||||
}
|
|
@ -0,0 +1,28 @@
|
|||
/**
|
||||
* 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;
|
||||
|
||||
public enum TracingHeaderFormat {
|
||||
SINGLE_ID_FORMAT, // <client-req-id>
|
||||
|
||||
TWO_ID_FORMAT, // <correlation-id>:<client-req-id>
|
||||
|
||||
ALL_ID_FORMAT; // <client-correlation-id>:<client-req-id>:<filesystem-id>
|
||||
// :<primary-req-id>:<stream-id>:<hdfs-operation>:<retry-count>
|
||||
}
|
|
@ -729,6 +729,28 @@ Consult the javadocs for `org.apache.hadoop.fs.azurebfs.constants.ConfigurationK
|
|||
`org.apache.hadoop.fs.azurebfs.AbfsConfiguration` for the full list
|
||||
of configuration options and their default values.
|
||||
|
||||
### <a name="clientcorrelationoptions"></a> Client Correlation Options
|
||||
|
||||
#### <a name="clientcorrelationid"></a> 1. Client CorrelationId Option
|
||||
|
||||
Config `fs.azure.client.correlationid` provides an option to correlate client
|
||||
requests using this client-provided identifier. This Id will be visible in Azure
|
||||
Storage Analytics logs in the `request-id-header` field.
|
||||
Reference: [Storage Analytics log format](https://docs.microsoft.com/en-us/rest/api/storageservices/storage-analytics-log-format)
|
||||
|
||||
This config accepts a string which can be maximum of 72 characters and should
|
||||
contain alphanumeric characters and/or hyphens only. Defaults to empty string if
|
||||
input is invalid.
|
||||
|
||||
#### <a name="tracingcontextformat"></a> 1. Correlation IDs Display Options
|
||||
|
||||
Config `fs.azure.tracingcontext.format` provides an option to select the format
|
||||
of IDs included in the `request-id-header`. This config accepts a String value
|
||||
corresponding to the following enum options.
|
||||
`SINGLE_ID_FORMAT` : clientRequestId
|
||||
`ALL_ID_FORMAT` : all IDs (default)
|
||||
`TWO_ID_FORMAT` : clientCorrelationId:clientRequestId
|
||||
|
||||
### <a name="flushconfigoptions"></a> Flush Options
|
||||
|
||||
#### <a name="abfsflushconfigoptions"></a> 1. Azure Blob File System Flush Options
|
||||
|
|
|
@ -35,6 +35,7 @@ import org.apache.hadoop.conf.Configuration;
|
|||
import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.fs.azurebfs.constants.FSOperationType;
|
||||
import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException;
|
||||
import org.apache.hadoop.fs.azurebfs.security.AbfsDelegationTokenManager;
|
||||
import org.apache.hadoop.fs.azurebfs.services.AbfsOutputStream;
|
||||
|
@ -45,6 +46,8 @@ import org.apache.hadoop.fs.azure.metrics.AzureFileSystemInstrumentation;
|
|||
import org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes;
|
||||
import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants;
|
||||
import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException;
|
||||
import org.apache.hadoop.fs.azurebfs.utils.TracingContext;
|
||||
import org.apache.hadoop.fs.azurebfs.utils.TracingHeaderFormat;
|
||||
import org.apache.hadoop.fs.azurebfs.utils.UriUtils;
|
||||
import org.apache.hadoop.fs.contract.ContractTestUtils;
|
||||
import org.apache.hadoop.fs.permission.FsPermission;
|
||||
|
@ -139,17 +142,42 @@ public abstract class AbstractAbfsIntegrationTest extends
|
|||
}
|
||||
}
|
||||
|
||||
protected boolean getIsNamespaceEnabled(AzureBlobFileSystem fs)
|
||||
throws IOException {
|
||||
return fs.getIsNamespaceEnabled(getTestTracingContext(fs, false));
|
||||
}
|
||||
|
||||
public TracingContext getTestTracingContext(AzureBlobFileSystem fs,
|
||||
boolean needsPrimaryReqId) {
|
||||
String correlationId, fsId;
|
||||
TracingHeaderFormat format;
|
||||
if (fs == null) {
|
||||
correlationId = "test-corr-id";
|
||||
fsId = "test-filesystem-id";
|
||||
format = TracingHeaderFormat.ALL_ID_FORMAT;
|
||||
} else {
|
||||
AbfsConfiguration abfsConf = fs.getAbfsStore().getAbfsConfiguration();
|
||||
correlationId = abfsConf.getClientCorrelationId();
|
||||
fsId = fs.getFileSystemId();
|
||||
format = abfsConf.getTracingHeaderFormat();
|
||||
}
|
||||
return new TracingContext(correlationId, fsId,
|
||||
FSOperationType.TEST_OP, needsPrimaryReqId, format, null);
|
||||
}
|
||||
|
||||
|
||||
@Before
|
||||
public void setup() throws Exception {
|
||||
//Create filesystem first to make sure getWasbFileSystem() can return an existing filesystem.
|
||||
createFileSystem();
|
||||
|
||||
// Only live account without namespace support can run ABFS&WASB compatibility tests
|
||||
if (!isIPAddress
|
||||
&& (abfsConfig.getAuthType(accountName) != AuthType.SAS)
|
||||
&& !abfs.getIsNamespaceEnabled()) {
|
||||
final URI wasbUri = new URI(abfsUrlToWasbUrl(getTestUrl(), abfsConfig.isHttpsAlwaysUsed()));
|
||||
// Only live account without namespace support can run ABFS&WASB
|
||||
// compatibility tests
|
||||
if (!isIPAddress && (abfsConfig.getAuthType(accountName) != AuthType.SAS)
|
||||
&& !abfs.getIsNamespaceEnabled(getTestTracingContext(
|
||||
getFileSystem(), false))) {
|
||||
final URI wasbUri = new URI(
|
||||
abfsUrlToWasbUrl(getTestUrl(), abfsConfig.isHttpsAlwaysUsed()));
|
||||
final AzureNativeFileSystemStore azureNativeFileSystemStore =
|
||||
new AzureNativeFileSystemStore();
|
||||
|
||||
|
@ -180,22 +208,23 @@ public abstract class AbstractAbfsIntegrationTest extends
|
|||
if (abfs == null) {
|
||||
return;
|
||||
}
|
||||
TracingContext tracingContext = getTestTracingContext(getFileSystem(), false);
|
||||
|
||||
if (usingFilesystemForSASTests) {
|
||||
abfsConfig.set(FS_AZURE_ACCOUNT_AUTH_TYPE_PROPERTY_NAME, AuthType.SharedKey.name());
|
||||
AzureBlobFileSystem tempFs = (AzureBlobFileSystem) FileSystem.newInstance(rawConfig);
|
||||
tempFs.getAbfsStore().deleteFilesystem();
|
||||
tempFs.getAbfsStore().deleteFilesystem(tracingContext);
|
||||
}
|
||||
else if (!useConfiguredFileSystem) {
|
||||
// Delete all uniquely created filesystem from the account
|
||||
final AzureBlobFileSystemStore abfsStore = abfs.getAbfsStore();
|
||||
abfsStore.deleteFilesystem();
|
||||
abfsStore.deleteFilesystem(tracingContext);
|
||||
|
||||
AbfsRestOperationException ex = intercept(AbfsRestOperationException.class,
|
||||
new Callable<Hashtable<String, String>>() {
|
||||
@Override
|
||||
public Hashtable<String, String> call() throws Exception {
|
||||
return abfsStore.getFilesystemProperties();
|
||||
return abfsStore.getFilesystemProperties(tracingContext);
|
||||
}
|
||||
});
|
||||
if (FILE_SYSTEM_NOT_FOUND.getStatusCode() != ex.getStatusCode()) {
|
||||
|
@ -439,7 +468,8 @@ public abstract class AbstractAbfsIntegrationTest extends
|
|||
abfss.getAbfsConfiguration().setDisableOutputStreamFlush(false);
|
||||
|
||||
return (AbfsOutputStream) abfss.createFile(path, fs.getFsStatistics(),
|
||||
true, FsPermission.getDefault(), FsPermission.getUMask(fs.getConf()));
|
||||
true, FsPermission.getDefault(), FsPermission.getUMask(fs.getConf()),
|
||||
getTestTracingContext(fs, false));
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -62,7 +62,9 @@ public final class ITestAbfsClient extends AbstractAbfsIntegrationTest {
|
|||
AbfsClient abfsClient = fs.getAbfsClient();
|
||||
|
||||
try {
|
||||
AbfsRestOperation op = abfsClient.listPath("/", true, LIST_MAX_RESULTS, "===========");
|
||||
AbfsRestOperation op = abfsClient
|
||||
.listPath("/", true, LIST_MAX_RESULTS, "===========",
|
||||
getTestTracingContext(fs, true));
|
||||
Assert.assertTrue(false);
|
||||
} catch (AbfsRestOperationException ex) {
|
||||
Assert.assertEquals("InvalidQueryParameterValue", ex.getErrorCode().getErrorCode());
|
||||
|
@ -135,7 +137,8 @@ public final class ITestAbfsClient extends AbstractAbfsIntegrationTest {
|
|||
private List<ListResultEntrySchema> listPath(String directory)
|
||||
throws IOException {
|
||||
return getFileSystem().getAbfsClient()
|
||||
.listPath(directory, false, getListMaxResults(), null).getResult()
|
||||
.listPath(directory, false, getListMaxResults(), null,
|
||||
getTestTracingContext(getFileSystem(), true)).getResult()
|
||||
.getListResultSchema().paths();
|
||||
}
|
||||
|
||||
|
|
|
@ -76,7 +76,8 @@ public class ITestAbfsDurationTrackers extends AbstractAbfsIntegrationTest {
|
|||
out.hflush();
|
||||
|
||||
// GET the file.
|
||||
in = fs.getAbfsStore().openFileForRead(testFilePath, fs.getFsStatistics());
|
||||
in = fs.getAbfsStore().openFileForRead(testFilePath,
|
||||
fs.getFsStatistics(), getTestTracingContext(fs, false));
|
||||
int res = in.read();
|
||||
LOG.info("Result of Read: {}", res);
|
||||
|
||||
|
|
|
@ -68,7 +68,8 @@ public class ITestAbfsInputStreamStatistics
|
|||
try {
|
||||
|
||||
outputStream = createAbfsOutputStreamWithFlushEnabled(fs, initValuesPath);
|
||||
inputStream = abfss.openFileForRead(initValuesPath, fs.getFsStatistics());
|
||||
inputStream = abfss.openFileForRead(initValuesPath, fs.getFsStatistics(),
|
||||
getTestTracingContext(fs, false));
|
||||
|
||||
AbfsInputStreamStatisticsImpl stats =
|
||||
(AbfsInputStreamStatisticsImpl) inputStream.getStreamStatistics();
|
||||
|
@ -112,7 +113,8 @@ public class ITestAbfsInputStreamStatistics
|
|||
//Writing a default buffer in a file.
|
||||
out.write(defBuffer);
|
||||
out.hflush();
|
||||
in = abfss.openFileForRead(seekStatPath, fs.getFsStatistics());
|
||||
in = abfss.openFileForRead(seekStatPath, fs.getFsStatistics(),
|
||||
getTestTracingContext(fs, false));
|
||||
|
||||
/*
|
||||
* Writing 1MB buffer to the file, this would make the fCursor(Current
|
||||
|
@ -202,7 +204,8 @@ public class ITestAbfsInputStreamStatistics
|
|||
*/
|
||||
out.write(defBuffer);
|
||||
out.hflush();
|
||||
in = abfss.openFileForRead(readStatPath, fs.getFsStatistics());
|
||||
in = abfss.openFileForRead(readStatPath, fs.getFsStatistics(),
|
||||
getTestTracingContext(fs, false));
|
||||
|
||||
/*
|
||||
* Doing file read 10 times.
|
||||
|
@ -273,14 +276,15 @@ public class ITestAbfsInputStreamStatistics
|
|||
out.hflush();
|
||||
|
||||
// AbfsRestOperation Instance required for eTag.
|
||||
AbfsRestOperation abfsRestOperation =
|
||||
fs.getAbfsClient().getPathStatus(nullStatFilePath.toUri().getPath(), false);
|
||||
AbfsRestOperation abfsRestOperation = fs.getAbfsClient()
|
||||
.getPathStatus(nullStatFilePath.toUri().getPath(), false,
|
||||
getTestTracingContext(fs, false));
|
||||
|
||||
// AbfsInputStream with no StreamStatistics.
|
||||
in = new AbfsInputStream(fs.getAbfsClient(), null,
|
||||
nullStatFilePath.toUri().getPath(), ONE_KB,
|
||||
abfsInputStreamContext,
|
||||
abfsRestOperation.getResult().getResponseHeader("ETag"));
|
||||
nullStatFilePath.toUri().getPath(), ONE_KB, abfsInputStreamContext,
|
||||
abfsRestOperation.getResult().getResponseHeader("ETag"),
|
||||
getTestTracingContext(fs, false));
|
||||
|
||||
// Verifying that AbfsInputStream Operations works with null statistics.
|
||||
assertNotEquals("AbfsInputStream read() with null statistics should "
|
||||
|
@ -323,7 +327,8 @@ public class ITestAbfsInputStreamStatistics
|
|||
out.write(defBuffer);
|
||||
out.close();
|
||||
|
||||
in = abfss.openFileForRead(readAheadCountersPath, fs.getFsStatistics());
|
||||
in = abfss.openFileForRead(readAheadCountersPath, fs.getFsStatistics(),
|
||||
getTestTracingContext(fs, false));
|
||||
|
||||
/*
|
||||
* Reading 1KB after each i * KB positions. Hence the reads are from 0
|
||||
|
@ -390,7 +395,8 @@ public class ITestAbfsInputStreamStatistics
|
|||
abfsOutputStream.hflush();
|
||||
|
||||
abfsInputStream =
|
||||
abfss.openFileForRead(actionHttpGetRequestPath, fs.getFsStatistics());
|
||||
abfss.openFileForRead(actionHttpGetRequestPath,
|
||||
fs.getFsStatistics(), getTestTracingContext(fs, false));
|
||||
abfsInputStream.read();
|
||||
IOStatistics ioStatistics = extractStatistics(fs);
|
||||
LOG.info("AbfsInputStreamStats info: {}",
|
||||
|
|
|
@ -38,6 +38,7 @@ import org.apache.hadoop.fs.Path;
|
|||
import org.apache.hadoop.fs.RemoteIterator;
|
||||
import org.apache.hadoop.fs.azurebfs.services.AbfsListStatusRemoteIterator;
|
||||
import org.apache.hadoop.fs.azurebfs.services.ListingSupport;
|
||||
import org.apache.hadoop.fs.azurebfs.utils.TracingContext;
|
||||
|
||||
import static org.mockito.ArgumentMatchers.any;
|
||||
import static org.mockito.ArgumentMatchers.anyBoolean;
|
||||
|
@ -64,7 +65,8 @@ public class ITestAbfsListStatusRemoteIterator extends AbstractAbfsIntegrationTe
|
|||
|
||||
ListingSupport listngSupport = Mockito.spy(getFileSystem().getAbfsStore());
|
||||
RemoteIterator<FileStatus> fsItr = new AbfsListStatusRemoteIterator(
|
||||
getFileSystem().getFileStatus(testDir), listngSupport);
|
||||
getFileSystem().getFileStatus(testDir), listngSupport,
|
||||
getTestTracingContext(getFileSystem(), true));
|
||||
Assertions.assertThat(fsItr)
|
||||
.describedAs("RemoteIterator should be instance of "
|
||||
+ "AbfsListStatusRemoteIterator by default")
|
||||
|
@ -88,7 +90,8 @@ public class ITestAbfsListStatusRemoteIterator extends AbstractAbfsIntegrationTe
|
|||
verify(listngSupport, Mockito.atLeast(minNumberOfInvokations))
|
||||
.listStatus(any(Path.class), nullable(String.class),
|
||||
anyList(), anyBoolean(),
|
||||
nullable(String.class));
|
||||
nullable(String.class),
|
||||
any(TracingContext.class));
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -100,7 +103,8 @@ public class ITestAbfsListStatusRemoteIterator extends AbstractAbfsIntegrationTe
|
|||
|
||||
ListingSupport listngSupport = Mockito.spy(getFileSystem().getAbfsStore());
|
||||
RemoteIterator<FileStatus> fsItr = new AbfsListStatusRemoteIterator(
|
||||
getFileSystem().getFileStatus(testDir), listngSupport);
|
||||
getFileSystem().getFileStatus(testDir), listngSupport,
|
||||
getTestTracingContext(getFileSystem(), true));
|
||||
Assertions.assertThat(fsItr)
|
||||
.describedAs("RemoteIterator should be instance of "
|
||||
+ "AbfsListStatusRemoteIterator by default")
|
||||
|
@ -129,7 +133,8 @@ public class ITestAbfsListStatusRemoteIterator extends AbstractAbfsIntegrationTe
|
|||
verify(listngSupport, Mockito.atLeast(minNumberOfInvokations))
|
||||
.listStatus(any(Path.class), nullable(String.class),
|
||||
anyList(), anyBoolean(),
|
||||
nullable(String.class));
|
||||
nullable(String.class),
|
||||
any(TracingContext.class));
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -205,7 +210,8 @@ public class ITestAbfsListStatusRemoteIterator extends AbstractAbfsIntegrationTe
|
|||
setPageSize(10);
|
||||
RemoteIterator fsItr =
|
||||
new AbfsListStatusRemoteIterator(getFileSystem().getFileStatus(testDir),
|
||||
getFileSystem().getAbfsStore());
|
||||
getFileSystem().getAbfsStore(),
|
||||
getTestTracingContext(getFileSystem(), true));
|
||||
fsItr = Mockito.spy(fsItr);
|
||||
Mockito.doReturn(false).when(fsItr).hasNext();
|
||||
|
||||
|
@ -253,7 +259,7 @@ public class ITestAbfsListStatusRemoteIterator extends AbstractAbfsIntegrationTe
|
|||
ListingSupport lsSupport =getMockListingSupport(exceptionMessage);
|
||||
RemoteIterator fsItr =
|
||||
new AbfsListStatusRemoteIterator(getFileSystem().getFileStatus(testDir),
|
||||
lsSupport);
|
||||
lsSupport, getTestTracingContext(getFileSystem(), true));
|
||||
|
||||
Assertions.assertThatThrownBy(() -> fsItr.next())
|
||||
.describedAs(
|
||||
|
@ -276,19 +282,20 @@ public class ITestAbfsListStatusRemoteIterator extends AbstractAbfsIntegrationTe
|
|||
private ListingSupport getMockListingSupport(String exceptionMessage) {
|
||||
return new ListingSupport() {
|
||||
@Override
|
||||
public FileStatus[] listStatus(Path path) throws IOException {
|
||||
public FileStatus[] listStatus(Path path, TracingContext tracingContext) throws IOException {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public FileStatus[] listStatus(Path path, String startFrom)
|
||||
public FileStatus[] listStatus(Path path, String startFrom, TracingContext tracingContext)
|
||||
throws IOException {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String listStatus(Path path, String startFrom,
|
||||
List<FileStatus> fileStatuses, boolean fetchAll, String continuation)
|
||||
List<FileStatus> fileStatuses, boolean fetchAll,
|
||||
String continuation, TracingContext tracingContext)
|
||||
throws IOException {
|
||||
throw new IOException(exceptionMessage);
|
||||
}
|
||||
|
|
|
@ -28,6 +28,10 @@ import org.junit.runners.Parameterized;
|
|||
import org.apache.hadoop.fs.FSDataInputStream;
|
||||
import org.apache.hadoop.fs.FSDataOutputStream;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.fs.azurebfs.constants.FSOperationType;
|
||||
import org.apache.hadoop.fs.azurebfs.services.AbfsInputStream;
|
||||
import org.apache.hadoop.fs.azurebfs.services.AbfsOutputStream;
|
||||
import org.apache.hadoop.fs.azurebfs.utils.TracingHeaderValidator;
|
||||
|
||||
import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.APPENDBLOB_MAX_WRITE_BUFFER_SIZE;
|
||||
import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.DEFAULT_READ_BUFFER_SIZE;
|
||||
|
@ -65,11 +69,9 @@ public class ITestAbfsReadWriteAndSeek extends AbstractAbfsScaleTest {
|
|||
private void testReadWriteAndSeek(int bufferSize) throws Exception {
|
||||
final AzureBlobFileSystem fs = getFileSystem();
|
||||
final AbfsConfiguration abfsConfiguration = fs.getAbfsStore().getAbfsConfiguration();
|
||||
|
||||
abfsConfiguration.setWriteBufferSize(bufferSize);
|
||||
abfsConfiguration.setReadBufferSize(bufferSize);
|
||||
|
||||
|
||||
final byte[] b = new byte[2 * bufferSize];
|
||||
new Random().nextBytes(b);
|
||||
|
||||
|
@ -80,13 +82,58 @@ public class ITestAbfsReadWriteAndSeek extends AbstractAbfsScaleTest {
|
|||
final byte[] readBuffer = new byte[2 * bufferSize];
|
||||
int result;
|
||||
try (FSDataInputStream inputStream = fs.open(TEST_PATH)) {
|
||||
((AbfsInputStream) inputStream.getWrappedStream()).registerListener(
|
||||
new TracingHeaderValidator(abfsConfiguration.getClientCorrelationId(),
|
||||
fs.getFileSystemId(), FSOperationType.READ, true, 0,
|
||||
((AbfsInputStream) inputStream.getWrappedStream())
|
||||
.getStreamID()));
|
||||
inputStream.seek(bufferSize);
|
||||
result = inputStream.read(readBuffer, bufferSize, bufferSize);
|
||||
assertNotEquals(-1, result);
|
||||
|
||||
//to test tracingHeader for case with bypassReadAhead == true
|
||||
inputStream.seek(0);
|
||||
byte[] temp = new byte[5];
|
||||
int t = inputStream.read(temp, 0, 1);
|
||||
|
||||
inputStream.seek(0);
|
||||
result = inputStream.read(readBuffer, 0, bufferSize);
|
||||
}
|
||||
assertNotEquals("data read in final read()", -1, result);
|
||||
assertArrayEquals(readBuffer, b);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testReadAheadRequestID() throws java.io.IOException {
|
||||
final AzureBlobFileSystem fs = getFileSystem();
|
||||
final AbfsConfiguration abfsConfiguration = fs.getAbfsStore().getAbfsConfiguration();
|
||||
int bufferSize = MIN_BUFFER_SIZE;
|
||||
abfsConfiguration.setReadBufferSize(bufferSize);
|
||||
|
||||
final byte[] b = new byte[bufferSize * 10];
|
||||
new Random().nextBytes(b);
|
||||
try (FSDataOutputStream stream = fs.create(TEST_PATH)) {
|
||||
((AbfsOutputStream) stream.getWrappedStream()).registerListener(
|
||||
new TracingHeaderValidator(abfsConfiguration.getClientCorrelationId(),
|
||||
fs.getFileSystemId(), FSOperationType.WRITE, false, 0,
|
||||
((AbfsOutputStream) stream.getWrappedStream())
|
||||
.getStreamID()));
|
||||
stream.write(b);
|
||||
}
|
||||
|
||||
final byte[] readBuffer = new byte[4 * bufferSize];
|
||||
int result;
|
||||
fs.registerListener(
|
||||
new TracingHeaderValidator(abfsConfiguration.getClientCorrelationId(),
|
||||
fs.getFileSystemId(), FSOperationType.OPEN, false, 0));
|
||||
try (FSDataInputStream inputStream = fs.open(TEST_PATH)) {
|
||||
((AbfsInputStream) inputStream.getWrappedStream()).registerListener(
|
||||
new TracingHeaderValidator(abfsConfiguration.getClientCorrelationId(),
|
||||
fs.getFileSystemId(), FSOperationType.READ, false, 0,
|
||||
((AbfsInputStream) inputStream.getWrappedStream())
|
||||
.getStreamID()));
|
||||
result = inputStream.read(readBuffer, 0, bufferSize*4);
|
||||
}
|
||||
fs.registerListener(null);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -19,12 +19,15 @@
|
|||
package org.apache.hadoop.fs.azurebfs;
|
||||
|
||||
import java.io.FileNotFoundException;
|
||||
import java.io.IOException;
|
||||
import java.util.Random;
|
||||
|
||||
import org.junit.Test;
|
||||
|
||||
import org.apache.hadoop.fs.FSDataOutputStream;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.fs.azurebfs.constants.FSOperationType;
|
||||
import org.apache.hadoop.fs.azurebfs.utils.TracingHeaderValidator;
|
||||
import org.apache.hadoop.fs.contract.ContractTestUtils;
|
||||
|
||||
/**
|
||||
|
@ -76,4 +79,14 @@ public class ITestAzureBlobFileSystemAppend extends
|
|||
fs.mkdirs(folderPath);
|
||||
fs.append(folderPath);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testTracingForAppend() throws IOException {
|
||||
AzureBlobFileSystem fs = getFileSystem();
|
||||
fs.create(TEST_FILE_PATH);
|
||||
fs.registerListener(new TracingHeaderValidator(
|
||||
fs.getAbfsStore().getAbfsConfiguration().getClientCorrelationId(),
|
||||
fs.getFileSystemId(), FSOperationType.APPEND, false, 0));
|
||||
fs.append(TEST_FILE_PATH, 10);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -21,11 +21,14 @@ package org.apache.hadoop.fs.azurebfs;
|
|||
import java.io.IOException;
|
||||
import java.util.EnumSet;
|
||||
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.fs.XAttrSetFlag;
|
||||
import org.junit.Assume;
|
||||
import org.junit.Test;
|
||||
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.fs.XAttrSetFlag;
|
||||
import org.apache.hadoop.fs.azurebfs.constants.FSOperationType;
|
||||
import org.apache.hadoop.fs.azurebfs.utils.TracingHeaderValidator;
|
||||
|
||||
import static org.apache.hadoop.test.LambdaTestUtils.intercept;
|
||||
|
||||
/**
|
||||
|
@ -42,7 +45,8 @@ public class ITestAzureBlobFileSystemAttributes extends AbstractAbfsIntegrationT
|
|||
@Test
|
||||
public void testSetGetXAttr() throws Exception {
|
||||
AzureBlobFileSystem fs = getFileSystem();
|
||||
Assume.assumeTrue(fs.getIsNamespaceEnabled());
|
||||
AbfsConfiguration conf = fs.getAbfsStore().getAbfsConfiguration();
|
||||
Assume.assumeTrue(getIsNamespaceEnabled(fs));
|
||||
|
||||
byte[] attributeValue1 = fs.getAbfsStore().encodeAttribute("hi");
|
||||
byte[] attributeValue2 = fs.getAbfsStore().encodeAttribute("你好");
|
||||
|
@ -55,8 +59,13 @@ public class ITestAzureBlobFileSystemAttributes extends AbstractAbfsIntegrationT
|
|||
assertNull(fs.getXAttr(testFile, attributeName1));
|
||||
|
||||
// after setting the xAttr on the file, the value should be retrievable
|
||||
fs.registerListener(
|
||||
new TracingHeaderValidator(conf.getClientCorrelationId(),
|
||||
fs.getFileSystemId(), FSOperationType.SET_ATTR, true, 0));
|
||||
fs.setXAttr(testFile, attributeName1, attributeValue1);
|
||||
fs.setListenerOperation(FSOperationType.GET_ATTR);
|
||||
assertArrayEquals(attributeValue1, fs.getXAttr(testFile, attributeName1));
|
||||
fs.registerListener(null);
|
||||
|
||||
// after setting a second xAttr on the file, the first xAttr values should not be overwritten
|
||||
fs.setXAttr(testFile, attributeName2, attributeValue2);
|
||||
|
@ -67,7 +76,7 @@ public class ITestAzureBlobFileSystemAttributes extends AbstractAbfsIntegrationT
|
|||
@Test
|
||||
public void testSetGetXAttrCreateReplace() throws Exception {
|
||||
AzureBlobFileSystem fs = getFileSystem();
|
||||
Assume.assumeTrue(fs.getIsNamespaceEnabled());
|
||||
Assume.assumeTrue(getIsNamespaceEnabled(fs));
|
||||
byte[] attributeValue = fs.getAbfsStore().encodeAttribute("one");
|
||||
String attributeName = "user.someAttribute";
|
||||
Path testFile = path("createReplaceXAttr");
|
||||
|
@ -84,7 +93,7 @@ public class ITestAzureBlobFileSystemAttributes extends AbstractAbfsIntegrationT
|
|||
@Test
|
||||
public void testSetGetXAttrReplace() throws Exception {
|
||||
AzureBlobFileSystem fs = getFileSystem();
|
||||
Assume.assumeTrue(fs.getIsNamespaceEnabled());
|
||||
Assume.assumeTrue(getIsNamespaceEnabled(fs));
|
||||
byte[] attributeValue1 = fs.getAbfsStore().encodeAttribute("one");
|
||||
byte[] attributeValue2 = fs.getAbfsStore().encodeAttribute("two");
|
||||
String attributeName = "user.someAttribute";
|
||||
|
|
|
@ -209,55 +209,55 @@ public class ITestAzureBlobFileSystemAuthorization extends AbstractAbfsIntegrati
|
|||
|
||||
@Test
|
||||
public void testSetOwnerUnauthorized() throws Exception {
|
||||
Assume.assumeTrue(this.getFileSystem().getIsNamespaceEnabled());
|
||||
Assume.assumeTrue(getIsNamespaceEnabled(getFileSystem()));
|
||||
runTest(FileSystemOperations.SetOwner, true);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSetPermissionUnauthorized() throws Exception {
|
||||
Assume.assumeTrue(this.getFileSystem().getIsNamespaceEnabled());
|
||||
Assume.assumeTrue(getIsNamespaceEnabled(getFileSystem()));
|
||||
runTest(FileSystemOperations.SetPermissions, true);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testModifyAclEntriesUnauthorized() throws Exception {
|
||||
Assume.assumeTrue(this.getFileSystem().getIsNamespaceEnabled());
|
||||
Assume.assumeTrue(getIsNamespaceEnabled(getFileSystem()));
|
||||
runTest(FileSystemOperations.ModifyAclEntries, true);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testRemoveAclEntriesUnauthorized() throws Exception {
|
||||
Assume.assumeTrue(this.getFileSystem().getIsNamespaceEnabled());
|
||||
Assume.assumeTrue(getIsNamespaceEnabled(getFileSystem()));
|
||||
runTest(FileSystemOperations.RemoveAclEntries, true);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testRemoveDefaultAclUnauthorized() throws Exception {
|
||||
Assume.assumeTrue(this.getFileSystem().getIsNamespaceEnabled());
|
||||
Assume.assumeTrue(getIsNamespaceEnabled(getFileSystem()));
|
||||
runTest(FileSystemOperations.RemoveDefaultAcl, true);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testRemoveAclUnauthorized() throws Exception {
|
||||
Assume.assumeTrue(this.getFileSystem().getIsNamespaceEnabled());
|
||||
Assume.assumeTrue(getIsNamespaceEnabled(getFileSystem()));
|
||||
runTest(FileSystemOperations.RemoveAcl, true);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSetAclUnauthorized() throws Exception {
|
||||
Assume.assumeTrue(this.getFileSystem().getIsNamespaceEnabled());
|
||||
Assume.assumeTrue(getIsNamespaceEnabled(getFileSystem()));
|
||||
runTest(FileSystemOperations.SetAcl, true);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGetAclStatusAuthorized() throws Exception {
|
||||
Assume.assumeTrue(this.getFileSystem().getIsNamespaceEnabled());
|
||||
Assume.assumeTrue(getIsNamespaceEnabled(getFileSystem()));
|
||||
runTest(FileSystemOperations.GetAcl, false);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGetAclStatusUnauthorized() throws Exception {
|
||||
Assume.assumeTrue(this.getFileSystem().getIsNamespaceEnabled());
|
||||
Assume.assumeTrue(getIsNamespaceEnabled(getFileSystem()));
|
||||
runTest(FileSystemOperations.GetAcl, true);
|
||||
}
|
||||
|
||||
|
|
|
@ -43,7 +43,7 @@ public class ITestAzureBlobFileSystemBackCompat extends
|
|||
public void testBlobBackCompat() throws Exception {
|
||||
final AzureBlobFileSystem fs = this.getFileSystem();
|
||||
Assume.assumeFalse("This test does not support namespace enabled account",
|
||||
this.getFileSystem().getIsNamespaceEnabled());
|
||||
getIsNamespaceEnabled(getFileSystem()));
|
||||
String storageConnectionString = getBlobConnectionString();
|
||||
CloudStorageAccount storageAccount = CloudStorageAccount.parse(storageConnectionString);
|
||||
CloudBlobClient blobClient = storageAccount.createCloudBlobClient();
|
||||
|
|
|
@ -188,7 +188,9 @@ public class ITestAzureBlobFileSystemCheckAccess
|
|||
// acts as noop
|
||||
AzureBlobFileSystemStore mockAbfsStore =
|
||||
Mockito.mock(AzureBlobFileSystemStore.class);
|
||||
Mockito.when(mockAbfsStore.getIsNamespaceEnabled()).thenReturn(true);
|
||||
Mockito.when(mockAbfsStore
|
||||
.getIsNamespaceEnabled(getTestTracingContext(getFileSystem(), false)))
|
||||
.thenReturn(true);
|
||||
Field abfsStoreField = AzureBlobFileSystem.class.getDeclaredField(
|
||||
"abfsStore");
|
||||
abfsStoreField.setAccessible(true);
|
||||
|
|
|
@ -37,12 +37,15 @@ import org.apache.hadoop.fs.permission.FsAction;
|
|||
import org.apache.hadoop.fs.permission.FsPermission;
|
||||
import org.apache.hadoop.test.GenericTestUtils;
|
||||
|
||||
import org.apache.hadoop.fs.azurebfs.constants.FSOperationType;
|
||||
import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException;
|
||||
import org.apache.hadoop.fs.azurebfs.contracts.exceptions.ConcurrentWriteOperationDetectedException;
|
||||
import org.apache.hadoop.fs.azurebfs.services.AbfsClient;
|
||||
import org.apache.hadoop.fs.azurebfs.services.TestAbfsClient;
|
||||
import org.apache.hadoop.fs.azurebfs.services.AbfsHttpOperation;
|
||||
import org.apache.hadoop.fs.azurebfs.services.AbfsRestOperation;
|
||||
import org.apache.hadoop.fs.azurebfs.services.TestAbfsClient;
|
||||
import org.apache.hadoop.fs.azurebfs.utils.TracingContext;
|
||||
import org.apache.hadoop.fs.azurebfs.utils.TracingHeaderValidator;
|
||||
|
||||
import static java.net.HttpURLConnection.HTTP_CONFLICT;
|
||||
import static java.net.HttpURLConnection.HTTP_INTERNAL_ERROR;
|
||||
|
@ -95,7 +98,12 @@ public class ITestAzureBlobFileSystemCreate extends
|
|||
fail("Should've thrown");
|
||||
} catch (FileNotFoundException expected) {
|
||||
}
|
||||
fs.registerListener(new TracingHeaderValidator(
|
||||
fs.getAbfsStore().getAbfsConfiguration().getClientCorrelationId(),
|
||||
fs.getFileSystemId(), FSOperationType.MKDIR, false, 0));
|
||||
fs.mkdirs(TEST_FOLDER_PATH);
|
||||
fs.registerListener(null);
|
||||
|
||||
fs.createNonRecursive(testFile, true, 1024, (short) 1, 1024, null)
|
||||
.close();
|
||||
assertIsFile(fs, testFile);
|
||||
|
@ -262,8 +270,12 @@ public class ITestAzureBlobFileSystemCreate extends
|
|||
fs.getInstrumentationMap());
|
||||
|
||||
// Case 2: Not Overwrite - File pre-exists
|
||||
fs.registerListener(new TracingHeaderValidator(
|
||||
fs.getAbfsStore().getAbfsConfiguration().getClientCorrelationId(),
|
||||
fs.getFileSystemId(), FSOperationType.CREATE, false, 0));
|
||||
intercept(FileAlreadyExistsException.class,
|
||||
() -> fs.create(nonOverwriteFile, false));
|
||||
fs.registerListener(null);
|
||||
|
||||
// One request to server to create path should be issued
|
||||
createRequestCount++;
|
||||
|
@ -289,7 +301,11 @@ public class ITestAzureBlobFileSystemCreate extends
|
|||
fs.getInstrumentationMap());
|
||||
|
||||
// Case 4: Overwrite - File pre-exists
|
||||
fs.registerListener(new TracingHeaderValidator(
|
||||
fs.getAbfsStore().getAbfsConfiguration().getClientCorrelationId(),
|
||||
fs.getFileSystemId(), FSOperationType.CREATE, true, 0));
|
||||
fs.create(overwriteFilePath, true);
|
||||
fs.registerListener(null);
|
||||
|
||||
if (enableConditionalCreateOverwrite) {
|
||||
// Three requests will be sent to server to create path,
|
||||
|
@ -346,7 +362,8 @@ public class ITestAzureBlobFileSystemCreate extends
|
|||
|
||||
AzureBlobFileSystemStore abfsStore = fs.getAbfsStore();
|
||||
abfsStore = setAzureBlobSystemStoreField(abfsStore, "client", mockClient);
|
||||
boolean isNamespaceEnabled = abfsStore.getIsNamespaceEnabled();
|
||||
boolean isNamespaceEnabled = abfsStore
|
||||
.getIsNamespaceEnabled(getTestTracingContext(fs, false));
|
||||
|
||||
AbfsRestOperation successOp = mock(
|
||||
AbfsRestOperation.class);
|
||||
|
@ -377,14 +394,14 @@ public class ITestAzureBlobFileSystemCreate extends
|
|||
.createPath(any(String.class), eq(true), eq(false),
|
||||
isNamespaceEnabled ? any(String.class) : eq(null),
|
||||
isNamespaceEnabled ? any(String.class) : eq(null),
|
||||
any(boolean.class), eq(null));
|
||||
any(boolean.class), eq(null), any(TracingContext.class));
|
||||
|
||||
doThrow(fileNotFoundResponseEx) // Scn1: GFS fails with Http404
|
||||
.doThrow(serverErrorResponseEx) // Scn2: GFS fails with Http500
|
||||
.doReturn(successOp) // Scn3: create overwrite=true fails with Http412
|
||||
.doReturn(successOp) // Scn4: create overwrite=true fails with Http500
|
||||
.when(mockClient)
|
||||
.getPathStatus(any(String.class), eq(false));
|
||||
.getPathStatus(any(String.class), eq(false), any(TracingContext.class));
|
||||
|
||||
// mock for overwrite=true
|
||||
doThrow(
|
||||
|
@ -395,7 +412,7 @@ public class ITestAzureBlobFileSystemCreate extends
|
|||
.createPath(any(String.class), eq(true), eq(true),
|
||||
isNamespaceEnabled ? any(String.class) : eq(null),
|
||||
isNamespaceEnabled ? any(String.class) : eq(null),
|
||||
any(boolean.class), eq(null));
|
||||
any(boolean.class), eq(null), any(TracingContext.class));
|
||||
|
||||
// Scn1: GFS fails with Http404
|
||||
// Sequence of events expected:
|
||||
|
@ -461,7 +478,8 @@ public class ITestAzureBlobFileSystemCreate extends
|
|||
Path testPath = new Path("testFile");
|
||||
intercept(
|
||||
exceptionClass,
|
||||
() -> abfsStore.createFile(testPath, null, true, permission, umask));
|
||||
() -> abfsStore.createFile(testPath, null, true, permission, umask,
|
||||
getTestTracingContext(getFileSystem(), true)));
|
||||
}
|
||||
|
||||
private AbfsRestOperationException getMockAbfsRestOperationException(int status) {
|
||||
|
|
|
@ -398,7 +398,8 @@ public class ITestAzureBlobFileSystemDelegationSAS extends AbstractAbfsIntegrati
|
|||
String src = "/testABC/test.xt";
|
||||
fs.create(new Path(src));
|
||||
AbfsRestOperation abfsHttpRestOperation = fs.getAbfsClient()
|
||||
.renamePath(src, "/testABC" + "/abc.txt", null);
|
||||
.renamePath(src, "/testABC" + "/abc.txt", null,
|
||||
getTestTracingContext(fs, false));
|
||||
AbfsHttpOperation result = abfsHttpRestOperation.getResult();
|
||||
String url = result.getSignatureMaskedUrl();
|
||||
String encodedUrl = result.getSignatureMaskedEncodedUrl();
|
||||
|
@ -414,7 +415,8 @@ public class ITestAzureBlobFileSystemDelegationSAS extends AbstractAbfsIntegrati
|
|||
public void testSignatureMaskOnExceptionMessage() throws Exception {
|
||||
intercept(IOException.class, "sig=XXXX",
|
||||
() -> getFileSystem().getAbfsClient()
|
||||
.renamePath("testABC/test.xt", "testABC/abc.txt", null));
|
||||
.renamePath("testABC/test.xt", "testABC/abc.txt", null,
|
||||
getTestTracingContext(getFileSystem(), false)));
|
||||
}
|
||||
|
||||
@Test
|
||||
|
|
|
@ -30,6 +30,7 @@ import org.assertj.core.api.Assertions;
|
|||
import org.junit.Assume;
|
||||
import org.junit.Test;
|
||||
|
||||
import org.apache.hadoop.fs.azurebfs.constants.FSOperationType;
|
||||
import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException;
|
||||
import org.apache.hadoop.fs.azurebfs.services.AbfsClient;
|
||||
import org.apache.hadoop.fs.azurebfs.services.AbfsHttpOperation;
|
||||
|
@ -37,6 +38,8 @@ import org.apache.hadoop.fs.azurebfs.services.AbfsRestOperation;
|
|||
import org.apache.hadoop.fs.azurebfs.services.TestAbfsClient;
|
||||
import org.apache.hadoop.fs.azurebfs.services.TestAbfsPerfTracker;
|
||||
import org.apache.hadoop.fs.azurebfs.utils.TestMockHelpers;
|
||||
import org.apache.hadoop.fs.azurebfs.utils.TracingContext;
|
||||
import org.apache.hadoop.fs.azurebfs.utils.TracingHeaderValidator;
|
||||
import org.apache.hadoop.fs.FileAlreadyExistsException;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
|
@ -149,9 +152,13 @@ public class ITestAzureBlobFileSystemDelete extends
|
|||
|
||||
es.shutdownNow();
|
||||
Path dir = new Path("/test");
|
||||
fs.registerListener(new TracingHeaderValidator(
|
||||
fs.getAbfsStore().getAbfsConfiguration().getClientCorrelationId(),
|
||||
fs.getFileSystemId(), FSOperationType.DELETE, false, 0));
|
||||
// first try a non-recursive delete, expect failure
|
||||
intercept(FileAlreadyExistsException.class,
|
||||
() -> fs.delete(dir, false));
|
||||
fs.registerListener(null);
|
||||
assertDeleted(fs, dir, true);
|
||||
assertPathDoesNotExist(fs, "deleted", dir);
|
||||
|
||||
|
@ -222,13 +229,14 @@ public class ITestAzureBlobFileSystemDelete extends
|
|||
intercept(AbfsRestOperationException.class,
|
||||
() -> fs.getAbfsStore().delete(
|
||||
new Path("/NonExistingPath"),
|
||||
false));
|
||||
false, getTestTracingContext(fs, false)));
|
||||
|
||||
intercept(AbfsRestOperationException.class,
|
||||
() -> client.deletePath(
|
||||
"/NonExistingPath",
|
||||
false,
|
||||
null));
|
||||
null,
|
||||
getTestTracingContext(fs, true)));
|
||||
|
||||
// mock idempotency check to mimic retried case
|
||||
AbfsClient mockClient = TestAbfsClient.getMockAbfsClient(
|
||||
|
@ -241,7 +249,8 @@ public class ITestAzureBlobFileSystemDelete extends
|
|||
mockStore,
|
||||
"abfsPerfTracker",
|
||||
TestAbfsPerfTracker.getAPerfTrackerInstance(this.getConfiguration()));
|
||||
doCallRealMethod().when(mockStore).delete(new Path("/NonExistingPath"), false);
|
||||
doCallRealMethod().when(mockStore).delete(new Path("/NonExistingPath"),
|
||||
false, getTestTracingContext(fs, false));
|
||||
|
||||
// Case 2: Mimic retried case
|
||||
// Idempotency check on Delete always returns success
|
||||
|
@ -252,13 +261,15 @@ public class ITestAzureBlobFileSystemDelete extends
|
|||
idempotencyRetOp.hardSetResult(HTTP_OK);
|
||||
|
||||
doReturn(idempotencyRetOp).when(mockClient).deleteIdempotencyCheckOp(any());
|
||||
when(mockClient.deletePath("/NonExistingPath", false,
|
||||
null)).thenCallRealMethod();
|
||||
TracingContext tracingContext = getTestTracingContext(fs, false);
|
||||
when(mockClient.deletePath("/NonExistingPath", false, null, tracingContext))
|
||||
.thenCallRealMethod();
|
||||
|
||||
Assertions.assertThat(mockClient.deletePath(
|
||||
"/NonExistingPath",
|
||||
false,
|
||||
null)
|
||||
null,
|
||||
tracingContext)
|
||||
.getResult()
|
||||
.getStatusCode())
|
||||
.describedAs("Idempotency check reports successful "
|
||||
|
@ -266,7 +277,7 @@ public class ITestAzureBlobFileSystemDelete extends
|
|||
.isEqualTo(idempotencyRetOp.getResult().getStatusCode());
|
||||
|
||||
// Call from AzureBlobFileSystemStore should not fail either
|
||||
mockStore.delete(new Path("/NonExistingPath"), false);
|
||||
mockStore.delete(new Path("/NonExistingPath"), false, getTestTracingContext(fs, false));
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -67,7 +67,7 @@ public class ITestAzureBlobFileSystemFileStatus extends
|
|||
|
||||
String errorInStatus = "error in " + fileStatus + " from " + fs;
|
||||
|
||||
if (!fs.getIsNamespaceEnabled()) {
|
||||
if (!getIsNamespaceEnabled(fs)) {
|
||||
assertEquals(errorInStatus + ": owner",
|
||||
fs.getOwnerUser(), fileStatus.getOwner());
|
||||
assertEquals(errorInStatus + ": group",
|
||||
|
|
|
@ -29,8 +29,12 @@ import java.util.concurrent.Executors;
|
|||
import java.util.concurrent.Future;
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.StreamCapabilities;
|
||||
import org.apache.hadoop.fs.azurebfs.constants.FSOperationType;
|
||||
import org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys;
|
||||
import org.apache.hadoop.fs.azurebfs.services.AbfsOutputStream;
|
||||
import org.apache.hadoop.fs.azurebfs.utils.TracingHeaderValidator;
|
||||
import org.hamcrest.core.IsEqual;
|
||||
import org.hamcrest.core.IsNot;
|
||||
import org.junit.Test;
|
||||
|
@ -41,6 +45,7 @@ import org.apache.hadoop.fs.FileStatus;
|
|||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
|
||||
import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_APPEND_BLOB_KEY;
|
||||
import static org.apache.hadoop.fs.contract.ContractTestUtils.assertHasStreamCapabilities;
|
||||
import static org.apache.hadoop.fs.contract.ContractTestUtils.assertLacksStreamCapabilities;
|
||||
|
||||
|
@ -301,6 +306,26 @@ public class ITestAzureBlobFileSystemFlush extends AbstractAbfsScaleTest {
|
|||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testTracingHeaderForAppendBlob() throws Exception {
|
||||
Configuration config = new Configuration(this.getRawConfiguration());
|
||||
config.set(FS_AZURE_APPEND_BLOB_KEY, "abfss:/");
|
||||
config.set(TestConfigurationKeys.FS_AZURE_TEST_APPENDBLOB_ENABLED, "true");
|
||||
AzureBlobFileSystem fs = (AzureBlobFileSystem) FileSystem
|
||||
.newInstance(config);
|
||||
|
||||
byte[] buf = new byte[10];
|
||||
new Random().nextBytes(buf);
|
||||
FSDataOutputStream out = fs.create(new Path("/testFile"));
|
||||
((AbfsOutputStream) out.getWrappedStream()).registerListener(
|
||||
new TracingHeaderValidator(
|
||||
fs.getAbfsStore().getAbfsConfiguration().getClientCorrelationId(),
|
||||
fs.getFileSystemId(), FSOperationType.WRITE, false, 0,
|
||||
((AbfsOutputStream) out.getWrappedStream()).getStreamID()));
|
||||
out.write(buf);
|
||||
out.hsync();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testStreamCapabilitiesWithFlushDisabled() throws Exception {
|
||||
final AzureBlobFileSystem fs = this.getFileSystem();
|
||||
|
|
|
@ -26,13 +26,18 @@ import org.junit.Test;
|
|||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FSDataOutputStream;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.fs.azurebfs.constants.FSOperationType;
|
||||
import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException;
|
||||
import org.apache.hadoop.fs.azurebfs.services.AbfsClient;
|
||||
import org.apache.hadoop.fs.azurebfs.services.AbfsLease;
|
||||
import org.apache.hadoop.fs.azurebfs.services.AbfsOutputStream;
|
||||
import org.apache.hadoop.fs.azurebfs.utils.Listener;
|
||||
import org.apache.hadoop.fs.azurebfs.utils.TracingContext;
|
||||
import org.apache.hadoop.fs.azurebfs.utils.TracingHeaderValidator;
|
||||
import org.apache.hadoop.test.GenericTestUtils;
|
||||
import org.apache.hadoop.test.LambdaTestUtils;
|
||||
|
||||
import static org.mockito.ArgumentMatchers.any;
|
||||
import static org.mockito.ArgumentMatchers.anyInt;
|
||||
import static org.mockito.ArgumentMatchers.anyString;
|
||||
import static org.mockito.Mockito.doThrow;
|
||||
|
@ -210,7 +215,11 @@ public class ITestAzureBlobFileSystemLease extends AbstractAbfsIntegrationTest {
|
|||
out.write(0);
|
||||
out.hsync();
|
||||
|
||||
fs.registerListener(new TracingHeaderValidator(
|
||||
getConfiguration().getClientCorrelationId(), fs.getFileSystemId(),
|
||||
FSOperationType.BREAK_LEASE, false, 0));
|
||||
fs.breakLease(testFilePath);
|
||||
fs.registerListener(null);
|
||||
|
||||
LambdaTestUtils.intercept(IOException.class, ERR_LEASE_EXPIRED, () -> {
|
||||
out.write(1);
|
||||
|
@ -308,29 +317,38 @@ public class ITestAzureBlobFileSystemLease extends AbstractAbfsIntegrationTest {
|
|||
final AzureBlobFileSystem fs = getCustomFileSystem(testFilePath.getParent(), 1);
|
||||
fs.mkdirs(testFilePath.getParent());
|
||||
fs.createNewFile(testFilePath);
|
||||
TracingContext tracingContext = getTestTracingContext(fs, true);
|
||||
Listener listener = new TracingHeaderValidator(
|
||||
getConfiguration().getClientCorrelationId(), fs.getFileSystemId(),
|
||||
FSOperationType.TEST_OP, true, 0);
|
||||
tracingContext.setListener(listener);
|
||||
|
||||
AbfsLease lease = new AbfsLease(fs.getAbfsClient(), testFilePath.toUri().getPath());
|
||||
AbfsLease lease = new AbfsLease(fs.getAbfsClient(),
|
||||
testFilePath.toUri().getPath(), tracingContext);
|
||||
Assert.assertNotNull("Did not successfully lease file", lease.getLeaseID());
|
||||
listener.setOperation(FSOperationType.RELEASE_LEASE);
|
||||
lease.free();
|
||||
lease.getTracingContext().setListener(null);
|
||||
Assert.assertEquals("Unexpected acquire retry count", 0, lease.getAcquireRetryCount());
|
||||
|
||||
AbfsClient mockClient = spy(fs.getAbfsClient());
|
||||
|
||||
doThrow(new AbfsLease.LeaseException("failed to acquire 1"))
|
||||
.doThrow(new AbfsLease.LeaseException("failed to acquire 2"))
|
||||
.doCallRealMethod()
|
||||
.when(mockClient).acquireLease(anyString(), anyInt());
|
||||
.doCallRealMethod().when(mockClient)
|
||||
.acquireLease(anyString(), anyInt(), any(TracingContext.class));
|
||||
|
||||
lease = new AbfsLease(mockClient, testFilePath.toUri().getPath(), 5, 1);
|
||||
lease = new AbfsLease(mockClient, testFilePath.toUri().getPath(), 5, 1, tracingContext);
|
||||
Assert.assertNotNull("Acquire lease should have retried", lease.getLeaseID());
|
||||
lease.free();
|
||||
Assert.assertEquals("Unexpected acquire retry count", 2, lease.getAcquireRetryCount());
|
||||
|
||||
doThrow(new AbfsLease.LeaseException("failed to acquire"))
|
||||
.when(mockClient).acquireLease(anyString(), anyInt());
|
||||
doThrow(new AbfsLease.LeaseException("failed to acquire")).when(mockClient)
|
||||
.acquireLease(anyString(), anyInt(), any(TracingContext.class));
|
||||
|
||||
LambdaTestUtils.intercept(AzureBlobFileSystemException.class, () -> {
|
||||
new AbfsLease(mockClient, testFilePath.toUri().getPath(), 5, 1);
|
||||
new AbfsLease(mockClient, testFilePath.toUri().getPath(), 5, 1,
|
||||
tracingContext);
|
||||
});
|
||||
}
|
||||
}
|
||||
|
|
|
@ -35,6 +35,8 @@ import org.apache.hadoop.fs.FSDataOutputStream;
|
|||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.LocatedFileStatus;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.fs.azurebfs.constants.FSOperationType;
|
||||
import org.apache.hadoop.fs.azurebfs.utils.TracingHeaderValidator;
|
||||
import org.apache.hadoop.fs.contract.ContractTestUtils;
|
||||
|
||||
import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.AZURE_LIST_MAX_RESULTS;
|
||||
|
@ -83,6 +85,9 @@ public class ITestAzureBlobFileSystemListStatus extends
|
|||
}
|
||||
|
||||
es.shutdownNow();
|
||||
fs.registerListener(
|
||||
new TracingHeaderValidator(getConfiguration().getClientCorrelationId(),
|
||||
fs.getFileSystemId(), FSOperationType.LISTSTATUS, true, 0));
|
||||
FileStatus[] files = fs.listStatus(new Path("/"));
|
||||
assertEquals(TEST_FILES_NUMBER, files.length /* user directory */);
|
||||
}
|
||||
|
|
|
@ -45,8 +45,9 @@ public class ITestAzureBlobFileSystemMkDir extends AbstractAbfsIntegrationTest {
|
|||
|
||||
@Test
|
||||
public void testCreateDirWithExistingDir() throws Exception {
|
||||
Assume.assumeTrue(DEFAULT_FS_AZURE_ENABLE_MKDIR_OVERWRITE || !getFileSystem()
|
||||
.getIsNamespaceEnabled());
|
||||
Assume.assumeTrue(
|
||||
DEFAULT_FS_AZURE_ENABLE_MKDIR_OVERWRITE || !getIsNamespaceEnabled(
|
||||
getFileSystem()));
|
||||
final AzureBlobFileSystem fs = getFileSystem();
|
||||
Path path = new Path("testFolder");
|
||||
assertMkdirs(fs, path);
|
||||
|
@ -58,7 +59,7 @@ public class ITestAzureBlobFileSystemMkDir extends AbstractAbfsIntegrationTest {
|
|||
Assume.assumeFalse("Ignore test until default overwrite is set to false",
|
||||
DEFAULT_FS_AZURE_ENABLE_MKDIR_OVERWRITE);
|
||||
Assume.assumeTrue("Ignore test for Non-HNS accounts",
|
||||
getFileSystem().getIsNamespaceEnabled());
|
||||
getIsNamespaceEnabled(getFileSystem()));
|
||||
//execute test only for HNS account with default overwrite=false
|
||||
Configuration config = new Configuration(this.getRawConfiguration());
|
||||
config.set(FS_AZURE_ENABLE_MKDIR_OVERWRITE, Boolean.toString(false));
|
||||
|
@ -74,8 +75,8 @@ public class ITestAzureBlobFileSystemMkDir extends AbstractAbfsIntegrationTest {
|
|||
@Test
|
||||
public void createDirWithExistingFilename() throws Exception {
|
||||
Assume.assumeFalse("Ignore test until default overwrite is set to false",
|
||||
DEFAULT_FS_AZURE_ENABLE_MKDIR_OVERWRITE && getFileSystem()
|
||||
.getIsNamespaceEnabled());
|
||||
DEFAULT_FS_AZURE_ENABLE_MKDIR_OVERWRITE && getIsNamespaceEnabled(
|
||||
getFileSystem()));
|
||||
final AzureBlobFileSystem fs = getFileSystem();
|
||||
Path path = new Path("testFilePath");
|
||||
fs.create(path);
|
||||
|
|
|
@ -36,6 +36,7 @@ import org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys;
|
|||
import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException;
|
||||
import org.apache.hadoop.fs.azurebfs.contracts.services.AzureServiceErrorCode;
|
||||
import org.apache.hadoop.fs.azurebfs.services.AuthType;
|
||||
import org.apache.hadoop.fs.azurebfs.utils.TracingContext;
|
||||
import org.apache.hadoop.io.IOUtils;
|
||||
|
||||
import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_ACCOUNT_OAUTH_CLIENT_ID;
|
||||
|
@ -129,26 +130,29 @@ public class ITestAzureBlobFileSystemOauth extends AbstractAbfsIntegrationTest{
|
|||
|
||||
// Use abfsStore in this test to verify the ERROR code in AbfsRestOperationException
|
||||
AzureBlobFileSystemStore abfsStore = fs.getAbfsStore();
|
||||
TracingContext tracingContext = getTestTracingContext(fs, true);
|
||||
// TEST READ FS
|
||||
Map<String, String> properties = abfsStore.getFilesystemProperties();
|
||||
Map<String, String> properties = abfsStore.getFilesystemProperties(tracingContext);
|
||||
// TEST READ FOLDER
|
||||
assertTrue(fs.exists(EXISTED_FOLDER_PATH));
|
||||
|
||||
// TEST DELETE FOLDER
|
||||
try {
|
||||
abfsStore.delete(EXISTED_FOLDER_PATH, true);
|
||||
abfsStore.delete(EXISTED_FOLDER_PATH, true, tracingContext);
|
||||
} catch (AbfsRestOperationException e) {
|
||||
assertEquals(AzureServiceErrorCode.AUTHORIZATION_PERMISSION_MISS_MATCH, e.getErrorCode());
|
||||
}
|
||||
|
||||
// TEST READ FILE
|
||||
try (InputStream inputStream = abfsStore.openFileForRead(EXISTED_FILE_PATH, null)) {
|
||||
try (InputStream inputStream = abfsStore.openFileForRead(EXISTED_FILE_PATH, null,
|
||||
tracingContext)) {
|
||||
assertTrue(inputStream.read() != 0);
|
||||
}
|
||||
|
||||
// TEST WRITE FILE
|
||||
try {
|
||||
abfsStore.openFileForWrite(EXISTED_FILE_PATH, fs.getFsStatistics(), true);
|
||||
abfsStore.openFileForWrite(EXISTED_FILE_PATH, fs.getFsStatistics(), true,
|
||||
tracingContext);
|
||||
} catch (AbfsRestOperationException e) {
|
||||
assertEquals(AzureServiceErrorCode.AUTHORIZATION_PERMISSION_MISS_MATCH, e.getErrorCode());
|
||||
} finally {
|
||||
|
|
|
@ -76,7 +76,7 @@ public class ITestAzureBlobFileSystemPermission extends AbstractAbfsIntegrationT
|
|||
public void testFilePermission() throws Exception {
|
||||
|
||||
final AzureBlobFileSystem fs = this.getFileSystem();
|
||||
Assume.assumeTrue(fs.getIsNamespaceEnabled());
|
||||
Assume.assumeTrue(getIsNamespaceEnabled(fs));
|
||||
fs.getConf().set(CommonConfigurationKeys.FS_PERMISSIONS_UMASK_KEY, DEFAULT_UMASK_VALUE);
|
||||
path = new Path(testRoot, UUID.randomUUID().toString());
|
||||
|
||||
|
@ -92,7 +92,7 @@ public class ITestAzureBlobFileSystemPermission extends AbstractAbfsIntegrationT
|
|||
@Test
|
||||
public void testFolderPermission() throws Exception {
|
||||
final AzureBlobFileSystem fs = this.getFileSystem();
|
||||
Assume.assumeTrue(fs.getIsNamespaceEnabled());
|
||||
Assume.assumeTrue(getIsNamespaceEnabled(fs));
|
||||
fs.getConf().set(CommonConfigurationKeys.FS_PERMISSIONS_UMASK_KEY, "027");
|
||||
|
||||
path = new Path(testRoot, UUID.randomUUID().toString());
|
||||
|
|
|
@ -114,7 +114,7 @@ public class ITestAzureBlobFileSystemRandomRead extends
|
|||
@Test
|
||||
public void testRandomRead() throws Exception {
|
||||
Assume.assumeFalse("This test does not support namespace enabled account",
|
||||
this.getFileSystem().getIsNamespaceEnabled());
|
||||
getIsNamespaceEnabled(getFileSystem()));
|
||||
Path testPath = new Path(TEST_FILE_PREFIX + "_testRandomRead");
|
||||
assumeHugeFileExists(testPath);
|
||||
|
||||
|
@ -445,7 +445,7 @@ public class ITestAzureBlobFileSystemRandomRead extends
|
|||
@Ignore("HADOOP-16915")
|
||||
public void testRandomReadPerformance() throws Exception {
|
||||
Assume.assumeFalse("This test does not support namespace enabled account",
|
||||
this.getFileSystem().getIsNamespaceEnabled());
|
||||
getIsNamespaceEnabled(getFileSystem()));
|
||||
Path testPath = new Path(TEST_FILE_PREFIX + "_testRandomReadPerformance");
|
||||
assumeHugeFileExists(testPath);
|
||||
|
||||
|
@ -506,7 +506,8 @@ public class ITestAzureBlobFileSystemRandomRead extends
|
|||
final AzureBlobFileSystem fs = createTestFile(testFile, 16 * MEGABYTE,
|
||||
1 * MEGABYTE, config);
|
||||
String eTag = fs.getAbfsClient()
|
||||
.getPathStatus(testFile.toUri().getPath(), false)
|
||||
.getPathStatus(testFile.toUri().getPath(), false,
|
||||
getTestTracingContext(fs, false))
|
||||
.getResult()
|
||||
.getResponseHeader(ETAG);
|
||||
|
||||
|
|
|
@ -240,8 +240,8 @@ public class ITestAzureBlobFileSystemRename extends
|
|||
AbfsRestOperation idempotencyRetOp = mock(AbfsRestOperation.class);
|
||||
when(idempotencyRetOp.getResult()).thenReturn(idempotencyRetHttpOp);
|
||||
doReturn(idempotencyRetOp).when(client).renameIdempotencyCheckOp(any(),
|
||||
any(), any());
|
||||
when(client.renamePath(any(), any(), any())).thenCallRealMethod();
|
||||
any(), any(), any());
|
||||
when(client.renamePath(any(), any(), any(), any())).thenCallRealMethod();
|
||||
|
||||
// rename on non-existing source file will trigger idempotency check
|
||||
if (idempotencyRetHttpOp.getStatusCode() == HTTP_OK) {
|
||||
|
@ -249,7 +249,8 @@ public class ITestAzureBlobFileSystemRename extends
|
|||
Assertions.assertThat(client.renamePath(
|
||||
"/NonExistingsourcepath",
|
||||
"/destpath",
|
||||
null)
|
||||
null,
|
||||
getTestTracingContext(fs, true))
|
||||
.getResult()
|
||||
.getStatusCode())
|
||||
.describedAs("Idempotency check reports recent successful "
|
||||
|
@ -261,7 +262,8 @@ public class ITestAzureBlobFileSystemRename extends
|
|||
() -> client.renamePath(
|
||||
"/NonExistingsourcepath",
|
||||
"/destpath",
|
||||
""));
|
||||
"",
|
||||
getTestTracingContext(fs, true)));
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -321,7 +323,8 @@ public class ITestAzureBlobFileSystemRename extends
|
|||
Assertions.assertThat(testClient.renameIdempotencyCheckOp(
|
||||
renameRequestStartTime,
|
||||
op,
|
||||
destinationPath.toUri().getPath())
|
||||
destinationPath.toUri().getPath(),
|
||||
getTestTracingContext(fs, true))
|
||||
.getResult()
|
||||
.getStatusCode())
|
||||
.describedAs(assertMessage)
|
||||
|
|
|
@ -107,7 +107,8 @@ public class ITestAzureBlobFileSystemStoreListStatusWithRange extends
|
|||
@Test
|
||||
public void testListWithRange() throws IOException {
|
||||
try {
|
||||
FileStatus[] listResult = store.listStatus(new Path(path), startFrom);
|
||||
FileStatus[] listResult = store.listStatus(new Path(path), startFrom,
|
||||
getTestTracingContext(fs, true));
|
||||
if (!expectedResult) {
|
||||
Assert.fail("Excepting failure with IllegalArgumentException");
|
||||
}
|
||||
|
|
|
@ -32,7 +32,9 @@ import org.apache.hadoop.fs.FileSystem;
|
|||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.fs.PathIOException;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.azurebfs.constants.FSOperationType;
|
||||
import org.apache.hadoop.fs.azurebfs.utils.AclTestHelpers;
|
||||
import org.apache.hadoop.fs.azurebfs.utils.TracingHeaderValidator;
|
||||
import org.apache.hadoop.fs.permission.AclEntry;
|
||||
import org.apache.hadoop.fs.permission.AclStatus;
|
||||
import org.apache.hadoop.fs.permission.FsAction;
|
||||
|
@ -88,7 +90,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest {
|
|||
@Test
|
||||
public void testModifyAclEntries() throws Exception {
|
||||
final AzureBlobFileSystem fs = this.getFileSystem();
|
||||
assumeTrue(fs.getIsNamespaceEnabled());
|
||||
assumeTrue(getIsNamespaceEnabled(fs));
|
||||
path = new Path(testRoot, UUID.randomUUID().toString());
|
||||
fs.mkdirs(path, FsPermission.createImmutable((short) RWX_RX));
|
||||
|
||||
|
@ -121,7 +123,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest {
|
|||
@Test
|
||||
public void testModifyAclEntriesOnlyAccess() throws Exception {
|
||||
final AzureBlobFileSystem fs = this.getFileSystem();
|
||||
assumeTrue(fs.getIsNamespaceEnabled());
|
||||
assumeTrue(getIsNamespaceEnabled(fs));
|
||||
path = new Path(testRoot, UUID.randomUUID().toString());
|
||||
fs.create(path).close();
|
||||
fs.setPermission(path, FsPermission.createImmutable((short) RW_R));
|
||||
|
@ -145,7 +147,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest {
|
|||
@Test
|
||||
public void testModifyAclEntriesOnlyDefault() throws Exception {
|
||||
final AzureBlobFileSystem fs = this.getFileSystem();
|
||||
assumeTrue(fs.getIsNamespaceEnabled());
|
||||
assumeTrue(getIsNamespaceEnabled(fs));
|
||||
path = new Path(testRoot, UUID.randomUUID().toString());
|
||||
FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) RWX_RX));
|
||||
List<AclEntry> aclSpec = Lists.newArrayList(
|
||||
|
@ -168,7 +170,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest {
|
|||
@Test
|
||||
public void testModifyAclEntriesMinimal() throws Exception {
|
||||
final AzureBlobFileSystem fs = this.getFileSystem();
|
||||
assumeTrue(fs.getIsNamespaceEnabled());
|
||||
assumeTrue(getIsNamespaceEnabled(fs));
|
||||
path = new Path(testRoot, UUID.randomUUID().toString());
|
||||
fs.create(path).close();
|
||||
fs.setPermission(path, FsPermission.createImmutable((short) RW_R));
|
||||
|
@ -186,7 +188,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest {
|
|||
@Test
|
||||
public void testModifyAclEntriesMinimalDefault() throws Exception {
|
||||
final AzureBlobFileSystem fs = this.getFileSystem();
|
||||
assumeTrue(fs.getIsNamespaceEnabled());
|
||||
assumeTrue(getIsNamespaceEnabled(fs));
|
||||
path = new Path(testRoot, UUID.randomUUID().toString());
|
||||
FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) RWX_RX));
|
||||
List<AclEntry> aclSpec = Lists.newArrayList(
|
||||
|
@ -206,7 +208,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest {
|
|||
@Test
|
||||
public void testModifyAclEntriesCustomMask() throws Exception {
|
||||
final AzureBlobFileSystem fs = this.getFileSystem();
|
||||
assumeTrue(fs.getIsNamespaceEnabled());
|
||||
assumeTrue(getIsNamespaceEnabled(fs));
|
||||
path = new Path(testRoot, UUID.randomUUID().toString());
|
||||
fs.create(path).close();
|
||||
fs.setPermission(path, FsPermission.createImmutable((short) RW_R));
|
||||
|
@ -225,7 +227,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest {
|
|||
@Test
|
||||
public void testModifyAclEntriesStickyBit() throws Exception {
|
||||
final AzureBlobFileSystem fs = this.getFileSystem();
|
||||
assumeTrue(fs.getIsNamespaceEnabled());
|
||||
assumeTrue(getIsNamespaceEnabled(fs));
|
||||
path = new Path(testRoot, UUID.randomUUID().toString());
|
||||
FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) 01750));
|
||||
List<AclEntry> aclSpec = Lists.newArrayList(
|
||||
|
@ -255,7 +257,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest {
|
|||
@Test(expected=FileNotFoundException.class)
|
||||
public void testModifyAclEntriesPathNotFound() throws Exception {
|
||||
final AzureBlobFileSystem fs = this.getFileSystem();
|
||||
assumeTrue(fs.getIsNamespaceEnabled());
|
||||
assumeTrue(getIsNamespaceEnabled(fs));
|
||||
path = new Path(testRoot, UUID.randomUUID().toString());
|
||||
// Path has not been created.
|
||||
List<AclEntry> aclSpec = Lists.newArrayList(
|
||||
|
@ -269,7 +271,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest {
|
|||
@Test (expected=Exception.class)
|
||||
public void testModifyAclEntriesDefaultOnFile() throws Exception {
|
||||
final AzureBlobFileSystem fs = this.getFileSystem();
|
||||
assumeTrue(fs.getIsNamespaceEnabled());
|
||||
assumeTrue(getIsNamespaceEnabled(fs));
|
||||
path = new Path(testRoot, UUID.randomUUID().toString());
|
||||
fs.create(path).close();
|
||||
fs.setPermission(path, FsPermission.createImmutable((short) RW_R));
|
||||
|
@ -281,7 +283,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest {
|
|||
@Test
|
||||
public void testModifyAclEntriesWithDefaultMask() throws Exception {
|
||||
final AzureBlobFileSystem fs = this.getFileSystem();
|
||||
assumeTrue(fs.getIsNamespaceEnabled());
|
||||
assumeTrue(getIsNamespaceEnabled(fs));
|
||||
path = new Path(testRoot, UUID.randomUUID().toString());
|
||||
FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) RWX_RX));
|
||||
List<AclEntry> aclSpec = Lists.newArrayList(
|
||||
|
@ -305,7 +307,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest {
|
|||
@Test
|
||||
public void testModifyAclEntriesWithAccessMask() throws Exception {
|
||||
final AzureBlobFileSystem fs = this.getFileSystem();
|
||||
assumeTrue(fs.getIsNamespaceEnabled());
|
||||
assumeTrue(getIsNamespaceEnabled(fs));
|
||||
path = new Path(testRoot, UUID.randomUUID().toString());
|
||||
FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) RWX_RX));
|
||||
List<AclEntry> aclSpec = Lists.newArrayList(
|
||||
|
@ -326,7 +328,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest {
|
|||
@Test(expected=PathIOException.class)
|
||||
public void testModifyAclEntriesWithDuplicateEntries() throws Exception {
|
||||
final AzureBlobFileSystem fs = this.getFileSystem();
|
||||
assumeTrue(fs.getIsNamespaceEnabled());
|
||||
assumeTrue(getIsNamespaceEnabled(fs));
|
||||
path = new Path(testRoot, UUID.randomUUID().toString());
|
||||
FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) RWX_RX));
|
||||
List<AclEntry> aclSpec = Lists.newArrayList(
|
||||
|
@ -342,7 +344,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest {
|
|||
@Test
|
||||
public void testRemoveAclEntries() throws Exception {
|
||||
final AzureBlobFileSystem fs = this.getFileSystem();
|
||||
assumeTrue(fs.getIsNamespaceEnabled());
|
||||
assumeTrue(getIsNamespaceEnabled(fs));
|
||||
path = new Path(testRoot, UUID.randomUUID().toString());
|
||||
FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) RWX_RX));
|
||||
List<AclEntry> aclSpec = Lists.newArrayList(
|
||||
|
@ -370,7 +372,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest {
|
|||
@Test
|
||||
public void testRemoveAclEntriesOnlyAccess() throws Exception {
|
||||
final AzureBlobFileSystem fs = this.getFileSystem();
|
||||
assumeTrue(fs.getIsNamespaceEnabled());
|
||||
assumeTrue(getIsNamespaceEnabled(fs));
|
||||
path = new Path(testRoot, UUID.randomUUID().toString());
|
||||
fs.create(path).close();
|
||||
fs.setPermission(path, FsPermission.createImmutable((short) RW_R));
|
||||
|
@ -395,7 +397,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest {
|
|||
@Test
|
||||
public void testRemoveAclEntriesOnlyDefault() throws Exception {
|
||||
final AzureBlobFileSystem fs = this.getFileSystem();
|
||||
assumeTrue(fs.getIsNamespaceEnabled());
|
||||
assumeTrue(getIsNamespaceEnabled(fs));
|
||||
path = new Path(testRoot, UUID.randomUUID().toString());
|
||||
FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) RWX_RX));
|
||||
List<AclEntry> aclSpec = Lists.newArrayList(
|
||||
|
@ -422,7 +424,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest {
|
|||
@Test
|
||||
public void testRemoveAclEntriesMinimal() throws Exception {
|
||||
final AzureBlobFileSystem fs = this.getFileSystem();
|
||||
assumeTrue(fs.getIsNamespaceEnabled());
|
||||
assumeTrue(getIsNamespaceEnabled(fs));
|
||||
path = new Path(testRoot, UUID.randomUUID().toString());
|
||||
fs.create(path).close();
|
||||
fs.setPermission(path, FsPermission.createImmutable((short) RWX_RW));
|
||||
|
@ -445,7 +447,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest {
|
|||
@Test
|
||||
public void testRemoveAclEntriesMinimalDefault() throws Exception {
|
||||
final AzureBlobFileSystem fs = this.getFileSystem();
|
||||
assumeTrue(fs.getIsNamespaceEnabled());
|
||||
assumeTrue(getIsNamespaceEnabled(fs));
|
||||
path = new Path(testRoot, UUID.randomUUID().toString());
|
||||
FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) RWX_RX));
|
||||
List<AclEntry> aclSpec = Lists.newArrayList(
|
||||
|
@ -473,7 +475,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest {
|
|||
@Test
|
||||
public void testRemoveAclEntriesStickyBit() throws Exception {
|
||||
final AzureBlobFileSystem fs = this.getFileSystem();
|
||||
assumeTrue(fs.getIsNamespaceEnabled());
|
||||
assumeTrue(getIsNamespaceEnabled(fs));
|
||||
path = new Path(testRoot, UUID.randomUUID().toString());
|
||||
FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) 01750));
|
||||
List<AclEntry> aclSpec = Lists.newArrayList(
|
||||
|
@ -501,7 +503,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest {
|
|||
@Test(expected=FileNotFoundException.class)
|
||||
public void testRemoveAclEntriesPathNotFound() throws Exception {
|
||||
final AzureBlobFileSystem fs = this.getFileSystem();
|
||||
assumeTrue(fs.getIsNamespaceEnabled());
|
||||
assumeTrue(getIsNamespaceEnabled(fs));
|
||||
path = new Path(testRoot, UUID.randomUUID().toString());
|
||||
// Path has not been created.
|
||||
List<AclEntry> aclSpec = Lists.newArrayList(
|
||||
|
@ -512,7 +514,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest {
|
|||
@Test(expected=PathIOException.class)
|
||||
public void testRemoveAclEntriesAccessMask() throws Exception {
|
||||
final AzureBlobFileSystem fs = this.getFileSystem();
|
||||
assumeTrue(fs.getIsNamespaceEnabled());
|
||||
assumeTrue(getIsNamespaceEnabled(fs));
|
||||
path = new Path(testRoot, UUID.randomUUID().toString());
|
||||
FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) RWX_RX));
|
||||
List<AclEntry> aclSpec = Lists.newArrayList(
|
||||
|
@ -526,7 +528,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest {
|
|||
@Test(expected=PathIOException.class)
|
||||
public void testRemoveAclEntriesDefaultMask() throws Exception {
|
||||
final AzureBlobFileSystem fs = this.getFileSystem();
|
||||
assumeTrue(fs.getIsNamespaceEnabled());
|
||||
assumeTrue(getIsNamespaceEnabled(fs));
|
||||
path = new Path(testRoot, UUID.randomUUID().toString());
|
||||
FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) RWX_RX));
|
||||
List<AclEntry> aclSpec = Lists.newArrayList(
|
||||
|
@ -540,7 +542,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest {
|
|||
@Test(expected=PathIOException.class)
|
||||
public void testRemoveAclEntriesWithDuplicateEntries() throws Exception {
|
||||
final AzureBlobFileSystem fs = this.getFileSystem();
|
||||
assumeTrue(fs.getIsNamespaceEnabled());
|
||||
assumeTrue(getIsNamespaceEnabled(fs));
|
||||
path = new Path(testRoot, UUID.randomUUID().toString());
|
||||
FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) RWX_RX));
|
||||
List<AclEntry> aclSpec = Lists.newArrayList(
|
||||
|
@ -556,7 +558,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest {
|
|||
@Test
|
||||
public void testRemoveDefaultAcl() throws Exception {
|
||||
final AzureBlobFileSystem fs = this.getFileSystem();
|
||||
assumeTrue(fs.getIsNamespaceEnabled());
|
||||
assumeTrue(getIsNamespaceEnabled(fs));
|
||||
path = new Path(testRoot, UUID.randomUUID().toString());
|
||||
FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) RWX_RX));
|
||||
List<AclEntry> aclSpec = Lists.newArrayList(
|
||||
|
@ -578,7 +580,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest {
|
|||
@Test
|
||||
public void testRemoveDefaultAclOnlyAccess() throws Exception {
|
||||
final AzureBlobFileSystem fs = this.getFileSystem();
|
||||
assumeTrue(fs.getIsNamespaceEnabled());
|
||||
assumeTrue(getIsNamespaceEnabled(fs));
|
||||
path = new Path(testRoot, UUID.randomUUID().toString());
|
||||
fs.create(path).close();
|
||||
fs.setPermission(path, FsPermission.createImmutable((short) RW_R));
|
||||
|
@ -600,7 +602,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest {
|
|||
@Test
|
||||
public void testRemoveDefaultAclOnlyDefault() throws Exception {
|
||||
final AzureBlobFileSystem fs = this.getFileSystem();
|
||||
assumeTrue(fs.getIsNamespaceEnabled());
|
||||
assumeTrue(getIsNamespaceEnabled(fs));
|
||||
path = new Path(testRoot, UUID.randomUUID().toString());
|
||||
FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) RWX_RX));
|
||||
List<AclEntry> aclSpec = Lists.newArrayList(
|
||||
|
@ -616,7 +618,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest {
|
|||
@Test
|
||||
public void testRemoveDefaultAclMinimal() throws Exception {
|
||||
final AzureBlobFileSystem fs = this.getFileSystem();
|
||||
assumeTrue(fs.getIsNamespaceEnabled());
|
||||
assumeTrue(getIsNamespaceEnabled(fs));
|
||||
path = new Path(testRoot, UUID.randomUUID().toString());
|
||||
FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) RWX_RX));
|
||||
fs.removeDefaultAcl(path);
|
||||
|
@ -629,7 +631,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest {
|
|||
@Test
|
||||
public void testRemoveDefaultAclStickyBit() throws Exception {
|
||||
final AzureBlobFileSystem fs = this.getFileSystem();
|
||||
assumeTrue(fs.getIsNamespaceEnabled());
|
||||
assumeTrue(getIsNamespaceEnabled(fs));
|
||||
path = new Path(testRoot, UUID.randomUUID().toString());
|
||||
FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) 01750));
|
||||
List<AclEntry> aclSpec = Lists.newArrayList(
|
||||
|
@ -651,7 +653,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest {
|
|||
@Test(expected=FileNotFoundException.class)
|
||||
public void testRemoveDefaultAclPathNotFound() throws Exception {
|
||||
final AzureBlobFileSystem fs = this.getFileSystem();
|
||||
assumeTrue(fs.getIsNamespaceEnabled());
|
||||
assumeTrue(getIsNamespaceEnabled(fs));
|
||||
path = new Path(testRoot, UUID.randomUUID().toString());
|
||||
// Path has not been created.
|
||||
fs.removeDefaultAcl(path);
|
||||
|
@ -660,7 +662,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest {
|
|||
@Test
|
||||
public void testRemoveAcl() throws Exception {
|
||||
final AzureBlobFileSystem fs = this.getFileSystem();
|
||||
assumeTrue(fs.getIsNamespaceEnabled());
|
||||
assumeTrue(getIsNamespaceEnabled(fs));
|
||||
path = new Path(testRoot, UUID.randomUUID().toString());
|
||||
FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) RWX_RX));
|
||||
List<AclEntry> aclSpec = Lists.newArrayList(
|
||||
|
@ -682,7 +684,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest {
|
|||
@Test
|
||||
public void testRemoveAclMinimalAcl() throws Exception {
|
||||
final AzureBlobFileSystem fs = this.getFileSystem();
|
||||
assumeTrue(fs.getIsNamespaceEnabled());
|
||||
assumeTrue(getIsNamespaceEnabled(fs));
|
||||
path = new Path(testRoot, UUID.randomUUID().toString());
|
||||
fs.create(path).close();
|
||||
fs.setPermission(path, FsPermission.createImmutable((short) RW_R));
|
||||
|
@ -696,7 +698,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest {
|
|||
@Test
|
||||
public void testRemoveAclStickyBit() throws Exception {
|
||||
final AzureBlobFileSystem fs = this.getFileSystem();
|
||||
assumeTrue(fs.getIsNamespaceEnabled());
|
||||
assumeTrue(getIsNamespaceEnabled(fs));
|
||||
path = new Path(testRoot, UUID.randomUUID().toString());
|
||||
FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) 01750));
|
||||
List<AclEntry> aclSpec = Lists.newArrayList(
|
||||
|
@ -716,7 +718,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest {
|
|||
@Test
|
||||
public void testRemoveAclOnlyDefault() throws Exception {
|
||||
final AzureBlobFileSystem fs = this.getFileSystem();
|
||||
assumeTrue(fs.getIsNamespaceEnabled());
|
||||
assumeTrue(getIsNamespaceEnabled(fs));
|
||||
path = new Path(testRoot, UUID.randomUUID().toString());
|
||||
FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) RWX_RX));
|
||||
List<AclEntry> aclSpec = Lists.newArrayList(
|
||||
|
@ -735,7 +737,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest {
|
|||
@Test(expected=FileNotFoundException.class)
|
||||
public void testRemoveAclPathNotFound() throws Exception {
|
||||
final AzureBlobFileSystem fs = this.getFileSystem();
|
||||
assumeTrue(fs.getIsNamespaceEnabled());
|
||||
assumeTrue(getIsNamespaceEnabled(fs));
|
||||
path = new Path(testRoot, UUID.randomUUID().toString());
|
||||
// Path has not been created.
|
||||
fs.removeAcl(path);
|
||||
|
@ -744,7 +746,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest {
|
|||
@Test
|
||||
public void testSetAcl() throws Exception {
|
||||
final AzureBlobFileSystem fs = this.getFileSystem();
|
||||
assumeTrue(fs.getIsNamespaceEnabled());
|
||||
assumeTrue(getIsNamespaceEnabled(fs));
|
||||
path = new Path(testRoot, UUID.randomUUID().toString());
|
||||
FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) RWX_RX));
|
||||
List<AclEntry> aclSpec = Lists.newArrayList(
|
||||
|
@ -770,7 +772,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest {
|
|||
@Test
|
||||
public void testSetAclOnlyAccess() throws Exception {
|
||||
final AzureBlobFileSystem fs = this.getFileSystem();
|
||||
assumeTrue(fs.getIsNamespaceEnabled());
|
||||
assumeTrue(getIsNamespaceEnabled(fs));
|
||||
path = new Path(testRoot, UUID.randomUUID().toString());
|
||||
fs.create(path).close();
|
||||
fs.setPermission(path, FsPermission.createImmutable((short) RW_R));
|
||||
|
@ -791,7 +793,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest {
|
|||
@Test
|
||||
public void testSetAclOnlyDefault() throws Exception {
|
||||
final AzureBlobFileSystem fs = this.getFileSystem();
|
||||
assumeTrue(fs.getIsNamespaceEnabled());
|
||||
assumeTrue(getIsNamespaceEnabled(fs));
|
||||
path = new Path(testRoot, UUID.randomUUID().toString());
|
||||
FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) RWX_RX));
|
||||
List<AclEntry> aclSpec = Lists.newArrayList(
|
||||
|
@ -811,7 +813,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest {
|
|||
@Test
|
||||
public void testSetAclMinimal() throws Exception {
|
||||
final AzureBlobFileSystem fs = this.getFileSystem();
|
||||
assumeTrue(fs.getIsNamespaceEnabled());
|
||||
assumeTrue(getIsNamespaceEnabled(fs));
|
||||
path = new Path(testRoot, UUID.randomUUID().toString());
|
||||
fs.create(path).close();
|
||||
fs.setPermission(path, FsPermission.createImmutable((short) RW_R_R));
|
||||
|
@ -835,7 +837,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest {
|
|||
@Test
|
||||
public void testSetAclMinimalDefault() throws Exception {
|
||||
final AzureBlobFileSystem fs = this.getFileSystem();
|
||||
assumeTrue(fs.getIsNamespaceEnabled());
|
||||
assumeTrue(getIsNamespaceEnabled(fs));
|
||||
path = new Path(testRoot, UUID.randomUUID().toString());
|
||||
FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) RWX_RX));
|
||||
List<AclEntry> aclSpec = Lists.newArrayList(
|
||||
|
@ -855,7 +857,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest {
|
|||
@Test
|
||||
public void testSetAclCustomMask() throws Exception {
|
||||
final AzureBlobFileSystem fs = this.getFileSystem();
|
||||
assumeTrue(fs.getIsNamespaceEnabled());
|
||||
assumeTrue(getIsNamespaceEnabled(fs));
|
||||
path = new Path(testRoot, UUID.randomUUID().toString());
|
||||
fs.create(path).close();
|
||||
fs.setPermission(path, FsPermission.createImmutable((short) RW_R));
|
||||
|
@ -877,7 +879,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest {
|
|||
@Test
|
||||
public void testSetAclStickyBit() throws Exception {
|
||||
final AzureBlobFileSystem fs = this.getFileSystem();
|
||||
assumeTrue(fs.getIsNamespaceEnabled());
|
||||
assumeTrue(getIsNamespaceEnabled(fs));
|
||||
path = new Path(testRoot, UUID.randomUUID().toString());
|
||||
FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) 01750));
|
||||
List<AclEntry> aclSpec = Lists.newArrayList(
|
||||
|
@ -903,7 +905,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest {
|
|||
@Test(expected=FileNotFoundException.class)
|
||||
public void testSetAclPathNotFound() throws Exception {
|
||||
final AzureBlobFileSystem fs = this.getFileSystem();
|
||||
assumeTrue(fs.getIsNamespaceEnabled());
|
||||
assumeTrue(getIsNamespaceEnabled(fs));
|
||||
path = new Path(testRoot, UUID.randomUUID().toString());
|
||||
// Path has not been created.
|
||||
List<AclEntry> aclSpec = Lists.newArrayList(
|
||||
|
@ -917,7 +919,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest {
|
|||
@Test(expected=Exception.class)
|
||||
public void testSetAclDefaultOnFile() throws Exception {
|
||||
final AzureBlobFileSystem fs = this.getFileSystem();
|
||||
assumeTrue(fs.getIsNamespaceEnabled());
|
||||
assumeTrue(getIsNamespaceEnabled(fs));
|
||||
path = new Path(testRoot, UUID.randomUUID().toString());
|
||||
fs.create(path).close();
|
||||
fs.setPermission(path, FsPermission.createImmutable((short) RW_R));
|
||||
|
@ -929,7 +931,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest {
|
|||
@Test
|
||||
public void testSetAclDoesNotChangeDefaultMask() throws Exception {
|
||||
final AzureBlobFileSystem fs = this.getFileSystem();
|
||||
assumeTrue(fs.getIsNamespaceEnabled());
|
||||
assumeTrue(getIsNamespaceEnabled(fs));
|
||||
path = new Path(testRoot, UUID.randomUUID().toString());
|
||||
FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) RWX_RX));
|
||||
List<AclEntry> aclSpec = Lists.newArrayList(
|
||||
|
@ -953,7 +955,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest {
|
|||
@Test(expected=PathIOException.class)
|
||||
public void testSetAclWithDuplicateEntries() throws Exception {
|
||||
final AzureBlobFileSystem fs = this.getFileSystem();
|
||||
assumeTrue(fs.getIsNamespaceEnabled());
|
||||
assumeTrue(getIsNamespaceEnabled(fs));
|
||||
path = new Path(testRoot, UUID.randomUUID().toString());
|
||||
FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) RWX_RX));
|
||||
List<AclEntry> aclSpec = Lists.newArrayList(
|
||||
|
@ -965,7 +967,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest {
|
|||
@Test
|
||||
public void testSetPermission() throws Exception {
|
||||
final AzureBlobFileSystem fs = this.getFileSystem();
|
||||
assumeTrue(fs.getIsNamespaceEnabled());
|
||||
assumeTrue(getIsNamespaceEnabled(fs));
|
||||
path = new Path(testRoot, UUID.randomUUID().toString());
|
||||
FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) RWX_RX));
|
||||
List<AclEntry> aclSpec = Lists.newArrayList(
|
||||
|
@ -992,7 +994,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest {
|
|||
@Test
|
||||
public void testSetPermissionOnlyAccess() throws Exception {
|
||||
final AzureBlobFileSystem fs = this.getFileSystem();
|
||||
assumeTrue(fs.getIsNamespaceEnabled());
|
||||
assumeTrue(getIsNamespaceEnabled(fs));
|
||||
path = new Path(testRoot, UUID.randomUUID().toString());
|
||||
fs.create(path).close();
|
||||
fs.setPermission(path, FsPermission.createImmutable((short) RW_R));
|
||||
|
@ -1014,7 +1016,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest {
|
|||
@Test
|
||||
public void testSetPermissionOnlyDefault() throws Exception {
|
||||
final AzureBlobFileSystem fs = this.getFileSystem();
|
||||
assumeTrue(fs.getIsNamespaceEnabled());
|
||||
assumeTrue(getIsNamespaceEnabled(fs));
|
||||
path = new Path(testRoot, UUID.randomUUID().toString());
|
||||
FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) RWX_RX));
|
||||
List<AclEntry> aclSpec = Lists.newArrayList(
|
||||
|
@ -1038,7 +1040,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest {
|
|||
@Test
|
||||
public void testDefaultAclNewFile() throws Exception {
|
||||
final AzureBlobFileSystem fs = this.getFileSystem();
|
||||
assumeTrue(fs.getIsNamespaceEnabled());
|
||||
assumeTrue(getIsNamespaceEnabled(fs));
|
||||
path = new Path(testRoot, UUID.randomUUID().toString());
|
||||
FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) RWX_RX));
|
||||
List<AclEntry> aclSpec = Lists.newArrayList(
|
||||
|
@ -1058,7 +1060,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest {
|
|||
@Ignore // wait umask fix to be deployed
|
||||
public void testOnlyAccessAclNewFile() throws Exception {
|
||||
final AzureBlobFileSystem fs = this.getFileSystem();
|
||||
assumeTrue(fs.getIsNamespaceEnabled());
|
||||
assumeTrue(getIsNamespaceEnabled(fs));
|
||||
path = new Path(testRoot, UUID.randomUUID().toString());
|
||||
FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) RWX_RX));
|
||||
List<AclEntry> aclSpec = Lists.newArrayList(
|
||||
|
@ -1075,7 +1077,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest {
|
|||
@Test
|
||||
public void testDefaultMinimalAclNewFile() throws Exception {
|
||||
final AzureBlobFileSystem fs = this.getFileSystem();
|
||||
assumeTrue(fs.getIsNamespaceEnabled());
|
||||
assumeTrue(getIsNamespaceEnabled(fs));
|
||||
path = new Path(testRoot, UUID.randomUUID().toString());
|
||||
FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) RWX_RX));
|
||||
List<AclEntry> aclSpec = Lists.newArrayList(
|
||||
|
@ -1094,7 +1096,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest {
|
|||
@Test
|
||||
public void testDefaultAclNewDir() throws Exception {
|
||||
final AzureBlobFileSystem fs = this.getFileSystem();
|
||||
assumeTrue(fs.getIsNamespaceEnabled());
|
||||
assumeTrue(getIsNamespaceEnabled(fs));
|
||||
path = new Path(testRoot, UUID.randomUUID().toString());
|
||||
FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) RWX_RX));
|
||||
List<AclEntry> aclSpec = Lists.newArrayList(
|
||||
|
@ -1120,7 +1122,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest {
|
|||
@Test
|
||||
public void testOnlyAccessAclNewDir() throws Exception {
|
||||
final AzureBlobFileSystem fs = this.getFileSystem();
|
||||
assumeTrue(fs.getIsNamespaceEnabled());
|
||||
assumeTrue(getIsNamespaceEnabled(fs));
|
||||
path = new Path(testRoot, UUID.randomUUID().toString());
|
||||
FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) RWX_RX));
|
||||
List<AclEntry> aclSpec = Lists.newArrayList(
|
||||
|
@ -1137,7 +1139,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest {
|
|||
@Test
|
||||
public void testDefaultMinimalAclNewDir() throws Exception {
|
||||
final AzureBlobFileSystem fs = this.getFileSystem();
|
||||
assumeTrue(fs.getIsNamespaceEnabled());
|
||||
assumeTrue(getIsNamespaceEnabled(fs));
|
||||
path = new Path(testRoot, UUID.randomUUID().toString());
|
||||
FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) RWX_RX));
|
||||
List<AclEntry> aclSpec = Lists.newArrayList(
|
||||
|
@ -1159,7 +1161,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest {
|
|||
@Test
|
||||
public void testDefaultAclNewFileWithMode() throws Exception {
|
||||
final AzureBlobFileSystem fs = this.getFileSystem();
|
||||
assumeTrue(fs.getIsNamespaceEnabled());
|
||||
assumeTrue(getIsNamespaceEnabled(fs));
|
||||
path = new Path(testRoot, UUID.randomUUID().toString());
|
||||
FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) RWX_RX_RX));
|
||||
List<AclEntry> aclSpec = Lists.newArrayList(
|
||||
|
@ -1181,7 +1183,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest {
|
|||
@Test
|
||||
public void testDefaultAclNewDirWithMode() throws Exception {
|
||||
final AzureBlobFileSystem fs = this.getFileSystem();
|
||||
assumeTrue(fs.getIsNamespaceEnabled());
|
||||
assumeTrue(getIsNamespaceEnabled(fs));
|
||||
path = new Path(testRoot, UUID.randomUUID().toString());
|
||||
FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) RWX_RX_RX));
|
||||
List<AclEntry> aclSpec = Lists.newArrayList(
|
||||
|
@ -1205,7 +1207,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest {
|
|||
@Test
|
||||
public void testDefaultAclRenamedFile() throws Exception {
|
||||
final AzureBlobFileSystem fs = this.getFileSystem();
|
||||
assumeTrue(fs.getIsNamespaceEnabled());
|
||||
assumeTrue(getIsNamespaceEnabled(fs));
|
||||
path = new Path(testRoot, UUID.randomUUID().toString());
|
||||
Path dirPath = new Path(path, "dir");
|
||||
FileSystem.mkdirs(fs, dirPath, FsPermission.createImmutable((short) RWX_RX));
|
||||
|
@ -1216,7 +1218,12 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest {
|
|||
fs.create(filePath).close();
|
||||
fs.setPermission(filePath, FsPermission.createImmutable((short) RW_R));
|
||||
Path renamedFilePath = new Path(dirPath, "file1");
|
||||
|
||||
fs.registerListener(new TracingHeaderValidator(
|
||||
fs.getAbfsStore().getAbfsConfiguration().getClientCorrelationId(),
|
||||
fs.getFileSystemId(), FSOperationType.RENAME, true, 0));
|
||||
fs.rename(filePath, renamedFilePath);
|
||||
fs.registerListener(null);
|
||||
AclEntry[] expected = new AclEntry[] { };
|
||||
AclStatus s = fs.getAclStatus(renamedFilePath);
|
||||
AclEntry[] returned = s.getEntries().toArray(new AclEntry[0]);
|
||||
|
@ -1227,7 +1234,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest {
|
|||
@Test
|
||||
public void testDefaultAclRenamedDir() throws Exception {
|
||||
final AzureBlobFileSystem fs = this.getFileSystem();
|
||||
assumeTrue(fs.getIsNamespaceEnabled());
|
||||
assumeTrue(getIsNamespaceEnabled(fs));
|
||||
path = new Path(testRoot, UUID.randomUUID().toString());
|
||||
Path dirPath = new Path(path, "dir");
|
||||
FileSystem.mkdirs(fs, dirPath, FsPermission.createImmutable((short) RWX_RX));
|
||||
|
@ -1248,39 +1255,58 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest {
|
|||
@Test
|
||||
public void testEnsureAclOperationWorksForRoot() throws Exception {
|
||||
final AzureBlobFileSystem fs = this.getFileSystem();
|
||||
assumeTrue(fs.getIsNamespaceEnabled());
|
||||
assumeTrue(getIsNamespaceEnabled(fs));
|
||||
|
||||
Path rootPath = new Path("/");
|
||||
|
||||
List<AclEntry> aclSpec1 = Lists.newArrayList(
|
||||
aclEntry(DEFAULT, GROUP, FOO, ALL),
|
||||
aclEntry(ACCESS, GROUP, BAR, ALL));
|
||||
|
||||
fs.registerListener(new TracingHeaderValidator(
|
||||
fs.getAbfsStore().getAbfsConfiguration().getClientCorrelationId(),
|
||||
fs.getFileSystemId(), FSOperationType.SET_ACL, true, 0));
|
||||
fs.setAcl(rootPath, aclSpec1);
|
||||
|
||||
fs.setListenerOperation(FSOperationType.GET_ACL_STATUS);
|
||||
fs.getAclStatus(rootPath);
|
||||
|
||||
fs.setListenerOperation(FSOperationType.SET_OWNER);
|
||||
fs.setOwner(rootPath, TEST_OWNER, TEST_GROUP);
|
||||
fs.setListenerOperation(FSOperationType.SET_PERMISSION);
|
||||
fs.setPermission(rootPath, new FsPermission("777"));
|
||||
|
||||
List<AclEntry> aclSpec2 = Lists.newArrayList(
|
||||
aclEntry(DEFAULT, USER, FOO, ALL),
|
||||
aclEntry(ACCESS, USER, BAR, ALL));
|
||||
fs.setListenerOperation(FSOperationType.MODIFY_ACL);
|
||||
fs.modifyAclEntries(rootPath, aclSpec2);
|
||||
fs.setListenerOperation(FSOperationType.REMOVE_ACL_ENTRIES);
|
||||
fs.removeAclEntries(rootPath, aclSpec2);
|
||||
fs.setListenerOperation(FSOperationType.REMOVE_DEFAULT_ACL);
|
||||
fs.removeDefaultAcl(rootPath);
|
||||
fs.setListenerOperation(FSOperationType.REMOVE_ACL);
|
||||
fs.removeAcl(rootPath);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSetOwnerForNonNamespaceEnabledAccount() throws Exception {
|
||||
final AzureBlobFileSystem fs = this.getFileSystem();
|
||||
Assume.assumeTrue(!fs.getIsNamespaceEnabled());
|
||||
AbfsConfiguration conf = fs.getAbfsStore().getAbfsConfiguration();
|
||||
Assume.assumeTrue(!getIsNamespaceEnabled(fs));
|
||||
final Path filePath = new Path(methodName.getMethodName());
|
||||
fs.create(filePath);
|
||||
|
||||
assertTrue(fs.exists(filePath));
|
||||
|
||||
TracingHeaderValidator tracingHeaderValidator = new TracingHeaderValidator(
|
||||
conf.getClientCorrelationId(), fs.getFileSystemId(),
|
||||
FSOperationType.GET_FILESTATUS, false, 0);
|
||||
fs.registerListener(tracingHeaderValidator);
|
||||
FileStatus oldFileStatus = fs.getFileStatus(filePath);
|
||||
tracingHeaderValidator.setOperation(FSOperationType.SET_OWNER);
|
||||
fs.setOwner(filePath, TEST_OWNER, TEST_GROUP);
|
||||
fs.registerListener(null);
|
||||
FileStatus newFileStatus = fs.getFileStatus(filePath);
|
||||
|
||||
assertEquals(oldFileStatus.getOwner(), newFileStatus.getOwner());
|
||||
|
@ -1290,7 +1316,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest {
|
|||
@Test
|
||||
public void testSetPermissionForNonNamespaceEnabledAccount() throws Exception {
|
||||
final AzureBlobFileSystem fs = this.getFileSystem();
|
||||
Assume.assumeTrue(!fs.getIsNamespaceEnabled());
|
||||
Assume.assumeTrue(!getIsNamespaceEnabled(fs));
|
||||
final Path filePath = new Path(methodName.getMethodName());
|
||||
fs.create(filePath);
|
||||
|
||||
|
@ -1309,7 +1335,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest {
|
|||
@Test
|
||||
public void testModifyAclEntriesForNonNamespaceEnabledAccount() throws Exception {
|
||||
final AzureBlobFileSystem fs = this.getFileSystem();
|
||||
Assume.assumeTrue(!fs.getIsNamespaceEnabled());
|
||||
Assume.assumeTrue(!getIsNamespaceEnabled(fs));
|
||||
final Path filePath = new Path(methodName.getMethodName());
|
||||
fs.create(filePath);
|
||||
try {
|
||||
|
@ -1326,7 +1352,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest {
|
|||
@Test
|
||||
public void testRemoveAclEntriesEntriesForNonNamespaceEnabledAccount() throws Exception {
|
||||
final AzureBlobFileSystem fs = this.getFileSystem();
|
||||
Assume.assumeTrue(!fs.getIsNamespaceEnabled());
|
||||
Assume.assumeTrue(!getIsNamespaceEnabled(fs));
|
||||
final Path filePath = new Path(methodName.getMethodName());
|
||||
fs.create(filePath);
|
||||
try {
|
||||
|
@ -1343,7 +1369,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest {
|
|||
@Test
|
||||
public void testRemoveDefaultAclForNonNamespaceEnabledAccount() throws Exception {
|
||||
final AzureBlobFileSystem fs = this.getFileSystem();
|
||||
Assume.assumeTrue(!fs.getIsNamespaceEnabled());
|
||||
Assume.assumeTrue(!getIsNamespaceEnabled(fs));
|
||||
final Path filePath = new Path(methodName.getMethodName());
|
||||
fs.create(filePath);
|
||||
try {
|
||||
|
@ -1357,7 +1383,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest {
|
|||
@Test
|
||||
public void testRemoveAclForNonNamespaceEnabledAccount() throws Exception {
|
||||
final AzureBlobFileSystem fs = this.getFileSystem();
|
||||
Assume.assumeTrue(!fs.getIsNamespaceEnabled());
|
||||
Assume.assumeTrue(!getIsNamespaceEnabled(fs));
|
||||
final Path filePath = new Path(methodName.getMethodName());
|
||||
fs.create(filePath);
|
||||
try {
|
||||
|
@ -1371,7 +1397,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest {
|
|||
@Test
|
||||
public void testSetAclForNonNamespaceEnabledAccount() throws Exception {
|
||||
final AzureBlobFileSystem fs = this.getFileSystem();
|
||||
Assume.assumeTrue(!fs.getIsNamespaceEnabled());
|
||||
Assume.assumeTrue(!getIsNamespaceEnabled(fs));
|
||||
final Path filePath = new Path(methodName.getMethodName());
|
||||
fs.create(filePath);
|
||||
try {
|
||||
|
@ -1388,7 +1414,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest {
|
|||
@Test
|
||||
public void testGetAclStatusForNonNamespaceEnabledAccount() throws Exception {
|
||||
final AzureBlobFileSystem fs = this.getFileSystem();
|
||||
Assume.assumeTrue(!fs.getIsNamespaceEnabled());
|
||||
Assume.assumeTrue(!getIsNamespaceEnabled(fs));
|
||||
final Path filePath = new Path(methodName.getMethodName());
|
||||
fs.create(filePath);
|
||||
try {
|
||||
|
|
|
@ -34,6 +34,7 @@ import java.util.Map;
|
|||
import java.util.Optional;
|
||||
import java.util.Random;
|
||||
|
||||
import org.apache.hadoop.fs.azurebfs.utils.TracingContext;
|
||||
import org.apache.hadoop.fs.contract.ContractTestUtils;
|
||||
import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
|
||||
import org.apache.hadoop.util.Lists;
|
||||
|
@ -112,11 +113,13 @@ public class ITestCustomerProvidedKey extends AbstractAbfsIntegrationTest {
|
|||
AbfsClient abfsClient = fs.getAbfsClient();
|
||||
int length = FILE_SIZE;
|
||||
byte[] buffer = new byte[length];
|
||||
final AbfsRestOperation op = abfsClient.getPathStatus(fileName, false);
|
||||
TracingContext tracingContext = getTestTracingContext(fs, false);
|
||||
final AbfsRestOperation op = abfsClient.getPathStatus(fileName, false,
|
||||
tracingContext);
|
||||
final String eTag = op.getResult()
|
||||
.getResponseHeader(HttpHeaderConfigurations.ETAG);
|
||||
AbfsRestOperation abfsRestOperation = abfsClient
|
||||
.read(fileName, 0, buffer, 0, length, eTag, null);
|
||||
.read(fileName, 0, buffer, 0, length, eTag, null, tracingContext);
|
||||
assertCPKHeaders(abfsRestOperation, true);
|
||||
assertResponseHeader(abfsRestOperation, true, X_MS_ENCRYPTION_KEY_SHA256,
|
||||
getCPKSha(fs));
|
||||
|
@ -160,11 +163,13 @@ public class ITestCustomerProvidedKey extends AbstractAbfsIntegrationTest {
|
|||
AbfsClient abfsClient = fs.getAbfsClient();
|
||||
int length = INT_512;
|
||||
byte[] buffer = new byte[length * 4];
|
||||
final AbfsRestOperation op = abfsClient.getPathStatus(fileName, false);
|
||||
TracingContext tracingContext = getTestTracingContext(fs, false);
|
||||
final AbfsRestOperation op = abfsClient
|
||||
.getPathStatus(fileName, false, tracingContext);
|
||||
final String eTag = op.getResult()
|
||||
.getResponseHeader(HttpHeaderConfigurations.ETAG);
|
||||
AbfsRestOperation abfsRestOperation = abfsClient
|
||||
.read(fileName, 0, buffer, 0, length, eTag, null);
|
||||
.read(fileName, 0, buffer, 0, length, eTag, null, tracingContext);
|
||||
assertCPKHeaders(abfsRestOperation, false);
|
||||
assertResponseHeader(abfsRestOperation, false, X_MS_ENCRYPTION_KEY_SHA256,
|
||||
getCPKSha(fs));
|
||||
|
@ -182,7 +187,8 @@ public class ITestCustomerProvidedKey extends AbstractAbfsIntegrationTest {
|
|||
try (AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.newInstance(conf);
|
||||
AbfsClient abfsClient2 = fs2.getAbfsClient()) {
|
||||
LambdaTestUtils.intercept(IOException.class, () -> {
|
||||
abfsClient2.read(fileName, 0, buffer, 0, length, eTag, null);
|
||||
abfsClient2.read(fileName, 0, buffer, 0, length, eTag, null,
|
||||
getTestTracingContext(fs, false));
|
||||
});
|
||||
}
|
||||
}
|
||||
|
@ -200,7 +206,7 @@ public class ITestCustomerProvidedKey extends AbstractAbfsIntegrationTest {
|
|||
byte[] buffer = getRandomBytesArray(5);
|
||||
AbfsClient abfsClient = fs.getAbfsClient();
|
||||
AbfsRestOperation abfsRestOperation = abfsClient
|
||||
.append(fileName, buffer, appendRequestParameters, null);
|
||||
.append(fileName, buffer, appendRequestParameters, null, getTestTracingContext(fs, false));
|
||||
assertCPKHeaders(abfsRestOperation, true);
|
||||
assertResponseHeader(abfsRestOperation, true, X_MS_ENCRYPTION_KEY_SHA256,
|
||||
getCPKSha(fs));
|
||||
|
@ -216,7 +222,8 @@ public class ITestCustomerProvidedKey extends AbstractAbfsIntegrationTest {
|
|||
try (AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.newInstance(conf);
|
||||
AbfsClient abfsClient2 = fs2.getAbfsClient()) {
|
||||
LambdaTestUtils.intercept(IOException.class, () -> {
|
||||
abfsClient2.append(fileName, buffer, appendRequestParameters, null);
|
||||
abfsClient2.append(fileName, buffer, appendRequestParameters, null,
|
||||
getTestTracingContext(fs, false));
|
||||
});
|
||||
}
|
||||
|
||||
|
@ -225,7 +232,8 @@ public class ITestCustomerProvidedKey extends AbstractAbfsIntegrationTest {
|
|||
try (AzureBlobFileSystem fs3 = (AzureBlobFileSystem) FileSystem
|
||||
.get(conf); AbfsClient abfsClient3 = fs3.getAbfsClient()) {
|
||||
LambdaTestUtils.intercept(IOException.class, () -> {
|
||||
abfsClient3.append(fileName, buffer, appendRequestParameters, null);
|
||||
abfsClient3.append(fileName, buffer, appendRequestParameters, null,
|
||||
getTestTracingContext(fs, false));
|
||||
});
|
||||
}
|
||||
}
|
||||
|
@ -243,7 +251,8 @@ public class ITestCustomerProvidedKey extends AbstractAbfsIntegrationTest {
|
|||
byte[] buffer = getRandomBytesArray(5);
|
||||
AbfsClient abfsClient = fs.getAbfsClient();
|
||||
AbfsRestOperation abfsRestOperation = abfsClient
|
||||
.append(fileName, buffer, appendRequestParameters, null);
|
||||
.append(fileName, buffer, appendRequestParameters, null,
|
||||
getTestTracingContext(fs, false));
|
||||
assertCPKHeaders(abfsRestOperation, false);
|
||||
assertResponseHeader(abfsRestOperation, false, X_MS_ENCRYPTION_KEY_SHA256,
|
||||
"");
|
||||
|
@ -259,7 +268,8 @@ public class ITestCustomerProvidedKey extends AbstractAbfsIntegrationTest {
|
|||
try (AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.newInstance(conf);
|
||||
AbfsClient abfsClient2 = fs2.getAbfsClient()) {
|
||||
LambdaTestUtils.intercept(IOException.class, () -> {
|
||||
abfsClient2.append(fileName, buffer, appendRequestParameters, null);
|
||||
abfsClient2.append(fileName, buffer, appendRequestParameters, null,
|
||||
getTestTracingContext(fs, false));
|
||||
});
|
||||
}
|
||||
}
|
||||
|
@ -405,7 +415,8 @@ public class ITestCustomerProvidedKey extends AbstractAbfsIntegrationTest {
|
|||
createFileAndGetContent(fs, testDirName + "/bbb", FILE_SIZE);
|
||||
AbfsClient abfsClient = fs.getAbfsClient();
|
||||
AbfsRestOperation abfsRestOperation = abfsClient
|
||||
.listPath(testDirName, false, INT_50, null);
|
||||
.listPath(testDirName, false, INT_50, null,
|
||||
getTestTracingContext(fs, false));
|
||||
assertListstatus(fs, abfsRestOperation, testPath);
|
||||
|
||||
// Trying with different CPK headers
|
||||
|
@ -415,7 +426,9 @@ public class ITestCustomerProvidedKey extends AbstractAbfsIntegrationTest {
|
|||
"different-1234567890123456789012");
|
||||
AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.newInstance(conf);
|
||||
AbfsClient abfsClient2 = fs2.getAbfsClient();
|
||||
abfsRestOperation = abfsClient2.listPath(testDirName, false, INT_50, null);
|
||||
TracingContext tracingContext = getTestTracingContext(fs, false);
|
||||
abfsRestOperation = abfsClient2.listPath(testDirName, false, INT_50,
|
||||
null, tracingContext);
|
||||
assertListstatus(fs, abfsRestOperation, testPath);
|
||||
|
||||
if (isWithCPK) {
|
||||
|
@ -424,7 +437,7 @@ public class ITestCustomerProvidedKey extends AbstractAbfsIntegrationTest {
|
|||
AzureBlobFileSystem fs3 = (AzureBlobFileSystem) FileSystem.get(conf);
|
||||
AbfsClient abfsClient3 = fs3.getAbfsClient();
|
||||
abfsRestOperation = abfsClient3
|
||||
.listPath(testDirName, false, INT_50, null);
|
||||
.listPath(testDirName, false, INT_50, null, tracingContext);
|
||||
assertListstatus(fs, abfsRestOperation, testPath);
|
||||
}
|
||||
}
|
||||
|
@ -463,11 +476,13 @@ public class ITestCustomerProvidedKey extends AbstractAbfsIntegrationTest {
|
|||
FsAction.EXECUTE, FsAction.EXECUTE);
|
||||
FsPermission umask = new FsPermission(FsAction.NONE, FsAction.NONE,
|
||||
FsAction.NONE);
|
||||
boolean isNamespaceEnabled = fs.getIsNamespaceEnabled();
|
||||
TracingContext tracingContext = getTestTracingContext(fs, false);
|
||||
boolean isNamespaceEnabled = fs.getIsNamespaceEnabled(tracingContext);
|
||||
AbfsRestOperation abfsRestOperation = abfsClient
|
||||
.createPath(testFileName, true, true,
|
||||
isNamespaceEnabled ? getOctalNotation(permission) : null,
|
||||
isNamespaceEnabled ? getOctalNotation(umask) : null, false, null);
|
||||
isNamespaceEnabled ? getOctalNotation(umask) : null, false, null,
|
||||
tracingContext);
|
||||
assertCPKHeaders(abfsRestOperation, isWithCPK);
|
||||
assertResponseHeader(abfsRestOperation, isWithCPK,
|
||||
X_MS_ENCRYPTION_KEY_SHA256, getCPKSha(fs));
|
||||
|
@ -505,7 +520,8 @@ public class ITestCustomerProvidedKey extends AbstractAbfsIntegrationTest {
|
|||
String newName = "/newName";
|
||||
AbfsClient abfsClient = fs.getAbfsClient();
|
||||
AbfsRestOperation abfsRestOperation = abfsClient
|
||||
.renamePath(testFileName, newName, null);
|
||||
.renamePath(testFileName, newName, null,
|
||||
getTestTracingContext(fs, false));
|
||||
assertCPKHeaders(abfsRestOperation, false);
|
||||
assertNoCPKResponseHeadersPresent(abfsRestOperation);
|
||||
|
||||
|
@ -548,7 +564,8 @@ public class ITestCustomerProvidedKey extends AbstractAbfsIntegrationTest {
|
|||
try (AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.newInstance(conf);
|
||||
AbfsClient abfsClient2 = fs2.getAbfsClient()) {
|
||||
LambdaTestUtils.intercept(IOException.class, () -> {
|
||||
abfsClient2.flush(testFileName, 0, false, false, null, null);
|
||||
abfsClient2.flush(testFileName, 0, false, false, null, null,
|
||||
getTestTracingContext(fs, false));
|
||||
});
|
||||
}
|
||||
|
||||
|
@ -558,14 +575,16 @@ public class ITestCustomerProvidedKey extends AbstractAbfsIntegrationTest {
|
|||
try (AzureBlobFileSystem fs3 = (AzureBlobFileSystem) FileSystem
|
||||
.get(conf); AbfsClient abfsClient3 = fs3.getAbfsClient()) {
|
||||
LambdaTestUtils.intercept(IOException.class, () -> {
|
||||
abfsClient3.flush(testFileName, 0, false, false, null, null);
|
||||
abfsClient3.flush(testFileName, 0, false, false, null, null,
|
||||
getTestTracingContext(fs, false));
|
||||
});
|
||||
}
|
||||
}
|
||||
|
||||
// With correct CPK
|
||||
AbfsRestOperation abfsRestOperation = abfsClient
|
||||
.flush(testFileName, 0, false, false, null, null);
|
||||
.flush(testFileName, 0, false, false, null, null,
|
||||
getTestTracingContext(fs, false));
|
||||
assertCPKHeaders(abfsRestOperation, isWithCPK);
|
||||
assertResponseHeader(abfsRestOperation, isWithCPK,
|
||||
X_MS_ENCRYPTION_KEY_SHA256, expectedCPKSha);
|
||||
|
@ -594,7 +613,8 @@ public class ITestCustomerProvidedKey extends AbstractAbfsIntegrationTest {
|
|||
properties.put("key", "val");
|
||||
AbfsRestOperation abfsRestOperation = abfsClient
|
||||
.setPathProperties(testFileName,
|
||||
convertXmsPropertiesToCommaSeparatedString(properties));
|
||||
convertXmsPropertiesToCommaSeparatedString(properties),
|
||||
getTestTracingContext(fs, false));
|
||||
assertCPKHeaders(abfsRestOperation, isWithCPK);
|
||||
assertResponseHeader(abfsRestOperation, isWithCPK,
|
||||
X_MS_ENCRYPTION_KEY_SHA256, getCPKSha(fs));
|
||||
|
@ -619,8 +639,9 @@ public class ITestCustomerProvidedKey extends AbstractAbfsIntegrationTest {
|
|||
createFileAndGetContent(fs, testFileName, FILE_SIZE);
|
||||
|
||||
AbfsClient abfsClient = fs.getAbfsClient();
|
||||
TracingContext tracingContext = getTestTracingContext(fs, false);
|
||||
AbfsRestOperation abfsRestOperation = abfsClient
|
||||
.getPathStatus(testFileName, false);
|
||||
.getPathStatus(testFileName, false, tracingContext);
|
||||
assertCPKHeaders(abfsRestOperation, false);
|
||||
assertResponseHeader(abfsRestOperation, isWithCPK,
|
||||
X_MS_ENCRYPTION_KEY_SHA256, getCPKSha(fs));
|
||||
|
@ -629,7 +650,7 @@ public class ITestCustomerProvidedKey extends AbstractAbfsIntegrationTest {
|
|||
assertResponseHeader(abfsRestOperation, false,
|
||||
X_MS_REQUEST_SERVER_ENCRYPTED, "");
|
||||
|
||||
abfsRestOperation = abfsClient.getPathStatus(testFileName, true);
|
||||
abfsRestOperation = abfsClient.getPathStatus(testFileName, true, tracingContext);
|
||||
assertCPKHeaders(abfsRestOperation, isWithCPK);
|
||||
assertResponseHeader(abfsRestOperation, isWithCPK,
|
||||
X_MS_ENCRYPTION_KEY_SHA256, getCPKSha(fs));
|
||||
|
@ -660,7 +681,8 @@ public class ITestCustomerProvidedKey extends AbstractAbfsIntegrationTest {
|
|||
|
||||
AbfsClient abfsClient = fs.getAbfsClient();
|
||||
AbfsRestOperation abfsRestOperation = abfsClient
|
||||
.deletePath(testFileName, false, null);
|
||||
.deletePath(testFileName, false, null,
|
||||
getTestTracingContext(fs, false));
|
||||
assertCPKHeaders(abfsRestOperation, false);
|
||||
assertNoCPKResponseHeadersPresent(abfsRestOperation);
|
||||
|
||||
|
@ -681,13 +703,14 @@ public class ITestCustomerProvidedKey extends AbstractAbfsIntegrationTest {
|
|||
private void testSetPermission(final boolean isWithCPK) throws Exception {
|
||||
final AzureBlobFileSystem fs = getAbfs(isWithCPK);
|
||||
final String testFileName = "/" + methodName.getMethodName();
|
||||
Assume.assumeTrue(fs.getIsNamespaceEnabled());
|
||||
Assume.assumeTrue(fs.getIsNamespaceEnabled(getTestTracingContext(fs, false)));
|
||||
createFileAndGetContent(fs, testFileName, FILE_SIZE);
|
||||
AbfsClient abfsClient = fs.getAbfsClient();
|
||||
FsPermission permission = new FsPermission(FsAction.EXECUTE,
|
||||
FsAction.EXECUTE, FsAction.EXECUTE);
|
||||
AbfsRestOperation abfsRestOperation = abfsClient
|
||||
.setPermission(testFileName, permission.toString());
|
||||
.setPermission(testFileName, permission.toString(),
|
||||
getTestTracingContext(fs, false));
|
||||
assertCPKHeaders(abfsRestOperation, false);
|
||||
assertNoCPKResponseHeadersPresent(abfsRestOperation);
|
||||
}
|
||||
|
@ -705,7 +728,8 @@ public class ITestCustomerProvidedKey extends AbstractAbfsIntegrationTest {
|
|||
private void testSetAcl(final boolean isWithCPK) throws Exception {
|
||||
final AzureBlobFileSystem fs = getAbfs(isWithCPK);
|
||||
final String testFileName = "/" + methodName.getMethodName();
|
||||
Assume.assumeTrue(fs.getIsNamespaceEnabled());
|
||||
TracingContext tracingContext = getTestTracingContext(fs, false);
|
||||
Assume.assumeTrue(fs.getIsNamespaceEnabled(tracingContext));
|
||||
createFileAndGetContent(fs, testFileName, FILE_SIZE);
|
||||
AbfsClient abfsClient = fs.getAbfsClient();
|
||||
|
||||
|
@ -714,7 +738,8 @@ public class ITestCustomerProvidedKey extends AbstractAbfsIntegrationTest {
|
|||
.deserializeAclSpec(AclEntry.aclSpecToString(aclSpec));
|
||||
|
||||
AbfsRestOperation abfsRestOperation = abfsClient
|
||||
.setAcl(testFileName, AbfsAclHelper.serializeAclSpec(aclEntries));
|
||||
.setAcl(testFileName, AbfsAclHelper.serializeAclSpec(aclEntries),
|
||||
tracingContext);
|
||||
assertCPKHeaders(abfsRestOperation, false);
|
||||
assertNoCPKResponseHeadersPresent(abfsRestOperation);
|
||||
}
|
||||
|
@ -732,10 +757,12 @@ public class ITestCustomerProvidedKey extends AbstractAbfsIntegrationTest {
|
|||
private void testGetAcl(final boolean isWithCPK) throws Exception {
|
||||
final AzureBlobFileSystem fs = getAbfs(isWithCPK);
|
||||
final String testFileName = "/" + methodName.getMethodName();
|
||||
Assume.assumeTrue(fs.getIsNamespaceEnabled());
|
||||
TracingContext tracingContext = getTestTracingContext(fs, false);
|
||||
Assume.assumeTrue(fs.getIsNamespaceEnabled(tracingContext));
|
||||
createFileAndGetContent(fs, testFileName, FILE_SIZE);
|
||||
AbfsClient abfsClient = fs.getAbfsClient();
|
||||
AbfsRestOperation abfsRestOperation = abfsClient.getAclStatus(testFileName);
|
||||
AbfsRestOperation abfsRestOperation =
|
||||
abfsClient.getAclStatus(testFileName, tracingContext);
|
||||
assertCPKHeaders(abfsRestOperation, false);
|
||||
assertNoCPKResponseHeadersPresent(abfsRestOperation);
|
||||
}
|
||||
|
@ -763,7 +790,7 @@ public class ITestCustomerProvidedKey extends AbstractAbfsIntegrationTest {
|
|||
fs.create(new Path(testFileName));
|
||||
AbfsClient abfsClient = fs.getAbfsClient();
|
||||
AbfsRestOperation abfsRestOperation = abfsClient
|
||||
.checkAccess(testFileName, "rwx");
|
||||
.checkAccess(testFileName, "rwx", getTestTracingContext(fs, false));
|
||||
assertCPKHeaders(abfsRestOperation, false);
|
||||
assertNoCPKResponseHeadersPresent(abfsRestOperation);
|
||||
}
|
||||
|
|
|
@ -26,6 +26,7 @@ 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.utils.TracingContext;
|
||||
|
||||
/**
|
||||
* Test FileSystemProperties.
|
||||
|
@ -64,8 +65,10 @@ public class ITestFileSystemProperties extends AbstractAbfsIntegrationTest {
|
|||
|
||||
final Hashtable<String, String> properties = new Hashtable<>();
|
||||
properties.put("key", "{ value: value }");
|
||||
fs.getAbfsStore().setFilesystemProperties(properties);
|
||||
Hashtable<String, String> fetchedProperties = fs.getAbfsStore().getFilesystemProperties();
|
||||
TracingContext tracingContext = getTestTracingContext(fs, true);
|
||||
fs.getAbfsStore().setFilesystemProperties(properties, tracingContext);
|
||||
Hashtable<String, String> fetchedProperties = fs.getAbfsStore()
|
||||
.getFilesystemProperties(tracingContext);
|
||||
|
||||
assertEquals(properties, fetchedProperties);
|
||||
}
|
||||
|
@ -76,9 +79,10 @@ public class ITestFileSystemProperties extends AbstractAbfsIntegrationTest {
|
|||
final Hashtable<String, String> properties = new Hashtable<>();
|
||||
properties.put("key", "{ value: valueTest }");
|
||||
touch(TEST_PATH);
|
||||
fs.getAbfsStore().setPathProperties(TEST_PATH, properties);
|
||||
Hashtable<String, String> fetchedProperties =
|
||||
fs.getAbfsStore().getPathStatus(TEST_PATH);
|
||||
TracingContext tracingContext = getTestTracingContext(fs, true);
|
||||
fs.getAbfsStore().setPathProperties(TEST_PATH, properties, tracingContext);
|
||||
Hashtable<String, String> fetchedProperties = fs.getAbfsStore()
|
||||
.getPathStatus(TEST_PATH, tracingContext);
|
||||
|
||||
assertEquals(properties, fetchedProperties);
|
||||
}
|
||||
|
@ -88,8 +92,10 @@ public class ITestFileSystemProperties extends AbstractAbfsIntegrationTest {
|
|||
final AzureBlobFileSystem fs = getFileSystem();
|
||||
final Hashtable<String, String> properties = new Hashtable<>();
|
||||
properties.put("key", "{ value: value歲 }");
|
||||
fs.getAbfsStore().setFilesystemProperties(properties);
|
||||
Hashtable<String, String> fetchedProperties = fs.getAbfsStore().getFilesystemProperties();
|
||||
TracingContext tracingContext = getTestTracingContext(fs, true);
|
||||
fs.getAbfsStore().setFilesystemProperties(properties, tracingContext);
|
||||
Hashtable<String, String> fetchedProperties = fs.getAbfsStore()
|
||||
.getFilesystemProperties(tracingContext);
|
||||
|
||||
assertEquals(properties, fetchedProperties);
|
||||
}
|
||||
|
@ -100,8 +106,10 @@ public class ITestFileSystemProperties extends AbstractAbfsIntegrationTest {
|
|||
final Hashtable<String, String> properties = new Hashtable<>();
|
||||
properties.put("key", "{ value: valueTest兩 }");
|
||||
touch(TEST_PATH);
|
||||
fs.getAbfsStore().setPathProperties(TEST_PATH, properties);
|
||||
Hashtable<String, String> fetchedProperties = fs.getAbfsStore().getPathStatus(TEST_PATH);
|
||||
TracingContext tracingContext = getTestTracingContext(fs, true);
|
||||
fs.getAbfsStore().setPathProperties(TEST_PATH, properties, tracingContext);
|
||||
Hashtable<String, String> fetchedProperties = fs.getAbfsStore()
|
||||
.getPathStatus(TEST_PATH, tracingContext);
|
||||
|
||||
assertEquals(properties, fetchedProperties);
|
||||
}
|
||||
|
@ -111,8 +119,10 @@ public class ITestFileSystemProperties extends AbstractAbfsIntegrationTest {
|
|||
final AzureBlobFileSystem fs = getFileSystem();
|
||||
final Hashtable<String, String> properties = new Hashtable<>();
|
||||
properties.put("containerForDevTest", "true");
|
||||
fs.getAbfsStore().setFilesystemProperties(properties);
|
||||
Hashtable<String, String> fetchedProperties = fs.getAbfsStore().getFilesystemProperties();
|
||||
TracingContext tracingContext = getTestTracingContext(fs, true);
|
||||
fs.getAbfsStore().setFilesystemProperties(properties, tracingContext);
|
||||
Hashtable<String, String> fetchedProperties = fs.getAbfsStore()
|
||||
.getFilesystemProperties(tracingContext);
|
||||
|
||||
assertEquals(properties, fetchedProperties);
|
||||
}
|
||||
|
|
|
@ -21,7 +21,6 @@ import java.io.FileNotFoundException;
|
|||
import java.io.IOException;
|
||||
import java.util.UUID;
|
||||
|
||||
import org.apache.hadoop.fs.azurebfs.enums.Trilean;
|
||||
import org.junit.Assume;
|
||||
import org.junit.Test;
|
||||
import org.assertj.core.api.Assertions;
|
||||
|
@ -32,7 +31,10 @@ import org.apache.hadoop.fs.azurebfs.services.AbfsClient;
|
|||
import org.apache.hadoop.fs.azurebfs.services.AbfsRestOperation;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.fs.azurebfs.enums.Trilean;
|
||||
import org.apache.hadoop.fs.azurebfs.utils.TracingContext;
|
||||
|
||||
import static org.mockito.ArgumentMatchers.any;
|
||||
import static org.mockito.ArgumentMatchers.anyString;
|
||||
import static org.mockito.Mockito.doReturn;
|
||||
import static org.mockito.Mockito.mock;
|
||||
|
@ -65,7 +67,7 @@ public class ITestGetNameSpaceEnabled extends AbstractAbfsIntegrationTest {
|
|||
Assume.assumeTrue("Skip this test because the account being used for test is a non XNS account",
|
||||
isUsingXNSAccount);
|
||||
assertTrue("Expecting getIsNamespaceEnabled() return true",
|
||||
getFileSystem().getIsNamespaceEnabled());
|
||||
getIsNamespaceEnabled(getFileSystem()));
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -73,26 +75,26 @@ public class ITestGetNameSpaceEnabled extends AbstractAbfsIntegrationTest {
|
|||
Assume.assumeFalse("Skip this test because the account being used for test is a XNS account",
|
||||
isUsingXNSAccount);
|
||||
assertFalse("Expecting getIsNamespaceEnabled() return false",
|
||||
getFileSystem().getIsNamespaceEnabled());
|
||||
getIsNamespaceEnabled(getFileSystem()));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGetIsNamespaceEnabledWhenConfigIsTrue() throws Exception {
|
||||
AzureBlobFileSystem fs = getNewFSWithHnsConf(TRUE_STR);
|
||||
Assertions.assertThat(fs.getIsNamespaceEnabled()).describedAs(
|
||||
Assertions.assertThat(getIsNamespaceEnabled(fs)).describedAs(
|
||||
"getIsNamespaceEnabled should return true when the "
|
||||
+ "config is set as true").isTrue();
|
||||
fs.getAbfsStore().deleteFilesystem();
|
||||
fs.getAbfsStore().deleteFilesystem(getTestTracingContext(fs, false));
|
||||
unsetAndAssert();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGetIsNamespaceEnabledWhenConfigIsFalse() throws Exception {
|
||||
AzureBlobFileSystem fs = getNewFSWithHnsConf(FALSE_STR);
|
||||
Assertions.assertThat(fs.getIsNamespaceEnabled()).describedAs(
|
||||
Assertions.assertThat(getIsNamespaceEnabled(fs)).describedAs(
|
||||
"getIsNamespaceEnabled should return false when the "
|
||||
+ "config is set as false").isFalse();
|
||||
fs.getAbfsStore().deleteFilesystem();
|
||||
fs.getAbfsStore().deleteFilesystem(getTestTracingContext(fs, false));
|
||||
unsetAndAssert();
|
||||
}
|
||||
|
||||
|
@ -101,11 +103,11 @@ public class ITestGetNameSpaceEnabled extends AbstractAbfsIntegrationTest {
|
|||
DEFAULT_FS_AZURE_ACCOUNT_IS_HNS_ENABLED);
|
||||
boolean expectedValue = this.getConfiguration()
|
||||
.getBoolean(FS_AZURE_TEST_NAMESPACE_ENABLED_ACCOUNT, false);
|
||||
Assertions.assertThat(fs.getIsNamespaceEnabled()).describedAs(
|
||||
Assertions.assertThat(getIsNamespaceEnabled(fs)).describedAs(
|
||||
"getIsNamespaceEnabled should return the value "
|
||||
+ "configured for fs.azure.test.namespace.enabled")
|
||||
.isEqualTo(expectedValue);
|
||||
fs.getAbfsStore().deleteFilesystem();
|
||||
fs.getAbfsStore().deleteFilesystem(getTestTracingContext(fs, false));
|
||||
}
|
||||
|
||||
private AzureBlobFileSystem getNewFSWithHnsConf(
|
||||
|
@ -178,7 +180,8 @@ public class ITestGetNameSpaceEnabled extends AbstractAbfsIntegrationTest {
|
|||
.setNamespaceEnabled(Trilean.getTrilean(invalidConf));
|
||||
AbfsClient mockClient =
|
||||
callAbfsGetIsNamespaceEnabledAndReturnMockAbfsClient();
|
||||
verify(mockClient, times(1)).getAclStatus(anyString());
|
||||
verify(mockClient, times(1))
|
||||
.getAclStatus(anyString(), any(TracingContext.class));
|
||||
}
|
||||
|
||||
private void ensureGetAclCallIsNeverMadeForValidConf(String validConf)
|
||||
|
@ -187,14 +190,16 @@ public class ITestGetNameSpaceEnabled extends AbstractAbfsIntegrationTest {
|
|||
.setNamespaceEnabled(Trilean.getTrilean(validConf));
|
||||
AbfsClient mockClient =
|
||||
callAbfsGetIsNamespaceEnabledAndReturnMockAbfsClient();
|
||||
verify(mockClient, never()).getAclStatus(anyString());
|
||||
verify(mockClient, never())
|
||||
.getAclStatus(anyString(), any(TracingContext.class));
|
||||
}
|
||||
|
||||
private void unsetConfAndEnsureGetAclCallIsMadeOnce() throws IOException {
|
||||
this.getFileSystem().getAbfsStore().setNamespaceEnabled(Trilean.UNKNOWN);
|
||||
AbfsClient mockClient =
|
||||
callAbfsGetIsNamespaceEnabledAndReturnMockAbfsClient();
|
||||
verify(mockClient, times(1)).getAclStatus(anyString());
|
||||
verify(mockClient, times(1))
|
||||
.getAclStatus(anyString(), any(TracingContext.class));
|
||||
}
|
||||
|
||||
private AbfsClient callAbfsGetIsNamespaceEnabledAndReturnMockAbfsClient()
|
||||
|
@ -203,9 +208,9 @@ public class ITestGetNameSpaceEnabled extends AbstractAbfsIntegrationTest {
|
|||
final AzureBlobFileSystemStore abfsStore = abfs.getAbfsStore();
|
||||
final AbfsClient mockClient = mock(AbfsClient.class);
|
||||
doReturn(mock(AbfsRestOperation.class)).when(mockClient)
|
||||
.getAclStatus(anyString());
|
||||
.getAclStatus(anyString(), any(TracingContext.class));
|
||||
abfsStore.setClient(mockClient);
|
||||
abfs.getIsNamespaceEnabled();
|
||||
getIsNamespaceEnabled(abfs);
|
||||
return mockClient;
|
||||
}
|
||||
|
||||
|
|
|
@ -54,7 +54,8 @@ public class ITestSharedKeyAuth extends AbstractAbfsIntegrationTest {
|
|||
+ "Authorization header is formed correctly including the "
|
||||
+ "signature.\", 403",
|
||||
() -> {
|
||||
abfsClient.getAclStatus("/");
|
||||
abfsClient
|
||||
.getAclStatus("/", getTestTracingContext(getFileSystem(), false));
|
||||
});
|
||||
}
|
||||
|
||||
|
|
|
@ -58,7 +58,7 @@ public class ITestWasbAbfsCompatibility extends AbstractAbfsIntegrationTest {
|
|||
AzureBlobFileSystem fs = getFileSystem();
|
||||
// test only valid for non-namespace enabled account
|
||||
Assume.assumeFalse("Namespace enabled account does not support this test,",
|
||||
fs.getIsNamespaceEnabled());
|
||||
getIsNamespaceEnabled(fs));
|
||||
|
||||
NativeAzureFileSystem wasb = getWasbFileSystem();
|
||||
|
||||
|
@ -93,7 +93,7 @@ public class ITestWasbAbfsCompatibility extends AbstractAbfsIntegrationTest {
|
|||
AzureBlobFileSystem abfs = getFileSystem();
|
||||
// test only valid for non-namespace enabled account
|
||||
Assume.assumeFalse("Namespace enabled account does not support this test",
|
||||
abfs.getIsNamespaceEnabled());
|
||||
getIsNamespaceEnabled(abfs));
|
||||
|
||||
NativeAzureFileSystem wasb = getWasbFileSystem();
|
||||
|
||||
|
@ -133,7 +133,7 @@ public class ITestWasbAbfsCompatibility extends AbstractAbfsIntegrationTest {
|
|||
AzureBlobFileSystem abfs = getFileSystem();
|
||||
// test only valid for non-namespace enabled account
|
||||
Assume.assumeFalse("Namespace enabled account does not support this test",
|
||||
abfs.getIsNamespaceEnabled());
|
||||
getIsNamespaceEnabled(abfs));
|
||||
|
||||
NativeAzureFileSystem wasb = getWasbFileSystem();
|
||||
|
||||
|
@ -168,7 +168,7 @@ public class ITestWasbAbfsCompatibility extends AbstractAbfsIntegrationTest {
|
|||
AzureBlobFileSystem abfs = getFileSystem();
|
||||
// test only valid for non-namespace enabled account
|
||||
Assume.assumeFalse("Namespace enabled account does not support this test",
|
||||
abfs.getIsNamespaceEnabled());
|
||||
getIsNamespaceEnabled(abfs));
|
||||
|
||||
NativeAzureFileSystem wasb = getWasbFileSystem();
|
||||
|
||||
|
|
|
@ -0,0 +1,201 @@
|
|||
/**
|
||||
* 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.lang.reflect.InvocationTargetException;
|
||||
import java.lang.reflect.Method;
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.assertj.core.api.Assertions;
|
||||
import org.junit.Assume;
|
||||
import org.junit.AssumptionViolatedException;
|
||||
import org.junit.Ignore;
|
||||
import org.junit.Test;
|
||||
|
||||
import org.apache.hadoop.fs.CommonPathCapabilities;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants;
|
||||
import org.apache.hadoop.fs.azurebfs.constants.FSOperationType;
|
||||
import org.apache.hadoop.fs.azurebfs.enums.Trilean;
|
||||
import org.apache.hadoop.fs.azurebfs.services.AbfsRestOperation;
|
||||
import org.apache.hadoop.fs.azurebfs.services.AuthType;
|
||||
import org.apache.hadoop.fs.azurebfs.utils.TracingContext;
|
||||
import org.apache.hadoop.fs.azurebfs.utils.TracingHeaderFormat;
|
||||
import org.apache.hadoop.fs.azurebfs.utils.TracingHeaderValidator;
|
||||
import org.apache.hadoop.fs.permission.FsAction;
|
||||
import org.apache.hadoop.fs.permission.FsPermission;
|
||||
import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
|
||||
|
||||
import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.EMPTY_STRING;
|
||||
import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_CLIENT_CORRELATIONID;
|
||||
import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.MIN_BUFFER_SIZE;
|
||||
|
||||
public class TestTracingContext extends AbstractAbfsIntegrationTest {
|
||||
private static final String[] CLIENT_CORRELATIONID_LIST = {
|
||||
"valid-corr-id-123", "inval!d", ""};
|
||||
private static final int HTTP_CREATED = 201;
|
||||
|
||||
public TestTracingContext() throws Exception {
|
||||
super();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testClientCorrelationId() throws Exception {
|
||||
checkCorrelationConfigValidation(CLIENT_CORRELATIONID_LIST[0], true);
|
||||
checkCorrelationConfigValidation(CLIENT_CORRELATIONID_LIST[1], false);
|
||||
checkCorrelationConfigValidation(CLIENT_CORRELATIONID_LIST[2], false);
|
||||
}
|
||||
|
||||
private String getOctalNotation(FsPermission fsPermission) {
|
||||
Preconditions.checkNotNull(fsPermission, "fsPermission");
|
||||
return String
|
||||
.format(AbfsHttpConstants.PERMISSION_FORMAT, fsPermission.toOctal());
|
||||
}
|
||||
|
||||
private String getRelativePath(final Path path) {
|
||||
Preconditions.checkNotNull(path, "path");
|
||||
return path.toUri().getPath();
|
||||
}
|
||||
|
||||
public void checkCorrelationConfigValidation(String clientCorrelationId,
|
||||
boolean includeInHeader) throws Exception {
|
||||
Configuration conf = getRawConfiguration();
|
||||
conf.set(FS_AZURE_CLIENT_CORRELATIONID, clientCorrelationId);
|
||||
AzureBlobFileSystem fs = (AzureBlobFileSystem) FileSystem.newInstance(conf);
|
||||
|
||||
String correlationID = fs.getClientCorrelationId();
|
||||
if (includeInHeader) {
|
||||
Assertions.assertThat(correlationID)
|
||||
.describedAs("Correlation ID should match config when valid")
|
||||
.isEqualTo(clientCorrelationId);
|
||||
} else {
|
||||
Assertions.assertThat(correlationID)
|
||||
.describedAs("Invalid ID should be replaced with empty string")
|
||||
.isEqualTo(EMPTY_STRING);
|
||||
}
|
||||
TracingContext tracingContext = new TracingContext(clientCorrelationId,
|
||||
fs.getFileSystemId(), FSOperationType.TEST_OP,
|
||||
TracingHeaderFormat.ALL_ID_FORMAT, null);
|
||||
boolean isNamespaceEnabled = fs.getIsNamespaceEnabled(tracingContext);
|
||||
String path = getRelativePath(new Path("/testDir"));
|
||||
String permission = isNamespaceEnabled
|
||||
? getOctalNotation(FsPermission.getDirDefault())
|
||||
: null;
|
||||
String umask = isNamespaceEnabled
|
||||
? getOctalNotation(FsPermission.getUMask(fs.getConf()))
|
||||
: null;
|
||||
|
||||
//request should not fail for invalid clientCorrelationID
|
||||
AbfsRestOperation op = fs.getAbfsClient()
|
||||
.createPath(path, false, true, permission, umask, false, null,
|
||||
tracingContext);
|
||||
|
||||
int statusCode = op.getResult().getStatusCode();
|
||||
Assertions.assertThat(statusCode).describedAs("Request should not fail")
|
||||
.isEqualTo(HTTP_CREATED);
|
||||
|
||||
String requestHeader = op.getResult().getClientRequestId().replace("[", "")
|
||||
.replace("]", "");
|
||||
Assertions.assertThat(requestHeader)
|
||||
.describedAs("Client Request Header should match TracingContext")
|
||||
.isEqualTo(tracingContext.getHeader());
|
||||
}
|
||||
|
||||
@Ignore
|
||||
@Test
|
||||
//call test methods from the respective test classes
|
||||
//can be ignored when running all tests as these get covered
|
||||
public void runCorrelationTestForAllMethods() throws Exception {
|
||||
Map<AbstractAbfsIntegrationTest, Method> testClasses = new HashMap<>();
|
||||
|
||||
testClasses.put(new ITestAzureBlobFileSystemListStatus(), //liststatus
|
||||
ITestAzureBlobFileSystemListStatus.class.getMethod("testListPath"));
|
||||
testClasses.put(new ITestAbfsReadWriteAndSeek(MIN_BUFFER_SIZE), //open,
|
||||
// read, write
|
||||
ITestAbfsReadWriteAndSeek.class.getMethod("testReadAheadRequestID"));
|
||||
testClasses.put(new ITestAbfsReadWriteAndSeek(MIN_BUFFER_SIZE), //read (bypassreadahead)
|
||||
ITestAbfsReadWriteAndSeek.class
|
||||
.getMethod("testReadAndWriteWithDifferentBufferSizesAndSeek"));
|
||||
testClasses.put(new ITestAzureBlobFileSystemAppend(), //append
|
||||
ITestAzureBlobFileSystemAppend.class.getMethod("testTracingForAppend"));
|
||||
testClasses.put(new ITestAzureBlobFileSystemFlush(),
|
||||
ITestAzureBlobFileSystemFlush.class.getMethod(
|
||||
"testTracingHeaderForAppendBlob")); //outputstream (appendblob)
|
||||
testClasses.put(new ITestAzureBlobFileSystemCreate(),
|
||||
ITestAzureBlobFileSystemCreate.class
|
||||
.getMethod("testDefaultCreateOverwriteFileTest")); //create
|
||||
testClasses.put(new ITestAzureBlobFilesystemAcl(),
|
||||
ITestAzureBlobFilesystemAcl.class
|
||||
.getMethod("testDefaultAclRenamedFile")); //rename
|
||||
testClasses.put(new ITestAzureBlobFileSystemDelete(),
|
||||
ITestAzureBlobFileSystemDelete.class
|
||||
.getMethod("testDeleteFirstLevelDirectory")); //delete
|
||||
testClasses.put(new ITestAzureBlobFileSystemCreate(),
|
||||
ITestAzureBlobFileSystemCreate.class
|
||||
.getMethod("testCreateNonRecursive")); //mkdirs
|
||||
testClasses.put(new ITestAzureBlobFileSystemAttributes(),
|
||||
ITestAzureBlobFileSystemAttributes.class
|
||||
.getMethod("testSetGetXAttr")); //setxattr, getxattr
|
||||
testClasses.put(new ITestAzureBlobFilesystemAcl(),
|
||||
ITestAzureBlobFilesystemAcl.class.getMethod(
|
||||
"testEnsureAclOperationWorksForRoot")); // setacl, getaclstatus,
|
||||
// setowner, setpermission, modifyaclentries,
|
||||
// removeaclentries, removedefaultacl, removeacl
|
||||
|
||||
for (AbstractAbfsIntegrationTest testClass : testClasses.keySet()) {
|
||||
try {
|
||||
testClass.setup();
|
||||
testClasses.get(testClass).invoke(testClass);
|
||||
testClass.teardown();
|
||||
} catch (InvocationTargetException e) {
|
||||
if (!(e.getCause() instanceof AssumptionViolatedException)) {
|
||||
throw new IOException(testClasses.get(testClass).getName()
|
||||
+ " failed tracing context validation test");
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testExternalOps() throws Exception {
|
||||
//validate tracing header for access, hasPathCapability
|
||||
AzureBlobFileSystem fs = getFileSystem();
|
||||
|
||||
fs.registerListener(new TracingHeaderValidator(
|
||||
fs.getAbfsStore().getAbfsConfiguration().getClientCorrelationId(),
|
||||
fs.getFileSystemId(), FSOperationType.HAS_PATH_CAPABILITY, false,
|
||||
0));
|
||||
|
||||
// unset namespaceEnabled to call getAcl -> trigger tracing header validator
|
||||
fs.getAbfsStore().setNamespaceEnabled(Trilean.UNKNOWN);
|
||||
fs.hasPathCapability(new Path("/"), CommonPathCapabilities.FS_ACLS);
|
||||
|
||||
Assume.assumeTrue(getIsNamespaceEnabled(getFileSystem()));
|
||||
Assume.assumeTrue(getConfiguration().isCheckAccessEnabled());
|
||||
Assume.assumeTrue(getAuthType() == AuthType.OAuth);
|
||||
|
||||
fs.setListenerOperation(FSOperationType.ACCESS);
|
||||
fs.getAbfsStore().setNamespaceEnabled(Trilean.TRUE);
|
||||
fs.access(new Path("/"), FsAction.READ);
|
||||
}
|
||||
}
|
|
@ -31,6 +31,7 @@ import org.apache.hadoop.fs.azurebfs.AbfsConfiguration;
|
|||
import org.apache.hadoop.fs.azurebfs.AbstractAbfsIntegrationTest;
|
||||
import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystem;
|
||||
import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystemStore;
|
||||
import org.apache.hadoop.fs.azurebfs.utils.TracingContext;
|
||||
import org.junit.Test;
|
||||
|
||||
import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.ONE_MB;
|
||||
|
@ -118,7 +119,8 @@ public class ITestAbfsInputStream extends AbstractAbfsIntegrationTest {
|
|||
doThrow(new IOException())
|
||||
.doCallRealMethod()
|
||||
.when(abfsInputStream)
|
||||
.readRemote(anyLong(), any(), anyInt(), anyInt());
|
||||
.readRemote(anyLong(), any(), anyInt(), anyInt(),
|
||||
any(TracingContext.class));
|
||||
|
||||
iStream = new FSDataInputStream(abfsInputStream);
|
||||
verifyBeforeSeek(abfsInputStream);
|
||||
|
|
|
@ -28,6 +28,7 @@ import org.apache.hadoop.fs.FileSystem;
|
|||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.fs.azurebfs.AbfsConfiguration;
|
||||
import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystem;
|
||||
import org.apache.hadoop.fs.azurebfs.utils.TracingContext;
|
||||
|
||||
import static java.lang.Math.max;
|
||||
import static java.lang.Math.min;
|
||||
|
@ -270,7 +271,8 @@ public class ITestAbfsInputStreamReadFooter extends ITestAbfsInputStream {
|
|||
.getWrappedStream();
|
||||
abfsInputStream = spy(abfsInputStream);
|
||||
doReturn(10).doReturn(10).doCallRealMethod().when(abfsInputStream)
|
||||
.readRemote(anyLong(), any(), anyInt(), anyInt());
|
||||
.readRemote(anyLong(), any(), anyInt(), anyInt(),
|
||||
any(TracingContext.class));
|
||||
|
||||
iStream = new FSDataInputStream(abfsInputStream);
|
||||
seek(iStream, seekPos);
|
||||
|
@ -319,7 +321,8 @@ public class ITestAbfsInputStreamReadFooter extends ITestAbfsInputStream {
|
|||
- someDataLength;
|
||||
doReturn(10).doReturn(secondReturnSize).doCallRealMethod()
|
||||
.when(abfsInputStream)
|
||||
.readRemote(anyLong(), any(), anyInt(), anyInt());
|
||||
.readRemote(anyLong(), any(), anyInt(), anyInt(),
|
||||
any(TracingContext.class));
|
||||
|
||||
iStream = new FSDataInputStream(abfsInputStream);
|
||||
seek(iStream, seekPos);
|
||||
|
|
|
@ -28,6 +28,7 @@ import org.apache.hadoop.fs.FileSystem;
|
|||
import org.apache.hadoop.fs.FSDataInputStream;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystem;
|
||||
import org.apache.hadoop.fs.azurebfs.utils.TracingContext;
|
||||
|
||||
import static org.mockito.ArgumentMatchers.any;
|
||||
import static org.mockito.ArgumentMatchers.anyInt;
|
||||
|
@ -251,7 +252,8 @@ public class ITestAbfsInputStreamSmallFileReads extends ITestAbfsInputStream {
|
|||
.doReturn(10)
|
||||
.doCallRealMethod()
|
||||
.when(abfsInputStream)
|
||||
.readRemote(anyLong(), any(), anyInt(), anyInt());
|
||||
.readRemote(anyLong(), any(), anyInt(), anyInt(),
|
||||
any(TracingContext.class));
|
||||
|
||||
iStream = new FSDataInputStream(abfsInputStream);
|
||||
seek(iStream, seekPos);
|
||||
|
@ -301,7 +303,8 @@ public class ITestAbfsInputStreamSmallFileReads extends ITestAbfsInputStream {
|
|||
.doReturn(secondReturnSize)
|
||||
.doCallRealMethod()
|
||||
.when(abfsInputStream)
|
||||
.readRemote(anyLong(), any(), anyInt(), anyInt());
|
||||
.readRemote(anyLong(), any(), anyInt(), anyInt(),
|
||||
any(TracingContext.class));
|
||||
|
||||
iStream = new FSDataInputStream(abfsInputStream);
|
||||
seek(iStream, seekPos);
|
||||
|
|
|
@ -38,6 +38,7 @@ import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemExc
|
|||
import org.apache.hadoop.fs.azurebfs.contracts.exceptions.TimeoutException;
|
||||
import org.apache.hadoop.fs.azurebfs.contracts.services.ReadBufferStatus;
|
||||
import org.apache.hadoop.fs.azurebfs.utils.TestCachedSASToken;
|
||||
import org.apache.hadoop.fs.azurebfs.utils.TracingContext;
|
||||
|
||||
import static org.mockito.ArgumentMatchers.any;
|
||||
import static org.mockito.Mockito.doReturn;
|
||||
|
@ -93,7 +94,8 @@ public class TestAbfsInputStream extends
|
|||
return client;
|
||||
}
|
||||
|
||||
private AbfsInputStream getAbfsInputStream(AbfsClient mockAbfsClient, String fileName) {
|
||||
private AbfsInputStream getAbfsInputStream(AbfsClient mockAbfsClient,
|
||||
String fileName) throws IOException {
|
||||
AbfsInputStreamContext inputStreamContext = new AbfsInputStreamContext(-1);
|
||||
// Create AbfsInputStream with the client instance
|
||||
AbfsInputStream inputStream = new AbfsInputStream(
|
||||
|
@ -102,7 +104,8 @@ public class TestAbfsInputStream extends
|
|||
FORWARD_SLASH + fileName,
|
||||
THREE_KB,
|
||||
inputStreamContext.withReadBufferSize(ONE_KB).withReadAheadQueueDepth(10).withReadAheadBlockSize(ONE_KB),
|
||||
"eTag");
|
||||
"eTag",
|
||||
getTestTracingContext(null, false));
|
||||
|
||||
inputStream.setCachedSasToken(
|
||||
TestCachedSASToken.getTestCachedSASTokenInstance());
|
||||
|
@ -117,7 +120,7 @@ public class TestAbfsInputStream extends
|
|||
int readAheadQueueDepth,
|
||||
int readBufferSize,
|
||||
boolean alwaysReadBufferSize,
|
||||
int readAheadBlockSize) {
|
||||
int readAheadBlockSize) throws IOException {
|
||||
AbfsInputStreamContext inputStreamContext = new AbfsInputStreamContext(-1);
|
||||
// Create AbfsInputStream with the client instance
|
||||
AbfsInputStream inputStream = new AbfsInputStream(
|
||||
|
@ -129,7 +132,8 @@ public class TestAbfsInputStream extends
|
|||
.withReadAheadQueueDepth(readAheadQueueDepth)
|
||||
.withShouldReadBufferSizeAlways(alwaysReadBufferSize)
|
||||
.withReadAheadBlockSize(readAheadBlockSize),
|
||||
eTag);
|
||||
eTag,
|
||||
getTestTracingContext(getFileSystem(), false));
|
||||
|
||||
inputStream.setCachedSasToken(
|
||||
TestCachedSASToken.getTestCachedSASTokenInstance());
|
||||
|
@ -140,11 +144,13 @@ public class TestAbfsInputStream extends
|
|||
private void queueReadAheads(AbfsInputStream inputStream) {
|
||||
// Mimic AbfsInputStream readAhead queue requests
|
||||
ReadBufferManager.getBufferManager()
|
||||
.queueReadAhead(inputStream, 0, ONE_KB);
|
||||
.queueReadAhead(inputStream, 0, ONE_KB, inputStream.getTracingContext());
|
||||
ReadBufferManager.getBufferManager()
|
||||
.queueReadAhead(inputStream, ONE_KB, ONE_KB);
|
||||
.queueReadAhead(inputStream, ONE_KB, ONE_KB,
|
||||
inputStream.getTracingContext());
|
||||
ReadBufferManager.getBufferManager()
|
||||
.queueReadAhead(inputStream, TWO_KB, TWO_KB);
|
||||
.queueReadAhead(inputStream, TWO_KB, TWO_KB,
|
||||
inputStream.getTracingContext());
|
||||
}
|
||||
|
||||
private void verifyReadCallCount(AbfsClient client, int count) throws
|
||||
|
@ -154,7 +160,7 @@ public class TestAbfsInputStream extends
|
|||
Thread.sleep(1000);
|
||||
verify(client, times(count)).read(any(String.class), any(Long.class),
|
||||
any(byte[].class), any(Integer.class), any(Integer.class),
|
||||
any(String.class), any(String.class));
|
||||
any(String.class), any(String.class), any(TracingContext.class));
|
||||
}
|
||||
|
||||
private void checkEvictedStatus(AbfsInputStream inputStream, int position, boolean expectedToThrowException)
|
||||
|
@ -209,7 +215,7 @@ public class TestAbfsInputStream extends
|
|||
.when(client)
|
||||
.read(any(String.class), any(Long.class), any(byte[].class),
|
||||
any(Integer.class), any(Integer.class), any(String.class),
|
||||
any(String.class));
|
||||
any(String.class), any(TracingContext.class));
|
||||
|
||||
AbfsInputStream inputStream = getAbfsInputStream(client, "testFailedReadAhead.txt");
|
||||
|
||||
|
@ -243,7 +249,7 @@ public class TestAbfsInputStream extends
|
|||
.when(client)
|
||||
.read(any(String.class), any(Long.class), any(byte[].class),
|
||||
any(Integer.class), any(Integer.class), any(String.class),
|
||||
any(String.class));
|
||||
any(String.class), any(TracingContext.class));
|
||||
|
||||
AbfsInputStream inputStream = getAbfsInputStream(client, "testFailedReadAheadEviction.txt");
|
||||
|
||||
|
@ -258,7 +264,8 @@ public class TestAbfsInputStream extends
|
|||
// at java.util.Stack.peek(Stack.java:102)
|
||||
// at java.util.Stack.pop(Stack.java:84)
|
||||
// at org.apache.hadoop.fs.azurebfs.services.ReadBufferManager.queueReadAhead
|
||||
ReadBufferManager.getBufferManager().queueReadAhead(inputStream, 0, ONE_KB);
|
||||
ReadBufferManager.getBufferManager().queueReadAhead(inputStream, 0, ONE_KB,
|
||||
getTestTracingContext(getFileSystem(), true));
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -287,7 +294,7 @@ public class TestAbfsInputStream extends
|
|||
.when(client)
|
||||
.read(any(String.class), any(Long.class), any(byte[].class),
|
||||
any(Integer.class), any(Integer.class), any(String.class),
|
||||
any(String.class));
|
||||
any(String.class), any(TracingContext.class));
|
||||
|
||||
AbfsInputStream inputStream = getAbfsInputStream(client, "testOlderReadAheadFailure.txt");
|
||||
|
||||
|
@ -341,7 +348,7 @@ public class TestAbfsInputStream extends
|
|||
.when(client)
|
||||
.read(any(String.class), any(Long.class), any(byte[].class),
|
||||
any(Integer.class), any(Integer.class), any(String.class),
|
||||
any(String.class));
|
||||
any(String.class), any(TracingContext.class));
|
||||
|
||||
AbfsInputStream inputStream = getAbfsInputStream(client, "testSuccessfulReadAhead.txt");
|
||||
int beforeReadCompletedListSize = ReadBufferManager.getBufferManager().getCompletedReadListSize();
|
||||
|
@ -399,7 +406,7 @@ public class TestAbfsInputStream extends
|
|||
.when(client)
|
||||
.read(any(String.class), any(Long.class), any(byte[].class),
|
||||
any(Integer.class), any(Integer.class), any(String.class),
|
||||
any(String.class));
|
||||
any(String.class), any(TracingContext.class));
|
||||
|
||||
AbfsInputStream inputStream = getAbfsInputStream(client, "testReadAheadManagerForFailedReadAhead.txt");
|
||||
|
||||
|
@ -452,7 +459,7 @@ public class TestAbfsInputStream extends
|
|||
.when(client)
|
||||
.read(any(String.class), any(Long.class), any(byte[].class),
|
||||
any(Integer.class), any(Integer.class), any(String.class),
|
||||
any(String.class));
|
||||
any(String.class), any(TracingContext.class));
|
||||
|
||||
AbfsInputStream inputStream = getAbfsInputStream(client, "testReadAheadManagerForOlderReadAheadFailure.txt");
|
||||
|
||||
|
@ -506,7 +513,7 @@ public class TestAbfsInputStream extends
|
|||
.when(client)
|
||||
.read(any(String.class), any(Long.class), any(byte[].class),
|
||||
any(Integer.class), any(Integer.class), any(String.class),
|
||||
any(String.class));
|
||||
any(String.class), any(TracingContext.class));
|
||||
|
||||
AbfsInputStream inputStream = getAbfsInputStream(client, "testSuccessfulReadAhead.txt");
|
||||
|
||||
|
@ -645,7 +652,7 @@ public class TestAbfsInputStream extends
|
|||
ALWAYS_READ_BUFFER_SIZE_TEST_FILE_SIZE, config);
|
||||
byte[] byteBuffer = new byte[ONE_MB];
|
||||
AbfsInputStream inputStream = this.getAbfsStore(fs)
|
||||
.openFileForRead(testPath, null);
|
||||
.openFileForRead(testPath, null, getTestTracingContext(fs, false));
|
||||
|
||||
Assertions.assertThat(inputStream.getBufferSize())
|
||||
.describedAs("Unexpected AbfsInputStream buffer size")
|
||||
|
|
|
@ -26,9 +26,12 @@ import org.junit.Test;
|
|||
|
||||
import org.mockito.ArgumentCaptor;
|
||||
|
||||
import org.apache.hadoop.fs.azurebfs.AbfsConfiguration;
|
||||
import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.azurebfs.AbfsConfiguration;
|
||||
import org.apache.hadoop.fs.azurebfs.constants.FSOperationType;
|
||||
import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters;
|
||||
import org.apache.hadoop.fs.azurebfs.utils.TracingContext;
|
||||
import org.apache.hadoop.fs.azurebfs.utils.TracingHeaderFormat;
|
||||
|
||||
import static org.mockito.ArgumentMatchers.eq;
|
||||
import static org.mockito.ArgumentMatchers.isNull;
|
||||
|
@ -86,11 +89,17 @@ public final class TestAbfsOutputStream {
|
|||
abfsConf = new AbfsConfiguration(conf, accountName1);
|
||||
AbfsPerfTracker tracker = new AbfsPerfTracker("test", accountName1, abfsConf);
|
||||
when(client.getAbfsPerfTracker()).thenReturn(tracker);
|
||||
when(client.append(anyString(), any(byte[].class), any(AppendRequestParameters.class), any())).thenReturn(op);
|
||||
when(client.flush(anyString(), anyLong(), anyBoolean(), anyBoolean(), any(), isNull())).thenReturn(op);
|
||||
when(client.append(anyString(), any(byte[].class),
|
||||
any(AppendRequestParameters.class), any(), any(TracingContext.class)))
|
||||
.thenReturn(op);
|
||||
when(client.flush(anyString(), anyLong(), anyBoolean(), anyBoolean(), any(),
|
||||
isNull(), any(TracingContext.class))).thenReturn(op);
|
||||
|
||||
AbfsOutputStream out = new AbfsOutputStream(client, null, PATH, 0,
|
||||
populateAbfsOutputStreamContext(BUFFER_SIZE, true, false, false));
|
||||
populateAbfsOutputStreamContext(BUFFER_SIZE, true, false, false),
|
||||
new TracingContext(abfsConf.getClientCorrelationId(), "test-fs-id",
|
||||
FSOperationType.WRITE, abfsConf.getTracingHeaderFormat(),
|
||||
null));
|
||||
final byte[] b = new byte[WRITE_SIZE];
|
||||
new Random().nextBytes(b);
|
||||
out.write(b);
|
||||
|
@ -110,12 +119,13 @@ public final class TestAbfsOutputStream {
|
|||
WRITE_SIZE, 0, 2 * WRITE_SIZE, APPEND_MODE, false, null);
|
||||
|
||||
verify(client, times(1)).append(
|
||||
eq(PATH), any(byte[].class), refEq(firstReqParameters), any());
|
||||
eq(PATH), any(byte[].class), refEq(firstReqParameters), any(),
|
||||
any(TracingContext.class));
|
||||
verify(client, times(1)).append(
|
||||
eq(PATH), any(byte[].class), refEq(secondReqParameters), any());
|
||||
eq(PATH), any(byte[].class), refEq(secondReqParameters), any(), any(TracingContext.class));
|
||||
// confirm there were only 2 invocations in all
|
||||
verify(client, times(2)).append(
|
||||
eq(PATH), any(byte[].class), any(), any());
|
||||
eq(PATH), any(byte[].class), any(), any(), any(TracingContext.class));
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -131,13 +141,17 @@ public final class TestAbfsOutputStream {
|
|||
conf.set(accountKey1, accountValue1);
|
||||
abfsConf = new AbfsConfiguration(conf, accountName1);
|
||||
AbfsPerfTracker tracker = new AbfsPerfTracker("test", accountName1, abfsConf);
|
||||
TracingContext tracingContext = new TracingContext("test-corr-id",
|
||||
"test-fs-id", FSOperationType.WRITE,
|
||||
TracingHeaderFormat.ALL_ID_FORMAT, null);
|
||||
|
||||
when(client.getAbfsPerfTracker()).thenReturn(tracker);
|
||||
when(client.append(anyString(), any(byte[].class), any(AppendRequestParameters.class), any())).thenReturn(op);
|
||||
when(client.flush(anyString(), anyLong(), anyBoolean(), anyBoolean(), any(), isNull())).thenReturn(op);
|
||||
when(client.append(anyString(), any(byte[].class), any(AppendRequestParameters.class), any(), any(TracingContext.class))).thenReturn(op);
|
||||
when(client.flush(anyString(), anyLong(), anyBoolean(), anyBoolean(), any(), isNull(), any(TracingContext.class))).thenReturn(op);
|
||||
|
||||
AbfsOutputStream out = new AbfsOutputStream(client, null, PATH, 0,
|
||||
populateAbfsOutputStreamContext(BUFFER_SIZE, true, false, false));
|
||||
populateAbfsOutputStreamContext(BUFFER_SIZE, true, false, false),
|
||||
tracingContext);
|
||||
final byte[] b = new byte[WRITE_SIZE];
|
||||
new Random().nextBytes(b);
|
||||
|
||||
|
@ -152,21 +166,26 @@ public final class TestAbfsOutputStream {
|
|||
BUFFER_SIZE, 0, 5*WRITE_SIZE-BUFFER_SIZE, APPEND_MODE, false, null);
|
||||
|
||||
verify(client, times(1)).append(
|
||||
eq(PATH), any(byte[].class), refEq(firstReqParameters), any());
|
||||
eq(PATH), any(byte[].class), refEq(firstReqParameters), any(),
|
||||
any(TracingContext.class));
|
||||
verify(client, times(1)).append(
|
||||
eq(PATH), any(byte[].class), refEq(secondReqParameters), any());
|
||||
eq(PATH), any(byte[].class), refEq(secondReqParameters), any(),
|
||||
any(TracingContext.class));
|
||||
// confirm there were only 2 invocations in all
|
||||
verify(client, times(2)).append(
|
||||
eq(PATH), any(byte[].class), any(), any());
|
||||
eq(PATH), any(byte[].class), any(), any(),
|
||||
any(TracingContext.class));
|
||||
|
||||
ArgumentCaptor<String> acFlushPath = ArgumentCaptor.forClass(String.class);
|
||||
ArgumentCaptor<Long> acFlushPosition = ArgumentCaptor.forClass(Long.class);
|
||||
ArgumentCaptor<TracingContext> acTracingContext = ArgumentCaptor
|
||||
.forClass(TracingContext.class);
|
||||
ArgumentCaptor<Boolean> acFlushRetainUnCommittedData = ArgumentCaptor.forClass(Boolean.class);
|
||||
ArgumentCaptor<Boolean> acFlushClose = ArgumentCaptor.forClass(Boolean.class);
|
||||
ArgumentCaptor<String> acFlushSASToken = ArgumentCaptor.forClass(String.class);
|
||||
|
||||
verify(client, times(1)).flush(acFlushPath.capture(), acFlushPosition.capture(), acFlushRetainUnCommittedData.capture(), acFlushClose.capture(),
|
||||
acFlushSASToken.capture(), isNull());
|
||||
acFlushSASToken.capture(), isNull(), acTracingContext.capture());
|
||||
assertThat(Arrays.asList(PATH)).describedAs("path").isEqualTo(acFlushPath.getAllValues());
|
||||
assertThat(Arrays.asList(Long.valueOf(5*WRITE_SIZE))).describedAs("position").isEqualTo(acFlushPosition.getAllValues());
|
||||
assertThat(Arrays.asList(false)).describedAs("RetainUnCommittedData flag").isEqualTo(acFlushRetainUnCommittedData.getAllValues());
|
||||
|
@ -187,15 +206,19 @@ public final class TestAbfsOutputStream {
|
|||
conf.set(accountKey1, accountValue1);
|
||||
abfsConf = new AbfsConfiguration(conf, accountName1);
|
||||
AbfsPerfTracker tracker = new AbfsPerfTracker("test", accountName1, abfsConf);
|
||||
TracingContext tracingContext = new TracingContext(
|
||||
abfsConf.getClientCorrelationId(), "test-fs-id",
|
||||
FSOperationType.WRITE, abfsConf.getTracingHeaderFormat(), null);
|
||||
|
||||
when(client.getAbfsPerfTracker()).thenReturn(tracker);
|
||||
when(client.append(anyString(), any(byte[].class), any(AppendRequestParameters.class), any())).thenReturn(op);
|
||||
when(client.flush(anyString(), anyLong(), anyBoolean(), anyBoolean(), any(), isNull())).thenReturn(op);
|
||||
when(client.append(anyString(), any(byte[].class), any(AppendRequestParameters.class), any(), any(TracingContext.class))).thenReturn(op);
|
||||
when(client.flush(anyString(), anyLong(), anyBoolean(), anyBoolean(), any(), isNull(), any(TracingContext.class))).thenReturn(op);
|
||||
when(op.getSasToken()).thenReturn("testToken");
|
||||
when(op.getResult()).thenReturn(httpOp);
|
||||
|
||||
AbfsOutputStream out = new AbfsOutputStream(client, null, PATH, 0,
|
||||
populateAbfsOutputStreamContext(BUFFER_SIZE, true, false, false));
|
||||
populateAbfsOutputStreamContext(BUFFER_SIZE, true, false, false),
|
||||
tracingContext);
|
||||
final byte[] b = new byte[BUFFER_SIZE];
|
||||
new Random().nextBytes(b);
|
||||
|
||||
|
@ -210,21 +233,23 @@ public final class TestAbfsOutputStream {
|
|||
BUFFER_SIZE, 0, BUFFER_SIZE, APPEND_MODE, false, null);
|
||||
|
||||
verify(client, times(1)).append(
|
||||
eq(PATH), any(byte[].class), refEq(firstReqParameters), any());
|
||||
eq(PATH), any(byte[].class), refEq(firstReqParameters), any(), any(TracingContext.class));
|
||||
verify(client, times(1)).append(
|
||||
eq(PATH), any(byte[].class), refEq(secondReqParameters), any());
|
||||
eq(PATH), any(byte[].class), refEq(secondReqParameters), any(), any(TracingContext.class));
|
||||
// confirm there were only 2 invocations in all
|
||||
verify(client, times(2)).append(
|
||||
eq(PATH), any(byte[].class), any(), any());
|
||||
eq(PATH), any(byte[].class), any(), any(), any(TracingContext.class));
|
||||
|
||||
ArgumentCaptor<String> acFlushPath = ArgumentCaptor.forClass(String.class);
|
||||
ArgumentCaptor<Long> acFlushPosition = ArgumentCaptor.forClass(Long.class);
|
||||
ArgumentCaptor<TracingContext> acTracingContext = ArgumentCaptor
|
||||
.forClass(TracingContext.class);
|
||||
ArgumentCaptor<Boolean> acFlushRetainUnCommittedData = ArgumentCaptor.forClass(Boolean.class);
|
||||
ArgumentCaptor<Boolean> acFlushClose = ArgumentCaptor.forClass(Boolean.class);
|
||||
ArgumentCaptor<String> acFlushSASToken = ArgumentCaptor.forClass(String.class);
|
||||
|
||||
verify(client, times(1)).flush(acFlushPath.capture(), acFlushPosition.capture(), acFlushRetainUnCommittedData.capture(), acFlushClose.capture(),
|
||||
acFlushSASToken.capture(), isNull());
|
||||
acFlushSASToken.capture(), isNull(), acTracingContext.capture());
|
||||
assertThat(Arrays.asList(PATH)).describedAs("path").isEqualTo(acFlushPath.getAllValues());
|
||||
assertThat(Arrays.asList(Long.valueOf(2*BUFFER_SIZE))).describedAs("position").isEqualTo(acFlushPosition.getAllValues());
|
||||
assertThat(Arrays.asList(false)).describedAs("RetainUnCommittedData flag").isEqualTo(acFlushRetainUnCommittedData.getAllValues());
|
||||
|
@ -247,13 +272,19 @@ public final class TestAbfsOutputStream {
|
|||
AbfsPerfTracker tracker = new AbfsPerfTracker("test", accountName1, abfsConf);
|
||||
|
||||
when(client.getAbfsPerfTracker()).thenReturn(tracker);
|
||||
when(client.append(anyString(), any(byte[].class), any(AppendRequestParameters.class), any())).thenReturn(op);
|
||||
when(client.flush(anyString(), anyLong(), anyBoolean(), anyBoolean(), any(), isNull())).thenReturn(op);
|
||||
when(client.append(anyString(), any(byte[].class),
|
||||
any(AppendRequestParameters.class), any(), any(TracingContext.class)))
|
||||
.thenReturn(op);
|
||||
when(client.flush(anyString(), anyLong(), anyBoolean(), anyBoolean(),
|
||||
any(), isNull(), any(TracingContext.class))).thenReturn(op);
|
||||
when(op.getSasToken()).thenReturn("testToken");
|
||||
when(op.getResult()).thenReturn(httpOp);
|
||||
|
||||
AbfsOutputStream out = new AbfsOutputStream(client, null, PATH, 0,
|
||||
populateAbfsOutputStreamContext(BUFFER_SIZE, true, false, false));
|
||||
populateAbfsOutputStreamContext(BUFFER_SIZE, true, false, false),
|
||||
new TracingContext(abfsConf.getClientCorrelationId(), "test-fs-id",
|
||||
FSOperationType.WRITE, abfsConf.getTracingHeaderFormat(),
|
||||
null));
|
||||
final byte[] b = new byte[BUFFER_SIZE];
|
||||
new Random().nextBytes(b);
|
||||
|
||||
|
@ -268,12 +299,12 @@ public final class TestAbfsOutputStream {
|
|||
BUFFER_SIZE, 0, BUFFER_SIZE, APPEND_MODE, false, null);
|
||||
|
||||
verify(client, times(1)).append(
|
||||
eq(PATH), any(byte[].class), refEq(firstReqParameters), any());
|
||||
eq(PATH), any(byte[].class), refEq(firstReqParameters), any(), any(TracingContext.class));
|
||||
verify(client, times(1)).append(
|
||||
eq(PATH), any(byte[].class), refEq(secondReqParameters), any());
|
||||
eq(PATH), any(byte[].class), refEq(secondReqParameters), any(), any(TracingContext.class));
|
||||
// confirm there were only 2 invocations in all
|
||||
verify(client, times(2)).append(
|
||||
eq(PATH), any(byte[].class), any(), any());
|
||||
eq(PATH), any(byte[].class), any(), any(), any(TracingContext.class));
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -291,11 +322,17 @@ public final class TestAbfsOutputStream {
|
|||
AbfsPerfTracker tracker = new AbfsPerfTracker("test", accountName1, abfsConf);
|
||||
|
||||
when(client.getAbfsPerfTracker()).thenReturn(tracker);
|
||||
when(client.append(anyString(), any(byte[].class), any(AppendRequestParameters.class), any())).thenReturn(op);
|
||||
when(client.flush(anyString(), anyLong(), anyBoolean(), anyBoolean(), any(), isNull())).thenReturn(op);
|
||||
when(client.append(anyString(), any(byte[].class),
|
||||
any(AppendRequestParameters.class), any(), any(TracingContext.class)))
|
||||
.thenReturn(op);
|
||||
when(client.flush(anyString(), anyLong(), anyBoolean(), anyBoolean(), any(),
|
||||
isNull(), any(TracingContext.class))).thenReturn(op);
|
||||
|
||||
AbfsOutputStream out = new AbfsOutputStream(client, null, PATH, 0,
|
||||
populateAbfsOutputStreamContext(BUFFER_SIZE, true, false, true));
|
||||
populateAbfsOutputStreamContext(BUFFER_SIZE, true, false, true),
|
||||
new TracingContext(abfsConf.getClientCorrelationId(), "test-fs-id",
|
||||
FSOperationType.OPEN, abfsConf.getTracingHeaderFormat(),
|
||||
null));
|
||||
final byte[] b = new byte[BUFFER_SIZE];
|
||||
new Random().nextBytes(b);
|
||||
|
||||
|
@ -310,12 +347,12 @@ public final class TestAbfsOutputStream {
|
|||
BUFFER_SIZE, 0, BUFFER_SIZE, APPEND_MODE, true, null);
|
||||
|
||||
verify(client, times(1)).append(
|
||||
eq(PATH), any(byte[].class), refEq(firstReqParameters), any());
|
||||
eq(PATH), any(byte[].class), refEq(firstReqParameters), any(), any(TracingContext.class));
|
||||
verify(client, times(1)).append(
|
||||
eq(PATH), any(byte[].class), refEq(secondReqParameters), any());
|
||||
eq(PATH), any(byte[].class), refEq(secondReqParameters), any(), any(TracingContext.class));
|
||||
// confirm there were only 2 invocations in all
|
||||
verify(client, times(2)).append(
|
||||
eq(PATH), any(byte[].class), any(), any());
|
||||
eq(PATH), any(byte[].class), any(), any(), any(TracingContext.class));
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -332,13 +369,21 @@ public final class TestAbfsOutputStream {
|
|||
conf.set(accountKey1, accountValue1);
|
||||
abfsConf = new AbfsConfiguration(conf, accountName1);
|
||||
AbfsPerfTracker tracker = new AbfsPerfTracker("test", accountName1, abfsConf);
|
||||
TracingContext tracingContext = new TracingContext(
|
||||
abfsConf.getClientCorrelationId(), "test-fs-id",
|
||||
FSOperationType.WRITE, abfsConf.getTracingHeaderFormat(), null);
|
||||
|
||||
when(client.getAbfsPerfTracker()).thenReturn(tracker);
|
||||
when(client.append(anyString(), any(byte[].class), any(AppendRequestParameters.class), any())).thenReturn(op);
|
||||
when(client.flush(anyString(), anyLong(), anyBoolean(), anyBoolean(), any(), isNull())).thenReturn(op);
|
||||
when(client.append(anyString(), any(byte[].class),
|
||||
any(AppendRequestParameters.class), any(), any(TracingContext.class)))
|
||||
.thenReturn(op);
|
||||
when(client.flush(anyString(), anyLong(), anyBoolean(), anyBoolean(), any(),
|
||||
isNull(), any(TracingContext.class))).thenReturn(op);
|
||||
|
||||
AbfsOutputStream out = new AbfsOutputStream(client, null, PATH, 0,
|
||||
populateAbfsOutputStreamContext(BUFFER_SIZE, true, false, false));
|
||||
populateAbfsOutputStreamContext(BUFFER_SIZE, true, false, false), new TracingContext(abfsConf.getClientCorrelationId(), "test-fs-id",
|
||||
FSOperationType.OPEN, abfsConf.getTracingHeaderFormat(),
|
||||
null));
|
||||
final byte[] b = new byte[BUFFER_SIZE];
|
||||
new Random().nextBytes(b);
|
||||
|
||||
|
@ -353,21 +398,23 @@ public final class TestAbfsOutputStream {
|
|||
BUFFER_SIZE, 0, BUFFER_SIZE, APPEND_MODE, false, null);
|
||||
|
||||
verify(client, times(1)).append(
|
||||
eq(PATH), any(byte[].class), refEq(firstReqParameters), any());
|
||||
eq(PATH), any(byte[].class), refEq(firstReqParameters), any(), any(TracingContext.class));
|
||||
verify(client, times(1)).append(
|
||||
eq(PATH), any(byte[].class), refEq(secondReqParameters), any());
|
||||
eq(PATH), any(byte[].class), refEq(secondReqParameters), any(), any(TracingContext.class));
|
||||
// confirm there were only 2 invocations in all
|
||||
verify(client, times(2)).append(
|
||||
eq(PATH), any(byte[].class), any(), any());
|
||||
eq(PATH), any(byte[].class), any(), any(), any(TracingContext.class));
|
||||
|
||||
ArgumentCaptor<String> acFlushPath = ArgumentCaptor.forClass(String.class);
|
||||
ArgumentCaptor<Long> acFlushPosition = ArgumentCaptor.forClass(Long.class);
|
||||
ArgumentCaptor<TracingContext> acTracingContext = ArgumentCaptor
|
||||
.forClass(TracingContext.class);
|
||||
ArgumentCaptor<Boolean> acFlushRetainUnCommittedData = ArgumentCaptor.forClass(Boolean.class);
|
||||
ArgumentCaptor<Boolean> acFlushClose = ArgumentCaptor.forClass(Boolean.class);
|
||||
ArgumentCaptor<String> acFlushSASToken = ArgumentCaptor.forClass(String.class);
|
||||
|
||||
verify(client, times(1)).flush(acFlushPath.capture(), acFlushPosition.capture(), acFlushRetainUnCommittedData.capture(), acFlushClose.capture(),
|
||||
acFlushSASToken.capture(), isNull());
|
||||
acFlushSASToken.capture(), isNull(), acTracingContext.capture());
|
||||
assertThat(Arrays.asList(PATH)).describedAs("path").isEqualTo(acFlushPath.getAllValues());
|
||||
assertThat(Arrays.asList(Long.valueOf(2*BUFFER_SIZE))).describedAs("position").isEqualTo(acFlushPosition.getAllValues());
|
||||
assertThat(Arrays.asList(false)).describedAs("RetainUnCommittedData flag").isEqualTo(acFlushRetainUnCommittedData.getAllValues());
|
||||
|
@ -388,11 +435,17 @@ public final class TestAbfsOutputStream {
|
|||
abfsConf = new AbfsConfiguration(conf, accountName1);
|
||||
AbfsPerfTracker tracker = new AbfsPerfTracker("test", accountName1, abfsConf);
|
||||
when(client.getAbfsPerfTracker()).thenReturn(tracker);
|
||||
when(client.append(anyString(), any(byte[].class), any(AppendRequestParameters.class), any())).thenReturn(op);
|
||||
when(client.flush(anyString(), anyLong(), anyBoolean(), anyBoolean(), any(), isNull())).thenReturn(op);
|
||||
when(client.append(anyString(), any(byte[].class),
|
||||
any(AppendRequestParameters.class), any(), any(TracingContext.class)))
|
||||
.thenReturn(op);
|
||||
when(client.flush(anyString(), anyLong(), anyBoolean(), anyBoolean(), any(),
|
||||
isNull(), any(TracingContext.class))).thenReturn(op);
|
||||
|
||||
AbfsOutputStream out = new AbfsOutputStream(client, null, PATH, 0,
|
||||
populateAbfsOutputStreamContext(BUFFER_SIZE, true, false, false));
|
||||
populateAbfsOutputStreamContext(BUFFER_SIZE, true, false, false),
|
||||
new TracingContext(abfsConf.getClientCorrelationId(), "test-fs-id",
|
||||
FSOperationType.WRITE, abfsConf.getTracingHeaderFormat(),
|
||||
null));
|
||||
final byte[] b = new byte[BUFFER_SIZE];
|
||||
new Random().nextBytes(b);
|
||||
|
||||
|
@ -409,11 +462,11 @@ public final class TestAbfsOutputStream {
|
|||
BUFFER_SIZE, 0, BUFFER_SIZE, APPEND_MODE, false, null);
|
||||
|
||||
verify(client, times(1)).append(
|
||||
eq(PATH), any(byte[].class), refEq(firstReqParameters), any());
|
||||
eq(PATH), any(byte[].class), refEq(firstReqParameters), any(), any(TracingContext.class));
|
||||
verify(client, times(1)).append(
|
||||
eq(PATH), any(byte[].class), refEq(secondReqParameters), any());
|
||||
eq(PATH), any(byte[].class), refEq(secondReqParameters), any(), any(TracingContext.class));
|
||||
// confirm there were only 2 invocations in all
|
||||
verify(client, times(2)).append(
|
||||
eq(PATH), any(byte[].class), any(), any());
|
||||
eq(PATH), any(byte[].class), any(), any(), any(TracingContext.class));
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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.utils;
|
||||
|
||||
import org.apache.hadoop.fs.azurebfs.constants.FSOperationType;
|
||||
import org.assertj.core.api.Assertions;
|
||||
|
||||
import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.EMPTY_STRING;
|
||||
|
||||
/**
|
||||
* Used to validate correlation identifiers provided during testing against
|
||||
* values that get associated with a request through its TracingContext instance
|
||||
*/
|
||||
public class TracingHeaderValidator implements Listener {
|
||||
private String clientCorrelationId;
|
||||
private String fileSystemId;
|
||||
private String primaryRequestId = EMPTY_STRING;
|
||||
private boolean needsPrimaryRequestId;
|
||||
private String streamID = "";
|
||||
private FSOperationType operation;
|
||||
private int retryNum;
|
||||
private TracingHeaderFormat format;
|
||||
|
||||
private static final String GUID_PATTERN = "^[0-9a-fA-F]{8}-([0-9a-fA-F]{4}-){3}[0-9a-fA-F]{12}$";
|
||||
|
||||
@Override
|
||||
public void callTracingHeaderValidator(String tracingContextHeader,
|
||||
TracingHeaderFormat format) {
|
||||
this.format = format;
|
||||
validateTracingHeader(tracingContextHeader);
|
||||
}
|
||||
|
||||
@Override
|
||||
public TracingHeaderValidator getClone() {
|
||||
TracingHeaderValidator tracingHeaderValidator = new TracingHeaderValidator(
|
||||
clientCorrelationId, fileSystemId, operation, needsPrimaryRequestId,
|
||||
retryNum, streamID);
|
||||
tracingHeaderValidator.primaryRequestId = primaryRequestId;
|
||||
return tracingHeaderValidator;
|
||||
}
|
||||
|
||||
public TracingHeaderValidator(String clientCorrelationId, String fileSystemId,
|
||||
FSOperationType operation, boolean needsPrimaryRequestId, int retryNum) {
|
||||
this.clientCorrelationId = clientCorrelationId;
|
||||
this.fileSystemId = fileSystemId;
|
||||
this.operation = operation;
|
||||
this.retryNum = retryNum;
|
||||
this.needsPrimaryRequestId = needsPrimaryRequestId;
|
||||
}
|
||||
|
||||
public TracingHeaderValidator(String clientCorrelationId, String fileSystemId,
|
||||
FSOperationType operation, boolean needsPrimaryRequestId, int retryNum,
|
||||
String streamID) {
|
||||
this(clientCorrelationId, fileSystemId, operation, needsPrimaryRequestId,
|
||||
retryNum);
|
||||
this.streamID = streamID;
|
||||
}
|
||||
|
||||
private void validateTracingHeader(String tracingContextHeader) {
|
||||
String[] idList = tracingContextHeader.split(":");
|
||||
validateBasicFormat(idList);
|
||||
if (format != TracingHeaderFormat.ALL_ID_FORMAT) {
|
||||
return;
|
||||
}
|
||||
if (!primaryRequestId.isEmpty() && !idList[3].isEmpty()) {
|
||||
Assertions.assertThat(idList[3])
|
||||
.describedAs("PrimaryReqID should be common for these requests")
|
||||
.isEqualTo(primaryRequestId);
|
||||
}
|
||||
if (!streamID.isEmpty()) {
|
||||
Assertions.assertThat(idList[4])
|
||||
.describedAs("Stream id should be common for these requests")
|
||||
.isEqualTo(streamID);
|
||||
}
|
||||
}
|
||||
|
||||
private void validateBasicFormat(String[] idList) {
|
||||
if (format == TracingHeaderFormat.ALL_ID_FORMAT) {
|
||||
Assertions.assertThat(idList)
|
||||
.describedAs("header should have 7 elements").hasSize(7);
|
||||
} else if (format == TracingHeaderFormat.TWO_ID_FORMAT) {
|
||||
Assertions.assertThat(idList)
|
||||
.describedAs("header should have 2 elements").hasSize(2);
|
||||
} else {
|
||||
Assertions.assertThat(idList).describedAs("header should have 1 element")
|
||||
.hasSize(1);
|
||||
Assertions.assertThat(idList[0])
|
||||
.describedAs("Client request ID is a guid").matches(GUID_PATTERN);
|
||||
return;
|
||||
}
|
||||
|
||||
if (clientCorrelationId.matches("[a-zA-Z0-9-]*")) {
|
||||
Assertions.assertThat(idList[0])
|
||||
.describedAs("Correlation ID should match config")
|
||||
.isEqualTo(clientCorrelationId);
|
||||
} else {
|
||||
Assertions.assertThat(idList[0])
|
||||
.describedAs("Invalid config should be replaced with empty string")
|
||||
.isEmpty();
|
||||
}
|
||||
Assertions.assertThat(idList[1]).describedAs("Client request ID is a guid")
|
||||
.matches(GUID_PATTERN);
|
||||
|
||||
if (format != TracingHeaderFormat.ALL_ID_FORMAT) {
|
||||
return;
|
||||
}
|
||||
|
||||
Assertions.assertThat(idList[2]).describedAs("Filesystem ID incorrect")
|
||||
.isEqualTo(fileSystemId);
|
||||
if (needsPrimaryRequestId && !operation
|
||||
.equals(FSOperationType.READ)) {
|
||||
Assertions.assertThat(idList[3]).describedAs("should have primaryReqId")
|
||||
.isNotEmpty();
|
||||
}
|
||||
Assertions.assertThat(idList[5]).describedAs("Operation name incorrect")
|
||||
.isEqualTo(operation.toString());
|
||||
int retryCount = Integer.parseInt(idList[6]);
|
||||
Assertions.assertThat(retryCount)
|
||||
.describedAs("Retry was required due to issue on server side")
|
||||
.isEqualTo(retryNum);
|
||||
}
|
||||
|
||||
/**
|
||||
* Sets the value of expected Hadoop operation
|
||||
* @param operation Hadoop operation code (String of two characters)
|
||||
*/
|
||||
@Override
|
||||
public void setOperation(FSOperationType operation) {
|
||||
this.operation = operation;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void updatePrimaryRequestID(String primaryRequestId) {
|
||||
this.primaryRequestId = primaryRequestId;
|
||||
}
|
||||
}
|
Loading…
Reference in New Issue