From 59a4ea3f41b9a4a19386e1a449dd7eba58a773f0 Mon Sep 17 00:00:00 2001
From: Anmol Asrani
Date: Mon, 30 Jun 2025 00:56:00 -0700
Subject: [PATCH 01/25] ABFS WASB Compatibility testing
---
.../fs/azurebfs/AzureBlobFileSystemStore.java | 4 +-
.../azurebfs/constants/AbfsHttpConstants.java | 1 +
.../services/AppendRequestParameters.java | 18 +-
.../fs/azurebfs/services/AbfsBlobBlock.java | 45 +-
.../fs/azurebfs/services/AbfsBlobClient.java | 28 +-
.../fs/azurebfs/services/AbfsClient.java | 21 +-
.../fs/azurebfs/services/AbfsDfsClient.java | 15 +-
.../azurebfs/services/AbfsOutputStream.java | 56 +-
.../services/AzureBlobBlockManager.java | 8 +-
.../services/AzureBlobIngressHandler.java | 10 +-
.../services/AzureDFSBlockManager.java | 1 +
.../services/AzureDFSIngressHandler.java | 13 +-
.../AzureDfsToBlobIngressFallbackHandler.java | 4 +-
.../services/AzureIngressHandler.java | 1 +
.../fs/azurebfs/services/RenameAtomicity.java | 4 +-
.../azurebfs/AbstractAbfsIntegrationTest.java | 1 +
.../azurebfs/ITestAbfsCustomEncryption.java | 8 +-
.../ITestAzureBlobFileSystemAppend.java | 6 +-
.../ITestAzureBlobFileSystemChecksum.java | 46 +-
...ITestAzureBlobFileSystemDelegationSAS.java | 2 +-
.../ITestAzureBlobFileSystemRename.java | 2 +-
.../azurebfs/ITestFileSystemProperties.java | 6 +-
.../azurebfs/ITestWasbAbfsCompatibility.java | 1729 ++++++++++++++++-
.../fs/azurebfs/services/ITestAbfsClient.java | 2 +-
.../services/ITestAbfsOutputStream.java | 4 +-
.../services/ITestAbfsRestOperation.java | 2 +-
.../services/TestAbfsOutputStream.java | 111 +-
27 files changed, 2010 insertions(+), 138 deletions(-)
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java
index e3df803480562..2a5f67f5a596b 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java
@@ -503,7 +503,7 @@ public Hashtable getFilesystemProperties(
}
public void setFilesystemProperties(
- final Hashtable properties, TracingContext tracingContext)
+ final Hashtable properties, final String name, TracingContext tracingContext)
throws AzureBlobFileSystemException {
if (properties == null || properties.isEmpty()) {
LOG.trace("setFilesystemProperties no properties present");
@@ -518,7 +518,7 @@ public void setFilesystemProperties(
"setFilesystemProperties")) {
final AbfsRestOperation op = getClient()
- .setFilesystemProperties(properties, tracingContext);
+ .setFilesystemProperties(properties, name, tracingContext);
perfInfo.registerResult(op.getResult()).registerSuccess(true);
}
}
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/AbfsHttpConstants.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/AbfsHttpConstants.java
index daca4f8cea48d..8bee2c72b6b65 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/AbfsHttpConstants.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/AbfsHttpConstants.java
@@ -216,6 +216,7 @@ public static ApiVersion getCurrentVersion() {
public static final String XML_TAG_RESOURCE_TYPE = "ResourceType";
public static final String XML_TAG_INVALID_XML = "Invalid XML";
public static final String XML_TAG_HDI_ISFOLDER = "hdi_isfolder";
+ public static final String XML_TAG_HDI_PERMISSION = "hdi_permission";
public static final String XML_TAG_ETAG = "Etag";
public static final String XML_TAG_LAST_MODIFIED_TIME = "Last-Modified";
public static final String XML_TAG_CREATION_TIME = "Creation-Time";
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/AppendRequestParameters.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/AppendRequestParameters.java
index 51ad61cc38fb1..84fbb0e91b88f 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/AppendRequestParameters.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/AppendRequestParameters.java
@@ -37,6 +37,7 @@ public enum Mode {
private boolean isExpectHeaderEnabled;
private boolean isRetryDueToExpect;
private BlobAppendRequestParameters blobParams;
+ private final String md5;
/**
@@ -55,7 +56,8 @@ public AppendRequestParameters(final long position,
final Mode mode,
final boolean isAppendBlob,
final String leaseId,
- final boolean isExpectHeaderEnabled) {
+ final boolean isExpectHeaderEnabled,
+ final String md5) {
this.position = position;
this.offset = offset;
this.length = length;
@@ -65,6 +67,7 @@ public AppendRequestParameters(final long position,
this.isExpectHeaderEnabled = isExpectHeaderEnabled;
this.isRetryDueToExpect = false;
this.blobParams = null;
+ this.md5 = md5;
}
/**
@@ -85,7 +88,8 @@ public AppendRequestParameters(final long position,
final boolean isAppendBlob,
final String leaseId,
final boolean isExpectHeaderEnabled,
- final BlobAppendRequestParameters blobParams) {
+ final BlobAppendRequestParameters blobParams,
+ final String md5) {
this.position = position;
this.offset = offset;
this.length = length;
@@ -95,6 +99,7 @@ public AppendRequestParameters(final long position,
this.isExpectHeaderEnabled = isExpectHeaderEnabled;
this.isRetryDueToExpect = false;
this.blobParams = blobParams;
+ this.md5 = md5;
}
public long getPosition() {
@@ -146,6 +151,15 @@ public String getBlockId() {
return getBlobParams().getBlockId();
}
+ /**
+ * Gets the MD5 hash.
+ *
+ * @return the MD5 hash string
+ */
+ public String getMd5() {
+ return md5;
+ }
+
/**
* Sets whether the retry is due to the Expect header.
*
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsBlobBlock.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsBlobBlock.java
index 96cbe54284d42..325c9fbf10a00 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsBlobBlock.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsBlobBlock.java
@@ -20,10 +20,12 @@
import java.io.IOException;
import java.nio.charset.StandardCharsets;
+import java.util.UUID;
import org.apache.commons.codec.binary.Base64;
import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.BLOCK_ID_LENGTH;
+import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.ONE_MB;
/**
* Represents a block in Azure Blob Storage used by Azure Data Lake Storage (ADLS).
@@ -42,23 +44,42 @@ public class AbfsBlobBlock extends AbfsBlock {
* @param offset Used to generate blockId based on offset.
* @throws IOException exception is thrown.
*/
- AbfsBlobBlock(AbfsOutputStream outputStream, long offset) throws IOException {
+ AbfsBlobBlock(AbfsOutputStream outputStream, long offset, int blockIdLength) throws IOException {
super(outputStream, offset);
- this.blockId = generateBlockId(offset);
+ String streamId = getOutputStream().getStreamID();
+ UUID streamIdGuid = UUID.nameUUIDFromBytes(streamId.getBytes(StandardCharsets.UTF_8));
+ this.blockId = generateBlockId(offset, streamIdGuid, blockIdLength);
}
/**
- * Helper method that generates blockId.
- * @param position The offset needed to generate blockId.
- * @return String representing the block ID generated.
+ * Generates a Base64-encoded block ID string based on the given position, stream ID, and desired raw length.
+ *
+ * The block ID is composed using the stream UUID and the block index, which is derived from
+ * the given position divided by the output stream's buffer size. The resulting string is
+ * optionally adjusted to match the specified raw length, padded or trimmed as needed, and
+ * then Base64-encoded.
+ *
+ * @param position The byte position in the stream, used to compute the block index.
+ * @param streamId The UUID representing the stream, used as a prefix in the block ID.
+ * @param rawLength The desired length of the raw block ID string before Base64 encoding.
+ * If 0, no length adjustment is made.
+ * @return A Base64-encoded block ID string suitable for use in block-based storage APIs.
*/
- private String generateBlockId(long position) {
- String streamId = getOutputStream().getStreamID();
- String streamIdHash = Integer.toString(streamId.hashCode());
- String blockId = String.format("%d_%s", position, streamIdHash);
- byte[] blockIdByteArray = new byte[BLOCK_ID_LENGTH];
- System.arraycopy(blockId.getBytes(StandardCharsets.UTF_8), 0, blockIdByteArray, 0, Math.min(BLOCK_ID_LENGTH, blockId.length()));
- return new String(Base64.encodeBase64(blockIdByteArray), StandardCharsets.UTF_8);
+ private String generateBlockId(long position, UUID streamId, int rawLength) {
+ long blockIndex = position / getOutputStream().getBufferSize();
+ String rawBlockId = String.format("%s-%06d", streamId, blockIndex);
+
+ if (rawLength != 0) {
+ // Adjust to match expected decoded length
+ if (rawBlockId.length() < rawLength) {
+ rawBlockId = String.format("%-" + rawLength + "s", rawBlockId)
+ .replace(' ', '_');
+ } else if (rawBlockId.length() > rawLength) {
+ rawBlockId = rawBlockId.substring(0, rawLength);
+ }
+ }
+
+ return Base64.encodeBase64String(rawBlockId.getBytes(StandardCharsets.UTF_8));
}
/**
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsBlobClient.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsBlobClient.java
index a0567da97b283..e64e68010645d 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsBlobClient.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsBlobClient.java
@@ -129,6 +129,7 @@
import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.XML_TAG_BLOCK_NAME;
import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.XML_TAG_COMMITTED_BLOCKS;
import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.XML_TAG_HDI_ISFOLDER;
+import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.XML_TAG_HDI_PERMISSION;
import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.XML_TAG_NAME;
import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.XML_VERSION;
import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.XMS_PROPERTIES_ENCODING_ASCII;
@@ -262,7 +263,7 @@ public AbfsRestOperation createFilesystem(TracingContext tracingContext)
* @throws AzureBlobFileSystemException if rest operation fails.
*/
@Override
- public AbfsRestOperation setFilesystemProperties(final Hashtable properties,
+ public AbfsRestOperation setFilesystemProperties(final Hashtable properties, final String name,
TracingContext tracingContext) throws AzureBlobFileSystemException {
List requestHeaders = createDefaultHeaders();
/*
@@ -898,7 +899,9 @@ public AbfsRestOperation append(final String path,
requestHeaders.add(new AbfsHttpHeader(EXPECT, HUNDRED_CONTINUE));
}
if (isChecksumValidationEnabled()) {
- addCheckSumHeaderForWrite(requestHeaders, reqParams, buffer);
+ if (reqParams.getMd5() != null) {
+ requestHeaders.add(new AbfsHttpHeader(CONTENT_MD5, reqParams.getMd5()));
+ }
}
if (reqParams.isRetryDueToExpect()) {
String userAgentRetry = getUserAgent();
@@ -982,6 +985,11 @@ public AbfsRestOperation appendBlock(final String path,
if (requestParameters.getLeaseId() != null) {
requestHeaders.add(new AbfsHttpHeader(X_MS_LEASE_ID, requestParameters.getLeaseId()));
}
+ if (isChecksumValidationEnabled()) {
+ if (requestParameters.getMd5() != null) {
+ requestHeaders.add(new AbfsHttpHeader(CONTENT_MD5, requestParameters.getMd5()));
+ }
+ }
final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder();
abfsUriQueryBuilder.addQuery(QUERY_PARAM_COMP, APPEND_BLOCK);
String sasTokenForReuse = appendSASTokenToQuery(path, SASTokenProvider.WRITE_OPERATION, abfsUriQueryBuilder);
@@ -1032,7 +1040,7 @@ public AbfsRestOperation flush(final String path,
final String cachedSasToken,
final String leaseId,
final ContextEncryptionAdapter contextEncryptionAdapter,
- final TracingContext tracingContext) throws AzureBlobFileSystemException {
+ final TracingContext tracingContext, String blobMd5) throws AzureBlobFileSystemException {
throw new UnsupportedOperationException(
"Flush without blockIds not supported on Blob Endpoint");
}
@@ -1060,7 +1068,7 @@ public AbfsRestOperation flush(byte[] buffer,
final String leaseId,
final String eTag,
ContextEncryptionAdapter contextEncryptionAdapter,
- final TracingContext tracingContext) throws AzureBlobFileSystemException {
+ final TracingContext tracingContext, String blobMd5) throws AzureBlobFileSystemException {
final List requestHeaders = createDefaultHeaders();
addEncryptionKeyRequestHeaders(path, requestHeaders, false,
contextEncryptionAdapter, tracingContext);
@@ -1070,8 +1078,8 @@ public AbfsRestOperation flush(byte[] buffer,
if (leaseId != null) {
requestHeaders.add(new AbfsHttpHeader(X_MS_LEASE_ID, leaseId));
}
- String md5Hash = computeMD5Hash(buffer, 0, buffer.length);
- requestHeaders.add(new AbfsHttpHeader(X_MS_BLOB_CONTENT_MD5, md5Hash));
+ //String md5Hash = computeMD5Hash(buffer, 0, buffer.length);
+ requestHeaders.add(new AbfsHttpHeader(X_MS_BLOB_CONTENT_MD5, blobMd5));
final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder();
abfsUriQueryBuilder.addQuery(QUERY_PARAM_COMP, BLOCKLIST);
@@ -1097,7 +1105,7 @@ public AbfsRestOperation flush(byte[] buffer,
AbfsRestOperation op1 = getPathStatus(path, true, tracingContext,
contextEncryptionAdapter);
String metadataMd5 = op1.getResult().getResponseHeader(CONTENT_MD5);
- if (!md5Hash.equals(metadataMd5)) {
+ if (!blobMd5.equals(metadataMd5)) {
throw ex;
}
return op;
@@ -1914,7 +1922,9 @@ private List getMetadataHeadersList(final Hashtable fileStatusList = new ArrayList<>();
// We need to loop on continuation token until we get an entry or continuation token becomes null.
do {
- ListResponseData listResponseData = listPath(path, false, 1, continuationToken, tracingContext, null);
+ ListResponseData listResponseData = listPath(path, false, 1, null, tracingContext, null);
fileStatusList.addAll(listResponseData.getFileStatusList());
continuationToken = listResponseData.getContinuationToken();
} while (StringUtils.isNotEmpty(continuationToken) && fileStatusList.isEmpty());
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java
index 64d33d40a67a2..d581d97b1ab94 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java
@@ -526,7 +526,7 @@ public abstract AbfsRestOperation createFilesystem(TracingContext tracingContext
* @return executed rest operation containing response from server.
* @throws AzureBlobFileSystemException if rest operation fails.
*/
- public abstract AbfsRestOperation setFilesystemProperties(Hashtable properties,
+ public abstract AbfsRestOperation setFilesystemProperties(Hashtable properties, final String name,
TracingContext tracingContext) throws AzureBlobFileSystemException;
/**
@@ -882,7 +882,7 @@ public boolean appendSuccessCheckOp(AbfsRestOperation op, final String path,
public abstract AbfsRestOperation flush(String path, long position,
boolean retainUncommittedData, boolean isClose,
String cachedSasToken, String leaseId,
- ContextEncryptionAdapter contextEncryptionAdapter, TracingContext tracingContext)
+ ContextEncryptionAdapter contextEncryptionAdapter, TracingContext tracingContext, String blobMd5)
throws AzureBlobFileSystemException;
/**
@@ -905,7 +905,7 @@ public abstract AbfsRestOperation flush(byte[] buffer,
String leaseId,
String eTag,
ContextEncryptionAdapter contextEncryptionAdapter,
- TracingContext tracingContext) throws AzureBlobFileSystemException;
+ TracingContext tracingContext, String blobMd5) throws AzureBlobFileSystemException;
/**
* Set the properties of a file or directory.
@@ -1338,21 +1338,6 @@ private void appendIfNotEmpty(StringBuilder sb, String regEx,
}
}
- /**
- * Add MD5 hash as request header to the append request.
- * @param requestHeaders to be updated with checksum header
- * @param reqParams for getting offset and length
- * @param buffer for getting input data for MD5 computation
- * @throws AbfsRestOperationException if Md5 computation fails
- */
- protected void addCheckSumHeaderForWrite(List requestHeaders,
- final AppendRequestParameters reqParams, final byte[] buffer)
- throws AbfsRestOperationException {
- String md5Hash = computeMD5Hash(buffer, reqParams.getoffset(),
- reqParams.getLength());
- requestHeaders.add(new AbfsHttpHeader(CONTENT_MD5, md5Hash));
- }
-
/**
* To verify the checksum information received from server for the data read.
* @param buffer stores the data received from server.
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsDfsClient.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsDfsClient.java
index 2b1fa87756396..5390443d8c4c7 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsDfsClient.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsDfsClient.java
@@ -110,12 +110,14 @@
import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.TRUE;
import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.XMS_PROPERTIES_ENCODING_ASCII;
import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.ACCEPT;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.CONTENT_MD5;
import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.EXPECT;
import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.IF_MATCH;
import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.IF_NONE_MATCH;
import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.RANGE;
import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.USER_AGENT;
import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_HTTP_METHOD_OVERRIDE;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_BLOB_CONTENT_MD5;
import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_CLIENT_TRANSACTION_ID;
import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_EXISTING_RESOURCE_TYPE;
import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_LEASE_ACTION;
@@ -229,7 +231,7 @@ public AbfsRestOperation createFilesystem(TracingContext tracingContext)
* @throws AzureBlobFileSystemException if rest operation fails.
*/
@Override
- public AbfsRestOperation setFilesystemProperties(final Hashtable properties,
+ public AbfsRestOperation setFilesystemProperties(final Hashtable properties, final String name,
TracingContext tracingContext) throws AzureBlobFileSystemException {
final String commaSeparatedProperties;
try {
@@ -762,7 +764,9 @@ public AbfsRestOperation append(final String path,
// Add MD5 Hash of request content as request header if feature is enabled
if (isChecksumValidationEnabled()) {
- addCheckSumHeaderForWrite(requestHeaders, reqParams, buffer);
+ if (reqParams.getMd5() != null) {
+ requestHeaders.add(new AbfsHttpHeader(CONTENT_MD5, reqParams.getMd5()));
+ }
}
// AbfsInputStream/AbfsOutputStream reuse SAS tokens for better performance
@@ -855,7 +859,7 @@ public AbfsRestOperation flush(final String path,
final String cachedSasToken,
final String leaseId,
ContextEncryptionAdapter contextEncryptionAdapter,
- TracingContext tracingContext) throws AzureBlobFileSystemException {
+ TracingContext tracingContext, String blobMd5) throws AzureBlobFileSystemException {
final List requestHeaders = createDefaultHeaders();
addEncryptionKeyRequestHeaders(path, requestHeaders, false,
contextEncryptionAdapter, tracingContext);
@@ -865,6 +869,9 @@ public AbfsRestOperation flush(final String path,
if (leaseId != null) {
requestHeaders.add(new AbfsHttpHeader(X_MS_LEASE_ID, leaseId));
}
+ if (isChecksumValidationEnabled()) {
+ requestHeaders.add(new AbfsHttpHeader(X_MS_BLOB_CONTENT_MD5, blobMd5));
+ }
final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder();
abfsUriQueryBuilder.addQuery(QUERY_PARAM_ACTION, FLUSH_ACTION);
@@ -894,7 +901,7 @@ public AbfsRestOperation flush(byte[] buffer,
final String leaseId,
final String eTag,
final ContextEncryptionAdapter contextEncryptionAdapter,
- final TracingContext tracingContext) throws AzureBlobFileSystemException {
+ final TracingContext tracingContext, String blobMd5) throws AzureBlobFileSystemException {
throw new UnsupportedOperationException(
"Flush with blockIds not supported on DFS Endpoint");
}
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java
index 29f83ae6f03ff..62133c33899b3 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java
@@ -22,6 +22,9 @@
import java.io.IOException;
import java.io.OutputStream;
import java.net.HttpURLConnection;
+import java.security.MessageDigest;
+import java.security.NoSuchAlgorithmException;
+import java.util.Base64;
import java.util.concurrent.ConcurrentLinkedDeque;
import java.util.concurrent.Future;
import java.util.UUID;
@@ -150,6 +153,9 @@ public class AbfsOutputStream extends OutputStream implements Syncable,
/** The handler for managing Abfs client operations. */
private final AbfsClientHandler clientHandler;
+ private final MessageDigest md5;
+ private final MessageDigest fullBlobContentMd5;
+
public AbfsOutputStream(AbfsOutputStreamContext abfsOutputStreamContext)
throws IOException {
this.statistics = abfsOutputStreamContext.getStatistics();
@@ -202,6 +208,16 @@ public AbfsOutputStream(AbfsOutputStreamContext abfsOutputStreamContext)
this.clientHandler = abfsOutputStreamContext.getClientHandler();
createIngressHandler(serviceTypeAtInit,
abfsOutputStreamContext.getBlockFactory(), bufferSize, false, null);
+ try {
+ md5 = MessageDigest.getInstance("MD5");
+ fullBlobContentMd5 = MessageDigest.getInstance("MD5");
+ } catch (NoSuchAlgorithmException e) {
+ throw new IOException("MD5 algorithm not available", e);
+ }
+ }
+
+ public int getBufferSize() {
+ return bufferSize;
}
/**
@@ -438,6 +454,8 @@ public synchronized void write(final byte[] data, final int off, final int lengt
AbfsBlock block = createBlockIfNeeded(position);
int written = bufferData(block, data, off, length);
+ getMessageDigest().update(data, off, written);
+ getFullBlobContentMd5().update(data, off, written);
int remainingCapacity = block.remainingCapacity();
if (written < length) {
@@ -514,6 +532,7 @@ private void uploadBlockAsync(AbfsBlock blockToUpload,
outputStreamStatistics.bytesToUpload(bytesLength);
outputStreamStatistics.writeCurrentBuffer();
DataBlocks.BlockUploadData blockUploadData = blockToUpload.startUpload();
+ String md5Hash = getMd5();
final Future job =
executorService.submit(() -> {
AbfsPerfTracker tracker =
@@ -535,7 +554,7 @@ private void uploadBlockAsync(AbfsBlock blockToUpload,
* leaseId - The AbfsLeaseId for this request.
*/
AppendRequestParameters reqParams = new AppendRequestParameters(
- offset, 0, bytesLength, mode, false, leaseId, isExpectHeaderEnabled);
+ offset, 0, bytesLength, mode, false, leaseId, isExpectHeaderEnabled, md5Hash);
AbfsRestOperation op;
try {
op = remoteWrite(blockToUpload, blockUploadData, reqParams, tracingContext);
@@ -1170,4 +1189,39 @@ public Boolean areWriteOperationsTasksDone() {
}
return true;
}
+
+ /**
+ * Returns the MessageDigest used for computing the incremental MD5 hash
+ * of the data written so far.
+ *
+ * @return the MessageDigest used for partial MD5 calculation.
+ */
+ public MessageDigest getMessageDigest() {
+ return md5;
+ }
+
+ /**
+ * Returns the MessageDigest used for computing the MD5 hash
+ * of the full blob content.
+ *
+ * @return the MessageDigest used for full blob MD5 calculation.
+ */
+ public MessageDigest getFullBlobContentMd5() {
+ return fullBlobContentMd5;
+ }
+
+ /**
+ * Returns the Base64-encoded MD5 checksum based on the current digest state.
+ * This finalizes the digest calculation. Returns null if the digest is empty.
+ *
+ * @return the Base64-encoded MD5 string, or null if no digest is available.
+ */
+ public String getMd5() {
+ byte[] digest = getMessageDigest().digest();
+ String md5 = null;
+ if (digest.length != 0) {
+ md5 = Base64.getEncoder().encodeToString(digest);
+ }
+ return md5;
+ }
}
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AzureBlobBlockManager.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AzureBlobBlockManager.java
index c72bf721dc6f7..652ac1b1be21c 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AzureBlobBlockManager.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AzureBlobBlockManager.java
@@ -26,6 +26,7 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
+import org.apache.commons.codec.binary.Base64;
import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException;
import org.apache.hadoop.fs.azurebfs.utils.TracingContext;
import org.apache.hadoop.fs.store.DataBlocks;
@@ -46,6 +47,7 @@ public class AzureBlobBlockManager extends AzureBlockManager {
/** The list to store blockId, position, and status. */
private final LinkedList blockEntryList = new LinkedList<>();
+ int blockIdLength = 0;
/**
* Constructs an AzureBlobBlockManager.
@@ -82,8 +84,9 @@ protected synchronized AbfsBlock createBlockInternal(long position)
throws IOException {
if (getActiveBlock() == null) {
setBlockCount(getBlockCount() + 1);
- AbfsBlock activeBlock = new AbfsBlobBlock(getAbfsOutputStream(), position);
+ AbfsBlock activeBlock = new AbfsBlobBlock(getAbfsOutputStream(), position, blockIdLength);
activeBlock.setBlockEntry(addNewEntry(activeBlock.getBlockId(), activeBlock.getOffset()));
+ getAbfsOutputStream().getMessageDigest().reset();
setActiveBlock(activeBlock);
}
return getActiveBlock();
@@ -104,6 +107,9 @@ private List getBlockList(TracingContext tracingContext)
.getBlockList(getAbfsOutputStream().getPath(), tracingContext);
if (op != null && op.getResult() != null) {
committedBlockIdList = op.getResult().getBlockIdList();
+ if (!committedBlockIdList.isEmpty()) {
+ blockIdLength = Base64.decodeBase64(committedBlockIdList.get(0)).length;
+ }
}
return committedBlockIdList;
}
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AzureBlobIngressHandler.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AzureBlobIngressHandler.java
index 150d85d474a03..44c684e4506d7 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AzureBlobIngressHandler.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AzureBlobIngressHandler.java
@@ -20,6 +20,8 @@
import java.io.IOException;
import java.nio.charset.StandardCharsets;
+import java.security.MessageDigest;
+import java.util.Base64;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -179,11 +181,13 @@ protected synchronized AbfsRestOperation remoteFlush(final long offset,
tracingContextFlush.setIngressHandler(BLOB_FLUSH);
tracingContextFlush.setPosition(String.valueOf(offset));
LOG.trace("Flushing data at offset {} for path {}", offset, getAbfsOutputStream().getPath());
+ String fullBlobMd5 = Base64.getEncoder().encodeToString(getAbfsOutputStream().getFullBlobContentMd5().digest());
op = getClient().flush(blockListXml.getBytes(StandardCharsets.UTF_8),
getAbfsOutputStream().getPath(),
isClose, getAbfsOutputStream().getCachedSasTokenString(), leaseId,
- getETag(), getAbfsOutputStream().getContextEncryptionAdapter(), tracingContextFlush);
+ getETag(), getAbfsOutputStream().getContextEncryptionAdapter(), tracingContextFlush, fullBlobMd5);
setETag(op.getResult().getResponseHeader(HttpHeaderConfigurations.ETAG));
+ getAbfsOutputStream().getFullBlobContentMd5().reset();
} catch (AbfsRestOperationException ex) {
LOG.error("Error in remote flush requiring handler switch for path {}", getAbfsOutputStream().getPath(), ex);
if (shouldIngressHandlerBeSwitched(ex)) {
@@ -289,7 +293,9 @@ protected void writeAppendBlobCurrentBufferToService() throws IOException {
LOG.trace("Writing current buffer to service at offset {} and path {}", offset, getAbfsOutputStream().getPath());
AppendRequestParameters reqParams = new AppendRequestParameters(
offset, 0, bytesLength, AppendRequestParameters.Mode.APPEND_MODE,
- true, getAbfsOutputStream().getLeaseId(), getAbfsOutputStream().isExpectHeaderEnabled());
+ true, getAbfsOutputStream().getLeaseId(),
+ getAbfsOutputStream().isExpectHeaderEnabled(),
+ getAbfsOutputStream().getMd5());
AbfsRestOperation op;
try {
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AzureDFSBlockManager.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AzureDFSBlockManager.java
index f7a4542c3f2ae..c2fc28afcae03 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AzureDFSBlockManager.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AzureDFSBlockManager.java
@@ -62,6 +62,7 @@ protected synchronized AbfsBlock createBlockInternal(long position)
if (getActiveBlock() == null) {
setBlockCount(getBlockCount() + 1);
AbfsBlock activeBlock = new AbfsBlock(getAbfsOutputStream(), position);
+ getAbfsOutputStream().getMessageDigest().reset();
setActiveBlock(activeBlock);
}
return getActiveBlock();
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AzureDFSIngressHandler.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AzureDFSIngressHandler.java
index 9b6562f2da1e7..a5468c319ecc6 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AzureDFSIngressHandler.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AzureDFSIngressHandler.java
@@ -19,6 +19,8 @@
package org.apache.hadoop.fs.azurebfs.services;
import java.io.IOException;
+import java.security.MessageDigest;
+import java.util.Base64;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -178,13 +180,16 @@ protected synchronized AbfsRestOperation remoteFlush(final long offset,
tracingContextFlush.setIngressHandler(DFS_FLUSH);
tracingContextFlush.setPosition(String.valueOf(offset));
}
+ String fullBlobMd5 = Base64.getEncoder().encodeToString(getAbfsOutputStream().getFullBlobContentMd5().digest());
LOG.trace("Flushing data at offset {} and path {}", offset, getAbfsOutputStream().getPath());
- return getClient()
+ AbfsRestOperation op = getClient()
.flush(getAbfsOutputStream().getPath(), offset, retainUncommitedData,
isClose,
getAbfsOutputStream().getCachedSasTokenString(), leaseId,
getAbfsOutputStream().getContextEncryptionAdapter(),
- tracingContextFlush);
+ tracingContextFlush, fullBlobMd5);
+ getAbfsOutputStream().getFullBlobContentMd5().reset();
+ return op;
}
/**
@@ -225,7 +230,9 @@ protected void writeAppendBlobCurrentBufferToService() throws IOException {
LOG.trace("Writing current buffer to service at offset {} and path {}", offset, getAbfsOutputStream().getPath());
AppendRequestParameters reqParams = new AppendRequestParameters(
offset, 0, bytesLength, AppendRequestParameters.Mode.APPEND_MODE,
- true, getAbfsOutputStream().getLeaseId(), getAbfsOutputStream().isExpectHeaderEnabled());
+ true, getAbfsOutputStream().getLeaseId(),
+ getAbfsOutputStream().isExpectHeaderEnabled(),
+ getAbfsOutputStream().getMd5());
// Perform the remote write operation.
AbfsRestOperation op = remoteWrite(activeBlock, uploadData, reqParams,
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AzureDfsToBlobIngressFallbackHandler.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AzureDfsToBlobIngressFallbackHandler.java
index ba842cbb79b62..cfa01315f2ae8 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AzureDfsToBlobIngressFallbackHandler.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AzureDfsToBlobIngressFallbackHandler.java
@@ -230,7 +230,9 @@ protected void writeAppendBlobCurrentBufferToService() throws IOException {
LOG.trace("Writing current buffer to service at offset {} and path {}", offset, getAbfsOutputStream().getPath());
AppendRequestParameters reqParams = new AppendRequestParameters(
offset, 0, bytesLength, AppendRequestParameters.Mode.APPEND_MODE,
- true, getAbfsOutputStream().getLeaseId(), getAbfsOutputStream().isExpectHeaderEnabled());
+ true, getAbfsOutputStream().getLeaseId(),
+ getAbfsOutputStream().isExpectHeaderEnabled(),
+ getAbfsOutputStream().getMd5());
// Perform the remote write operation.
AbfsRestOperation op;
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AzureIngressHandler.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AzureIngressHandler.java
index 3072bdf5d04d6..135e2f5c86ca0 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AzureIngressHandler.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AzureIngressHandler.java
@@ -19,6 +19,7 @@
package org.apache.hadoop.fs.azurebfs.services;
import java.io.IOException;
+import java.security.MessageDigest;
import java.util.Objects;
import org.slf4j.Logger;
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/RenameAtomicity.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/RenameAtomicity.java
index 860d9eb527763..ad51faab1bbb5 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/RenameAtomicity.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/RenameAtomicity.java
@@ -213,7 +213,7 @@ void createRenamePendingJson(Path path, byte[] bytes)
= new AppendRequestParameters(0, 0,
bytes.length, AppendRequestParameters.Mode.APPEND_MODE, false, null,
abfsClient.getAbfsConfiguration().isExpectHeaderEnabled(),
- new BlobAppendRequestParameters(blockId, eTag));
+ new BlobAppendRequestParameters(blockId, eTag), null);
abfsClient.append(path.toUri().getPath(), bytes,
appendRequestParameters, null, null, tracingContext);
@@ -221,7 +221,7 @@ void createRenamePendingJson(Path path, byte[] bytes)
String blockList = generateBlockListXml(blockId);
// PutBlockList on the path.
abfsClient.flush(blockList.getBytes(StandardCharsets.UTF_8),
- path.toUri().getPath(), true, null, null, eTag, null, tracingContext);
+ path.toUri().getPath(), true, null, null, eTag, null, tracingContext, null);
}
/**
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsIntegrationTest.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsIntegrationTest.java
index e31df5eec65bb..b03e2cf56327b 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsIntegrationTest.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsIntegrationTest.java
@@ -210,6 +210,7 @@ public void setup() throws Exception {
if (rawConfig.get(keyProperty) == null) {
rawConfig.set(keyProperty, getAccountKey());
}
+ rawConfig.set("fs.azure.enable.append.support", "true");
azureNativeFileSystemStore.initialize(
wasbUri,
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsCustomEncryption.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsCustomEncryption.java
index 781cd701400f8..45de7b3d2348e 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsCustomEncryption.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsCustomEncryption.java
@@ -320,22 +320,22 @@ private AbfsRestOperation callOperation(AzureBlobFileSystem fs,
case WRITE:
if (ingressClient instanceof AbfsDfsClient) {
return ingressClient.flush(path, 3, false, false, null,
- null, encryptionAdapter, getTestTracingContext(fs, false));
+ null, encryptionAdapter, getTestTracingContext(fs, false), null);
} else {
byte[] buffer = generateBlockListXml(EMPTY_STRING).getBytes(StandardCharsets.UTF_8);
return ingressClient.flush(buffer, path, false, null,
- null, null, encryptionAdapter, getTestTracingContext(fs, false));
+ null, null, encryptionAdapter, getTestTracingContext(fs, false), null);
}
case APPEND:
if (ingressClient instanceof AbfsDfsClient) {
return ingressClient.append(path, "val".getBytes(),
new AppendRequestParameters(3, 0, 3, APPEND_MODE, false, null,
- true),
+ true, null, null),
null, encryptionAdapter, getTestTracingContext(fs, false));
} else {
return ingressClient.append(path, "val".getBytes(),
new AppendRequestParameters(3, 0, 3, APPEND_MODE, false, null,
- true, new BlobAppendRequestParameters(BLOCK_ID, null)),
+ true, new BlobAppendRequestParameters(BLOCK_ID, null), null),
null, encryptionAdapter, getTestTracingContext(fs, false));
}
case SET_ACL:
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemAppend.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemAppend.java
index 920c4964a559f..db23062813653 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemAppend.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemAppend.java
@@ -1132,7 +1132,7 @@ public void testFlushSuccessWithConnectionResetOnResponseValidMd5() throws Excep
Mockito.nullable(String.class),
Mockito.anyString(),
Mockito.nullable(ContextEncryptionAdapter.class),
- Mockito.any(TracingContext.class)
+ Mockito.any(TracingContext.class), Mockito.anyString()
);
out.hsync();
@@ -1145,7 +1145,7 @@ public void testFlushSuccessWithConnectionResetOnResponseValidMd5() throws Excep
Mockito.nullable(String.class),
Mockito.anyString(),
Mockito.nullable(ContextEncryptionAdapter.class),
- Mockito.any(TracingContext.class));
+ Mockito.any(TracingContext.class), Mockito.anyString());
}
}
@@ -1234,7 +1234,7 @@ public void testFlushSuccessWithConnectionResetOnResponseInvalidMd5() throws Exc
Mockito.nullable(String.class),
Mockito.anyString(),
Mockito.nullable(ContextEncryptionAdapter.class),
- Mockito.any(TracingContext.class)
+ Mockito.any(TracingContext.class), Mockito.anyString()
);
FSDataOutputStream os1 = createMockedOutputStream(fs,
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemChecksum.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemChecksum.java
index bf8c14ae5e14d..48bd369bd7eac 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemChecksum.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemChecksum.java
@@ -18,7 +18,10 @@
package org.apache.hadoop.fs.azurebfs;
+import java.io.IOException;
import java.nio.charset.StandardCharsets;
+import java.security.MessageDigest;
+import java.security.NoSuchAlgorithmException;
import java.security.SecureRandom;
import java.util.Arrays;
import java.util.HashSet;
@@ -28,6 +31,7 @@
import org.assertj.core.api.Assertions;
import org.junit.Assume;
import org.junit.Test;
+import org.mockito.Mock;
import org.mockito.Mockito;
import org.apache.hadoop.conf.Configuration;
@@ -61,11 +65,36 @@ public class ITestAzureBlobFileSystemChecksum extends AbstractAbfsIntegrationTes
private static final int MB_15 = 15 * ONE_MB;
private static final int MB_16 = 16 * ONE_MB;
private static final String INVALID_MD5_TEXT = "Text for Invalid MD5 Computation";
+ private final MessageDigest md;
public ITestAzureBlobFileSystemChecksum() throws Exception {
super();
+ try {
+ md = MessageDigest.getInstance("MD5");
+ } catch (NoSuchAlgorithmException e) {
+ throw new IOException("MD5 algorithm not available", e);
+ }
}
+ /**
+ * Computes the MD5 checksum of a specified portion of the input byte array.
+ *
+ * @param data The byte array containing the data to compute the MD5 checksum for.
+ * @param off The starting offset in the byte array.
+ * @param length The number of bytes to include in the checksum computation.
+ * @return The Base64-encoded MD5 checksum of the specified data, or null if the digest is empty.
+ * @throws IllegalArgumentException If the offset or length is invalid for the given byte array.
+ */
+public String getMd5(byte[] data, int off, int length) {
+ md.update(data, off, length);
+ byte[] digest = md.digest();
+ String md5 = null;
+ if (digest.length != 0) {
+ md5 = java.util.Base64.getEncoder().encodeToString(digest);
+ }
+ return md5;
+}
+
@Test
public void testWriteReadWithChecksum() throws Exception {
testWriteReadWithChecksumInternal(true);
@@ -84,10 +113,10 @@ public void testAppendWithChecksumAtDifferentOffsets() throws Exception {
byte[] data = generateRandomBytes(MB_4);
int pos = 0;
- pos += appendWithOffsetHelper(os, client, path, data, fs, pos, 0);
- pos += appendWithOffsetHelper(os, client, path, data, fs, pos, ONE_MB);
- pos += appendWithOffsetHelper(os, client, path, data, fs, pos, MB_2);
- appendWithOffsetHelper(os, client, path, data, fs, pos, MB_4 - 1);
+ pos += appendWithOffsetHelper(os, client, path, data, fs, pos, 0, getMd5(data, 0, data.length));
+ pos += appendWithOffsetHelper(os, client, path, data, fs, pos, ONE_MB, getMd5(data, ONE_MB, data.length - ONE_MB));
+ pos += appendWithOffsetHelper(os, client, path, data, fs, pos, MB_2, getMd5(data, MB_2, data.length-MB_2));
+ appendWithOffsetHelper(os, client, path, data, fs, pos, MB_4 - 1, getMd5(data, MB_4 - 1, data.length - (MB_4 - 1)));
fs.close();
}
@@ -118,14 +147,15 @@ public void testAbfsInvalidChecksumExceptionInAppend() throws Exception {
AzureBlobFileSystem fs = getConfiguredFileSystem(MB_4, MB_4, true);
AbfsClient spiedClient = Mockito.spy(fs.getAbfsStore().getClientHandler().getIngressClient());
Path path = path("testPath" + getMethodName());
- AbfsOutputStream os = (AbfsOutputStream) fs.create(path).getWrappedStream();
+ AbfsOutputStream os = Mockito.spy((AbfsOutputStream) fs.create(path).getWrappedStream());
byte[] data= generateRandomBytes(MB_4);
String invalidMD5Hash = spiedClient.computeMD5Hash(
INVALID_MD5_TEXT.getBytes(), 0, INVALID_MD5_TEXT.length());
Mockito.doReturn(invalidMD5Hash).when(spiedClient).computeMD5Hash(any(),
any(Integer.class), any(Integer.class));
+ Mockito.doReturn(invalidMD5Hash).when(os).getMd5();
AbfsRestOperationException ex = intercept(AbfsInvalidChecksumException.class, () -> {
- appendWithOffsetHelper(os, spiedClient, path, data, fs, 0, 0);
+ appendWithOffsetHelper(os, spiedClient, path, data, fs, 0, 0, invalidMD5Hash);
});
Assertions.assertThat(ex.getErrorCode())
@@ -197,12 +227,12 @@ private String generateBlockId(AbfsOutputStream os, long position) {
* @throws Exception
*/
private int appendWithOffsetHelper(AbfsOutputStream os, AbfsClient client, Path path,
- byte[] data, AzureBlobFileSystem fs, final int pos, final int offset) throws Exception {
+ byte[] data, AzureBlobFileSystem fs, final int pos, final int offset, String md5) throws Exception {
String blockId = generateBlockId(os, pos);
String eTag = os.getIngressHandler().getETag();
AppendRequestParameters reqParams = new AppendRequestParameters(
pos, offset, data.length - offset, APPEND_MODE, isAppendBlobEnabled(), null, true,
- new BlobAppendRequestParameters(blockId, eTag));
+ new BlobAppendRequestParameters(blockId, eTag), md5);
client.append(path.toUri().getPath(), data, reqParams, null, null,
getTestTracingContext(fs, false));
return reqParams.getLength();
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemDelegationSAS.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemDelegationSAS.java
index 378655405da66..31ef93fa4c100 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemDelegationSAS.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemDelegationSAS.java
@@ -436,7 +436,7 @@ public void testSetFileSystemProperties() throws Exception {
properties.put("FileSystemProperties", "true");
TracingContext tracingContext = getTestTracingContext(fs, true);
assertThrows(IOException.class, () -> fs.getAbfsStore()
- .setFilesystemProperties(properties, tracingContext));
+ .setFilesystemProperties(properties, "FileSystemProperties", tracingContext));
assertThrows(IOException.class,
() -> fs.getAbfsStore().getFilesystemProperties(tracingContext));
}
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemRename.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemRename.java
index 408f94d78d88c..14f2a14b2b64d 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemRename.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemRename.java
@@ -2128,7 +2128,7 @@ private void testRenamePreRenameFailureResolution(final AzureBlobFileSystem fs)
Mockito.anyBoolean(), Mockito.nullable(String.class),
Mockito.nullable(String.class), Mockito.anyString(),
Mockito.nullable(ContextEncryptionAdapter.class),
- Mockito.any(TracingContext.class));
+ Mockito.any(TracingContext.class), Mockito.anyString());
return createAnswer.callRealMethod();
};
RenameAtomicityTestUtils.addCreatePathMock(client,
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestFileSystemProperties.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestFileSystemProperties.java
index 3f2a4fe98802c..49446994ca922 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestFileSystemProperties.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestFileSystemProperties.java
@@ -77,7 +77,7 @@ public void testBase64FileSystemProperties() throws Exception {
final Hashtable properties = new Hashtable<>();
properties.put("key", "{ value: value }");
TracingContext tracingContext = getTestTracingContext(fs, true);
- fs.getAbfsStore().setFilesystemProperties(properties, tracingContext);
+ fs.getAbfsStore().setFilesystemProperties(properties, "key", tracingContext);
Hashtable fetchedProperties = fs.getAbfsStore()
.getFilesystemProperties(tracingContext);
@@ -105,7 +105,7 @@ public void testBase64InvalidFileSystemProperties() throws Exception {
final Hashtable properties = new Hashtable<>();
properties.put("key", "{ value: value歲 }");
TracingContext tracingContext = getTestTracingContext(fs, true);
- fs.getAbfsStore().setFilesystemProperties(properties, tracingContext);
+ fs.getAbfsStore().setFilesystemProperties(properties, "key", tracingContext);
Hashtable fetchedProperties = fs.getAbfsStore()
.getFilesystemProperties(tracingContext);
@@ -133,7 +133,7 @@ public void testSetFileSystemProperties() throws Exception {
final Hashtable properties = new Hashtable<>();
properties.put("containerForDevTest", "true");
TracingContext tracingContext = getTestTracingContext(fs, true);
- fs.getAbfsStore().setFilesystemProperties(properties, tracingContext);
+ fs.getAbfsStore().setFilesystemProperties(properties, "containerForDevTest", tracingContext);
Hashtable fetchedProperties = fs.getAbfsStore()
.getFilesystemProperties(tracingContext);
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestWasbAbfsCompatibility.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestWasbAbfsCompatibility.java
index 38d4d12b10f34..bbde4bd4a3257 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestWasbAbfsCompatibility.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestWasbAbfsCompatibility.java
@@ -18,34 +18,65 @@
package org.apache.hadoop.fs.azurebfs;
import java.io.BufferedReader;
+import java.io.IOException;
import java.io.InputStreamReader;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+import java.util.EnumSet;
+import java.util.UUID;
+import org.assertj.core.api.Assertions;
import org.junit.Assume;
import org.junit.Test;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
+import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
+import org.apache.hadoop.fs.XAttrSetFlag;
import org.apache.hadoop.fs.azure.NativeAzureFileSystem;
import org.apache.hadoop.fs.azurebfs.constants.AbfsServiceType;
import org.apache.hadoop.fs.contract.ContractTestUtils;
+import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_ABFS_ENABLE_CHECKSUM_VALIDATION;
+import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_INFINITE_LEASE_KEY;
+import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_LEASE_THREADS;
import static org.apache.hadoop.fs.contract.ContractTestUtils.assertDeleted;
import static org.apache.hadoop.fs.contract.ContractTestUtils.assertIsDirectory;
import static org.apache.hadoop.fs.contract.ContractTestUtils.assertMkdirs;
import static org.apache.hadoop.fs.contract.ContractTestUtils.assertPathExists;
+import static org.apache.hadoop.fs.contract.ContractTestUtils.file;
/**
* Test compatibility between ABFS client and WASB client.
*/
public class ITestWasbAbfsCompatibility extends AbstractAbfsIntegrationTest {
+
private static final String WASB_TEST_CONTEXT = "wasb test file";
+
private static final String ABFS_TEST_CONTEXT = "abfs test file";
+
private static final String TEST_CONTEXT = "THIS IS FOR TEST";
+ private static final String TEST_CONTEXT1 = "THIS IS FOR TEST1";
+
+ private static final byte[] ATTRIBUTE_VALUE_1 = "one".getBytes(
+ StandardCharsets.UTF_8);
+
+ private static final byte[] ATTRIBUTE_VALUE_2 = "two".getBytes(
+ StandardCharsets.UTF_8);
+
+ private static final String ATTRIBUTE_NAME_1 = "user_someAttribute";
+
+ private static final String ATTRIBUTE_NAME_2 = "user_someAttribute1";
+
+ private static final EnumSet CREATE_FLAG = EnumSet.of(
+ XAttrSetFlag.CREATE);
+
private static final Logger LOG =
LoggerFactory.getLogger(ITestWasbAbfsCompatibility.class);
@@ -66,7 +97,7 @@ public void testListFileStatus() throws Exception {
Path testFiles = path("/testfiles");
Path path1 = new Path(testFiles + "/~12/!008/3/abFsTestfile");
- try(FSDataOutputStream abfsStream = fs.create(path1, true)) {
+ try (FSDataOutputStream abfsStream = fs.create(path1, true)) {
abfsStream.write(ABFS_TEST_CONTEXT.getBytes());
abfsStream.flush();
abfsStream.hsync();
@@ -75,14 +106,16 @@ public void testListFileStatus() throws Exception {
// create file using wasb
Path path2 = new Path(testFiles + "/~12/!008/3/nativeFsTestfile");
LOG.info("{}", wasb.getUri());
- try(FSDataOutputStream nativeFsStream = wasb.create(path2, true)) {
+ try (FSDataOutputStream nativeFsStream = wasb.create(path2, true)) {
nativeFsStream.write(WASB_TEST_CONTEXT.getBytes());
nativeFsStream.flush();
nativeFsStream.hsync();
}
// list file using abfs and wasb
- FileStatus[] abfsFileStatus = fs.listStatus(new Path(testFiles + "/~12/!008/3/"));
- FileStatus[] nativeFsFileStatus = wasb.listStatus(new Path(testFiles + "/~12/!008/3/"));
+ FileStatus[] abfsFileStatus = fs.listStatus(
+ new Path(testFiles + "/~12/!008/3/"));
+ FileStatus[] nativeFsFileStatus = wasb.listStatus(
+ new Path(testFiles + "/~12/!008/3/"));
assertEquals(2, abfsFileStatus.length);
assertEquals(2, nativeFsFileStatus.length);
@@ -102,18 +135,13 @@ public void testReadFile() throws Exception {
NativeAzureFileSystem wasb = getWasbFileSystem();
Path testFile = path("/testReadFile");
- for (int i = 0; i< 4; i++) {
+ for (int i = 0; i < 4; i++) {
Path path = new Path(testFile + "/~12/!008/testfile" + i);
final FileSystem createFs = createFileWithAbfs[i] ? abfs : wasb;
// Read
final FileSystem readFs = readFileWithAbfs[i] ? abfs : wasb;
- if (createFs == abfs && readFs == wasb) {
- //Since flush changes the md5Hash value, md5 returned by GetBlobProperties will not match the one returned by GetBlob.
- Assume.assumeFalse(getIngressServiceType() == AbfsServiceType.BLOB);
- }
-
// Write
- try(FSDataOutputStream nativeFsStream = createFs.create(path, true)) {
+ try (FSDataOutputStream nativeFsStream = createFs.create(path, true)) {
nativeFsStream.write(TEST_CONTEXT.getBytes());
nativeFsStream.flush();
nativeFsStream.hsync();
@@ -122,7 +150,8 @@ public void testReadFile() throws Exception {
// Check file status
ContractTestUtils.assertIsFile(createFs, path);
- try(BufferedReader br =new BufferedReader(new InputStreamReader(readFs.open(path)))) {
+ try (BufferedReader br = new BufferedReader(
+ new InputStreamReader(readFs.open(path)))) {
String line = br.readLine();
assertEquals("Wrong text from " + readFs,
TEST_CONTEXT, line);
@@ -133,6 +162,89 @@ public void testReadFile() throws Exception {
}
}
+ @Test
+ public void testwriteFile() throws Exception {
+ AzureBlobFileSystem abfs = getFileSystem();
+ Assume.assumeFalse("Namespace enabled account does not support this test",
+ getIsNamespaceEnabled(abfs));
+ NativeAzureFileSystem wasb = getWasbFileSystem();
+
+ Path testFile = path("/testReadFile");
+ Path path = new Path(testFile + "/~12/!008/testfile" + 1);
+ // Write
+ try (FSDataOutputStream nativeFsStream = wasb.create(path, true)) {
+ nativeFsStream.write(TEST_CONTEXT.getBytes());
+ nativeFsStream.flush();
+ nativeFsStream.hsync();
+ }
+
+ // Check file status
+ ContractTestUtils.assertIsFile(wasb, path);
+
+ try (BufferedReader br = new BufferedReader(
+ new InputStreamReader(abfs.open(path)))) {
+ String line = br.readLine();
+ assertEquals("Wrong text from " + abfs,
+ TEST_CONTEXT, line);
+ }
+ try (FSDataOutputStream nativeFsStream = abfs.append(path)) {
+ nativeFsStream.write(TEST_CONTEXT.getBytes());
+ nativeFsStream.flush();
+ nativeFsStream.hsync();
+ }
+ }
+
+ @Test
+ public void testwriteFile1() throws Exception {
+ AzureBlobFileSystem abfs = getFileSystem();
+ Assume.assumeFalse("Namespace enabled account does not support this test",
+ getIsNamespaceEnabled(abfs));
+ NativeAzureFileSystem wasb = getWasbFileSystem();
+
+ Path testFile = path("/testReadFile");
+ Path path = new Path(testFile + "/~12/!008/testfile" + 2);
+ // Write
+ try (FSDataOutputStream nativeFsStream = abfs.create(path, true)) {
+ nativeFsStream.write(TEST_CONTEXT.getBytes());
+ nativeFsStream.flush();
+ nativeFsStream.hsync();
+ }
+
+ // Check file status
+ ContractTestUtils.assertIsFile(abfs, path);
+
+ try (FSDataOutputStream nativeFsStream = wasb.append(path)) {
+ nativeFsStream.write(TEST_CONTEXT.getBytes());
+ nativeFsStream.flush();
+ nativeFsStream.hsync();
+ }
+
+ try (FSDataOutputStream nativeFsStream = abfs.append(path)) {
+ nativeFsStream.write(TEST_CONTEXT1.getBytes());
+ nativeFsStream.flush();
+ nativeFsStream.hsync();
+ }
+ }
+
+ @Test
+ public void testazcopywasbcompatibility() throws Exception {
+ AzureBlobFileSystem abfs = getFileSystem();
+ Assume.assumeFalse("Namespace enabled account does not support this test",
+ getIsNamespaceEnabled(abfs));
+ NativeAzureFileSystem wasb = getWasbFileSystem();
+
+ Path testFile = path("/testReadFile");
+ Path path = new Path(testFile + "/~12/!008/testfile" + 2);
+ createAzCopyFile(path);
+
+ try (FSDataOutputStream nativeFsStream = abfs.append(path)) {
+ nativeFsStream.write(TEST_CONTEXT1.getBytes());
+ nativeFsStream.flush();
+ nativeFsStream.hsync();
+ }
+ }
+
+
@Test
public void testDir() throws Exception {
boolean[] createDirWithAbfs = new boolean[]{false, true, false, true};
@@ -152,7 +264,8 @@ public void testDir() throws Exception {
final FileSystem createFs = createDirWithAbfs[i] ? abfs : wasb;
assertTrue(createFs.mkdirs(path));
//check
- assertPathExists(createFs, "Created dir not found with " + createFs, path);
+ assertPathExists(createFs, "Created dir not found with " + createFs,
+ path);
//read
final FileSystem readFs = readDirWithAbfs[i] ? abfs : wasb;
assertPathExists(readFs, "Created dir not found with " + readFs,
@@ -164,9 +277,11 @@ public void testDir() throws Exception {
@Test
- public void testUrlConversion(){
- String abfsUrl = "abfs://abcde-1111-1111-1111-1111@xxxx.dfs.xxx.xxx.xxxx.xxxx";
- String wabsUrl = "wasb://abcde-1111-1111-1111-1111@xxxx.blob.xxx.xxx.xxxx.xxxx";
+ public void testUrlConversion() {
+ String abfsUrl
+ = "abfs://abcde-1111-1111-1111-1111@xxxx.dfs.xxx.xxx.xxxx.xxxx";
+ String wabsUrl
+ = "wasb://abcde-1111-1111-1111-1111@xxxx.blob.xxx.xxx.xxxx.xxxx";
assertEquals(abfsUrl, wasbUrlToAbfsUrl(wabsUrl));
assertEquals(wabsUrl, abfsUrlToWasbUrl(abfsUrl, false));
}
@@ -201,4 +316,1586 @@ public void testSetWorkingDirectory() throws Exception {
assertEquals(path3, wasb.getWorkingDirectory());
assertEquals(path3, abfs.getWorkingDirectory());
}
+
+ // Scenario wise testing
+
+ //Scenario 1: - Create and write via WASB, read via ABFS
+ @Test
+ public void testScenario1() throws Exception {
+ AzureBlobFileSystem abfs = getFileSystem();
+ Assume.assumeFalse("Namespace enabled account does not support this test",
+ getIsNamespaceEnabled(abfs));
+ NativeAzureFileSystem wasb = getWasbFileSystem();
+
+ Path testFile = path("/testReadFile");
+ Path path = new Path(testFile + "/~12/!008/testfile" + 1);
+
+ // Write
+ try (FSDataOutputStream nativeFsStream = wasb.create(path, true)) {
+ nativeFsStream.write(TEST_CONTEXT.getBytes());
+ nativeFsStream.flush();
+ nativeFsStream.hsync();
+ }
+ // Check file status
+ ContractTestUtils.assertIsFile(wasb, path);
+
+ try (BufferedReader br = new BufferedReader(
+ new InputStreamReader(abfs.open(path)))) {
+ String line = br.readLine();
+ assertEquals("Wrong text from " + abfs,
+ TEST_CONTEXT, line);
+ }
+
+ // Remove file
+ assertDeleted(abfs, path, true);
+ }
+
+ //Scenario 2: - Create and write via WASB, read via ABFS and then write the same file via ABFS
+ @Test
+ public void testScenario2() throws Exception {
+ AzureBlobFileSystem abfs = getFileSystem();
+ Assume.assumeFalse("Namespace enabled account does not support this test",
+ getIsNamespaceEnabled(abfs));
+ NativeAzureFileSystem wasb = getWasbFileSystem();
+
+ Path testFile = path("/testReadFile");
+ Path path = new Path(testFile + "/~12/!008/testfile" + 2);
+
+ // Write
+ try (FSDataOutputStream nativeFsStream = wasb.create(path, true)) {
+ nativeFsStream.write(TEST_CONTEXT.getBytes());
+ nativeFsStream.flush();
+ nativeFsStream.hsync();
+ }
+ // Check file status
+ ContractTestUtils.assertIsFile(wasb, path);
+
+ try (BufferedReader br = new BufferedReader(
+ new InputStreamReader(abfs.open(path)))) {
+ String line = br.readLine();
+ assertEquals("Wrong text from " + abfs,
+ TEST_CONTEXT, line);
+ }
+
+ // Write
+ try (FSDataOutputStream abfsOutputStream = abfs.append(path)) {
+ abfsOutputStream.write(TEST_CONTEXT1.getBytes());
+ abfsOutputStream.flush();
+ abfsOutputStream.hsync();
+ }
+
+ // Remove file
+ assertDeleted(abfs, path, true);
+ }
+
+ //Scenario 3: - Create and write via ABFS and the read via WASB
+ @Test
+ public void testScenario3() throws Exception {
+ AzureBlobFileSystem abfs = getFileSystem();
+ Assume.assumeFalse("Namespace enabled account does not support this test",
+ getIsNamespaceEnabled(abfs));
+ NativeAzureFileSystem wasb = getWasbFileSystem();
+
+ Path testFile = path("/testReadFile");
+ Path path = new Path(testFile + "/~12/!008/testfile" + 3);
+
+ // Write
+ try (FSDataOutputStream nativeFsStream = abfs.create(path, true)) {
+ nativeFsStream.write(TEST_CONTEXT.getBytes());
+ nativeFsStream.flush();
+ nativeFsStream.hsync();
+ }
+
+ // Check file status
+ ContractTestUtils.assertIsFile(abfs, path);
+
+ try (BufferedReader br = new BufferedReader(
+ new InputStreamReader(wasb.open(path)))) {
+ String line = br.readLine();
+ assertEquals("Wrong text from " + wasb,
+ TEST_CONTEXT, line);
+ }
+ // Remove file
+ assertDeleted(abfs, path, true);
+ }
+
+ //Scenario 4:- Create via WASB, write via ABFS and then write via WASB
+ @Test
+ public void testScenario4() throws Exception {
+ AzureBlobFileSystem abfs = getFileSystem();
+ Assume.assumeFalse("Namespace enabled account does not support this test",
+ getIsNamespaceEnabled(abfs));
+ NativeAzureFileSystem wasb = getWasbFileSystem();
+
+ Path testFile = path("/testReadFile");
+ Path path = new Path(testFile + "/~12/!008/testfile" + 4);
+
+ // Write
+ wasb.create(path, true);
+ try (FSDataOutputStream abfsOutputStream = abfs.append(path)) {
+ abfsOutputStream.write(TEST_CONTEXT.getBytes());
+ abfsOutputStream.flush();
+ abfsOutputStream.hsync();
+ }
+
+ try (FSDataOutputStream nativeFsStream = abfs.append(path)) {
+ nativeFsStream.write(TEST_CONTEXT1.getBytes());
+ nativeFsStream.flush();
+ nativeFsStream.hsync();
+ }
+
+ // Check file status
+ ContractTestUtils.assertIsFile(abfs, path);
+ // Remove file
+ assertDeleted(abfs, path, true);
+ }
+
+ //Scenario 5:- Create via ABFS, write via WASB, read via ABFS (Checksum validation disabled)
+ @Test
+ public void testScenario5() throws Exception {
+ Configuration conf = getRawConfiguration();
+ conf.setBoolean(FS_AZURE_ABFS_ENABLE_CHECKSUM_VALIDATION, false);
+ FileSystem fileSystem = FileSystem.newInstance(conf);
+ AzureBlobFileSystem abfs = (AzureBlobFileSystem) fileSystem;
+ Assume.assumeFalse("Namespace enabled account does not support this test",
+ getIsNamespaceEnabled(abfs));
+ NativeAzureFileSystem wasb = getWasbFileSystem();
+
+ Path testFile = path("/testReadFile");
+ Path path = new Path(testFile + "/~12/!008/testfile" + 5);
+
+ // Write
+ abfs.create(path, true);
+ try (FSDataOutputStream nativeFsStream = wasb.append(path)) {
+ nativeFsStream.write(TEST_CONTEXT.getBytes());
+ nativeFsStream.flush();
+ nativeFsStream.hsync();
+ }
+
+ // Check file status
+ ContractTestUtils.assertIsFile(abfs, path);
+
+ try (BufferedReader br = new BufferedReader(
+ new InputStreamReader(abfs.open(path)))) {
+ String line = br.readLine();
+ assertEquals("Wrong text from " + abfs,
+ TEST_CONTEXT, line);
+ }
+
+ // Remove file
+ assertDeleted(abfs, path, true);
+ }
+
+ //Scenario 6: - Create via ABFS, write via WASB, read via ABFS (Checksum validation enabled)
+ @Test
+ public void testScenario6() throws Exception {
+ Configuration conf = getRawConfiguration();
+ conf.setBoolean(FS_AZURE_ABFS_ENABLE_CHECKSUM_VALIDATION, true);
+ FileSystem fileSystem = FileSystem.newInstance(conf);
+ AzureBlobFileSystem abfs = (AzureBlobFileSystem) fileSystem;
+ Assume.assumeFalse("Namespace enabled account does not support this test",
+ getIsNamespaceEnabled(abfs));
+ NativeAzureFileSystem wasb = getWasbFileSystem();
+
+ Path testFile = path("/testReadFile");
+ Path path = new Path(testFile + "/~12/!008/testfile" + 6);
+
+ // Write
+ abfs.create(path, true);
+ try (FSDataOutputStream nativeFsStream = wasb.append(path)) {
+ nativeFsStream.write(TEST_CONTEXT.getBytes());
+ nativeFsStream.flush();
+ nativeFsStream.hsync();
+ }
+
+ // Check file status
+ ContractTestUtils.assertIsFile(abfs, path);
+
+ try (BufferedReader br = new BufferedReader(
+ new InputStreamReader(abfs.open(path)))) {
+ String line = br.readLine();
+ assertEquals("Wrong text from " + abfs,
+ TEST_CONTEXT, line);
+ }
+
+ // Remove file
+ assertDeleted(abfs, path, true);
+ }
+
+ // Scenario 7 :- Create via WASB and then create overwrite true using ABFS
+ @Test
+ public void testScenario7() throws Exception {
+ Configuration conf = getRawConfiguration();
+ conf.setBoolean(FS_AZURE_ABFS_ENABLE_CHECKSUM_VALIDATION, true);
+ FileSystem fileSystem = FileSystem.newInstance(conf);
+ AzureBlobFileSystem abfs = (AzureBlobFileSystem) fileSystem;
+ Assume.assumeFalse("Namespace enabled account does not support this test",
+ getIsNamespaceEnabled(abfs));
+ NativeAzureFileSystem wasb = getWasbFileSystem();
+
+ Path testFile = path("/testReadFile");
+ Path path = new Path(testFile + "/~12/!008/testfile" + 7);
+
+ try (FSDataOutputStream nativeFsStream = wasb.create(path, true)) {
+ nativeFsStream.write(TEST_CONTEXT.getBytes());
+ nativeFsStream.flush();
+ nativeFsStream.hsync();
+ }
+
+ // Check file status
+ ContractTestUtils.assertIsFile(abfs, path);
+
+ try (BufferedReader br = new BufferedReader(
+ new InputStreamReader(abfs.open(path)))) {
+ String line = br.readLine();
+ assertEquals("Wrong text from " + abfs,
+ TEST_CONTEXT, line);
+ }
+ abfs.create(path, true);
+
+ // Remove file
+ assertDeleted(abfs, path, true);
+ }
+
+ // Scenario 8 :- Create via WASB and then create overwrite false using ABFS
+ @Test
+ public void testScenario8() throws Exception {
+ Configuration conf = getRawConfiguration();
+ conf.setBoolean(FS_AZURE_ABFS_ENABLE_CHECKSUM_VALIDATION, true);
+ FileSystem fileSystem = FileSystem.newInstance(conf);
+ AzureBlobFileSystem abfs = (AzureBlobFileSystem) fileSystem;
+ Assume.assumeFalse("Namespace enabled account does not support this test",
+ getIsNamespaceEnabled(abfs));
+ NativeAzureFileSystem wasb = getWasbFileSystem();
+
+ Path testFile = path("/testReadFile");
+ Path path = new Path(testFile + "/~12/!008/testfile" + 8);
+
+ try (FSDataOutputStream nativeFsStream = wasb.create(path, true)) {
+ nativeFsStream.write(TEST_CONTEXT.getBytes());
+ nativeFsStream.flush();
+ nativeFsStream.hsync();
+ }
+
+ // Check file status
+ ContractTestUtils.assertIsFile(abfs, path);
+
+ try (BufferedReader br = new BufferedReader(
+ new InputStreamReader(abfs.open(path)))) {
+ String line = br.readLine();
+ assertEquals("Wrong text from " + abfs,
+ TEST_CONTEXT, line);
+ }
+ try {
+ abfs.create(path, false);
+ } catch (Exception e) {
+ assertTrue(e.getMessage().contains("AlreadyExists"));
+ }
+
+ // Remove file
+ assertDeleted(abfs, path, true);
+ }
+
+ // Scenario 9 :- Create via ABFS and then create overwrite true using WASB
+ @Test
+ public void testScenario9() throws Exception {
+ Configuration conf = getRawConfiguration();
+ conf.setBoolean(FS_AZURE_ABFS_ENABLE_CHECKSUM_VALIDATION, true);
+ FileSystem fileSystem = FileSystem.newInstance(conf);
+ AzureBlobFileSystem abfs = (AzureBlobFileSystem) fileSystem;
+ Assume.assumeFalse("Namespace enabled account does not support this test",
+ getIsNamespaceEnabled(abfs));
+ NativeAzureFileSystem wasb = getWasbFileSystem();
+
+ Path testFile = path("/testReadFile");
+ Path path = new Path(testFile + "/~12/!008/testfile" + 9);
+
+ try (FSDataOutputStream nativeFsStream = abfs.create(path, true)) {
+ nativeFsStream.write(TEST_CONTEXT.getBytes());
+ nativeFsStream.flush();
+ nativeFsStream.hsync();
+ }
+
+ // Check file status
+ ContractTestUtils.assertIsFile(abfs, path);
+
+ try (BufferedReader br = new BufferedReader(
+ new InputStreamReader(abfs.open(path)))) {
+ String line = br.readLine();
+ assertEquals("Wrong text from " + abfs,
+ TEST_CONTEXT, line);
+ }
+ wasb.create(path, true);
+
+ // Remove file
+ assertDeleted(abfs, path, true);
+ }
+
+ // Scenario 10 :- Create via ABFS and then create overwrite false using WASB
+ @Test
+ public void testScenario10() throws Exception {
+ Configuration conf = getRawConfiguration();
+ conf.setBoolean(FS_AZURE_ABFS_ENABLE_CHECKSUM_VALIDATION, true);
+ FileSystem fileSystem = FileSystem.newInstance(conf);
+ AzureBlobFileSystem abfs = (AzureBlobFileSystem) fileSystem;
+ Assume.assumeFalse("Namespace enabled account does not support this test",
+ getIsNamespaceEnabled(abfs));
+ NativeAzureFileSystem wasb = getWasbFileSystem();
+
+ Path testFile = path("/testReadFile");
+ Path path = new Path(testFile + "/~12/!008/testfile" + 10);
+
+ try (FSDataOutputStream nativeFsStream = abfs.create(path, true)) {
+ nativeFsStream.write(TEST_CONTEXT.getBytes());
+ nativeFsStream.flush();
+ nativeFsStream.hsync();
+ }
+
+ // Check file status
+ ContractTestUtils.assertIsFile(abfs, path);
+
+ try (BufferedReader br = new BufferedReader(
+ new InputStreamReader(abfs.open(path)))) {
+ String line = br.readLine();
+ assertEquals("Wrong text from " + abfs,
+ TEST_CONTEXT, line);
+ }
+ try {
+ wasb.create(path, false);
+ } catch (Exception e) {
+ assertTrue(e.getMessage().toLowerCase().contains("exists"));
+ }
+
+ // Remove file
+ assertDeleted(abfs, path, true);
+ }
+
+ // Scenario 11 :- Create via ABFS and then write via WASB and delete via ABFS
+ @Test
+ public void testScenario11() throws Exception {
+ Configuration conf = getRawConfiguration();
+ conf.setBoolean(FS_AZURE_ABFS_ENABLE_CHECKSUM_VALIDATION, true);
+ FileSystem fileSystem = FileSystem.newInstance(conf);
+ AzureBlobFileSystem abfs = (AzureBlobFileSystem) fileSystem;
+ Assume.assumeFalse("Namespace enabled account does not support this test",
+ getIsNamespaceEnabled(abfs));
+ NativeAzureFileSystem wasb = getWasbFileSystem();
+
+ Path testFile = path("/testReadFile");
+ Path path = new Path(testFile + "/~12/!008/testfile" + 11);
+
+ // Write
+ abfs.create(path, true);
+ try (FSDataOutputStream nativeFsStream = wasb.append(path)) {
+ nativeFsStream.write(TEST_CONTEXT.getBytes());
+ nativeFsStream.flush();
+ nativeFsStream.hsync();
+ }
+
+ // Check file status
+ ContractTestUtils.assertIsFile(abfs, path);
+
+ try (BufferedReader br = new BufferedReader(
+ new InputStreamReader(abfs.open(path)))) {
+ String line = br.readLine();
+ assertEquals("Wrong text from " + abfs,
+ TEST_CONTEXT, line);
+ }
+ abfs.delete(path, true);
+ }
+
+ // Scenario 12 :- Create and write via ABFS and delete via WASB
+ @Test
+ public void testScenario12() throws Exception {
+ Configuration conf = getRawConfiguration();
+ conf.setBoolean(FS_AZURE_ABFS_ENABLE_CHECKSUM_VALIDATION, true);
+ FileSystem fileSystem = FileSystem.newInstance(conf);
+ AzureBlobFileSystem abfs = (AzureBlobFileSystem) fileSystem;
+ Assume.assumeFalse("Namespace enabled account does not support this test",
+ getIsNamespaceEnabled(abfs));
+ NativeAzureFileSystem wasb = getWasbFileSystem();
+
+ Path testFile = path("/testReadFile");
+ Path path = new Path(testFile + "/~12/!008/testfile" + 12);
+
+ // Write
+ try (FSDataOutputStream nativeFsStream = abfs.create(path, true)) {
+ nativeFsStream.write(TEST_CONTEXT.getBytes());
+ nativeFsStream.flush();
+ nativeFsStream.hsync();
+ }
+
+ // Check file status
+ ContractTestUtils.assertIsFile(abfs, path);
+
+ try (BufferedReader br = new BufferedReader(
+ new InputStreamReader(abfs.open(path)))) {
+ String line = br.readLine();
+ assertEquals("Wrong text from " + abfs,
+ TEST_CONTEXT, line);
+ }
+ wasb.delete(path, true);
+ }
+
+ // Scenario 13:- Create via ABFS, write via WASB, and read via wasb
+ @Test
+ public void testScenario13() throws Exception {
+ Configuration conf = getRawConfiguration();
+ conf.setBoolean(FS_AZURE_ABFS_ENABLE_CHECKSUM_VALIDATION, true);
+ FileSystem fileSystem = FileSystem.newInstance(conf);
+ AzureBlobFileSystem abfs = (AzureBlobFileSystem) fileSystem;
+ Assume.assumeFalse("Namespace enabled account does not support this test",
+ getIsNamespaceEnabled(abfs));
+ NativeAzureFileSystem wasb = getWasbFileSystem();
+
+ Path testFile = path("/testReadFile");
+ Path path = new Path(testFile + "/~12/!008/testfile" + 13);
+
+ // Write
+ abfs.create(path, true);
+ try (FSDataOutputStream nativeFsStream = wasb.append(path)) {
+ nativeFsStream.write(TEST_CONTEXT.getBytes());
+ nativeFsStream.flush();
+ nativeFsStream.hsync();
+ }
+
+ // Check file status
+ ContractTestUtils.assertIsFile(abfs, path);
+
+ try (BufferedReader br = new BufferedReader(
+ new InputStreamReader(wasb.open(path)))) {
+ String line = br.readLine();
+ assertEquals("Wrong text from " + wasb,
+ TEST_CONTEXT, line);
+ }
+ abfs.delete(path, true);
+ }
+
+ // Scenario 14:- Create via ABFS, write via WASB, and delete via wasb
+ @Test
+ public void testScenario14() throws Exception {
+ Configuration conf = getRawConfiguration();
+ conf.setBoolean(FS_AZURE_ABFS_ENABLE_CHECKSUM_VALIDATION, true);
+ FileSystem fileSystem = FileSystem.newInstance(conf);
+ AzureBlobFileSystem abfs = (AzureBlobFileSystem) fileSystem;
+ Assume.assumeFalse("Namespace enabled account does not support this test",
+ getIsNamespaceEnabled(abfs));
+ NativeAzureFileSystem wasb = getWasbFileSystem();
+
+ Path testFile = path("/testReadFile");
+ Path path = new Path(testFile + "/~12/!008/testfile" + 14);
+
+ // Write
+ abfs.create(path, true);
+ try (FSDataOutputStream nativeFsStream = wasb.append(path)) {
+ nativeFsStream.write(TEST_CONTEXT.getBytes());
+ nativeFsStream.flush();
+ nativeFsStream.hsync();
+ }
+
+ // Check file status
+ ContractTestUtils.assertIsFile(abfs, path);
+
+ try (BufferedReader br = new BufferedReader(
+ new InputStreamReader(wasb.open(path)))) {
+ String line = br.readLine();
+ assertEquals("Wrong text from " + wasb,
+ TEST_CONTEXT, line);
+ }
+ wasb.delete(path, true);
+ }
+
+ // Scenario 15 :- Create and write via WASB and delete via ABFS
+ @Test
+ public void testScenario15() throws Exception {
+ Configuration conf = getRawConfiguration();
+ conf.setBoolean(FS_AZURE_ABFS_ENABLE_CHECKSUM_VALIDATION, true);
+ FileSystem fileSystem = FileSystem.newInstance(conf);
+ AzureBlobFileSystem abfs = (AzureBlobFileSystem) fileSystem;
+ Assume.assumeFalse("Namespace enabled account does not support this test",
+ getIsNamespaceEnabled(abfs));
+ NativeAzureFileSystem wasb = getWasbFileSystem();
+
+ Path testFile = path("/testReadFile");
+ Path path = new Path(testFile + "/~12/!008/testfile" + 15);
+
+ // Write
+ try (FSDataOutputStream nativeFsStream = wasb.create(path, true)) {
+ nativeFsStream.write(TEST_CONTEXT.getBytes());
+ nativeFsStream.flush();
+ nativeFsStream.hsync();
+ }
+
+ // Check file status
+ ContractTestUtils.assertIsFile(abfs, path);
+
+ try (BufferedReader br = new BufferedReader(
+ new InputStreamReader(wasb.open(path)))) {
+ String line = br.readLine();
+ assertEquals("Wrong text from " + wasb,
+ TEST_CONTEXT, line);
+ }
+ abfs.delete(path, true);
+ }
+
+ // Scenario 16: Create via WASB, write via ABFS, and delete via WASB
+ @Test
+ public void testScenario16() throws Exception {
+ Configuration conf = getRawConfiguration();
+ conf.setBoolean(FS_AZURE_ABFS_ENABLE_CHECKSUM_VALIDATION, true);
+ FileSystem fileSystem = FileSystem.newInstance(conf);
+ AzureBlobFileSystem abfs = (AzureBlobFileSystem) fileSystem;
+ Assume.assumeFalse("Namespace enabled account does not support this test",
+ getIsNamespaceEnabled(abfs));
+ NativeAzureFileSystem wasb = getWasbFileSystem();
+
+ Path testFile = path("/testReadFile");
+ Path path = new Path(testFile + "/~12/!008/testfile" + 16);
+
+ // Write
+ wasb.create(path, true);
+ try (FSDataOutputStream nativeFsStream = abfs.append(path)) {
+ nativeFsStream.write(TEST_CONTEXT.getBytes());
+ nativeFsStream.flush();
+ nativeFsStream.hsync();
+ }
+
+ // Check file status
+ ContractTestUtils.assertIsFile(abfs, path);
+
+ try (BufferedReader br = new BufferedReader(
+ new InputStreamReader(abfs.open(path)))) {
+ String line = br.readLine();
+ assertEquals("Wrong text from " + abfs,
+ TEST_CONTEXT, line);
+ }
+ wasb.delete(path, true);
+ }
+
+ // Scenario 17: Create, setXAttr and getXAttr via ABFS
+ @Test
+ public void testScenario17() throws Exception {
+ Configuration conf = getRawConfiguration();
+ conf.setBoolean(FS_AZURE_ABFS_ENABLE_CHECKSUM_VALIDATION, true);
+ FileSystem fileSystem = FileSystem.newInstance(conf);
+ AzureBlobFileSystem abfs = (AzureBlobFileSystem) fileSystem;
+ Assume.assumeFalse("Namespace enabled account does not support this test",
+ getIsNamespaceEnabled(abfs));
+ NativeAzureFileSystem wasb = getWasbFileSystem();
+
+ Path testFile = path("/testReadFile");
+ Path path = new Path(testFile + "/~12/!008/testfile" + 17);
+ // Write
+ try (FSDataOutputStream nativeFsStream = abfs.create(path, true)) {
+ nativeFsStream.write(TEST_CONTEXT.getBytes());
+ nativeFsStream.flush();
+ nativeFsStream.hsync();
+ }
+ // --- VALIDATE FILE ---
+ FileStatus status = abfs.getFileStatus(path);
+ assertIsFile(path, status);
+
+ // --- SET XATTR #1 ---
+ abfs.setXAttr(path, ATTRIBUTE_NAME_1, ATTRIBUTE_VALUE_1);
+ byte[] readValue = abfs.getXAttr(path, ATTRIBUTE_NAME_1);
+ System.out.println("XAttr raw bytes: " + Arrays.toString(readValue));
+ System.out.println("XAttr as string: " + new String(readValue, StandardCharsets.UTF_8));
+ assertAttributeEqual(readValue, ATTRIBUTE_VALUE_1, "one");
+
+ // --- SET XATTR #2 ---
+ abfs.setXAttr(path, ATTRIBUTE_NAME_2, ATTRIBUTE_VALUE_2);
+ readValue = abfs.getXAttr(path, ATTRIBUTE_NAME_2);
+ assertAttributeEqual(readValue, ATTRIBUTE_VALUE_2, "two");
+
+ // --- VERIFY XATTR #1 AGAIN ---
+ readValue = abfs.getXAttr(path, ATTRIBUTE_NAME_1);
+ assertAttributeEqual(readValue, ATTRIBUTE_VALUE_1, "one");
+
+ abfs.delete(path, true);
+ }
+
+ // Scenario 17: Create, setXAttr and getXAttr via WASB
+ @Test
+ public void testScenario18() throws Exception {
+ Configuration conf = getRawConfiguration();
+ conf.setBoolean(FS_AZURE_ABFS_ENABLE_CHECKSUM_VALIDATION, true);
+ FileSystem fileSystem = FileSystem.newInstance(conf);
+ AzureBlobFileSystem abfs = (AzureBlobFileSystem) fileSystem;
+ Assume.assumeFalse("Namespace enabled account does not support this test",
+ getIsNamespaceEnabled(abfs));
+ NativeAzureFileSystem wasb = getWasbFileSystem();
+
+ Path testFile = path("/testReadFile");
+ Path path = new Path(testFile + "/~12/!008/testfile" + 18);
+ // Write
+ try (FSDataOutputStream nativeFsStream = wasb.create(path, true)) {
+ nativeFsStream.write(TEST_CONTEXT.getBytes());
+ nativeFsStream.flush();
+ nativeFsStream.hsync();
+ }
+ // --- VALIDATE FILE ---
+ FileStatus status = wasb.getFileStatus(path);
+ assertIsFile(path, status);
+
+ // --- SET XATTR #1 ---
+ wasb.setXAttr(path, ATTRIBUTE_NAME_1, ATTRIBUTE_VALUE_1);
+ byte[] readValue = wasb.getXAttr(path, ATTRIBUTE_NAME_1);
+ System.out.println("XAttr raw bytes: " + Arrays.toString(readValue));
+ System.out.println("XAttr as string: " + new String(readValue, StandardCharsets.UTF_8));
+ assertAttributeEqual(readValue, ATTRIBUTE_VALUE_1, "one");
+
+ // --- SET XATTR #2 ---
+ wasb.setXAttr(path, ATTRIBUTE_NAME_2, ATTRIBUTE_VALUE_2);
+ readValue = wasb.getXAttr(path, ATTRIBUTE_NAME_2);
+ assertAttributeEqual(readValue, ATTRIBUTE_VALUE_2, "two");
+
+ // --- VERIFY XATTR #1 AGAIN ---
+ readValue = wasb.getXAttr(path, ATTRIBUTE_NAME_1);
+ assertAttributeEqual(readValue, ATTRIBUTE_VALUE_1, "one");
+
+ wasb.delete(path, true);
+ }
+
+ // Scenario 19: Create, setXAttr via wasb and getXAttr via ABFS
+ @Test
+ public void testScenario19() throws Exception {
+ Configuration conf = getRawConfiguration();
+ conf.setBoolean(FS_AZURE_ABFS_ENABLE_CHECKSUM_VALIDATION, true);
+ FileSystem fileSystem = FileSystem.newInstance(conf);
+ AzureBlobFileSystem abfs = (AzureBlobFileSystem) fileSystem;
+ Assume.assumeFalse("Namespace enabled account does not support this test",
+ getIsNamespaceEnabled(abfs));
+ NativeAzureFileSystem wasb = getWasbFileSystem();
+
+ Path testFile = path("/testReadFile");
+ Path path = new Path(testFile + "/~12/!008/testfile" + 19);
+ // Write
+ try (FSDataOutputStream nativeFsStream = wasb.create(path, true)) {
+ nativeFsStream.write(TEST_CONTEXT.getBytes());
+ nativeFsStream.flush();
+ nativeFsStream.hsync();
+ }
+ // --- VALIDATE FILE ---
+ FileStatus status = wasb.getFileStatus(path);
+ assertIsFile(path, status);
+
+ // --- SET XATTR #1 ---
+ wasb.setXAttr(path, ATTRIBUTE_NAME_1, ATTRIBUTE_VALUE_1);
+ byte[] readValue = abfs.getXAttr(path, ATTRIBUTE_NAME_1);
+ System.out.println("XAttr raw bytes: " + Arrays.toString(readValue));
+ System.out.println("XAttr as string: " + new String(readValue, StandardCharsets.UTF_8));
+ assertAttributeEqual(readValue, ATTRIBUTE_VALUE_1, "one");
+
+ // --- SET XATTR #2 ---
+ wasb.setXAttr(path, ATTRIBUTE_NAME_2, ATTRIBUTE_VALUE_2);
+ readValue = abfs.getXAttr(path, ATTRIBUTE_NAME_2);
+ assertAttributeEqual(readValue, ATTRIBUTE_VALUE_2, "two");
+
+ // --- VERIFY XATTR #1 AGAIN ---
+ readValue = abfs.getXAttr(path, ATTRIBUTE_NAME_1);
+ assertAttributeEqual(readValue, ATTRIBUTE_VALUE_1, "one");
+
+ wasb.delete(path, true);
+ }
+
+ // Scenario 20: Create, setXAttr via wasb and getXAttr via ABFS and create overwrite via ABFS
+ @Test
+ public void testScenario20() throws Exception {
+ Configuration conf = getRawConfiguration();
+ conf.setBoolean(FS_AZURE_ABFS_ENABLE_CHECKSUM_VALIDATION, true);
+ FileSystem fileSystem = FileSystem.newInstance(conf);
+ AzureBlobFileSystem abfs = (AzureBlobFileSystem) fileSystem;
+ Assume.assumeFalse("Namespace enabled account does not support this test",
+ getIsNamespaceEnabled(abfs));
+ NativeAzureFileSystem wasb = getWasbFileSystem();
+
+ Path testFile = path("/testReadFile");
+ Path path = new Path(testFile + "/~12/!008/testfile" + 20);
+ // Write
+ try (FSDataOutputStream nativeFsStream = wasb.create(path, true)) {
+ nativeFsStream.write(TEST_CONTEXT.getBytes());
+ nativeFsStream.flush();
+ nativeFsStream.hsync();
+ }
+ // --- VALIDATE FILE ---
+ FileStatus status = wasb.getFileStatus(path);
+ assertIsFile(path, status);
+
+ // --- SET XATTR #1 ---
+ wasb.setXAttr(path, ATTRIBUTE_NAME_1, ATTRIBUTE_VALUE_1);
+ byte[] readValue = abfs.getXAttr(path, ATTRIBUTE_NAME_1);
+ System.out.println("XAttr raw bytes: " + Arrays.toString(readValue));
+ System.out.println("XAttr as string: " + new String(readValue, StandardCharsets.UTF_8));
+ assertAttributeEqual(readValue, ATTRIBUTE_VALUE_1, "one");
+
+ // --- SET XATTR #2 ---
+ wasb.setXAttr(path, ATTRIBUTE_NAME_2, ATTRIBUTE_VALUE_2);
+ readValue = abfs.getXAttr(path, ATTRIBUTE_NAME_2);
+ assertAttributeEqual(readValue, ATTRIBUTE_VALUE_2, "two");
+
+ // --- VERIFY XATTR #1 AGAIN ---
+ readValue = abfs.getXAttr(path, ATTRIBUTE_NAME_1);
+ assertAttributeEqual(readValue, ATTRIBUTE_VALUE_1, "one");
+
+ abfs.create(path, true);
+ wasb.delete(path, true);
+ }
+
+ // Scenario 21: Create, setXAttr ABFS, getXAttr WASB and create overwrite via WASB
+ @Test
+ public void testScenario21() throws Exception {
+ Configuration conf = getRawConfiguration();
+ conf.setBoolean(FS_AZURE_ABFS_ENABLE_CHECKSUM_VALIDATION, true);
+ FileSystem fileSystem = FileSystem.newInstance(conf);
+ AzureBlobFileSystem abfs = (AzureBlobFileSystem) fileSystem;
+ Assume.assumeFalse("Namespace enabled account does not support this test",
+ getIsNamespaceEnabled(abfs));
+ NativeAzureFileSystem wasb = getWasbFileSystem();
+
+ Path testFile = path("/testReadFile");
+ Path path = new Path(testFile + "/~12/!008/testfile" + 21);
+ // Write
+ try (FSDataOutputStream nativeFsStream = abfs.create(path, true)) {
+ nativeFsStream.write(TEST_CONTEXT.getBytes());
+ nativeFsStream.flush();
+ nativeFsStream.hsync();
+ }
+ // --- VALIDATE FILE ---
+ FileStatus status = wasb.getFileStatus(path);
+ assertIsFile(path, status);
+
+ // --- SET XATTR #1 ---
+ abfs.setXAttr(path, ATTRIBUTE_NAME_1, ATTRIBUTE_VALUE_1);
+ byte[] readValue = wasb.getXAttr(path, ATTRIBUTE_NAME_1);
+ System.out.println("XAttr raw bytes: " + Arrays.toString(readValue));
+ System.out.println("XAttr as string: " + new String(readValue, StandardCharsets.UTF_8));
+ assertAttributeEqual(readValue, ATTRIBUTE_VALUE_1, "one");
+
+ // --- SET XATTR #2 ---
+ abfs.setXAttr(path, ATTRIBUTE_NAME_2, ATTRIBUTE_VALUE_2);
+ readValue = wasb.getXAttr(path, ATTRIBUTE_NAME_2);
+ assertAttributeEqual(readValue, ATTRIBUTE_VALUE_2, "two");
+
+ // --- VERIFY XATTR #1 AGAIN ---
+ readValue = wasb.getXAttr(path, ATTRIBUTE_NAME_1);
+ assertAttributeEqual(readValue, ATTRIBUTE_VALUE_1, "one");
+
+ wasb.create(path, true);
+ wasb.delete(path, true);
+ }
+
+ // Scenario 22: Create via WASB, setXAttr ABFS, getXAttr wasb and create overwrite via WASB
+ @Test
+ public void testScenario22() throws Exception {
+ Configuration conf = getRawConfiguration();
+ conf.setBoolean(FS_AZURE_ABFS_ENABLE_CHECKSUM_VALIDATION, true);
+ FileSystem fileSystem = FileSystem.newInstance(conf);
+ AzureBlobFileSystem abfs = (AzureBlobFileSystem) fileSystem;
+ Assume.assumeFalse("Namespace enabled account does not support this test",
+ getIsNamespaceEnabled(abfs));
+ NativeAzureFileSystem wasb = getWasbFileSystem();
+
+ Path testFile = path("/testReadFile");
+ Path path = new Path(testFile + "/~12/!008/testfile" + 22);
+ // Write
+ try (FSDataOutputStream nativeFsStream = wasb.create(path, true)) {
+ nativeFsStream.write(TEST_CONTEXT.getBytes());
+ nativeFsStream.flush();
+ nativeFsStream.hsync();
+ }
+ // --- VALIDATE FILE ---
+ FileStatus status = wasb.getFileStatus(path);
+ assertIsFile(path, status);
+
+ // --- SET XATTR #1 ---
+ abfs.setXAttr(path, ATTRIBUTE_NAME_1, ATTRIBUTE_VALUE_1);
+ byte[] readValue = wasb.getXAttr(path, ATTRIBUTE_NAME_1);
+ System.out.println("XAttr raw bytes: " + Arrays.toString(readValue));
+ System.out.println("XAttr as string: " + new String(readValue, StandardCharsets.UTF_8));
+ assertAttributeEqual(readValue, ATTRIBUTE_VALUE_1, "one");
+
+ // --- SET XATTR #2 ---
+ abfs.setXAttr(path, ATTRIBUTE_NAME_2, ATTRIBUTE_VALUE_2);
+ readValue = wasb.getXAttr(path, ATTRIBUTE_NAME_2);
+ assertAttributeEqual(readValue, ATTRIBUTE_VALUE_2, "two");
+
+ // --- VERIFY XATTR #1 AGAIN ---
+ readValue = wasb.getXAttr(path, ATTRIBUTE_NAME_1);
+ assertAttributeEqual(readValue, ATTRIBUTE_VALUE_1, "one");
+
+ wasb.create(path, true);
+ wasb.delete(path, true);
+ }
+
+ // Scenario 23: Create via WASB, setXAttr ABFS, then setXAttr via WASB and getXAttr via ABFS
+ @Test
+ public void testScenario23() throws Exception {
+ Configuration conf = getRawConfiguration();
+ conf.setBoolean(FS_AZURE_ABFS_ENABLE_CHECKSUM_VALIDATION, true);
+ FileSystem fileSystem = FileSystem.newInstance(conf);
+ AzureBlobFileSystem abfs = (AzureBlobFileSystem) fileSystem;
+ Assume.assumeFalse("Namespace enabled account does not support this test",
+ getIsNamespaceEnabled(abfs));
+ NativeAzureFileSystem wasb = getWasbFileSystem();
+
+ Path testFile = path("/testReadFile");
+ Path path = new Path(testFile + "/~12/!008/testfile" + 23);
+ // Write
+ try (FSDataOutputStream nativeFsStream = wasb.create(path, true)) {
+ nativeFsStream.write(TEST_CONTEXT.getBytes());
+ nativeFsStream.flush();
+ nativeFsStream.hsync();
+ }
+ // --- VALIDATE FILE ---
+ FileStatus status = wasb.getFileStatus(path);
+ assertIsFile(path, status);
+
+ // --- SET XATTR #1 ---
+ abfs.setXAttr(path, ATTRIBUTE_NAME_1, ATTRIBUTE_VALUE_1);
+ byte[] readValue = abfs.getXAttr(path, ATTRIBUTE_NAME_1);
+ System.out.println("XAttr raw bytes: " + Arrays.toString(readValue));
+ System.out.println("XAttr as string: " + new String(readValue, StandardCharsets.UTF_8));
+ assertAttributeEqual(readValue, ATTRIBUTE_VALUE_1, "one");
+
+ // --- SET XATTR #2 ---
+ wasb.setXAttr(path, ATTRIBUTE_NAME_2, ATTRIBUTE_VALUE_2, CREATE_FLAG);
+ readValue = abfs.getXAttr(path, ATTRIBUTE_NAME_2);
+ assertAttributeEqual(readValue, ATTRIBUTE_VALUE_2, "two");
+
+ // --- VERIFY XATTR #1 AGAIN ---
+ readValue = abfs.getXAttr(path, ATTRIBUTE_NAME_1);
+ assertAttributeEqual(readValue, ATTRIBUTE_VALUE_1, "one");
+
+ wasb.delete(path, true);
+ }
+
+ // Scenario 24: Create via ABFS, then setXAttr via WASB and getXAttr via ABFS
+ @Test
+ public void testScenario24() throws Exception {
+ Configuration conf = getRawConfiguration();
+ conf.setBoolean(FS_AZURE_ABFS_ENABLE_CHECKSUM_VALIDATION, true);
+ FileSystem fileSystem = FileSystem.newInstance(conf);
+ AzureBlobFileSystem abfs = (AzureBlobFileSystem) fileSystem;
+ Assume.assumeFalse("Namespace enabled account does not support this test",
+ getIsNamespaceEnabled(abfs));
+ NativeAzureFileSystem wasb = getWasbFileSystem();
+
+ Path testFile = path("/testReadFile");
+ Path path = new Path(testFile + "/~12/!008/testfile" + 24);
+ // Write
+ try (FSDataOutputStream nativeFsStream = abfs.create(path, true)) {
+ nativeFsStream.write(TEST_CONTEXT.getBytes());
+ nativeFsStream.flush();
+ nativeFsStream.hsync();
+ }
+ // --- VALIDATE FILE ---
+ FileStatus status = wasb.getFileStatus(path);
+ assertIsFile(path, status);
+
+ // --- SET XATTR #1 ---
+ wasb.setXAttr(path, ATTRIBUTE_NAME_1, ATTRIBUTE_VALUE_1);
+ byte[] readValue = abfs.getXAttr(path, ATTRIBUTE_NAME_1);
+ System.out.println("XAttr raw bytes: " + Arrays.toString(readValue));
+ System.out.println("XAttr as string: " + new String(readValue, StandardCharsets.UTF_8));
+ assertAttributeEqual(readValue, ATTRIBUTE_VALUE_1, "one");
+
+ // --- SET XATTR #2 ---
+ wasb.setXAttr(path, ATTRIBUTE_NAME_2, ATTRIBUTE_VALUE_2, CREATE_FLAG);
+ readValue = abfs.getXAttr(path, ATTRIBUTE_NAME_2);
+ assertAttributeEqual(readValue, ATTRIBUTE_VALUE_2, "two");
+
+ // --- VERIFY XATTR #1 AGAIN ---
+ readValue = abfs.getXAttr(path, ATTRIBUTE_NAME_1);
+ assertAttributeEqual(readValue, ATTRIBUTE_VALUE_1, "one");
+
+ wasb.delete(path, true);
+ }
+
+ // Scenario 24: Create via WASB, then setXAttr getXAttr via ABFS and delete via WASB
+ @Test
+ public void testScenario25() throws Exception {
+ Configuration conf = getRawConfiguration();
+ conf.setBoolean(FS_AZURE_ABFS_ENABLE_CHECKSUM_VALIDATION, true);
+ FileSystem fileSystem = FileSystem.newInstance(conf);
+ AzureBlobFileSystem abfs = (AzureBlobFileSystem) fileSystem;
+ Assume.assumeFalse("Namespace enabled account does not support this test",
+ getIsNamespaceEnabled(abfs));
+ NativeAzureFileSystem wasb = getWasbFileSystem();
+
+ Path testFile = path("/testReadFile");
+ Path path = new Path(testFile + "/~12/!008/testfile" + 25);
+ // Write
+ try (FSDataOutputStream nativeFsStream = wasb.create(path, true)) {
+ nativeFsStream.write(TEST_CONTEXT.getBytes());
+ nativeFsStream.flush();
+ nativeFsStream.hsync();
+ }
+ // --- VALIDATE FILE ---
+ FileStatus status = wasb.getFileStatus(path);
+ assertIsFile(path, status);
+
+ // --- SET XATTR #1 ---
+ abfs.setXAttr(path, ATTRIBUTE_NAME_1, ATTRIBUTE_VALUE_1);
+ byte[] readValue = abfs.getXAttr(path, ATTRIBUTE_NAME_1);
+ System.out.println("XAttr raw bytes: " + Arrays.toString(readValue));
+ System.out.println("XAttr as string: " + new String(readValue, StandardCharsets.UTF_8));
+ assertAttributeEqual(readValue, ATTRIBUTE_VALUE_1, "one");
+
+ // --- SET XATTR #2 ---
+ abfs.setXAttr(path, ATTRIBUTE_NAME_2, ATTRIBUTE_VALUE_2, CREATE_FLAG);
+ readValue = abfs.getXAttr(path, ATTRIBUTE_NAME_2);
+ assertAttributeEqual(readValue, ATTRIBUTE_VALUE_2, "two");
+
+ // --- VERIFY XATTR #1 AGAIN ---
+ readValue = abfs.getXAttr(path, ATTRIBUTE_NAME_1);
+ assertAttributeEqual(readValue, ATTRIBUTE_VALUE_1, "one");
+
+ wasb.delete(path, true);
+ }
+
+ // Scenario 26: Create via ABFS, then setXAttr getXAttr via WASB and delete via WASB
+ @Test
+ public void testScenario26() throws Exception {
+ Configuration conf = getRawConfiguration();
+ conf.setBoolean(FS_AZURE_ABFS_ENABLE_CHECKSUM_VALIDATION, true);
+ FileSystem fileSystem = FileSystem.newInstance(conf);
+ AzureBlobFileSystem abfs = (AzureBlobFileSystem) fileSystem;
+ Assume.assumeFalse("Namespace enabled account does not support this test",
+ getIsNamespaceEnabled(abfs));
+ NativeAzureFileSystem wasb = getWasbFileSystem();
+
+ Path testFile = path("/testReadFile");
+ Path path = new Path(testFile + "/~12/!008/testfile" + 26);
+ // Write
+ try (FSDataOutputStream nativeFsStream = abfs.create(path, true)) {
+ nativeFsStream.write(TEST_CONTEXT.getBytes());
+ nativeFsStream.flush();
+ nativeFsStream.hsync();
+ }
+ // --- VALIDATE FILE ---
+ FileStatus status = abfs.getFileStatus(path);
+ assertIsFile(path, status);
+
+ // --- SET XATTR #1 ---
+ wasb.setXAttr(path, ATTRIBUTE_NAME_1, ATTRIBUTE_VALUE_1);
+ byte[] readValue = wasb.getXAttr(path, ATTRIBUTE_NAME_1);
+ System.out.println("XAttr raw bytes: " + Arrays.toString(readValue));
+ System.out.println("XAttr as string: " + new String(readValue, StandardCharsets.UTF_8));
+ assertAttributeEqual(readValue, ATTRIBUTE_VALUE_1, "one");
+
+ // --- SET XATTR #2 ---
+ wasb.setXAttr(path, ATTRIBUTE_NAME_2, ATTRIBUTE_VALUE_2, CREATE_FLAG);
+ readValue = wasb.getXAttr(path, ATTRIBUTE_NAME_2);
+ assertAttributeEqual(readValue, ATTRIBUTE_VALUE_2, "two");
+
+ // --- VERIFY XATTR #1 AGAIN ---
+ readValue = wasb.getXAttr(path, ATTRIBUTE_NAME_1);
+ assertAttributeEqual(readValue, ATTRIBUTE_VALUE_1, "one");
+
+ wasb.delete(path, true);
+ }
+
+ // Scenario 27: Create and write via ABFS, rename via wasb
+ @Test
+ public void testScenario27() throws Exception {
+ Configuration conf = getRawConfiguration();
+ conf.setBoolean(FS_AZURE_ABFS_ENABLE_CHECKSUM_VALIDATION, true);
+ FileSystem fileSystem = FileSystem.newInstance(conf);
+ AzureBlobFileSystem abfs = (AzureBlobFileSystem) fileSystem;
+ Assume.assumeFalse("Namespace enabled account does not support this test",
+ getIsNamespaceEnabled(abfs));
+ NativeAzureFileSystem wasb = getWasbFileSystem();
+
+ Path testFile = path("/testReadFile");
+ Path testPath1 = new Path(testFile + "/~12/!008/testfile" + 27);
+ Path testPath2 = new Path(testFile + "/~12/!008/testfile" + 28);
+
+ // Write
+ try (FSDataOutputStream nativeFsStream = abfs.create(testPath1, true)) {
+ nativeFsStream.write(TEST_CONTEXT.getBytes());
+ nativeFsStream.flush();
+ nativeFsStream.hsync();
+ }
+
+ // Check file status
+ ContractTestUtils.assertIsFile(abfs, testPath1);
+
+ try (BufferedReader br = new BufferedReader(
+ new InputStreamReader(abfs.open(testPath1)))) {
+ String line = br.readLine();
+ assertEquals("Wrong text from " + abfs,
+ TEST_CONTEXT, line);
+ }
+ // --- RENAME FILE ---
+ boolean renamed = wasb.rename(testPath1, testPath2);
+ System.out.println("Rename successful: " + renamed);
+
+ // --- LIST FILES IN DIRECTORY ---
+ Path parentDir = new Path(testFile + "/~12/!008");
+ FileStatus[] fileStatuses = wasb.listStatus(parentDir);
+
+ for (FileStatus status : fileStatuses) {
+ System.out.println("File: " + status.getPath());
+ }
+ wasb.delete(testPath2, true);
+ }
+
+ // Scenario 28: Create and write via WASB, rename via ABFS, list via ABFS
+ @Test
+ public void testScenario28() throws Exception {
+ Configuration conf = getRawConfiguration();
+ conf.setBoolean(FS_AZURE_ABFS_ENABLE_CHECKSUM_VALIDATION, true);
+ FileSystem fileSystem = FileSystem.newInstance(conf);
+ AzureBlobFileSystem abfs = (AzureBlobFileSystem) fileSystem;
+ Assume.assumeFalse("Namespace enabled account does not support this test",
+ getIsNamespaceEnabled(abfs));
+ NativeAzureFileSystem wasb = getWasbFileSystem();
+
+ Path testFile = path("/testReadFile");
+ Path testPath1 = new Path(testFile + "/~12/!008/testfile" + 29);
+ Path testPath2 = new Path(testFile + "/~12/!008/testfile" + 30);
+
+ // Write
+ try (FSDataOutputStream nativeFsStream = wasb.create(testPath1, true)) {
+ nativeFsStream.write(TEST_CONTEXT.getBytes());
+ nativeFsStream.flush();
+ nativeFsStream.hsync();
+ }
+
+ // Check file status
+ ContractTestUtils.assertIsFile(abfs, testPath1);
+
+ try (BufferedReader br = new BufferedReader(
+ new InputStreamReader(abfs.open(testPath1)))) {
+ String line = br.readLine();
+ assertEquals("Wrong text from " + abfs,
+ TEST_CONTEXT, line);
+ }
+ // --- RENAME FILE ---
+ boolean renamed = abfs.rename(testPath1, testPath2);
+ System.out.println("Rename successful: " + renamed);
+
+ // --- LIST FILES IN DIRECTORY ---
+ Path parentDir = new Path(testFile + "/~12/!008");
+ FileStatus[] fileStatuses = abfs.listStatus(parentDir);
+
+ for (FileStatus status : fileStatuses) {
+ System.out.println("File: " + status.getPath());
+ }
+ wasb.delete(testPath2, true);
+ }
+
+ // Scenario 29: Create via WASB and write via ABFS, rename via ABFS, list via ABFS
+ @Test
+ public void testScenario29() throws Exception {
+ Configuration conf = getRawConfiguration();
+ conf.setBoolean(FS_AZURE_ABFS_ENABLE_CHECKSUM_VALIDATION, true);
+ FileSystem fileSystem = FileSystem.newInstance(conf);
+ AzureBlobFileSystem abfs = (AzureBlobFileSystem) fileSystem;
+ Assume.assumeFalse("Namespace enabled account does not support this test",
+ getIsNamespaceEnabled(abfs));
+ NativeAzureFileSystem wasb = getWasbFileSystem();
+
+ Path testFile = path("/testReadFile");
+ Path testPath1 = new Path(testFile + "/~12/!008/testfile" + 29);
+ Path testPath2 = new Path(testFile + "/~12/!008/testfile" + 30);
+
+ // Write
+ wasb.create(testPath1, true);
+ try (FSDataOutputStream nativeFsStream = abfs.append(testPath1)) {
+ nativeFsStream.write(TEST_CONTEXT.getBytes());
+ nativeFsStream.flush();
+ nativeFsStream.hsync();
+ }
+
+ // Check file status
+ ContractTestUtils.assertIsFile(abfs, testPath1);
+
+ try (BufferedReader br = new BufferedReader(
+ new InputStreamReader(abfs.open(testPath1)))) {
+ String line = br.readLine();
+ assertEquals("Wrong text from " + abfs,
+ TEST_CONTEXT, line);
+ }
+ // --- RENAME FILE ---
+ boolean renamed = abfs.rename(testPath1, testPath2);
+ System.out.println("Rename successful: " + renamed);
+
+ // --- LIST FILES IN DIRECTORY ---
+ Path parentDir = new Path(testFile + "/~12/!008");
+ FileStatus[] fileStatuses = abfs.listStatus(parentDir);
+
+ for (FileStatus status : fileStatuses) {
+ System.out.println("File: " + status.getPath());
+ }
+ wasb.delete(testPath2, true);
+ }
+
+ //Scenario 30: Create and write via WASB, rename via WASB, rename via ABFS, list via ABFS
+ @Test
+ public void testScenario30() throws Exception {
+ Configuration conf = getRawConfiguration();
+ conf.setBoolean(FS_AZURE_ABFS_ENABLE_CHECKSUM_VALIDATION, true);
+ FileSystem fileSystem = FileSystem.newInstance(conf);
+ AzureBlobFileSystem abfs = (AzureBlobFileSystem) fileSystem;
+ Assume.assumeFalse("Namespace enabled account does not support this test",
+ getIsNamespaceEnabled(abfs));
+ NativeAzureFileSystem wasb = getWasbFileSystem();
+
+ Path testFile = path("/testReadFile");
+ Path testPath1 = new Path(testFile + "/~12/!008/testfile" + 31);
+ Path testPath2 = new Path(testFile + "/~12/!008/testfile" + 32);
+ Path testPath3 = new Path(testFile + "/~12/!008/testfile" + 33);
+
+ // Write
+ try (FSDataOutputStream nativeFsStream = wasb.create(testPath1, true)) {
+ nativeFsStream.write(TEST_CONTEXT.getBytes());
+ nativeFsStream.flush();
+ nativeFsStream.hsync();
+ }
+
+ // Check file status
+ ContractTestUtils.assertIsFile(abfs, testPath1);
+
+ try (BufferedReader br = new BufferedReader(
+ new InputStreamReader(abfs.open(testPath1)))) {
+ String line = br.readLine();
+ assertEquals("Wrong text from " + abfs,
+ TEST_CONTEXT, line);
+ }
+ // --- RENAME FILE ---
+ boolean renamed = wasb.rename(testPath1, testPath2);
+ System.out.println("Rename successful: " + renamed);
+ // --- RENAME FILE ---
+ boolean renamed1 = abfs.rename(testPath2, testPath3);
+ System.out.println("Rename successful: " + renamed1);
+
+ // --- LIST FILES IN DIRECTORY ---
+ Path parentDir = new Path(testFile + "/~12/!008");
+ FileStatus[] fileStatuses = abfs.listStatus(parentDir);
+
+ for (FileStatus status : fileStatuses) {
+ System.out.println("File: " + status.getPath());
+ }
+ wasb.delete(testPath3, true);
+ }
+
+ //Scenario 31: Create and write via WASB, delete via WASB, rename via ABFS -> should fail
+ @Test
+ public void testScenario31() throws Exception {
+ Configuration conf = getRawConfiguration();
+ conf.setBoolean(FS_AZURE_ABFS_ENABLE_CHECKSUM_VALIDATION, true);
+ FileSystem fileSystem = FileSystem.newInstance(conf);
+ AzureBlobFileSystem abfs = (AzureBlobFileSystem) fileSystem;
+ Assume.assumeFalse("Namespace enabled account does not support this test",
+ getIsNamespaceEnabled(abfs));
+ NativeAzureFileSystem wasb = getWasbFileSystem();
+
+ Path testFile = path("/testReadFile");
+ Path testPath1 = new Path(testFile + "/~12/!008/testfile" + 31);
+ Path testPath2 = new Path(testFile + "/~12/!008/testfile" + 32);
+ Path testPath3 = new Path(testFile + "/~12/!008/testfile" + 33);
+
+ // Write
+ try (FSDataOutputStream nativeFsStream = wasb.create(testPath1, true)) {
+ nativeFsStream.write(TEST_CONTEXT.getBytes());
+ nativeFsStream.flush();
+ nativeFsStream.hsync();
+ }
+
+ // Check file status
+ ContractTestUtils.assertIsFile(abfs, testPath1);
+
+ try (BufferedReader br = new BufferedReader(
+ new InputStreamReader(abfs.open(testPath1)))) {
+ String line = br.readLine();
+ assertEquals("Wrong text from " + abfs,
+ TEST_CONTEXT, line);
+ }
+ wasb.delete(testPath1, true);
+ // --- RENAME FILE ---
+ boolean renamed = abfs.rename(testPath1, testPath2);
+ System.out.println("Rename successful: " + renamed);
+ Assertions.assertThat(renamed)
+ .as("Rename operation should have failed but returned true")
+ .isFalse();
+ }
+
+ //Scenario 32 :Create Dir & File via WASB → Rename Dir via ABFS → List Files via ABFS
+ @Test
+ public void testScenario32() throws Exception {
+ Configuration conf = getRawConfiguration();
+ conf.setBoolean(FS_AZURE_ABFS_ENABLE_CHECKSUM_VALIDATION, true);
+ FileSystem fileSystem = FileSystem.newInstance(conf);
+ AzureBlobFileSystem abfs = (AzureBlobFileSystem) fileSystem;
+ Assume.assumeFalse("Namespace enabled account does not support this test",
+ getIsNamespaceEnabled(abfs));
+ NativeAzureFileSystem wasb = getWasbFileSystem();
+
+ Path testFile = path("/testReadFile");
+ Path testFile1 = path("/testReadFile1");
+ Path testPath1 = new Path(testFile + "/~12/!008/testfile" + 50);
+ Path testPath2 = new Path(testFile + "/~12/!008/testfile" + 51);
+ Path testPath3 = new Path(testFile + "/~12/!008/testfile" + 52);
+
+ // Write
+ wasb.mkdirs(testFile);
+ try (FSDataOutputStream nativeFsStream = wasb.create(testPath1, true)) {
+ nativeFsStream.write(TEST_CONTEXT.getBytes());
+ nativeFsStream.flush();
+ nativeFsStream.hsync();
+ }
+ wasb.create(testPath2, true);
+ wasb.create(testPath3, true);
+
+ // Check file status
+ ContractTestUtils.assertIsFile(abfs, testPath1);
+
+ try (BufferedReader br = new BufferedReader(
+ new InputStreamReader(abfs.open(testPath1)))) {
+ String line = br.readLine();
+ assertEquals("Wrong text from " + abfs,
+ TEST_CONTEXT, line);
+ }
+ // --- RENAME DIR ---
+ boolean renamed = abfs.rename(testFile, testFile1);
+ System.out.println("Rename successful: " + renamed);
+ // --- LIST FILES IN DIRECTORY ---
+ listAllFilesAndDirs(abfs, testFile1);
+ }
+
+ //Scenario 33 :Create Dir & File via ABFS → Rename Dir via WASB → List Files via WASB
+ @Test
+ public void testScenario33() throws Exception {
+ Configuration conf = getRawConfiguration();
+ conf.setBoolean(FS_AZURE_ABFS_ENABLE_CHECKSUM_VALIDATION, true);
+ FileSystem fileSystem = FileSystem.newInstance(conf);
+ AzureBlobFileSystem abfs = (AzureBlobFileSystem) fileSystem;
+ Assume.assumeFalse("Namespace enabled account does not support this test",
+ getIsNamespaceEnabled(abfs));
+ NativeAzureFileSystem wasb = getWasbFileSystem();
+
+ Path testFile = path("/testReadFile");
+ Path testFile1 = path("/testReadFile1");
+ Path testPath1 = new Path(testFile + "/~12/!008/testfile" + 55);
+ Path testPath2 = new Path(testFile + "/~12/!008/testfile" + 56);
+ Path testPath3 = new Path(testFile + "/~12/!008/testfile" + 57);
+
+ // Write
+ abfs.mkdirs(testFile);
+ try (FSDataOutputStream nativeFsStream = abfs.create(testPath1, true)) {
+ nativeFsStream.write(TEST_CONTEXT.getBytes());
+ nativeFsStream.flush();
+ nativeFsStream.hsync();
+ }
+ abfs.create(testPath2, true);
+ abfs.create(testPath3, true);
+
+ // Check file status
+ ContractTestUtils.assertIsFile(abfs, testPath1);
+
+ try (BufferedReader br = new BufferedReader(
+ new InputStreamReader(abfs.open(testPath1)))) {
+ String line = br.readLine();
+ assertEquals("Wrong text from " + abfs,
+ TEST_CONTEXT, line);
+ }
+ // --- RENAME DIR ---
+ boolean renamed = wasb.rename(testFile, testFile1);
+ System.out.println("Rename successful: " + renamed);
+ // --- LIST FILES IN DIRECTORY ---
+ listAllFilesAndDirs(wasb, testFile1);
+ }
+
+ //Scenario 34: Create Dir via ABFS → Rename File inside Dir via WASB → List via ABFS
+ @Test
+ public void testScenario34() throws Exception {
+ Configuration conf = getRawConfiguration();
+ conf.setBoolean(FS_AZURE_ABFS_ENABLE_CHECKSUM_VALIDATION, true);
+ FileSystem fileSystem = FileSystem.newInstance(conf);
+ AzureBlobFileSystem abfs = (AzureBlobFileSystem) fileSystem;
+ Assume.assumeFalse("Namespace enabled account does not support this test",
+ getIsNamespaceEnabled(abfs));
+ NativeAzureFileSystem wasb = getWasbFileSystem();
+
+ Path testFile = path("/testReadFile");
+ Path testFile1 = path("/testReadFile1");
+ Path testPath1 = new Path(testFile + "/~12/!008/testfile" + 65);
+ Path testPath2 = new Path(testFile + "/~12/!008/testfile" + 66);
+ Path testPath3 = new Path(testFile + "/~12/!008/testfile" + 67);
+
+ // Write
+ abfs.mkdirs(testFile);
+ try (FSDataOutputStream nativeFsStream = abfs.create(testPath1, true)) {
+ nativeFsStream.write(TEST_CONTEXT.getBytes());
+ nativeFsStream.flush();
+ nativeFsStream.hsync();
+ }
+ abfs.create(testPath3, true);
+
+ // Check file status
+ ContractTestUtils.assertIsFile(abfs, testPath1);
+
+ try (BufferedReader br = new BufferedReader(
+ new InputStreamReader(abfs.open(testPath1)))) {
+ String line = br.readLine();
+ assertEquals("Wrong text from " + abfs,
+ TEST_CONTEXT, line);
+ }
+ // --- RENAME DIR ---
+ boolean renamed = wasb.rename(testPath1, testPath2);
+ System.out.println("Rename successful: " + renamed);
+ // --- LIST FILES IN DIRECTORY ---
+ listAllFilesAndDirs(abfs, testFile);
+ }
+
+ //Scenario 35: Create Dir via WASB → Rename File inside Dir via ABFS → List via WASB
+ @Test
+ public void testScenario35() throws Exception {
+ Configuration conf = getRawConfiguration();
+ conf.setBoolean(FS_AZURE_ABFS_ENABLE_CHECKSUM_VALIDATION, true);
+ FileSystem fileSystem = FileSystem.newInstance(conf);
+ AzureBlobFileSystem abfs = (AzureBlobFileSystem) fileSystem;
+ Assume.assumeFalse("Namespace enabled account does not support this test",
+ getIsNamespaceEnabled(abfs));
+ NativeAzureFileSystem wasb = getWasbFileSystem();
+
+ Path testFile = path("/testReadFile");
+ Path testFile1 = path("/testReadFile1");
+ Path testPath1 = new Path(testFile + "/~12/!008/testfile" + 75);
+ Path testPath2 = new Path(testFile + "/~12/!008/testfile" + 76);
+ Path testPath3 = new Path(testFile + "/~12/!008/testfile" + 77);
+
+ // Write
+ wasb.mkdirs(testFile);
+ try (FSDataOutputStream nativeFsStream = wasb.create(testPath1, true)) {
+ nativeFsStream.write(TEST_CONTEXT.getBytes());
+ nativeFsStream.flush();
+ nativeFsStream.hsync();
+ }
+ wasb.create(testPath3, true);
+
+ // Check file status
+ ContractTestUtils.assertIsFile(abfs, testPath1);
+
+ try (BufferedReader br = new BufferedReader(
+ new InputStreamReader(abfs.open(testPath1)))) {
+ String line = br.readLine();
+ assertEquals("Wrong text from " + abfs,
+ TEST_CONTEXT, line);
+ }
+ // --- RENAME DIR ---
+ boolean renamed = abfs.rename(testPath1, testPath2);
+ System.out.println("Rename successful: " + renamed);
+ // --- LIST FILES IN DIRECTORY ---
+ listAllFilesAndDirs(wasb, testFile);
+ }
+
+ //Scenario 36: Create via WASB → Rename to existing name via ABFS → List via WASB
+ @Test
+ public void testScenario36() throws Exception {
+ Configuration conf = getRawConfiguration();
+ conf.setBoolean(FS_AZURE_ABFS_ENABLE_CHECKSUM_VALIDATION, true);
+ FileSystem fileSystem = FileSystem.newInstance(conf);
+ AzureBlobFileSystem abfs = (AzureBlobFileSystem) fileSystem;
+ Assume.assumeFalse("Namespace enabled account does not support this test",
+ getIsNamespaceEnabled(abfs));
+ NativeAzureFileSystem wasb = getWasbFileSystem();
+
+ Path testFile = path("/testReadFile");
+ Path testFile1 = path("/testReadFile1");
+ Path testPath1 = new Path(testFile + "/~12/!008/testfile" + 75);
+ Path testPath2 = new Path(testFile + "/~12/!008/testfile" + 76);
+ Path testPath3 = new Path(testFile + "/~12/!008/testfile" + 77);
+
+ // Write
+ wasb.mkdirs(testFile);
+ try (FSDataOutputStream nativeFsStream = wasb.create(testPath1, true)) {
+ nativeFsStream.write(TEST_CONTEXT.getBytes());
+ nativeFsStream.flush();
+ nativeFsStream.hsync();
+ }
+ wasb.create(testPath3, true);
+
+ // Check file status
+ ContractTestUtils.assertIsFile(abfs, testPath1);
+
+ try (BufferedReader br = new BufferedReader(
+ new InputStreamReader(abfs.open(testPath1)))) {
+ String line = br.readLine();
+ assertEquals("Wrong text from " + abfs,
+ TEST_CONTEXT, line);
+ }
+ // --- RENAME DIR ---
+ boolean renamed = abfs.rename(testFile, testFile);
+ System.out.println("Rename successful: " + renamed);
+ Assertions.assertThat(renamed)
+ .as("Rename operation should have failed but returned true")
+ .isFalse();
+ }
+
+ //Scenario 37: Rename a non-existent file via WASB
+ @Test
+ public void testScenario37() throws Exception {
+ Configuration conf = getRawConfiguration();
+ conf.setBoolean(FS_AZURE_ABFS_ENABLE_CHECKSUM_VALIDATION, true);
+ FileSystem fileSystem = FileSystem.newInstance(conf);
+ AzureBlobFileSystem abfs = (AzureBlobFileSystem) fileSystem;
+ Assume.assumeFalse("Namespace enabled account does not support this test",
+ getIsNamespaceEnabled(abfs));
+ NativeAzureFileSystem wasb = getWasbFileSystem();
+
+ Path testFile = path("/testReadFile");
+ Path testFile1 = path("/testReadFile1");
+ Path testPath1 = new Path(testFile + "/~12/!008/testfile" + 75);
+ Path testPath2 = new Path(testFile + "/~12/!008/testfile" + 76);
+ Path testPath3 = new Path(testFile + "/~12/!008/testfile" + 77);
+
+ // Write
+ abfs.mkdirs(testFile);
+ try (FSDataOutputStream nativeFsStream = abfs.create(testPath1, true)) {
+ nativeFsStream.write(TEST_CONTEXT.getBytes());
+ nativeFsStream.flush();
+ nativeFsStream.hsync();
+ }
+ abfs.create(testPath3, true);
+
+ // Check file status
+ ContractTestUtils.assertIsFile(abfs, testPath1);
+
+ try (BufferedReader br = new BufferedReader(
+ new InputStreamReader(abfs.open(testPath1)))) {
+ String line = br.readLine();
+ assertEquals("Wrong text from " + abfs,
+ TEST_CONTEXT, line);
+ }
+ // --- RENAME NON EXISTENT FILE ---
+ boolean renamed = wasb.rename(testPath2, testPath3);
+ System.out.println("Rename successful: " + renamed);
+ Assertions.assertThat(renamed)
+ .as("Rename operation should have failed but returned true")
+ .isFalse();
+ }
+
+ // Scenario 38: Create via WASB, setXAttr and getXAttr WASB and create overwrite via WASB
+ @Test
+ public void testScenario38() throws Exception {
+ Configuration conf = getRawConfiguration();
+ conf.setBoolean(FS_AZURE_ABFS_ENABLE_CHECKSUM_VALIDATION, true);
+ FileSystem fileSystem = FileSystem.newInstance(conf);
+ AzureBlobFileSystem abfs = (AzureBlobFileSystem) fileSystem;
+ Assume.assumeFalse("Namespace enabled account does not support this test",
+ getIsNamespaceEnabled(abfs));
+ NativeAzureFileSystem wasb = getWasbFileSystem();
+
+ Path testFile = path("/testReadFile");
+ Path path = new Path(testFile + "/~12/!008/testfile" + 38);
+ // Write
+ try (FSDataOutputStream nativeFsStream = wasb.create(path, true)) {
+ nativeFsStream.write(TEST_CONTEXT.getBytes());
+ nativeFsStream.flush();
+ nativeFsStream.hsync();
+ }
+ // --- VALIDATE FILE ---
+ FileStatus status = wasb.getFileStatus(path);
+ assertIsFile(path, status);
+
+ // --- SET XATTR #1 ---
+ wasb.setXAttr(path, ATTRIBUTE_NAME_1, ATTRIBUTE_VALUE_1);
+ byte[] readValue = wasb.getXAttr(path, ATTRIBUTE_NAME_1);
+ System.out.println("XAttr raw bytes: " + Arrays.toString(readValue));
+ System.out.println("XAttr as string: " + new String(readValue, StandardCharsets.UTF_8));
+ assertAttributeEqual(readValue, ATTRIBUTE_VALUE_1, "one");
+
+ // --- SET XATTR #2 ---
+ wasb.setXAttr(path, ATTRIBUTE_NAME_2, ATTRIBUTE_VALUE_2);
+ readValue = wasb.getXAttr(path, ATTRIBUTE_NAME_2);
+ assertAttributeEqual(readValue, ATTRIBUTE_VALUE_2, "two");
+
+ // --- VERIFY XATTR #1 AGAIN ---
+ readValue = wasb.getXAttr(path, ATTRIBUTE_NAME_1);
+ assertAttributeEqual(readValue, ATTRIBUTE_VALUE_1, "one");
+
+ wasb.create(path, true);
+ wasb.delete(path, true);
+ }
+
+ // Scenario 39: Create and write via WASB, rename via wasb and list via WASB
+ @Test
+ public void testScenario39() throws Exception {
+ Configuration conf = getRawConfiguration();
+ conf.setBoolean(FS_AZURE_ABFS_ENABLE_CHECKSUM_VALIDATION, true);
+ FileSystem fileSystem = FileSystem.newInstance(conf);
+ AzureBlobFileSystem abfs = (AzureBlobFileSystem) fileSystem;
+ Assume.assumeFalse("Namespace enabled account does not support this test",
+ getIsNamespaceEnabled(abfs));
+ NativeAzureFileSystem wasb = getWasbFileSystem();
+
+ String testRunId = UUID.randomUUID().toString();
+ Path baseDir = path("/testScenario39_" + testRunId);
+ Path testFile = new Path(baseDir, "testReadFile");
+ Path testPath1 = new Path(testFile + "/~12/!008/testfile" + 1);
+ Path testPath2 = new Path(testFile + "/~12/!008/testfile" + 2);
+ Path testPath3 = new Path(testFile + "/~12/!008/testfile" + 3);
+
+ // Write
+ wasb.mkdirs(testFile);
+ try (FSDataOutputStream nativeFsStream = wasb.create(testPath1, true)) {
+ nativeFsStream.write(TEST_CONTEXT.getBytes());
+ nativeFsStream.flush();
+ nativeFsStream.hsync();
+ }
+ wasb.create(testPath3, true);
+
+ // Check file status
+ ContractTestUtils.assertIsFile(abfs, testPath1);
+
+ try (BufferedReader br = new BufferedReader(
+ new InputStreamReader(abfs.open(testPath1)))) {
+ String line = br.readLine();
+ assertEquals("Wrong text from " + abfs,
+ TEST_CONTEXT, line);
+ }
+ // --- RENAME DIR ---
+ boolean renamed = wasb.rename(testPath1, testPath2);
+ System.out.println("Rename successful: " + renamed);
+ // --- LIST FILES IN DIRECTORY ---
+ listAllFilesAndDirs(wasb, testFile);
+ }
+
+ public static void listAllFilesAndDirs(FileSystem fs, Path path) throws
+ IOException {
+ RemoteIterator iter = fs.listStatusIterator(path);
+
+ while (iter.hasNext()) {
+ FileStatus status = iter.next();
+
+ if (status.isDirectory()) {
+ System.out.println("Directory: " + status.getPath());
+ // Recursive call
+ listAllFilesAndDirs(fs, status.getPath());
+ } else {
+ System.out.println("File: " + status.getPath());
+ }
+ }
+ }
+
+ private static void assertIsFile(Path path, FileStatus status) {
+ if (status.isDirectory()) {
+ throw new AssertionError("File claims to be a directory: " + path + " " + status);
+ }
+ if (status.isSymlink()) {
+ throw new AssertionError("File claims to be a symlink: " + path + " " + status);
+ }
+ }
+
+ private static void assertAttributeEqual(byte[] actual, byte[] expected, String expectedDecoded) {
+ if (!Arrays.equals(actual, expected)) {
+ throw new AssertionError("Encoded attribute does not match expected bytes");
+ }
+ String decoded = new String(actual, StandardCharsets.UTF_8);
+ if (!decoded.equals(expectedDecoded)) {
+ throw new AssertionError("Decoded attribute does not match. Got: " + decoded + ", Expected: " + expectedDecoded);
+ }
+ }
}
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsClient.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsClient.java
index cba1643478f54..de89b3c5686aa 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsClient.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsClient.java
@@ -621,7 +621,7 @@ public void testExpectHundredContinue() throws Exception {
AppendRequestParameters appendRequestParameters
= new AppendRequestParameters(
BUFFER_OFFSET, BUFFER_OFFSET, BUFFER_LENGTH,
- AppendRequestParameters.Mode.APPEND_MODE, false, null, true);
+ AppendRequestParameters.Mode.APPEND_MODE, false, null, true, null);
byte[] buffer = getRandomBytesArray(BUFFER_LENGTH);
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsOutputStream.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsOutputStream.java
index dff94aae117da..e3f040a1bedb7 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsOutputStream.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsOutputStream.java
@@ -378,7 +378,7 @@ public void testNoNetworkCallsForFlush() throws Exception {
Mockito.verify(blobClient, Mockito.times(0)).
flush(Mockito.any(byte[].class), Mockito.anyString(), Mockito.anyBoolean(),
Mockito.anyString(), Mockito.anyString(), Mockito.anyString(), Mockito.any(),
- Mockito.any(TracingContext.class));
+ Mockito.any(TracingContext.class), Mockito.anyString());
}
private AbfsRestOperationException getMockAbfsRestOperationException(int status) {
@@ -424,6 +424,6 @@ public void testNoNetworkCallsForSecondFlush() throws Exception {
Mockito.any(TracingContext.class));
Mockito.verify(blobClient, Mockito.times(1)).
flush(Mockito.any(byte[].class), Mockito.anyString(), Mockito.anyBoolean(), Mockito.any(), Mockito.any(), Mockito.anyString(), Mockito.any(),
- Mockito.any(TracingContext.class));
+ Mockito.any(TracingContext.class), Mockito.anyString());
}
}
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsRestOperation.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsRestOperation.java
index 11c0e104a6a69..d6572443bb9fc 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsRestOperation.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsRestOperation.java
@@ -194,7 +194,7 @@ private AbfsRestOperation getRestOperation() throws Exception {
= new AppendRequestParameters(
BUFFER_OFFSET, BUFFER_OFFSET, BUFFER_LENGTH,
AppendRequestParameters.Mode.APPEND_MODE, false, null,
- expectHeaderEnabled);
+ expectHeaderEnabled, null);
byte[] buffer = getRandomBytesArray(5);
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsOutputStream.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsOutputStream.java
index 0e7cad909040d..67def1bb1d347 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsOutputStream.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsOutputStream.java
@@ -27,8 +27,11 @@
import org.junit.Test;
import org.mockito.ArgumentCaptor;
+import org.mockito.Mock;
+import org.mockito.Mockito;
import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.azurebfs.Abfs;
import org.apache.hadoop.fs.azurebfs.AbfsConfiguration;
import org.apache.hadoop.fs.azurebfs.constants.FSOperationType;
import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters;
@@ -117,19 +120,20 @@ public void verifyShortWriteRequest() throws Exception {
AbfsConfiguration abfsConf;
final Configuration conf = new Configuration();
conf.set(accountKey1, accountValue1);
- abfsConf = new AbfsConfiguration(conf, accountName1);
+ abfsConf = Mockito.spy(new AbfsConfiguration(conf, accountName1));
AbfsPerfTracker tracker = new AbfsPerfTracker("test", accountName1, abfsConf);
when(client.getAbfsPerfTracker()).thenReturn(tracker);
+ when(client.getAbfsConfiguration()).thenReturn(abfsConf);
when(client.append(anyString(), any(byte[].class),
any(AppendRequestParameters.class), any(),
any(), any(TracingContext.class)))
.thenReturn(op);
when(client.flush(anyString(), anyLong(), anyBoolean(), anyBoolean(), any(),
- isNull(), any(), any(TracingContext.class))).thenReturn(op);
+ isNull(), any(), any(TracingContext.class), anyString())).thenReturn(op);
when(clientHandler.getClient(any())).thenReturn(client);
when(clientHandler.getDfsClient()).thenReturn(client);
-
- AbfsOutputStream out = new AbfsOutputStream(
+ AbfsOutputStream out;
+ out = Mockito.spy(new AbfsOutputStream(
populateAbfsOutputStreamContext(
BUFFER_SIZE,
true,
@@ -141,7 +145,9 @@ public void verifyShortWriteRequest() throws Exception {
new TracingContext(abfsConf.getClientCorrelationId(), "test-fs-id",
FSOperationType.WRITE, abfsConf.getTracingHeaderFormat(),
null),
- createExecutorService(abfsConf)));
+ createExecutorService(abfsConf))));
+ when(out.getClient()).thenReturn(client);
+ when(out.getMd5()).thenReturn(null);
final byte[] b = new byte[WRITE_SIZE];
new Random().nextBytes(b);
out.write(b);
@@ -156,9 +162,9 @@ public void verifyShortWriteRequest() throws Exception {
out.hsync();
AppendRequestParameters firstReqParameters = new AppendRequestParameters(
- 0, 0, WRITE_SIZE, APPEND_MODE, false, null, true);
+ 0, 0, WRITE_SIZE, APPEND_MODE, false, null, true, null);
AppendRequestParameters secondReqParameters = new AppendRequestParameters(
- WRITE_SIZE, 0, 2 * WRITE_SIZE, APPEND_MODE, false, null, true);
+ WRITE_SIZE, 0, 2 * WRITE_SIZE, APPEND_MODE, false, null, true, null);
verify(client, times(1)).append(
eq(PATH), any(byte[].class), refEq(firstReqParameters), any(), any(),
@@ -182,19 +188,21 @@ public void verifyWriteRequest() throws Exception {
AbfsConfiguration abfsConf;
final Configuration conf = new Configuration();
conf.set(accountKey1, accountValue1);
- abfsConf = new AbfsConfiguration(conf, accountName1);
+ abfsConf = Mockito.spy(new AbfsConfiguration(conf, accountName1));
AbfsPerfTracker tracker = new AbfsPerfTracker("test", accountName1, abfsConf);
+ when(client.getAbfsPerfTracker()).thenReturn(tracker);
+ when(client.getAbfsConfiguration()).thenReturn(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(), any(), any(TracingContext.class))).thenReturn(op);
- when(client.flush(anyString(), anyLong(), anyBoolean(), anyBoolean(), any(), isNull(), any(), any(TracingContext.class))).thenReturn(op);
+ when(client.flush(anyString(), anyLong(), anyBoolean(), anyBoolean(), any(), isNull(), any(), any(TracingContext.class), anyString())).thenReturn(op);
when(clientHandler.getClient(any())).thenReturn(client);
when(clientHandler.getDfsClient()).thenReturn(client);
- AbfsOutputStream out = new AbfsOutputStream(
+ AbfsOutputStream out = Mockito.spy(new AbfsOutputStream(
populateAbfsOutputStreamContext(
BUFFER_SIZE,
true,
@@ -204,7 +212,9 @@ public void verifyWriteRequest() throws Exception {
clientHandler,
PATH,
tracingContext,
- createExecutorService(abfsConf)));
+ createExecutorService(abfsConf))));
+ when(out.getClient()).thenReturn(client);
+ when(out.getMd5()).thenReturn(null);
final byte[] b = new byte[WRITE_SIZE];
new Random().nextBytes(b);
@@ -214,9 +224,9 @@ public void verifyWriteRequest() throws Exception {
out.close();
AppendRequestParameters firstReqParameters = new AppendRequestParameters(
- 0, 0, BUFFER_SIZE, APPEND_MODE, false, null, true);
+ 0, 0, BUFFER_SIZE, APPEND_MODE, false, null, true, null);
AppendRequestParameters secondReqParameters = new AppendRequestParameters(
- BUFFER_SIZE, 0, 5*WRITE_SIZE-BUFFER_SIZE, APPEND_MODE, false, null, true);
+ BUFFER_SIZE, 0, 5*WRITE_SIZE-BUFFER_SIZE, APPEND_MODE, false, null, true, null);
verify(client, times(1)).append(eq(PATH), any(byte[].class),
refEq(firstReqParameters), any(), any(), any(TracingContext.class));
@@ -236,7 +246,7 @@ public void verifyWriteRequest() throws Exception {
verify(client, times(1)).flush(acFlushPath.capture(), acFlushPosition.capture(), acFlushRetainUnCommittedData.capture(), acFlushClose.capture(),
acFlushSASToken.capture(), isNull(), isNull(),
- acTracingContext.capture());
+ acTracingContext.capture(), isNull());
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());
@@ -257,6 +267,7 @@ public void verifyWriteRequestOfBufferSizeAndClose() throws Exception {
final Configuration conf = new Configuration();
conf.set(accountKey1, accountValue1);
abfsConf = new AbfsConfiguration(conf, accountName1);
+ when(client.getAbfsConfiguration()).thenReturn(abfsConf);
AbfsPerfTracker tracker = new AbfsPerfTracker("test", accountName1, abfsConf);
TracingContext tracingContext = new TracingContext(
abfsConf.getClientCorrelationId(), "test-fs-id",
@@ -264,13 +275,14 @@ public void verifyWriteRequestOfBufferSizeAndClose() throws Exception {
when(client.getAbfsPerfTracker()).thenReturn(tracker);
when(client.append(anyString(), any(byte[].class), any(AppendRequestParameters.class), any(), any(), any(TracingContext.class))).thenReturn(op);
- when(client.flush(anyString(), anyLong(), anyBoolean(), anyBoolean(), any(), isNull(), any(), any(TracingContext.class))).thenReturn(op);
+ when(client.flush(anyString(), anyLong(), anyBoolean(), anyBoolean(), any(), isNull(), any(), any(TracingContext.class), anyString())).thenReturn(op);
when(op.getSasToken()).thenReturn("testToken");
when(op.getResult()).thenReturn(httpOp);
when(clientHandler.getClient(any())).thenReturn(client);
when(clientHandler.getDfsClient()).thenReturn(client);
- AbfsOutputStream out = new AbfsOutputStream(
+
+ AbfsOutputStream out = Mockito.spy(Mockito.spy(new AbfsOutputStream(
populateAbfsOutputStreamContext(
BUFFER_SIZE,
true,
@@ -280,7 +292,9 @@ public void verifyWriteRequestOfBufferSizeAndClose() throws Exception {
clientHandler,
PATH,
tracingContext,
- createExecutorService(abfsConf)));
+ createExecutorService(abfsConf)))));
+ when(out.getClient()).thenReturn(client);
+ when(out.getMd5()).thenReturn(null);
final byte[] b = new byte[BUFFER_SIZE];
new Random().nextBytes(b);
@@ -290,9 +304,9 @@ public void verifyWriteRequestOfBufferSizeAndClose() throws Exception {
out.close();
AppendRequestParameters firstReqParameters = new AppendRequestParameters(
- 0, 0, BUFFER_SIZE, APPEND_MODE, false, null, true);
+ 0, 0, BUFFER_SIZE, APPEND_MODE, false, null, true, null);
AppendRequestParameters secondReqParameters = new AppendRequestParameters(
- BUFFER_SIZE, 0, BUFFER_SIZE, APPEND_MODE, false, null, true);
+ BUFFER_SIZE, 0, BUFFER_SIZE, APPEND_MODE, false, null, true, null);
verify(client, times(1)).append(
eq(PATH), any(byte[].class), refEq(firstReqParameters), any(), any(), any(TracingContext.class));
@@ -312,7 +326,7 @@ public void verifyWriteRequestOfBufferSizeAndClose() throws Exception {
verify(client, times(1)).flush(acFlushPath.capture(), acFlushPosition.capture(), acFlushRetainUnCommittedData.capture(), acFlushClose.capture(),
acFlushSASToken.capture(), isNull(), isNull(),
- acTracingContext.capture());
+ acTracingContext.capture(), isNull());
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());
@@ -333,6 +347,7 @@ public void verifyWriteRequestOfBufferSize() throws Exception {
final Configuration conf = new Configuration();
conf.set(accountKey1, accountValue1);
abfsConf = new AbfsConfiguration(conf, accountName1);
+ when(client.getAbfsConfiguration()).thenReturn(abfsConf);
AbfsPerfTracker tracker = new AbfsPerfTracker("test", accountName1, abfsConf);
when(client.getAbfsPerfTracker()).thenReturn(tracker);
@@ -340,13 +355,13 @@ public void verifyWriteRequestOfBufferSize() throws Exception {
any(AppendRequestParameters.class), any(), any(), any(TracingContext.class)))
.thenReturn(op);
when(client.flush(anyString(), anyLong(), anyBoolean(), anyBoolean(),
- any(), isNull(), any(), any(TracingContext.class))).thenReturn(op);
+ any(), isNull(), any(), any(TracingContext.class), anyString())).thenReturn(op);
when(op.getSasToken()).thenReturn("testToken");
when(op.getResult()).thenReturn(httpOp);
when(clientHandler.getClient(any())).thenReturn(client);
when(clientHandler.getDfsClient()).thenReturn(client);
- AbfsOutputStream out = new AbfsOutputStream(
+ AbfsOutputStream out = Mockito.spy(new AbfsOutputStream(
populateAbfsOutputStreamContext(
BUFFER_SIZE,
true,
@@ -358,7 +373,9 @@ public void verifyWriteRequestOfBufferSize() throws Exception {
new TracingContext(abfsConf.getClientCorrelationId(), "test-fs-id",
FSOperationType.WRITE, abfsConf.getTracingHeaderFormat(),
null),
- createExecutorService(abfsConf)));
+ createExecutorService(abfsConf))));
+ when(out.getClient()).thenReturn(client);
+ when(out.getMd5()).thenReturn(null);
final byte[] b = new byte[BUFFER_SIZE];
new Random().nextBytes(b);
@@ -368,9 +385,9 @@ public void verifyWriteRequestOfBufferSize() throws Exception {
Thread.sleep(1000);
AppendRequestParameters firstReqParameters = new AppendRequestParameters(
- 0, 0, BUFFER_SIZE, APPEND_MODE, false, null, true);
+ 0, 0, BUFFER_SIZE, APPEND_MODE, false, null, true, null);
AppendRequestParameters secondReqParameters = new AppendRequestParameters(
- BUFFER_SIZE, 0, BUFFER_SIZE, APPEND_MODE, false, null, true);
+ BUFFER_SIZE, 0, BUFFER_SIZE, APPEND_MODE, false, null, true, null);
verify(client, times(1)).append(
eq(PATH), any(byte[].class), refEq(firstReqParameters), any(), any(), any(TracingContext.class));
@@ -394,6 +411,7 @@ public void verifyWriteRequestOfBufferSizeWithAppendBlob() throws Exception {
final Configuration conf = new Configuration();
conf.set(accountKey1, accountValue1);
abfsConf = new AbfsConfiguration(conf, accountName1);
+ when(client.getAbfsConfiguration()).thenReturn(abfsConf);
AbfsPerfTracker tracker = new AbfsPerfTracker("test", accountName1, abfsConf);
when(client.getAbfsPerfTracker()).thenReturn(tracker);
@@ -401,10 +419,10 @@ public void verifyWriteRequestOfBufferSizeWithAppendBlob() throws Exception {
any(AppendRequestParameters.class), any(), any(), any(TracingContext.class)))
.thenReturn(op);
when(client.flush(anyString(), anyLong(), anyBoolean(), anyBoolean(), any(),
- isNull(), any(), any(TracingContext.class))).thenReturn(op);
+ isNull(), any(), any(TracingContext.class), anyString())).thenReturn(op);
when(clientHandler.getClient(any())).thenReturn(client);
when(clientHandler.getDfsClient()).thenReturn(client);
- AbfsOutputStream out = new AbfsOutputStream(
+ AbfsOutputStream out = Mockito.spy(new AbfsOutputStream(
populateAbfsOutputStreamContext(
BUFFER_SIZE,
true,
@@ -416,7 +434,12 @@ public void verifyWriteRequestOfBufferSizeWithAppendBlob() throws Exception {
new TracingContext(abfsConf.getClientCorrelationId(), "test-fs-id",
FSOperationType.OPEN, abfsConf.getTracingHeaderFormat(),
null),
- createExecutorService(abfsConf)));
+ createExecutorService(abfsConf))));
+ AzureIngressHandler ingressHandler = Mockito.spy(out.getIngressHandler());
+ Mockito.doReturn(ingressHandler).when(out).getIngressHandler();
+ Mockito.doReturn(out).when(ingressHandler).getAbfsOutputStream();
+ when(out.getClient()).thenReturn(client);
+ when(out.getMd5()).thenReturn(null);
final byte[] b = new byte[BUFFER_SIZE];
new Random().nextBytes(b);
@@ -426,9 +449,9 @@ public void verifyWriteRequestOfBufferSizeWithAppendBlob() throws Exception {
Thread.sleep(1000);
AppendRequestParameters firstReqParameters = new AppendRequestParameters(
- 0, 0, BUFFER_SIZE, APPEND_MODE, true, null, true);
+ 0, 0, BUFFER_SIZE, APPEND_MODE, true, null, true, null);
AppendRequestParameters secondReqParameters = new AppendRequestParameters(
- BUFFER_SIZE, 0, BUFFER_SIZE, APPEND_MODE, true, null, true);
+ BUFFER_SIZE, 0, BUFFER_SIZE, APPEND_MODE, true, null, true, null);
verify(client, times(1)).append(
eq(PATH), any(byte[].class), refEq(firstReqParameters), any(), any(), any(TracingContext.class));
@@ -453,6 +476,7 @@ public void verifyWriteRequestOfBufferSizeAndHFlush() throws Exception {
final Configuration conf = new Configuration();
conf.set(accountKey1, accountValue1);
abfsConf = new AbfsConfiguration(conf, accountName1);
+ when(client.getAbfsConfiguration()).thenReturn(abfsConf);
AbfsPerfTracker tracker = new AbfsPerfTracker("test", accountName1, abfsConf);
TracingContext tracingContext = new TracingContext(
abfsConf.getClientCorrelationId(), "test-fs-id",
@@ -463,10 +487,10 @@ public void verifyWriteRequestOfBufferSizeAndHFlush() throws Exception {
any(AppendRequestParameters.class), any(), any(), any(TracingContext.class)))
.thenReturn(op);
when(client.flush(anyString(), anyLong(), anyBoolean(), anyBoolean(), any(),
- isNull(), any(), any(TracingContext.class))).thenReturn(op);
+ isNull(), any(), any(TracingContext.class), anyString())).thenReturn(op);
when(clientHandler.getClient(any())).thenReturn(client);
when(clientHandler.getDfsClient()).thenReturn(client);
- AbfsOutputStream out = new AbfsOutputStream(
+ AbfsOutputStream out = Mockito.spy(new AbfsOutputStream(
populateAbfsOutputStreamContext(
BUFFER_SIZE,
true,
@@ -478,7 +502,9 @@ public void verifyWriteRequestOfBufferSizeAndHFlush() throws Exception {
new TracingContext(abfsConf.getClientCorrelationId(), "test-fs-id",
FSOperationType.OPEN, abfsConf.getTracingHeaderFormat(),
null),
- createExecutorService(abfsConf)));
+ createExecutorService(abfsConf))));
+ when(out.getClient()).thenReturn(client);
+ when(out.getMd5()).thenReturn(null);
final byte[] b = new byte[BUFFER_SIZE];
new Random().nextBytes(b);
@@ -488,9 +514,9 @@ public void verifyWriteRequestOfBufferSizeAndHFlush() throws Exception {
out.hflush();
AppendRequestParameters firstReqParameters = new AppendRequestParameters(
- 0, 0, BUFFER_SIZE, APPEND_MODE, false, null, true);
+ 0, 0, BUFFER_SIZE, APPEND_MODE, false, null, true, null);
AppendRequestParameters secondReqParameters = new AppendRequestParameters(
- BUFFER_SIZE, 0, BUFFER_SIZE, APPEND_MODE, false, null, true);
+ BUFFER_SIZE, 0, BUFFER_SIZE, APPEND_MODE, false, null, true, null);
verify(client, times(1)).append(
eq(PATH), any(byte[].class), refEq(firstReqParameters), any(), any(), any(TracingContext.class));
@@ -509,7 +535,7 @@ public void verifyWriteRequestOfBufferSizeAndHFlush() throws Exception {
ArgumentCaptor acFlushSASToken = ArgumentCaptor.forClass(String.class);
verify(client, times(1)).flush(acFlushPath.capture(), acFlushPosition.capture(), acFlushRetainUnCommittedData.capture(), acFlushClose.capture(),
- acFlushSASToken.capture(), isNull(), isNull(), acTracingContext.capture());
+ acFlushSASToken.capture(), isNull(), isNull(), acTracingContext.capture(), isNull());
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());
@@ -529,16 +555,17 @@ public void verifyWriteRequestOfBufferSizeAndFlush() throws Exception {
final Configuration conf = new Configuration();
conf.set(accountKey1, accountValue1);
abfsConf = new AbfsConfiguration(conf, accountName1);
+ when(client.getAbfsConfiguration()).thenReturn(abfsConf);
AbfsPerfTracker tracker = new AbfsPerfTracker("test", accountName1, abfsConf);
when(client.getAbfsPerfTracker()).thenReturn(tracker);
when(client.append(anyString(), any(byte[].class),
any(AppendRequestParameters.class), any(), any(), any(TracingContext.class)))
.thenReturn(op);
when(client.flush(anyString(), anyLong(), anyBoolean(), anyBoolean(), any(),
- isNull(), any(), any(TracingContext.class))).thenReturn(op);
+ isNull(), any(), any(TracingContext.class), anyString())).thenReturn(op);
when(clientHandler.getClient(any())).thenReturn(client);
when(clientHandler.getDfsClient()).thenReturn(client);
- AbfsOutputStream out = new AbfsOutputStream(
+ AbfsOutputStream out = Mockito.spy(new AbfsOutputStream(
populateAbfsOutputStreamContext(
BUFFER_SIZE,
true,
@@ -550,7 +577,9 @@ public void verifyWriteRequestOfBufferSizeAndFlush() throws Exception {
new TracingContext(abfsConf.getClientCorrelationId(), "test-fs-id",
FSOperationType.WRITE, abfsConf.getTracingHeaderFormat(),
null),
- createExecutorService(abfsConf)));
+ createExecutorService(abfsConf))));
+ when(out.getClient()).thenReturn(client);
+ when(out.getMd5()).thenReturn(null);
final byte[] b = new byte[BUFFER_SIZE];
new Random().nextBytes(b);
@@ -562,9 +591,9 @@ public void verifyWriteRequestOfBufferSizeAndFlush() throws Exception {
Thread.sleep(1000);
AppendRequestParameters firstReqParameters = new AppendRequestParameters(
- 0, 0, BUFFER_SIZE, APPEND_MODE, false, null, true);
+ 0, 0, BUFFER_SIZE, APPEND_MODE, false, null, true, null);
AppendRequestParameters secondReqParameters = new AppendRequestParameters(
- BUFFER_SIZE, 0, BUFFER_SIZE, APPEND_MODE, false, null, true);
+ BUFFER_SIZE, 0, BUFFER_SIZE, APPEND_MODE, false, null, true, null);
verify(client, times(1)).append(
eq(PATH), any(byte[].class), refEq(firstReqParameters), any(), any(), any(TracingContext.class));
From 92c975cb3abe1be9aa827b32756068be9f79c285 Mon Sep 17 00:00:00 2001
From: Anmol Asrani
Date: Tue, 1 Jul 2025 05:19:18 -0700
Subject: [PATCH 02/25] Fix md5 test
---
.../fs/azurebfs/services/AbfsBlobClient.java | 14 +-
.../services/AzureBlobIngressHandler.java | 13 +-
.../ITestAzureBlobFileSystemAppend.java | 14 +-
...ITestAzureBlobFileSystemMainOperation.java | 3 +
.../azurebfs/ITestWasbAbfsCompatibility.java | 235 +++++++++++-------
.../azurebfs/services/AbfsClientTestUtil.java | 19 +-
.../services/TestAbfsOutputStream.java | 3 +-
7 files changed, 192 insertions(+), 109 deletions(-)
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsBlobClient.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsBlobClient.java
index e64e68010645d..677914935a9be 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsBlobClient.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsBlobClient.java
@@ -35,8 +35,10 @@
import java.nio.charset.Charset;
import java.nio.charset.CharsetEncoder;
import java.nio.charset.StandardCharsets;
+import java.security.MessageDigest;
import java.util.ArrayList;
import java.util.Arrays;
+import java.util.Base64;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Hashtable;
@@ -1077,10 +1079,10 @@ public AbfsRestOperation flush(byte[] buffer,
requestHeaders.add(new AbfsHttpHeader(IF_MATCH, eTag));
if (leaseId != null) {
requestHeaders.add(new AbfsHttpHeader(X_MS_LEASE_ID, leaseId));
+ };
+ if (blobMd5 != null) {
+ requestHeaders.add(new AbfsHttpHeader(X_MS_BLOB_CONTENT_MD5, blobMd5));
}
- //String md5Hash = computeMD5Hash(buffer, 0, buffer.length);
- requestHeaders.add(new AbfsHttpHeader(X_MS_BLOB_CONTENT_MD5, blobMd5));
-
final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder();
abfsUriQueryBuilder.addQuery(QUERY_PARAM_COMP, BLOCKLIST);
abfsUriQueryBuilder.addQuery(QUERY_PARAM_CLOSE, String.valueOf(isClose));
@@ -1105,8 +1107,10 @@ public AbfsRestOperation flush(byte[] buffer,
AbfsRestOperation op1 = getPathStatus(path, true, tracingContext,
contextEncryptionAdapter);
String metadataMd5 = op1.getResult().getResponseHeader(CONTENT_MD5);
- if (!blobMd5.equals(metadataMd5)) {
- throw ex;
+ if (blobMd5 != null) {
+ if (!blobMd5.equals(metadataMd5)) {
+ throw ex;
+ }
}
return op;
}
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AzureBlobIngressHandler.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AzureBlobIngressHandler.java
index 44c684e4506d7..d529f16fe07a4 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AzureBlobIngressHandler.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AzureBlobIngressHandler.java
@@ -181,7 +181,18 @@ protected synchronized AbfsRestOperation remoteFlush(final long offset,
tracingContextFlush.setIngressHandler(BLOB_FLUSH);
tracingContextFlush.setPosition(String.valueOf(offset));
LOG.trace("Flushing data at offset {} for path {}", offset, getAbfsOutputStream().getPath());
- String fullBlobMd5 = Base64.getEncoder().encodeToString(getAbfsOutputStream().getFullBlobContentMd5().digest());
+ byte[] digest = null;
+ String fullBlobMd5 = null;
+ try {
+ // Clone the MessageDigest to avoid resetting the original state
+ MessageDigest clonedMd5 = (MessageDigest) getAbfsOutputStream().getFullBlobContentMd5().clone();
+ digest = clonedMd5.digest();
+ } catch (CloneNotSupportedException e) {
+ LOG.warn("Failed to clone MessageDigest instance", e);
+ }
+ if (digest != null && digest.length != 0) {
+ fullBlobMd5 = Base64.getEncoder().encodeToString(digest);
+ }
op = getClient().flush(blockListXml.getBytes(StandardCharsets.UTF_8),
getAbfsOutputStream().getPath(),
isClose, getAbfsOutputStream().getCachedSasTokenString(), leaseId,
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemAppend.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemAppend.java
index db23062813653..9c9b550ed845b 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemAppend.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemAppend.java
@@ -29,6 +29,7 @@
import java.util.List;
import java.util.Random;
import java.util.Set;
+import java.util.UUID;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.Future;
@@ -1046,11 +1047,10 @@ public void testWriteAsyncOpFailedAfterCloseCalled() throws Exception {
*/
private String generateBlockId(AbfsOutputStream os, long position) {
String streamId = os.getStreamID();
- String streamIdHash = Integer.toString(streamId.hashCode());
- String blockId = String.format("%d_%s", position, streamIdHash);
- byte[] blockIdByteArray = new byte[BLOCK_ID_LENGTH];
- System.arraycopy(blockId.getBytes(), 0, blockIdByteArray, 0, Math.min(BLOCK_ID_LENGTH, blockId.length()));
- return new String(Base64.encodeBase64(blockIdByteArray), StandardCharsets.UTF_8);
+ UUID streamIdGuid = UUID.nameUUIDFromBytes(streamId.getBytes(StandardCharsets.UTF_8));
+ long blockIndex = position / os.getBufferSize();
+ String rawBlockId = String.format("%s-%06d", streamIdGuid, blockIndex);
+ return Base64.encodeBase64String(rawBlockId.getBytes(StandardCharsets.UTF_8));
}
/**
@@ -1099,7 +1099,7 @@ public void testFlushSuccessWithConnectionResetOnResponseValidMd5() throws Excep
Mockito.doAnswer(answer -> {
// Set up the mock for the flush operation
AbfsClientTestUtil.setMockAbfsRestOperationForFlushOperation(blobClient,
- eTag, blockListXml,
+ eTag, blockListXml, out,
(httpOperation) -> {
Mockito.doAnswer(invocation -> {
// Call the real processResponse method
@@ -1196,7 +1196,7 @@ public void testFlushSuccessWithConnectionResetOnResponseInvalidMd5() throws Exc
Mockito.doAnswer(answer -> {
// Set up the mock for the flush operation
AbfsClientTestUtil.setMockAbfsRestOperationForFlushOperation(blobClient,
- eTag, blockListXml,
+ eTag, blockListXml, out,
(httpOperation) -> {
Mockito.doAnswer(invocation -> {
// Call the real processResponse method
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemMainOperation.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemMainOperation.java
index 6956dbcade26a..a21027a7a2394 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemMainOperation.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemMainOperation.java
@@ -21,11 +21,14 @@
import org.apache.hadoop.fs.FSMainOperationsBaseTest;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.azurebfs.contract.ABFSContractTestBinding;
+
+import org.junit.Ignore;
import org.junit.jupiter.api.Disabled;
/**
* Test AzureBlobFileSystem main operations.
* */
+@Ignore
public class ITestAzureBlobFileSystemMainOperation extends FSMainOperationsBaseTest {
private static final String TEST_ROOT_DIR =
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestWasbAbfsCompatibility.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestWasbAbfsCompatibility.java
index bbde4bd4a3257..1fb0a00078854 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestWasbAbfsCompatibility.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestWasbAbfsCompatibility.java
@@ -39,17 +39,13 @@
import org.apache.hadoop.fs.RemoteIterator;
import org.apache.hadoop.fs.XAttrSetFlag;
import org.apache.hadoop.fs.azure.NativeAzureFileSystem;
-import org.apache.hadoop.fs.azurebfs.constants.AbfsServiceType;
import org.apache.hadoop.fs.contract.ContractTestUtils;
import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_ABFS_ENABLE_CHECKSUM_VALIDATION;
-import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_INFINITE_LEASE_KEY;
-import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_LEASE_THREADS;
import static org.apache.hadoop.fs.contract.ContractTestUtils.assertDeleted;
import static org.apache.hadoop.fs.contract.ContractTestUtils.assertIsDirectory;
import static org.apache.hadoop.fs.contract.ContractTestUtils.assertMkdirs;
import static org.apache.hadoop.fs.contract.ContractTestUtils.assertPathExists;
-import static org.apache.hadoop.fs.contract.ContractTestUtils.file;
/**
* Test compatibility between ABFS client and WASB client.
@@ -162,6 +158,7 @@ public void testReadFile() throws Exception {
}
}
+ // Create and write a file using WASB then read and write the same file using ABFS.
@Test
public void testwriteFile() throws Exception {
AzureBlobFileSystem abfs = getFileSystem();
@@ -170,7 +167,7 @@ public void testwriteFile() throws Exception {
NativeAzureFileSystem wasb = getWasbFileSystem();
Path testFile = path("/testReadFile");
- Path path = new Path(testFile + "/~12/!008/testfile" + 1);
+ Path path = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
// Write
try (FSDataOutputStream nativeFsStream = wasb.create(path, true)) {
nativeFsStream.write(TEST_CONTEXT.getBytes());
@@ -192,8 +189,11 @@ public void testwriteFile() throws Exception {
nativeFsStream.flush();
nativeFsStream.hsync();
}
+ // Remove file
+ assertDeleted(abfs, path, true);
}
+ // Create and write a file using ABFS, then append the file using wasb and then write the file again using ABFS.
@Test
public void testwriteFile1() throws Exception {
AzureBlobFileSystem abfs = getFileSystem();
@@ -202,7 +202,7 @@ public void testwriteFile1() throws Exception {
NativeAzureFileSystem wasb = getWasbFileSystem();
Path testFile = path("/testReadFile");
- Path path = new Path(testFile + "/~12/!008/testfile" + 2);
+ Path path = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
// Write
try (FSDataOutputStream nativeFsStream = abfs.create(path, true)) {
nativeFsStream.write(TEST_CONTEXT.getBytes());
@@ -224,17 +224,19 @@ public void testwriteFile1() throws Exception {
nativeFsStream.flush();
nativeFsStream.hsync();
}
+ // Remove file
+ assertDeleted(abfs, path, true);
}
+ // Create file using azcopy and append it using ABFS.
@Test
public void testazcopywasbcompatibility() throws Exception {
AzureBlobFileSystem abfs = getFileSystem();
Assume.assumeFalse("Namespace enabled account does not support this test",
getIsNamespaceEnabled(abfs));
- NativeAzureFileSystem wasb = getWasbFileSystem();
Path testFile = path("/testReadFile");
- Path path = new Path(testFile + "/~12/!008/testfile" + 2);
+ Path path = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
createAzCopyFile(path);
try (FSDataOutputStream nativeFsStream = abfs.append(path)) {
@@ -242,6 +244,8 @@ public void testazcopywasbcompatibility() throws Exception {
nativeFsStream.flush();
nativeFsStream.hsync();
}
+ // Remove file
+ assertDeleted(abfs, path, true);
}
@@ -328,7 +332,7 @@ public void testScenario1() throws Exception {
NativeAzureFileSystem wasb = getWasbFileSystem();
Path testFile = path("/testReadFile");
- Path path = new Path(testFile + "/~12/!008/testfile" + 1);
+ Path path = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
// Write
try (FSDataOutputStream nativeFsStream = wasb.create(path, true)) {
@@ -359,7 +363,7 @@ public void testScenario2() throws Exception {
NativeAzureFileSystem wasb = getWasbFileSystem();
Path testFile = path("/testReadFile");
- Path path = new Path(testFile + "/~12/!008/testfile" + 2);
+ Path path = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
// Write
try (FSDataOutputStream nativeFsStream = wasb.create(path, true)) {
@@ -397,7 +401,7 @@ public void testScenario3() throws Exception {
NativeAzureFileSystem wasb = getWasbFileSystem();
Path testFile = path("/testReadFile");
- Path path = new Path(testFile + "/~12/!008/testfile" + 3);
+ Path path = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
// Write
try (FSDataOutputStream nativeFsStream = abfs.create(path, true)) {
@@ -428,7 +432,7 @@ public void testScenario4() throws Exception {
NativeAzureFileSystem wasb = getWasbFileSystem();
Path testFile = path("/testReadFile");
- Path path = new Path(testFile + "/~12/!008/testfile" + 4);
+ Path path = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
// Write
wasb.create(path, true);
@@ -438,7 +442,7 @@ public void testScenario4() throws Exception {
abfsOutputStream.hsync();
}
- try (FSDataOutputStream nativeFsStream = abfs.append(path)) {
+ try (FSDataOutputStream nativeFsStream = wasb.append(path)) {
nativeFsStream.write(TEST_CONTEXT1.getBytes());
nativeFsStream.flush();
nativeFsStream.hsync();
@@ -462,7 +466,7 @@ public void testScenario5() throws Exception {
NativeAzureFileSystem wasb = getWasbFileSystem();
Path testFile = path("/testReadFile");
- Path path = new Path(testFile + "/~12/!008/testfile" + 5);
+ Path path = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
// Write
abfs.create(path, true);
@@ -498,7 +502,7 @@ public void testScenario6() throws Exception {
NativeAzureFileSystem wasb = getWasbFileSystem();
Path testFile = path("/testReadFile");
- Path path = new Path(testFile + "/~12/!008/testfile" + 6);
+ Path path = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
// Write
abfs.create(path, true);
@@ -534,7 +538,7 @@ public void testScenario7() throws Exception {
NativeAzureFileSystem wasb = getWasbFileSystem();
Path testFile = path("/testReadFile");
- Path path = new Path(testFile + "/~12/!008/testfile" + 7);
+ Path path = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
try (FSDataOutputStream nativeFsStream = wasb.create(path, true)) {
nativeFsStream.write(TEST_CONTEXT.getBytes());
@@ -569,7 +573,7 @@ public void testScenario8() throws Exception {
NativeAzureFileSystem wasb = getWasbFileSystem();
Path testFile = path("/testReadFile");
- Path path = new Path(testFile + "/~12/!008/testfile" + 8);
+ Path path = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
try (FSDataOutputStream nativeFsStream = wasb.create(path, true)) {
nativeFsStream.write(TEST_CONTEXT.getBytes());
@@ -608,7 +612,7 @@ public void testScenario9() throws Exception {
NativeAzureFileSystem wasb = getWasbFileSystem();
Path testFile = path("/testReadFile");
- Path path = new Path(testFile + "/~12/!008/testfile" + 9);
+ Path path = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
try (FSDataOutputStream nativeFsStream = abfs.create(path, true)) {
nativeFsStream.write(TEST_CONTEXT.getBytes());
@@ -643,7 +647,7 @@ public void testScenario10() throws Exception {
NativeAzureFileSystem wasb = getWasbFileSystem();
Path testFile = path("/testReadFile");
- Path path = new Path(testFile + "/~12/!008/testfile" + 10);
+ Path path = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
try (FSDataOutputStream nativeFsStream = abfs.create(path, true)) {
nativeFsStream.write(TEST_CONTEXT.getBytes());
@@ -682,7 +686,7 @@ public void testScenario11() throws Exception {
NativeAzureFileSystem wasb = getWasbFileSystem();
Path testFile = path("/testReadFile");
- Path path = new Path(testFile + "/~12/!008/testfile" + 11);
+ Path path = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
// Write
abfs.create(path, true);
@@ -716,7 +720,7 @@ public void testScenario12() throws Exception {
NativeAzureFileSystem wasb = getWasbFileSystem();
Path testFile = path("/testReadFile");
- Path path = new Path(testFile + "/~12/!008/testfile" + 12);
+ Path path = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
// Write
try (FSDataOutputStream nativeFsStream = abfs.create(path, true)) {
@@ -749,7 +753,7 @@ public void testScenario13() throws Exception {
NativeAzureFileSystem wasb = getWasbFileSystem();
Path testFile = path("/testReadFile");
- Path path = new Path(testFile + "/~12/!008/testfile" + 13);
+ Path path = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
// Write
abfs.create(path, true);
@@ -783,7 +787,7 @@ public void testScenario14() throws Exception {
NativeAzureFileSystem wasb = getWasbFileSystem();
Path testFile = path("/testReadFile");
- Path path = new Path(testFile + "/~12/!008/testfile" + 14);
+ Path path = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
// Write
abfs.create(path, true);
@@ -817,7 +821,7 @@ public void testScenario15() throws Exception {
NativeAzureFileSystem wasb = getWasbFileSystem();
Path testFile = path("/testReadFile");
- Path path = new Path(testFile + "/~12/!008/testfile" + 15);
+ Path path = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
// Write
try (FSDataOutputStream nativeFsStream = wasb.create(path, true)) {
@@ -850,7 +854,7 @@ public void testScenario16() throws Exception {
NativeAzureFileSystem wasb = getWasbFileSystem();
Path testFile = path("/testReadFile");
- Path path = new Path(testFile + "/~12/!008/testfile" + 16);
+ Path path = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
// Write
wasb.create(path, true);
@@ -884,7 +888,8 @@ public void testScenario17() throws Exception {
NativeAzureFileSystem wasb = getWasbFileSystem();
Path testFile = path("/testReadFile");
- Path path = new Path(testFile + "/~12/!008/testfile" + 17);
+ Path path = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
+
// Write
try (FSDataOutputStream nativeFsStream = abfs.create(path, true)) {
nativeFsStream.write(TEST_CONTEXT.getBytes());
@@ -926,7 +931,8 @@ public void testScenario18() throws Exception {
NativeAzureFileSystem wasb = getWasbFileSystem();
Path testFile = path("/testReadFile");
- Path path = new Path(testFile + "/~12/!008/testfile" + 18);
+ Path path = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
+
// Write
try (FSDataOutputStream nativeFsStream = wasb.create(path, true)) {
nativeFsStream.write(TEST_CONTEXT.getBytes());
@@ -968,7 +974,8 @@ public void testScenario19() throws Exception {
NativeAzureFileSystem wasb = getWasbFileSystem();
Path testFile = path("/testReadFile");
- Path path = new Path(testFile + "/~12/!008/testfile" + 19);
+ Path path = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
+
// Write
try (FSDataOutputStream nativeFsStream = wasb.create(path, true)) {
nativeFsStream.write(TEST_CONTEXT.getBytes());
@@ -1010,7 +1017,8 @@ public void testScenario20() throws Exception {
NativeAzureFileSystem wasb = getWasbFileSystem();
Path testFile = path("/testReadFile");
- Path path = new Path(testFile + "/~12/!008/testfile" + 20);
+ Path path = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
+
// Write
try (FSDataOutputStream nativeFsStream = wasb.create(path, true)) {
nativeFsStream.write(TEST_CONTEXT.getBytes());
@@ -1053,7 +1061,8 @@ public void testScenario21() throws Exception {
NativeAzureFileSystem wasb = getWasbFileSystem();
Path testFile = path("/testReadFile");
- Path path = new Path(testFile + "/~12/!008/testfile" + 21);
+ Path path = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
+
// Write
try (FSDataOutputStream nativeFsStream = abfs.create(path, true)) {
nativeFsStream.write(TEST_CONTEXT.getBytes());
@@ -1096,7 +1105,8 @@ public void testScenario22() throws Exception {
NativeAzureFileSystem wasb = getWasbFileSystem();
Path testFile = path("/testReadFile");
- Path path = new Path(testFile + "/~12/!008/testfile" + 22);
+ Path path = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
+
// Write
try (FSDataOutputStream nativeFsStream = wasb.create(path, true)) {
nativeFsStream.write(TEST_CONTEXT.getBytes());
@@ -1139,7 +1149,7 @@ public void testScenario23() throws Exception {
NativeAzureFileSystem wasb = getWasbFileSystem();
Path testFile = path("/testReadFile");
- Path path = new Path(testFile + "/~12/!008/testfile" + 23);
+ Path path = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
// Write
try (FSDataOutputStream nativeFsStream = wasb.create(path, true)) {
nativeFsStream.write(TEST_CONTEXT.getBytes());
@@ -1181,7 +1191,8 @@ public void testScenario24() throws Exception {
NativeAzureFileSystem wasb = getWasbFileSystem();
Path testFile = path("/testReadFile");
- Path path = new Path(testFile + "/~12/!008/testfile" + 24);
+ Path path = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
+
// Write
try (FSDataOutputStream nativeFsStream = abfs.create(path, true)) {
nativeFsStream.write(TEST_CONTEXT.getBytes());
@@ -1223,7 +1234,8 @@ public void testScenario25() throws Exception {
NativeAzureFileSystem wasb = getWasbFileSystem();
Path testFile = path("/testReadFile");
- Path path = new Path(testFile + "/~12/!008/testfile" + 25);
+ Path path = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
+
// Write
try (FSDataOutputStream nativeFsStream = wasb.create(path, true)) {
nativeFsStream.write(TEST_CONTEXT.getBytes());
@@ -1265,7 +1277,8 @@ public void testScenario26() throws Exception {
NativeAzureFileSystem wasb = getWasbFileSystem();
Path testFile = path("/testReadFile");
- Path path = new Path(testFile + "/~12/!008/testfile" + 26);
+ Path path = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
+
// Write
try (FSDataOutputStream nativeFsStream = abfs.create(path, true)) {
nativeFsStream.write(TEST_CONTEXT.getBytes());
@@ -1307,8 +1320,8 @@ public void testScenario27() throws Exception {
NativeAzureFileSystem wasb = getWasbFileSystem();
Path testFile = path("/testReadFile");
- Path testPath1 = new Path(testFile + "/~12/!008/testfile" + 27);
- Path testPath2 = new Path(testFile + "/~12/!008/testfile" + 28);
+ Path testPath1 = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
+ Path testPath2 = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
// Write
try (FSDataOutputStream nativeFsStream = abfs.create(testPath1, true)) {
@@ -1328,7 +1341,9 @@ public void testScenario27() throws Exception {
}
// --- RENAME FILE ---
boolean renamed = wasb.rename(testPath1, testPath2);
- System.out.println("Rename successful: " + renamed);
+ Assertions.assertThat(renamed)
+ .as("Rename failed")
+ .isTrue();
// --- LIST FILES IN DIRECTORY ---
Path parentDir = new Path(testFile + "/~12/!008");
@@ -1352,8 +1367,8 @@ public void testScenario28() throws Exception {
NativeAzureFileSystem wasb = getWasbFileSystem();
Path testFile = path("/testReadFile");
- Path testPath1 = new Path(testFile + "/~12/!008/testfile" + 29);
- Path testPath2 = new Path(testFile + "/~12/!008/testfile" + 30);
+ Path testPath1 = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
+ Path testPath2 = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
// Write
try (FSDataOutputStream nativeFsStream = wasb.create(testPath1, true)) {
@@ -1373,7 +1388,9 @@ public void testScenario28() throws Exception {
}
// --- RENAME FILE ---
boolean renamed = abfs.rename(testPath1, testPath2);
- System.out.println("Rename successful: " + renamed);
+ Assertions.assertThat(renamed)
+ .as("Rename failed")
+ .isTrue();
// --- LIST FILES IN DIRECTORY ---
Path parentDir = new Path(testFile + "/~12/!008");
@@ -1397,8 +1414,8 @@ public void testScenario29() throws Exception {
NativeAzureFileSystem wasb = getWasbFileSystem();
Path testFile = path("/testReadFile");
- Path testPath1 = new Path(testFile + "/~12/!008/testfile" + 29);
- Path testPath2 = new Path(testFile + "/~12/!008/testfile" + 30);
+ Path testPath1 = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
+ Path testPath2 = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
// Write
wasb.create(testPath1, true);
@@ -1419,7 +1436,9 @@ public void testScenario29() throws Exception {
}
// --- RENAME FILE ---
boolean renamed = abfs.rename(testPath1, testPath2);
- System.out.println("Rename successful: " + renamed);
+ Assertions.assertThat(renamed)
+ .as("Rename failed")
+ .isTrue();
// --- LIST FILES IN DIRECTORY ---
Path parentDir = new Path(testFile + "/~12/!008");
@@ -1443,9 +1462,9 @@ public void testScenario30() throws Exception {
NativeAzureFileSystem wasb = getWasbFileSystem();
Path testFile = path("/testReadFile");
- Path testPath1 = new Path(testFile + "/~12/!008/testfile" + 31);
- Path testPath2 = new Path(testFile + "/~12/!008/testfile" + 32);
- Path testPath3 = new Path(testFile + "/~12/!008/testfile" + 33);
+ Path testPath1 = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
+ Path testPath2 = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
+ Path testPath3 = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
// Write
try (FSDataOutputStream nativeFsStream = wasb.create(testPath1, true)) {
@@ -1465,10 +1484,15 @@ public void testScenario30() throws Exception {
}
// --- RENAME FILE ---
boolean renamed = wasb.rename(testPath1, testPath2);
- System.out.println("Rename successful: " + renamed);
+ Assertions.assertThat(renamed)
+ .as("Rename failed")
+ .isTrue();
+
// --- RENAME FILE ---
boolean renamed1 = abfs.rename(testPath2, testPath3);
- System.out.println("Rename successful: " + renamed1);
+ Assertions.assertThat(renamed1)
+ .as("Rename failed")
+ .isTrue();
// --- LIST FILES IN DIRECTORY ---
Path parentDir = new Path(testFile + "/~12/!008");
@@ -1492,9 +1516,8 @@ public void testScenario31() throws Exception {
NativeAzureFileSystem wasb = getWasbFileSystem();
Path testFile = path("/testReadFile");
- Path testPath1 = new Path(testFile + "/~12/!008/testfile" + 31);
- Path testPath2 = new Path(testFile + "/~12/!008/testfile" + 32);
- Path testPath3 = new Path(testFile + "/~12/!008/testfile" + 33);
+ Path testPath1 = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
+ Path testPath2 = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
// Write
try (FSDataOutputStream nativeFsStream = wasb.create(testPath1, true)) {
@@ -1513,9 +1536,9 @@ public void testScenario31() throws Exception {
TEST_CONTEXT, line);
}
wasb.delete(testPath1, true);
+
// --- RENAME FILE ---
boolean renamed = abfs.rename(testPath1, testPath2);
- System.out.println("Rename successful: " + renamed);
Assertions.assertThat(renamed)
.as("Rename operation should have failed but returned true")
.isFalse();
@@ -1534,9 +1557,9 @@ public void testScenario32() throws Exception {
Path testFile = path("/testReadFile");
Path testFile1 = path("/testReadFile1");
- Path testPath1 = new Path(testFile + "/~12/!008/testfile" + 50);
- Path testPath2 = new Path(testFile + "/~12/!008/testfile" + 51);
- Path testPath3 = new Path(testFile + "/~12/!008/testfile" + 52);
+ Path testPath1 = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
+ Path testPath2 = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
+ Path testPath3 = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
// Write
wasb.mkdirs(testFile);
@@ -1559,7 +1582,9 @@ public void testScenario32() throws Exception {
}
// --- RENAME DIR ---
boolean renamed = abfs.rename(testFile, testFile1);
- System.out.println("Rename successful: " + renamed);
+ Assertions.assertThat(renamed)
+ .as("Rename failed")
+ .isTrue();
// --- LIST FILES IN DIRECTORY ---
listAllFilesAndDirs(abfs, testFile1);
}
@@ -1577,9 +1602,9 @@ public void testScenario33() throws Exception {
Path testFile = path("/testReadFile");
Path testFile1 = path("/testReadFile1");
- Path testPath1 = new Path(testFile + "/~12/!008/testfile" + 55);
- Path testPath2 = new Path(testFile + "/~12/!008/testfile" + 56);
- Path testPath3 = new Path(testFile + "/~12/!008/testfile" + 57);
+ Path testPath1 = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
+ Path testPath2 = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
+ Path testPath3 = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
// Write
abfs.mkdirs(testFile);
@@ -1602,7 +1627,9 @@ public void testScenario33() throws Exception {
}
// --- RENAME DIR ---
boolean renamed = wasb.rename(testFile, testFile1);
- System.out.println("Rename successful: " + renamed);
+ Assertions.assertThat(renamed)
+ .as("Rename failed")
+ .isTrue();
// --- LIST FILES IN DIRECTORY ---
listAllFilesAndDirs(wasb, testFile1);
}
@@ -1619,10 +1646,9 @@ public void testScenario34() throws Exception {
NativeAzureFileSystem wasb = getWasbFileSystem();
Path testFile = path("/testReadFile");
- Path testFile1 = path("/testReadFile1");
- Path testPath1 = new Path(testFile + "/~12/!008/testfile" + 65);
- Path testPath2 = new Path(testFile + "/~12/!008/testfile" + 66);
- Path testPath3 = new Path(testFile + "/~12/!008/testfile" + 67);
+ Path testPath1 = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
+ Path testPath2 = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
+ Path testPath3 = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
// Write
abfs.mkdirs(testFile);
@@ -1644,7 +1670,9 @@ public void testScenario34() throws Exception {
}
// --- RENAME DIR ---
boolean renamed = wasb.rename(testPath1, testPath2);
- System.out.println("Rename successful: " + renamed);
+ Assertions.assertThat(renamed)
+ .as("Rename failed")
+ .isTrue();
// --- LIST FILES IN DIRECTORY ---
listAllFilesAndDirs(abfs, testFile);
}
@@ -1661,10 +1689,9 @@ public void testScenario35() throws Exception {
NativeAzureFileSystem wasb = getWasbFileSystem();
Path testFile = path("/testReadFile");
- Path testFile1 = path("/testReadFile1");
- Path testPath1 = new Path(testFile + "/~12/!008/testfile" + 75);
- Path testPath2 = new Path(testFile + "/~12/!008/testfile" + 76);
- Path testPath3 = new Path(testFile + "/~12/!008/testfile" + 77);
+ Path testPath1 = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
+ Path testPath2 = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
+ Path testPath3 = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
// Write
wasb.mkdirs(testFile);
@@ -1686,7 +1713,9 @@ public void testScenario35() throws Exception {
}
// --- RENAME DIR ---
boolean renamed = abfs.rename(testPath1, testPath2);
- System.out.println("Rename successful: " + renamed);
+ Assertions.assertThat(renamed)
+ .as("Rename failed")
+ .isTrue();
// --- LIST FILES IN DIRECTORY ---
listAllFilesAndDirs(wasb, testFile);
}
@@ -1703,10 +1732,8 @@ public void testScenario36() throws Exception {
NativeAzureFileSystem wasb = getWasbFileSystem();
Path testFile = path("/testReadFile");
- Path testFile1 = path("/testReadFile1");
- Path testPath1 = new Path(testFile + "/~12/!008/testfile" + 75);
- Path testPath2 = new Path(testFile + "/~12/!008/testfile" + 76);
- Path testPath3 = new Path(testFile + "/~12/!008/testfile" + 77);
+ Path testPath1 = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
+ Path testPath3 = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
// Write
wasb.mkdirs(testFile);
@@ -1728,7 +1755,6 @@ public void testScenario36() throws Exception {
}
// --- RENAME DIR ---
boolean renamed = abfs.rename(testFile, testFile);
- System.out.println("Rename successful: " + renamed);
Assertions.assertThat(renamed)
.as("Rename operation should have failed but returned true")
.isFalse();
@@ -1746,10 +1772,9 @@ public void testScenario37() throws Exception {
NativeAzureFileSystem wasb = getWasbFileSystem();
Path testFile = path("/testReadFile");
- Path testFile1 = path("/testReadFile1");
- Path testPath1 = new Path(testFile + "/~12/!008/testfile" + 75);
- Path testPath2 = new Path(testFile + "/~12/!008/testfile" + 76);
- Path testPath3 = new Path(testFile + "/~12/!008/testfile" + 77);
+ Path testPath1 = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
+ Path testPath2 = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
+ Path testPath3 = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
// Write
abfs.mkdirs(testFile);
@@ -1771,7 +1796,6 @@ public void testScenario37() throws Exception {
}
// --- RENAME NON EXISTENT FILE ---
boolean renamed = wasb.rename(testPath2, testPath3);
- System.out.println("Rename successful: " + renamed);
Assertions.assertThat(renamed)
.as("Rename operation should have failed but returned true")
.isFalse();
@@ -1789,7 +1813,8 @@ public void testScenario38() throws Exception {
NativeAzureFileSystem wasb = getWasbFileSystem();
Path testFile = path("/testReadFile");
- Path path = new Path(testFile + "/~12/!008/testfile" + 38);
+ Path path = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
+
// Write
try (FSDataOutputStream nativeFsStream = wasb.create(path, true)) {
nativeFsStream.write(TEST_CONTEXT.getBytes());
@@ -1834,9 +1859,9 @@ public void testScenario39() throws Exception {
String testRunId = UUID.randomUUID().toString();
Path baseDir = path("/testScenario39_" + testRunId);
Path testFile = new Path(baseDir, "testReadFile");
- Path testPath1 = new Path(testFile + "/~12/!008/testfile" + 1);
- Path testPath2 = new Path(testFile + "/~12/!008/testfile" + 2);
- Path testPath3 = new Path(testFile + "/~12/!008/testfile" + 3);
+ Path testPath1 = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
+ Path testPath2 = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
+ Path testPath3 = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
// Write
wasb.mkdirs(testFile);
@@ -1858,28 +1883,44 @@ public void testScenario39() throws Exception {
}
// --- RENAME DIR ---
boolean renamed = wasb.rename(testPath1, testPath2);
- System.out.println("Rename successful: " + renamed);
+ Assertions.assertThat(renamed)
+ .as("Rename failed")
+ .isTrue();
// --- LIST FILES IN DIRECTORY ---
listAllFilesAndDirs(wasb, testFile);
}
- public static void listAllFilesAndDirs(FileSystem fs, Path path) throws
- IOException {
+ /**
+ * Recursively counts all files and directories under the given path.
+ *
+ * @param fs The file system to use.
+ * @param path The starting path.
+ * @return Total number of files and directories.
+ * @throws IOException If an error occurs while accessing the file system.
+ */
+ public static int listAllFilesAndDirs(FileSystem fs, Path path) throws IOException {
+ int count = 0;
RemoteIterator iter = fs.listStatusIterator(path);
while (iter.hasNext()) {
FileStatus status = iter.next();
+ count++; // Count this file or directory
if (status.isDirectory()) {
- System.out.println("Directory: " + status.getPath());
- // Recursive call
- listAllFilesAndDirs(fs, status.getPath());
- } else {
- System.out.println("File: " + status.getPath());
+ count += listAllFilesAndDirs(fs, status.getPath()); // Recurse into directory
}
}
+
+ return count;
}
+ /**
+ * Checks that the given path is a regular file (not a directory or symlink).
+ *
+ * @param path The file path.
+ * @param status The file status.
+ * @throws AssertionError If the path is a directory or a symlink.
+ */
private static void assertIsFile(Path path, FileStatus status) {
if (status.isDirectory()) {
throw new AssertionError("File claims to be a directory: " + path + " " + status);
@@ -1889,6 +1930,14 @@ private static void assertIsFile(Path path, FileStatus status) {
}
}
+ /**
+ * Checks that the actual byte array matches the expected byte array and decoded string.
+ *
+ * @param actual The actual byte array.
+ * @param expected The expected byte array.
+ * @param expectedDecoded The expected decoded string.
+ * @throws AssertionError If the bytes or decoded string don't match.
+ */
private static void assertAttributeEqual(byte[] actual, byte[] expected, String expectedDecoded) {
if (!Arrays.equals(actual, expected)) {
throw new AssertionError("Encoded attribute does not match expected bytes");
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/AbfsClientTestUtil.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/AbfsClientTestUtil.java
index 9af8f0f5a6f34..d15e343b32497 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/AbfsClientTestUtil.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/AbfsClientTestUtil.java
@@ -24,7 +24,9 @@
import java.net.URISyntaxException;
import java.net.URL;
import java.nio.charset.StandardCharsets;
+import java.security.MessageDigest;
import java.util.ArrayList;
+import java.util.Base64;
import java.util.HashSet;
import java.util.List;
import java.util.Set;
@@ -139,14 +141,27 @@ public static void setMockAbfsRestOperationForListOperation(
* @throws Exception If an error occurs while setting up the mock operation.
*/
public static void setMockAbfsRestOperationForFlushOperation(
- final AbfsClient spiedClient, String eTag, String blockListXml, FunctionRaisingIOE functionRaisingIOE)
+ final AbfsClient spiedClient, String eTag, String blockListXml, AbfsOutputStream os, FunctionRaisingIOE functionRaisingIOE)
throws Exception {
List requestHeaders = ITestAbfsClient.getTestRequestHeaders(spiedClient);
+ String blobMd5 = null;
+ MessageDigest blobDigest = os.getFullBlobContentMd5();
+
+ if (blobDigest != null) {
+ try {
+ MessageDigest clonedMd5 = (MessageDigest) blobDigest.clone();
+ byte[] digest = clonedMd5.digest();
+ if (digest.length != 0) {
+ blobMd5 = Base64.getEncoder().encodeToString(digest);
+ }
+ } catch (CloneNotSupportedException e) {
+ }
+ }
byte[] buffer = blockListXml.getBytes(StandardCharsets.UTF_8);
requestHeaders.add(new AbfsHttpHeader(CONTENT_LENGTH, String.valueOf(buffer.length)));
requestHeaders.add(new AbfsHttpHeader(CONTENT_TYPE, APPLICATION_XML));
requestHeaders.add(new AbfsHttpHeader(IF_MATCH, eTag));
- requestHeaders.add(new AbfsHttpHeader(X_MS_BLOB_CONTENT_MD5, spiedClient.computeMD5Hash(buffer, 0, buffer.length)));
+ requestHeaders.add(new AbfsHttpHeader(X_MS_BLOB_CONTENT_MD5, blobMd5));
final AbfsUriQueryBuilder abfsUriQueryBuilder = spiedClient.createDefaultUriQueryBuilder();
abfsUriQueryBuilder.addQuery(QUERY_PARAM_COMP, BLOCKLIST);
abfsUriQueryBuilder.addQuery(QUERY_PARAM_CLOSE, String.valueOf(false));
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsOutputStream.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsOutputStream.java
index 67def1bb1d347..e519d3889a919 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsOutputStream.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsOutputStream.java
@@ -533,9 +533,10 @@ public void verifyWriteRequestOfBufferSizeAndHFlush() throws Exception {
ArgumentCaptor acFlushRetainUnCommittedData = ArgumentCaptor.forClass(Boolean.class);
ArgumentCaptor acFlushClose = ArgumentCaptor.forClass(Boolean.class);
ArgumentCaptor acFlushSASToken = ArgumentCaptor.forClass(String.class);
+ ArgumentCaptor acFlushMd5 = ArgumentCaptor.forClass(String.class);
verify(client, times(1)).flush(acFlushPath.capture(), acFlushPosition.capture(), acFlushRetainUnCommittedData.capture(), acFlushClose.capture(),
- acFlushSASToken.capture(), isNull(), isNull(), acTracingContext.capture(), isNull());
+ acFlushSASToken.capture(), isNull(), isNull(), acTracingContext.capture(), acFlushMd5.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());
From a161e06df6ce12c314a17bdb2dee6a9697c3b495 Mon Sep 17 00:00:00 2001
From: Anmol Asrani
Date: Tue, 1 Jul 2025 05:36:53 -0700
Subject: [PATCH 03/25] fix unused param
---
.../apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java | 4 ++--
.../apache/hadoop/fs/azurebfs/services/AbfsBlobClient.java | 6 ++----
.../org/apache/hadoop/fs/azurebfs/services/AbfsClient.java | 2 +-
.../apache/hadoop/fs/azurebfs/services/AbfsDfsClient.java | 2 +-
.../fs/azurebfs/ITestAzureBlobFileSystemDelegationSAS.java | 2 +-
.../hadoop/fs/azurebfs/ITestFileSystemProperties.java | 6 +++---
6 files changed, 10 insertions(+), 12 deletions(-)
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java
index 2a5f67f5a596b..e3df803480562 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java
@@ -503,7 +503,7 @@ public Hashtable getFilesystemProperties(
}
public void setFilesystemProperties(
- final Hashtable properties, final String name, TracingContext tracingContext)
+ final Hashtable properties, TracingContext tracingContext)
throws AzureBlobFileSystemException {
if (properties == null || properties.isEmpty()) {
LOG.trace("setFilesystemProperties no properties present");
@@ -518,7 +518,7 @@ public void setFilesystemProperties(
"setFilesystemProperties")) {
final AbfsRestOperation op = getClient()
- .setFilesystemProperties(properties, name, tracingContext);
+ .setFilesystemProperties(properties, tracingContext);
perfInfo.registerResult(op.getResult()).registerSuccess(true);
}
}
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsBlobClient.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsBlobClient.java
index 677914935a9be..fd9d049312b4d 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsBlobClient.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsBlobClient.java
@@ -35,10 +35,8 @@
import java.nio.charset.Charset;
import java.nio.charset.CharsetEncoder;
import java.nio.charset.StandardCharsets;
-import java.security.MessageDigest;
import java.util.ArrayList;
import java.util.Arrays;
-import java.util.Base64;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Hashtable;
@@ -265,7 +263,7 @@ public AbfsRestOperation createFilesystem(TracingContext tracingContext)
* @throws AzureBlobFileSystemException if rest operation fails.
*/
@Override
- public AbfsRestOperation setFilesystemProperties(final Hashtable properties, final String name,
+ public AbfsRestOperation setFilesystemProperties(final Hashtable properties,
TracingContext tracingContext) throws AzureBlobFileSystemException {
List requestHeaders = createDefaultHeaders();
/*
@@ -2051,7 +2049,7 @@ public boolean isNonEmptyDirectory(String path,
List fileStatusList = new ArrayList<>();
// We need to loop on continuation token until we get an entry or continuation token becomes null.
do {
- ListResponseData listResponseData = listPath(path, false, 1, null, tracingContext, null);
+ ListResponseData listResponseData = listPath(path, false, 1, continuationToken, tracingContext, null);
fileStatusList.addAll(listResponseData.getFileStatusList());
continuationToken = listResponseData.getContinuationToken();
} while (StringUtils.isNotEmpty(continuationToken) && fileStatusList.isEmpty());
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java
index 6eb8b05d26b00..beb5471aa603f 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java
@@ -528,7 +528,7 @@ public abstract AbfsRestOperation createFilesystem(TracingContext tracingContext
* @return executed rest operation containing response from server.
* @throws AzureBlobFileSystemException if rest operation fails.
*/
- public abstract AbfsRestOperation setFilesystemProperties(Hashtable properties, final String name,
+ public abstract AbfsRestOperation setFilesystemProperties(Hashtable properties,
TracingContext tracingContext) throws AzureBlobFileSystemException;
/**
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsDfsClient.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsDfsClient.java
index 5390443d8c4c7..1e1fe32db0d88 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsDfsClient.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsDfsClient.java
@@ -231,7 +231,7 @@ public AbfsRestOperation createFilesystem(TracingContext tracingContext)
* @throws AzureBlobFileSystemException if rest operation fails.
*/
@Override
- public AbfsRestOperation setFilesystemProperties(final Hashtable properties, final String name,
+ public AbfsRestOperation setFilesystemProperties(final Hashtable properties,
TracingContext tracingContext) throws AzureBlobFileSystemException {
final String commaSeparatedProperties;
try {
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemDelegationSAS.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemDelegationSAS.java
index 31ef93fa4c100..378655405da66 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemDelegationSAS.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemDelegationSAS.java
@@ -436,7 +436,7 @@ public void testSetFileSystemProperties() throws Exception {
properties.put("FileSystemProperties", "true");
TracingContext tracingContext = getTestTracingContext(fs, true);
assertThrows(IOException.class, () -> fs.getAbfsStore()
- .setFilesystemProperties(properties, "FileSystemProperties", tracingContext));
+ .setFilesystemProperties(properties, tracingContext));
assertThrows(IOException.class,
() -> fs.getAbfsStore().getFilesystemProperties(tracingContext));
}
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestFileSystemProperties.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestFileSystemProperties.java
index 49446994ca922..3f2a4fe98802c 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestFileSystemProperties.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestFileSystemProperties.java
@@ -77,7 +77,7 @@ public void testBase64FileSystemProperties() throws Exception {
final Hashtable properties = new Hashtable<>();
properties.put("key", "{ value: value }");
TracingContext tracingContext = getTestTracingContext(fs, true);
- fs.getAbfsStore().setFilesystemProperties(properties, "key", tracingContext);
+ fs.getAbfsStore().setFilesystemProperties(properties, tracingContext);
Hashtable fetchedProperties = fs.getAbfsStore()
.getFilesystemProperties(tracingContext);
@@ -105,7 +105,7 @@ public void testBase64InvalidFileSystemProperties() throws Exception {
final Hashtable properties = new Hashtable<>();
properties.put("key", "{ value: value歲 }");
TracingContext tracingContext = getTestTracingContext(fs, true);
- fs.getAbfsStore().setFilesystemProperties(properties, "key", tracingContext);
+ fs.getAbfsStore().setFilesystemProperties(properties, tracingContext);
Hashtable fetchedProperties = fs.getAbfsStore()
.getFilesystemProperties(tracingContext);
@@ -133,7 +133,7 @@ public void testSetFileSystemProperties() throws Exception {
final Hashtable properties = new Hashtable<>();
properties.put("containerForDevTest", "true");
TracingContext tracingContext = getTestTracingContext(fs, true);
- fs.getAbfsStore().setFilesystemProperties(properties, "containerForDevTest", tracingContext);
+ fs.getAbfsStore().setFilesystemProperties(properties, tracingContext);
Hashtable fetchedProperties = fs.getAbfsStore()
.getFilesystemProperties(tracingContext);
From 538a2cdc159cd76a776c6750454e291ff49df24a Mon Sep 17 00:00:00 2001
From: Anmol Asrani
Date: Thu, 3 Jul 2025 00:27:14 -0700
Subject: [PATCH 04/25] Test changes
---
.../fs/azurebfs/services/AbfsBlobBlock.java | 10 +-
.../fs/azurebfs/services/AbfsDfsClient.java | 2 +-
.../services/AzureBlobBlockManager.java | 2 +-
.../services/AzureBlobIngressHandler.java | 3 +-
.../azurebfs/services/AzureBlockManager.java | 4 +-
.../services/AzureDFSIngressHandler.java | 39 ++++--
.../fs/azurebfs/services/RenameAtomicity.java | 13 +-
.../ITestAzureBlobFileSystemAppend.java | 9 +-
...ITestAzureBlobFileSystemMainOperation.java | 1 -
.../azurebfs/ITestWasbAbfsCompatibility.java | 117 +++++++++++-------
.../services/TestAbfsOutputStream.java | 8 +-
11 files changed, 127 insertions(+), 81 deletions(-)
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsBlobBlock.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsBlobBlock.java
index 325c9fbf10a00..9c76bcf663ced 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsBlobBlock.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsBlobBlock.java
@@ -36,6 +36,7 @@
public class AbfsBlobBlock extends AbfsBlock {
private final String blockId;
+ private final long blockIndex;
/**
* Gets the activeBlock and the blockId.
@@ -44,11 +45,12 @@ public class AbfsBlobBlock extends AbfsBlock {
* @param offset Used to generate blockId based on offset.
* @throws IOException exception is thrown.
*/
- AbfsBlobBlock(AbfsOutputStream outputStream, long offset, int blockIdLength) throws IOException {
+ AbfsBlobBlock(AbfsOutputStream outputStream, long offset, int blockIdLength, long blockIndex) throws IOException {
super(outputStream, offset);
+ this.blockIndex = blockIndex;
String streamId = getOutputStream().getStreamID();
UUID streamIdGuid = UUID.nameUUIDFromBytes(streamId.getBytes(StandardCharsets.UTF_8));
- this.blockId = generateBlockId(offset, streamIdGuid, blockIdLength);
+ this.blockId = generateBlockId(streamIdGuid, blockIdLength);
}
/**
@@ -59,14 +61,12 @@ public class AbfsBlobBlock extends AbfsBlock {
* optionally adjusted to match the specified raw length, padded or trimmed as needed, and
* then Base64-encoded.
*
- * @param position The byte position in the stream, used to compute the block index.
* @param streamId The UUID representing the stream, used as a prefix in the block ID.
* @param rawLength The desired length of the raw block ID string before Base64 encoding.
* If 0, no length adjustment is made.
* @return A Base64-encoded block ID string suitable for use in block-based storage APIs.
*/
- private String generateBlockId(long position, UUID streamId, int rawLength) {
- long blockIndex = position / getOutputStream().getBufferSize();
+ private String generateBlockId(UUID streamId, int rawLength) {
String rawBlockId = String.format("%s-%06d", streamId, blockIndex);
if (rawLength != 0) {
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsDfsClient.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsDfsClient.java
index 1e1fe32db0d88..d9f5a63796137 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsDfsClient.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsDfsClient.java
@@ -869,7 +869,7 @@ public AbfsRestOperation flush(final String path,
if (leaseId != null) {
requestHeaders.add(new AbfsHttpHeader(X_MS_LEASE_ID, leaseId));
}
- if (isChecksumValidationEnabled()) {
+ if (isChecksumValidationEnabled() && blobMd5 != null) {
requestHeaders.add(new AbfsHttpHeader(X_MS_BLOB_CONTENT_MD5, blobMd5));
}
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AzureBlobBlockManager.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AzureBlobBlockManager.java
index 652ac1b1be21c..06739dd462cca 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AzureBlobBlockManager.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AzureBlobBlockManager.java
@@ -84,7 +84,7 @@ protected synchronized AbfsBlock createBlockInternal(long position)
throws IOException {
if (getActiveBlock() == null) {
setBlockCount(getBlockCount() + 1);
- AbfsBlock activeBlock = new AbfsBlobBlock(getAbfsOutputStream(), position, blockIdLength);
+ AbfsBlock activeBlock = new AbfsBlobBlock(getAbfsOutputStream(), position, blockIdLength, getBlockCount());
activeBlock.setBlockEntry(addNewEntry(activeBlock.getBlockId(), activeBlock.getOffset()));
getAbfsOutputStream().getMessageDigest().reset();
setActiveBlock(activeBlock);
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AzureBlobIngressHandler.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AzureBlobIngressHandler.java
index d529f16fe07a4..05cdf3aa47d25 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AzureBlobIngressHandler.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AzureBlobIngressHandler.java
@@ -198,7 +198,6 @@ protected synchronized AbfsRestOperation remoteFlush(final long offset,
isClose, getAbfsOutputStream().getCachedSasTokenString(), leaseId,
getETag(), getAbfsOutputStream().getContextEncryptionAdapter(), tracingContextFlush, fullBlobMd5);
setETag(op.getResult().getResponseHeader(HttpHeaderConfigurations.ETAG));
- getAbfsOutputStream().getFullBlobContentMd5().reset();
} catch (AbfsRestOperationException ex) {
LOG.error("Error in remote flush requiring handler switch for path {}", getAbfsOutputStream().getPath(), ex);
if (shouldIngressHandlerBeSwitched(ex)) {
@@ -206,6 +205,8 @@ isClose, getAbfsOutputStream().getCachedSasTokenString(), leaseId,
}
LOG.error("Error in remote flush for path {} and offset {}", getAbfsOutputStream().getPath(), offset, ex);
throw ex;
+ } finally {
+ getAbfsOutputStream().getFullBlobContentMd5().reset();
}
return op;
}
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AzureBlockManager.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AzureBlockManager.java
index 426e0c8194f8a..73a65ab03ba63 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AzureBlockManager.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AzureBlockManager.java
@@ -125,7 +125,7 @@ protected DataBlocks.BlockFactory getBlockFactory() {
*
* @return the block count
*/
- protected long getBlockCount() {
+ public long getBlockCount() {
return blockCount;
}
@@ -134,7 +134,7 @@ protected long getBlockCount() {
*
* @param blockCount the count of blocks to set
*/
- public void setBlockCount(final long blockCount) {
+ protected void setBlockCount(final long blockCount) {
this.blockCount = blockCount;
}
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AzureDFSIngressHandler.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AzureDFSIngressHandler.java
index a5468c319ecc6..f27ce62af188c 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AzureDFSIngressHandler.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AzureDFSIngressHandler.java
@@ -25,6 +25,7 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException;
import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters;
import org.apache.hadoop.fs.azurebfs.utils.TracingContext;
import org.apache.hadoop.fs.store.DataBlocks;
@@ -180,15 +181,35 @@ protected synchronized AbfsRestOperation remoteFlush(final long offset,
tracingContextFlush.setIngressHandler(DFS_FLUSH);
tracingContextFlush.setPosition(String.valueOf(offset));
}
- String fullBlobMd5 = Base64.getEncoder().encodeToString(getAbfsOutputStream().getFullBlobContentMd5().digest());
- LOG.trace("Flushing data at offset {} and path {}", offset, getAbfsOutputStream().getPath());
- AbfsRestOperation op = getClient()
- .flush(getAbfsOutputStream().getPath(), offset, retainUncommitedData,
- isClose,
- getAbfsOutputStream().getCachedSasTokenString(), leaseId,
- getAbfsOutputStream().getContextEncryptionAdapter(),
- tracingContextFlush, fullBlobMd5);
- getAbfsOutputStream().getFullBlobContentMd5().reset();
+ byte[] digest = null;
+ String fullBlobMd5 = null;
+ try {
+ // Clone the MessageDigest to avoid resetting the original state
+ MessageDigest clonedMd5 = (MessageDigest) getAbfsOutputStream().getFullBlobContentMd5().clone();
+ digest = clonedMd5.digest();
+ } catch (CloneNotSupportedException e) {
+ LOG.warn("Failed to clone MessageDigest instance", e);
+ }
+ if (digest != null && digest.length != 0) {
+ fullBlobMd5 = Base64.getEncoder().encodeToString(digest);
+ }
+ LOG.trace("Flushing data at offset {} and path {}", offset,
+ getAbfsOutputStream().getPath());
+ AbfsRestOperation op;
+ try {
+ op = getClient()
+ .flush(getAbfsOutputStream().getPath(), offset, retainUncommitedData,
+ isClose,
+ getAbfsOutputStream().getCachedSasTokenString(), leaseId,
+ getAbfsOutputStream().getContextEncryptionAdapter(),
+ tracingContextFlush, fullBlobMd5);
+ } catch (AbfsRestOperationException ex) {
+ LOG.error("Error in remote flush for path {} and offset {}",
+ getAbfsOutputStream().getPath(), offset, ex);
+ throw ex;
+ } finally {
+ getAbfsOutputStream().getFullBlobContentMd5().reset();
+ }
return op;
}
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/RenameAtomicity.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/RenameAtomicity.java
index ad51faab1bbb5..fdec073e4fdfc 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/RenameAtomicity.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/RenameAtomicity.java
@@ -21,6 +21,7 @@
import java.io.IOException;
import java.nio.charset.Charset;
import java.nio.charset.StandardCharsets;
+import java.security.MessageDigest;
import java.util.Random;
import com.fasterxml.jackson.core.JsonProcessingException;
@@ -209,19 +210,21 @@ void createRenamePendingJson(Path path, byte[] bytes)
String eTag = extractEtagHeader(putBlobOp.getResult());
String blockId = generateBlockId();
+ String blockList = generateBlockListXml(blockId);
+ byte[] buffer = blockList.getBytes(StandardCharsets.UTF_8);
+ String computedMd5 = abfsClient.computeMD5Hash(buffer, 0, buffer.length);
+
AppendRequestParameters appendRequestParameters
= new AppendRequestParameters(0, 0,
bytes.length, AppendRequestParameters.Mode.APPEND_MODE, false, null,
abfsClient.getAbfsConfiguration().isExpectHeaderEnabled(),
- new BlobAppendRequestParameters(blockId, eTag), null);
+ new BlobAppendRequestParameters(blockId, eTag), computedMd5);
abfsClient.append(path.toUri().getPath(), bytes,
appendRequestParameters, null, null, tracingContext);
- String blockList = generateBlockListXml(blockId);
- // PutBlockList on the path.
- abfsClient.flush(blockList.getBytes(StandardCharsets.UTF_8),
- path.toUri().getPath(), true, null, null, eTag, null, tracingContext, null);
+ abfsClient.flush(buffer,
+ path.toUri().getPath(), true, null, null, eTag, null, tracingContext, computedMd5);
}
/**
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemAppend.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemAppend.java
index 9c9b550ed845b..9fe50d945a121 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemAppend.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemAppend.java
@@ -1042,13 +1042,12 @@ public void testWriteAsyncOpFailedAfterCloseCalled() throws Exception {
/**
* Helper method that generates blockId.
- * @param position The offset needed to generate blockId.
* @return String representing the block ID generated.
*/
- private String generateBlockId(AbfsOutputStream os, long position) {
+ private String generateBlockId(AbfsOutputStream os) {
String streamId = os.getStreamID();
UUID streamIdGuid = UUID.nameUUIDFromBytes(streamId.getBytes(StandardCharsets.UTF_8));
- long blockIndex = position / os.getBufferSize();
+ long blockIndex = os.getBlockManager().getBlockCount();
String rawBlockId = String.format("%s-%06d", streamIdGuid, blockIndex);
return Base64.encodeBase64String(rawBlockId.getBytes(StandardCharsets.UTF_8));
}
@@ -1093,7 +1092,7 @@ public void testFlushSuccessWithConnectionResetOnResponseValidMd5() throws Excep
new Random().nextBytes(bytes);
// Write some bytes and attempt to flush, which should retry
out.write(bytes);
- String blockId = generateBlockId(out, 0);
+ String blockId = generateBlockId(out);
String blockListXml = generateBlockListXml(blockId);
Mockito.doAnswer(answer -> {
@@ -1190,7 +1189,7 @@ public void testFlushSuccessWithConnectionResetOnResponseInvalidMd5() throws Exc
new Random().nextBytes(bytes);
// Write some bytes and attempt to flush, which should retry
out.write(bytes);
- String blockId = generateBlockId(out, 0);
+ String blockId = generateBlockId(out);
String blockListXml = generateBlockListXml(blockId);
Mockito.doAnswer(answer -> {
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemMainOperation.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemMainOperation.java
index a21027a7a2394..9851d09b3609c 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemMainOperation.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemMainOperation.java
@@ -28,7 +28,6 @@
/**
* Test AzureBlobFileSystem main operations.
* */
-@Ignore
public class ITestAzureBlobFileSystemMainOperation extends FSMainOperationsBaseTest {
private static final String TEST_ROOT_DIR =
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestWasbAbfsCompatibility.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestWasbAbfsCompatibility.java
index 1fb0a00078854..28f7cd1ca1520 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestWasbAbfsCompatibility.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestWasbAbfsCompatibility.java
@@ -164,6 +164,7 @@ public void testwriteFile() throws Exception {
AzureBlobFileSystem abfs = getFileSystem();
Assume.assumeFalse("Namespace enabled account does not support this test",
getIsNamespaceEnabled(abfs));
+ assumeBlobServiceType();
NativeAzureFileSystem wasb = getWasbFileSystem();
Path testFile = path("/testReadFile");
@@ -199,6 +200,8 @@ public void testwriteFile1() throws Exception {
AzureBlobFileSystem abfs = getFileSystem();
Assume.assumeFalse("Namespace enabled account does not support this test",
getIsNamespaceEnabled(abfs));
+ assumeBlobServiceType();
+ Assume.assumeFalse("Not valid for APPEND BLOB", isAppendBlobEnabled());
NativeAzureFileSystem wasb = getWasbFileSystem();
Path testFile = path("/testReadFile");
@@ -234,6 +237,7 @@ public void testazcopywasbcompatibility() throws Exception {
AzureBlobFileSystem abfs = getFileSystem();
Assume.assumeFalse("Namespace enabled account does not support this test",
getIsNamespaceEnabled(abfs));
+ Assume.assumeFalse("Not valid for APPEND BLOB", isAppendBlobEnabled());
Path testFile = path("/testReadFile");
Path path = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
@@ -360,6 +364,8 @@ public void testScenario2() throws Exception {
AzureBlobFileSystem abfs = getFileSystem();
Assume.assumeFalse("Namespace enabled account does not support this test",
getIsNamespaceEnabled(abfs));
+ assumeBlobServiceType();
+ Assume.assumeFalse("Not valid for APPEND BLOB", isAppendBlobEnabled());
NativeAzureFileSystem wasb = getWasbFileSystem();
Path testFile = path("/testReadFile");
@@ -398,6 +404,7 @@ public void testScenario3() throws Exception {
AzureBlobFileSystem abfs = getFileSystem();
Assume.assumeFalse("Namespace enabled account does not support this test",
getIsNamespaceEnabled(abfs));
+ Assume.assumeFalse("Not valid for APPEND BLOB", isAppendBlobEnabled());
NativeAzureFileSystem wasb = getWasbFileSystem();
Path testFile = path("/testReadFile");
@@ -429,6 +436,8 @@ public void testScenario4() throws Exception {
AzureBlobFileSystem abfs = getFileSystem();
Assume.assumeFalse("Namespace enabled account does not support this test",
getIsNamespaceEnabled(abfs));
+ assumeBlobServiceType();
+ Assume.assumeFalse("Not valid for APPEND BLOB", isAppendBlobEnabled());
NativeAzureFileSystem wasb = getWasbFileSystem();
Path testFile = path("/testReadFile");
@@ -463,6 +472,8 @@ public void testScenario5() throws Exception {
AzureBlobFileSystem abfs = (AzureBlobFileSystem) fileSystem;
Assume.assumeFalse("Namespace enabled account does not support this test",
getIsNamespaceEnabled(abfs));
+ assumeBlobServiceType();
+ Assume.assumeFalse("Not valid for APPEND BLOB", isAppendBlobEnabled());
NativeAzureFileSystem wasb = getWasbFileSystem();
Path testFile = path("/testReadFile");
@@ -499,6 +510,8 @@ public void testScenario6() throws Exception {
AzureBlobFileSystem abfs = (AzureBlobFileSystem) fileSystem;
Assume.assumeFalse("Namespace enabled account does not support this test",
getIsNamespaceEnabled(abfs));
+ Assume.assumeFalse("Not valid for APPEND BLOB", isAppendBlobEnabled());
+ assumeBlobServiceType();
NativeAzureFileSystem wasb = getWasbFileSystem();
Path testFile = path("/testReadFile");
@@ -535,6 +548,7 @@ public void testScenario7() throws Exception {
AzureBlobFileSystem abfs = (AzureBlobFileSystem) fileSystem;
Assume.assumeFalse("Namespace enabled account does not support this test",
getIsNamespaceEnabled(abfs));
+ Assume.assumeFalse("Not valid for APPEND BLOB", isAppendBlobEnabled());
NativeAzureFileSystem wasb = getWasbFileSystem();
Path testFile = path("/testReadFile");
@@ -570,6 +584,7 @@ public void testScenario8() throws Exception {
AzureBlobFileSystem abfs = (AzureBlobFileSystem) fileSystem;
Assume.assumeFalse("Namespace enabled account does not support this test",
getIsNamespaceEnabled(abfs));
+ Assume.assumeFalse("Not valid for APPEND BLOB", isAppendBlobEnabled());
NativeAzureFileSystem wasb = getWasbFileSystem();
Path testFile = path("/testReadFile");
@@ -609,6 +624,8 @@ public void testScenario9() throws Exception {
AzureBlobFileSystem abfs = (AzureBlobFileSystem) fileSystem;
Assume.assumeFalse("Namespace enabled account does not support this test",
getIsNamespaceEnabled(abfs));
+ assumeBlobServiceType();
+ Assume.assumeFalse("Not valid for APPEND BLOB", isAppendBlobEnabled());
NativeAzureFileSystem wasb = getWasbFileSystem();
Path testFile = path("/testReadFile");
@@ -683,6 +700,8 @@ public void testScenario11() throws Exception {
AzureBlobFileSystem abfs = (AzureBlobFileSystem) fileSystem;
Assume.assumeFalse("Namespace enabled account does not support this test",
getIsNamespaceEnabled(abfs));
+ assumeBlobServiceType();
+ Assume.assumeFalse("Not valid for APPEND BLOB", isAppendBlobEnabled());
NativeAzureFileSystem wasb = getWasbFileSystem();
Path testFile = path("/testReadFile");
@@ -750,6 +769,8 @@ public void testScenario13() throws Exception {
AzureBlobFileSystem abfs = (AzureBlobFileSystem) fileSystem;
Assume.assumeFalse("Namespace enabled account does not support this test",
getIsNamespaceEnabled(abfs));
+ assumeBlobServiceType();
+ Assume.assumeFalse("Not valid for APPEND BLOB", isAppendBlobEnabled());
NativeAzureFileSystem wasb = getWasbFileSystem();
Path testFile = path("/testReadFile");
@@ -784,6 +805,8 @@ public void testScenario14() throws Exception {
AzureBlobFileSystem abfs = (AzureBlobFileSystem) fileSystem;
Assume.assumeFalse("Namespace enabled account does not support this test",
getIsNamespaceEnabled(abfs));
+ assumeBlobServiceType();
+ Assume.assumeFalse("Not valid for APPEND BLOB", isAppendBlobEnabled());
NativeAzureFileSystem wasb = getWasbFileSystem();
Path testFile = path("/testReadFile");
@@ -851,6 +874,8 @@ public void testScenario16() throws Exception {
AzureBlobFileSystem abfs = (AzureBlobFileSystem) fileSystem;
Assume.assumeFalse("Namespace enabled account does not support this test",
getIsNamespaceEnabled(abfs));
+ assumeBlobServiceType();
+ Assume.assumeFalse("Not valid for APPEND BLOB", isAppendBlobEnabled());
NativeAzureFileSystem wasb = getWasbFileSystem();
Path testFile = path("/testReadFile");
@@ -903,8 +928,6 @@ public void testScenario17() throws Exception {
// --- SET XATTR #1 ---
abfs.setXAttr(path, ATTRIBUTE_NAME_1, ATTRIBUTE_VALUE_1);
byte[] readValue = abfs.getXAttr(path, ATTRIBUTE_NAME_1);
- System.out.println("XAttr raw bytes: " + Arrays.toString(readValue));
- System.out.println("XAttr as string: " + new String(readValue, StandardCharsets.UTF_8));
assertAttributeEqual(readValue, ATTRIBUTE_VALUE_1, "one");
// --- SET XATTR #2 ---
@@ -946,8 +969,6 @@ public void testScenario18() throws Exception {
// --- SET XATTR #1 ---
wasb.setXAttr(path, ATTRIBUTE_NAME_1, ATTRIBUTE_VALUE_1);
byte[] readValue = wasb.getXAttr(path, ATTRIBUTE_NAME_1);
- System.out.println("XAttr raw bytes: " + Arrays.toString(readValue));
- System.out.println("XAttr as string: " + new String(readValue, StandardCharsets.UTF_8));
assertAttributeEqual(readValue, ATTRIBUTE_VALUE_1, "one");
// --- SET XATTR #2 ---
@@ -989,8 +1010,6 @@ public void testScenario19() throws Exception {
// --- SET XATTR #1 ---
wasb.setXAttr(path, ATTRIBUTE_NAME_1, ATTRIBUTE_VALUE_1);
byte[] readValue = abfs.getXAttr(path, ATTRIBUTE_NAME_1);
- System.out.println("XAttr raw bytes: " + Arrays.toString(readValue));
- System.out.println("XAttr as string: " + new String(readValue, StandardCharsets.UTF_8));
assertAttributeEqual(readValue, ATTRIBUTE_VALUE_1, "one");
// --- SET XATTR #2 ---
@@ -1014,6 +1033,7 @@ public void testScenario20() throws Exception {
AzureBlobFileSystem abfs = (AzureBlobFileSystem) fileSystem;
Assume.assumeFalse("Namespace enabled account does not support this test",
getIsNamespaceEnabled(abfs));
+ Assume.assumeFalse("Not valid for APPEND BLOB", isAppendBlobEnabled());
NativeAzureFileSystem wasb = getWasbFileSystem();
Path testFile = path("/testReadFile");
@@ -1032,8 +1052,6 @@ public void testScenario20() throws Exception {
// --- SET XATTR #1 ---
wasb.setXAttr(path, ATTRIBUTE_NAME_1, ATTRIBUTE_VALUE_1);
byte[] readValue = abfs.getXAttr(path, ATTRIBUTE_NAME_1);
- System.out.println("XAttr raw bytes: " + Arrays.toString(readValue));
- System.out.println("XAttr as string: " + new String(readValue, StandardCharsets.UTF_8));
assertAttributeEqual(readValue, ATTRIBUTE_VALUE_1, "one");
// --- SET XATTR #2 ---
@@ -1058,6 +1076,8 @@ public void testScenario21() throws Exception {
AzureBlobFileSystem abfs = (AzureBlobFileSystem) fileSystem;
Assume.assumeFalse("Namespace enabled account does not support this test",
getIsNamespaceEnabled(abfs));
+ assumeBlobServiceType();
+ Assume.assumeFalse("Not valid for APPEND BLOB", isAppendBlobEnabled());
NativeAzureFileSystem wasb = getWasbFileSystem();
Path testFile = path("/testReadFile");
@@ -1076,8 +1096,6 @@ public void testScenario21() throws Exception {
// --- SET XATTR #1 ---
abfs.setXAttr(path, ATTRIBUTE_NAME_1, ATTRIBUTE_VALUE_1);
byte[] readValue = wasb.getXAttr(path, ATTRIBUTE_NAME_1);
- System.out.println("XAttr raw bytes: " + Arrays.toString(readValue));
- System.out.println("XAttr as string: " + new String(readValue, StandardCharsets.UTF_8));
assertAttributeEqual(readValue, ATTRIBUTE_VALUE_1, "one");
// --- SET XATTR #2 ---
@@ -1120,8 +1138,6 @@ public void testScenario22() throws Exception {
// --- SET XATTR #1 ---
abfs.setXAttr(path, ATTRIBUTE_NAME_1, ATTRIBUTE_VALUE_1);
byte[] readValue = wasb.getXAttr(path, ATTRIBUTE_NAME_1);
- System.out.println("XAttr raw bytes: " + Arrays.toString(readValue));
- System.out.println("XAttr as string: " + new String(readValue, StandardCharsets.UTF_8));
assertAttributeEqual(readValue, ATTRIBUTE_VALUE_1, "one");
// --- SET XATTR #2 ---
@@ -1163,8 +1179,6 @@ public void testScenario23() throws Exception {
// --- SET XATTR #1 ---
abfs.setXAttr(path, ATTRIBUTE_NAME_1, ATTRIBUTE_VALUE_1);
byte[] readValue = abfs.getXAttr(path, ATTRIBUTE_NAME_1);
- System.out.println("XAttr raw bytes: " + Arrays.toString(readValue));
- System.out.println("XAttr as string: " + new String(readValue, StandardCharsets.UTF_8));
assertAttributeEqual(readValue, ATTRIBUTE_VALUE_1, "one");
// --- SET XATTR #2 ---
@@ -1188,6 +1202,7 @@ public void testScenario24() throws Exception {
AzureBlobFileSystem abfs = (AzureBlobFileSystem) fileSystem;
Assume.assumeFalse("Namespace enabled account does not support this test",
getIsNamespaceEnabled(abfs));
+ Assume.assumeFalse("Not valid for APPEND BLOB", isAppendBlobEnabled());
NativeAzureFileSystem wasb = getWasbFileSystem();
Path testFile = path("/testReadFile");
@@ -1206,8 +1221,6 @@ public void testScenario24() throws Exception {
// --- SET XATTR #1 ---
wasb.setXAttr(path, ATTRIBUTE_NAME_1, ATTRIBUTE_VALUE_1);
byte[] readValue = abfs.getXAttr(path, ATTRIBUTE_NAME_1);
- System.out.println("XAttr raw bytes: " + Arrays.toString(readValue));
- System.out.println("XAttr as string: " + new String(readValue, StandardCharsets.UTF_8));
assertAttributeEqual(readValue, ATTRIBUTE_VALUE_1, "one");
// --- SET XATTR #2 ---
@@ -1249,8 +1262,6 @@ public void testScenario25() throws Exception {
// --- SET XATTR #1 ---
abfs.setXAttr(path, ATTRIBUTE_NAME_1, ATTRIBUTE_VALUE_1);
byte[] readValue = abfs.getXAttr(path, ATTRIBUTE_NAME_1);
- System.out.println("XAttr raw bytes: " + Arrays.toString(readValue));
- System.out.println("XAttr as string: " + new String(readValue, StandardCharsets.UTF_8));
assertAttributeEqual(readValue, ATTRIBUTE_VALUE_1, "one");
// --- SET XATTR #2 ---
@@ -1274,6 +1285,7 @@ public void testScenario26() throws Exception {
AzureBlobFileSystem abfs = (AzureBlobFileSystem) fileSystem;
Assume.assumeFalse("Namespace enabled account does not support this test",
getIsNamespaceEnabled(abfs));
+ Assume.assumeFalse("Not valid for APPEND BLOB", isAppendBlobEnabled());
NativeAzureFileSystem wasb = getWasbFileSystem();
Path testFile = path("/testReadFile");
@@ -1292,8 +1304,6 @@ public void testScenario26() throws Exception {
// --- SET XATTR #1 ---
wasb.setXAttr(path, ATTRIBUTE_NAME_1, ATTRIBUTE_VALUE_1);
byte[] readValue = wasb.getXAttr(path, ATTRIBUTE_NAME_1);
- System.out.println("XAttr raw bytes: " + Arrays.toString(readValue));
- System.out.println("XAttr as string: " + new String(readValue, StandardCharsets.UTF_8));
assertAttributeEqual(readValue, ATTRIBUTE_VALUE_1, "one");
// --- SET XATTR #2 ---
@@ -1317,6 +1327,7 @@ public void testScenario27() throws Exception {
AzureBlobFileSystem abfs = (AzureBlobFileSystem) fileSystem;
Assume.assumeFalse("Namespace enabled account does not support this test",
getIsNamespaceEnabled(abfs));
+ Assume.assumeFalse("Not valid for APPEND BLOB", isAppendBlobEnabled());
NativeAzureFileSystem wasb = getWasbFileSystem();
Path testFile = path("/testReadFile");
@@ -1347,11 +1358,10 @@ public void testScenario27() throws Exception {
// --- LIST FILES IN DIRECTORY ---
Path parentDir = new Path(testFile + "/~12/!008");
- FileStatus[] fileStatuses = wasb.listStatus(parentDir);
-
- for (FileStatus status : fileStatuses) {
- System.out.println("File: " + status.getPath());
- }
+ int noOfFiles = listAllFilesAndDirs(wasb, parentDir);
+ Assertions.assertThat(noOfFiles)
+ .as("Expected only 1 file or directory under path: %s", parentDir)
+ .isEqualTo(1);
wasb.delete(testPath2, true);
}
@@ -1394,11 +1404,10 @@ public void testScenario28() throws Exception {
// --- LIST FILES IN DIRECTORY ---
Path parentDir = new Path(testFile + "/~12/!008");
- FileStatus[] fileStatuses = abfs.listStatus(parentDir);
-
- for (FileStatus status : fileStatuses) {
- System.out.println("File: " + status.getPath());
- }
+ int noOfFiles = listAllFilesAndDirs(abfs, parentDir);
+ Assertions.assertThat(noOfFiles)
+ .as("Expected only 1 file or directory under path: %s", parentDir)
+ .isEqualTo(1);
wasb.delete(testPath2, true);
}
@@ -1411,6 +1420,8 @@ public void testScenario29() throws Exception {
AzureBlobFileSystem abfs = (AzureBlobFileSystem) fileSystem;
Assume.assumeFalse("Namespace enabled account does not support this test",
getIsNamespaceEnabled(abfs));
+ assumeBlobServiceType();
+ Assume.assumeFalse("Not valid for APPEND BLOB", isAppendBlobEnabled());
NativeAzureFileSystem wasb = getWasbFileSystem();
Path testFile = path("/testReadFile");
@@ -1442,11 +1453,10 @@ public void testScenario29() throws Exception {
// --- LIST FILES IN DIRECTORY ---
Path parentDir = new Path(testFile + "/~12/!008");
- FileStatus[] fileStatuses = abfs.listStatus(parentDir);
-
- for (FileStatus status : fileStatuses) {
- System.out.println("File: " + status.getPath());
- }
+ int noOfFiles = listAllFilesAndDirs(abfs, parentDir);
+ Assertions.assertThat(noOfFiles)
+ .as("Expected only 1 file or directory under path: %s", parentDir)
+ .isEqualTo(1);
wasb.delete(testPath2, true);
}
@@ -1496,11 +1506,10 @@ public void testScenario30() throws Exception {
// --- LIST FILES IN DIRECTORY ---
Path parentDir = new Path(testFile + "/~12/!008");
- FileStatus[] fileStatuses = abfs.listStatus(parentDir);
-
- for (FileStatus status : fileStatuses) {
- System.out.println("File: " + status.getPath());
- }
+ int noOfFiles = listAllFilesAndDirs(abfs, parentDir);
+ Assertions.assertThat(noOfFiles)
+ .as("Expected only 1 file or directory under path: %s", parentDir)
+ .isEqualTo(1);
wasb.delete(testPath3, true);
}
@@ -1586,7 +1595,10 @@ public void testScenario32() throws Exception {
.as("Rename failed")
.isTrue();
// --- LIST FILES IN DIRECTORY ---
- listAllFilesAndDirs(abfs, testFile1);
+ int listResult = listAllFilesAndDirs(abfs, testFile1);
+ Assertions.assertThat(listResult)
+ .as("Expected only 5 entries under path: %s", testFile1)
+ .isEqualTo(5);
}
//Scenario 33 :Create Dir & File via ABFS → Rename Dir via WASB → List Files via WASB
@@ -1598,6 +1610,7 @@ public void testScenario33() throws Exception {
AzureBlobFileSystem abfs = (AzureBlobFileSystem) fileSystem;
Assume.assumeFalse("Namespace enabled account does not support this test",
getIsNamespaceEnabled(abfs));
+ Assume.assumeFalse("Not valid for APPEND BLOB", isAppendBlobEnabled());
NativeAzureFileSystem wasb = getWasbFileSystem();
Path testFile = path("/testReadFile");
@@ -1631,7 +1644,10 @@ public void testScenario33() throws Exception {
.as("Rename failed")
.isTrue();
// --- LIST FILES IN DIRECTORY ---
- listAllFilesAndDirs(wasb, testFile1);
+ int listResult = listAllFilesAndDirs(wasb, testFile1);
+ Assertions.assertThat(listResult)
+ .as("Expected only 5 entries under path: %s", testFile1)
+ .isEqualTo(5);
}
//Scenario 34: Create Dir via ABFS → Rename File inside Dir via WASB → List via ABFS
@@ -1674,7 +1690,10 @@ public void testScenario34() throws Exception {
.as("Rename failed")
.isTrue();
// --- LIST FILES IN DIRECTORY ---
- listAllFilesAndDirs(abfs, testFile);
+ int listResult = listAllFilesAndDirs(abfs, testFile);
+ Assertions.assertThat(listResult)
+ .as("Expected only 4 entries under path: %s", testFile)
+ .isEqualTo(4);
}
//Scenario 35: Create Dir via WASB → Rename File inside Dir via ABFS → List via WASB
@@ -1717,7 +1736,10 @@ public void testScenario35() throws Exception {
.as("Rename failed")
.isTrue();
// --- LIST FILES IN DIRECTORY ---
- listAllFilesAndDirs(wasb, testFile);
+ int listResult = listAllFilesAndDirs(wasb, testFile);
+ Assertions.assertThat(listResult)
+ .as("Expected only 4 entries under path: %s", testFile)
+ .isEqualTo(4);
}
//Scenario 36: Create via WASB → Rename to existing name via ABFS → List via WASB
@@ -1828,8 +1850,6 @@ public void testScenario38() throws Exception {
// --- SET XATTR #1 ---
wasb.setXAttr(path, ATTRIBUTE_NAME_1, ATTRIBUTE_VALUE_1);
byte[] readValue = wasb.getXAttr(path, ATTRIBUTE_NAME_1);
- System.out.println("XAttr raw bytes: " + Arrays.toString(readValue));
- System.out.println("XAttr as string: " + new String(readValue, StandardCharsets.UTF_8));
assertAttributeEqual(readValue, ATTRIBUTE_VALUE_1, "one");
// --- SET XATTR #2 ---
@@ -1887,7 +1907,10 @@ public void testScenario39() throws Exception {
.as("Rename failed")
.isTrue();
// --- LIST FILES IN DIRECTORY ---
- listAllFilesAndDirs(wasb, testFile);
+ int listResult = listAllFilesAndDirs(wasb, testFile);
+ Assertions.assertThat(listResult)
+ .as("Expected only 4 entries under path: %s", testFile)
+ .isEqualTo(4);
}
/**
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsOutputStream.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsOutputStream.java
index e519d3889a919..4c8e668dc082c 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsOutputStream.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsOutputStream.java
@@ -27,11 +27,9 @@
import org.junit.Test;
import org.mockito.ArgumentCaptor;
-import org.mockito.Mock;
import org.mockito.Mockito;
import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.azurebfs.Abfs;
import org.apache.hadoop.fs.azurebfs.AbfsConfiguration;
import org.apache.hadoop.fs.azurebfs.constants.FSOperationType;
import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters;
@@ -243,10 +241,11 @@ public void verifyWriteRequest() throws Exception {
ArgumentCaptor acFlushRetainUnCommittedData = ArgumentCaptor.forClass(Boolean.class);
ArgumentCaptor acFlushClose = ArgumentCaptor.forClass(Boolean.class);
ArgumentCaptor acFlushSASToken = ArgumentCaptor.forClass(String.class);
+ ArgumentCaptor acMd5 = ArgumentCaptor.forClass(String.class);
verify(client, times(1)).flush(acFlushPath.capture(), acFlushPosition.capture(), acFlushRetainUnCommittedData.capture(), acFlushClose.capture(),
acFlushSASToken.capture(), isNull(), isNull(),
- acTracingContext.capture(), isNull());
+ acTracingContext.capture(), acMd5.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());
@@ -323,10 +322,11 @@ public void verifyWriteRequestOfBufferSizeAndClose() throws Exception {
ArgumentCaptor acFlushRetainUnCommittedData = ArgumentCaptor.forClass(Boolean.class);
ArgumentCaptor acFlushClose = ArgumentCaptor.forClass(Boolean.class);
ArgumentCaptor acFlushSASToken = ArgumentCaptor.forClass(String.class);
+ ArgumentCaptor acMd5 = ArgumentCaptor.forClass(String.class);
verify(client, times(1)).flush(acFlushPath.capture(), acFlushPosition.capture(), acFlushRetainUnCommittedData.capture(), acFlushClose.capture(),
acFlushSASToken.capture(), isNull(), isNull(),
- acTracingContext.capture(), isNull());
+ acTracingContext.capture(), acMd5.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());
From 2bd7ce016410df786afca114c4929500bb6e41ac Mon Sep 17 00:00:00 2001
From: Anmol Asrani
Date: Thu, 3 Jul 2025 01:01:01 -0700
Subject: [PATCH 05/25] remove unused imports
---
.../fs/azurebfs/services/AbfsBlobBlock.java | 4 ---
.../azurebfs/services/AbfsOutputStream.java | 26 ++++++++++++++-----
.../services/AzureBlobBlockManager.java | 11 +++++++-
.../services/AzureDFSIngressHandler.java | 3 +--
.../services/AzureIngressHandler.java | 1 -
.../fs/azurebfs/services/RenameAtomicity.java | 1 -
.../azurebfs/AbstractAbfsIntegrationTest.java | 4 ++-
.../ITestAzureBlobFileSystemAppend.java | 1 -
.../ITestAzureBlobFileSystemChecksum.java | 14 +++++-----
...ITestAzureBlobFileSystemMainOperation.java | 1 -
.../azurebfs/services/AbfsClientTestUtil.java | 14 ++++++----
11 files changed, 49 insertions(+), 31 deletions(-)
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsBlobBlock.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsBlobBlock.java
index 9c76bcf663ced..5939f049c11e1 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsBlobBlock.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsBlobBlock.java
@@ -24,9 +24,6 @@
import org.apache.commons.codec.binary.Base64;
-import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.BLOCK_ID_LENGTH;
-import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.ONE_MB;
-
/**
* Represents a block in Azure Blob Storage used by Azure Data Lake Storage (ADLS).
*
@@ -55,7 +52,6 @@ public class AbfsBlobBlock extends AbfsBlock {
/**
* Generates a Base64-encoded block ID string based on the given position, stream ID, and desired raw length.
- *
* The block ID is composed using the stream UUID and the block index, which is derived from
* the given position divided by the output stream's buffer size. The resulting string is
* optionally adjusted to match the specified raw length, padded or trimmed as needed, and
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java
index 62133c33899b3..1aa55aa46d944 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java
@@ -61,6 +61,7 @@
import org.apache.hadoop.fs.Syncable;
import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.APPEND_ACTION;
+import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.MD5;
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;
@@ -153,7 +154,17 @@ public class AbfsOutputStream extends OutputStream implements Syncable,
/** The handler for managing Abfs client operations. */
private final AbfsClientHandler clientHandler;
+ /**
+ * The `MessageDigest` instance used for computing the incremental MD5 hash
+ * of the data written so far. This is updated as data is written to the stream.
+ */
private final MessageDigest md5;
+
+ /**
+ * The `MessageDigest` instance used for computing the MD5 hash
+ * of the full blob content. This is updated with all data written to the stream
+ * and represents the complete MD5 checksum of the blob.
+ */
private final MessageDigest fullBlobContentMd5;
public AbfsOutputStream(AbfsOutputStreamContext abfsOutputStreamContext)
@@ -209,17 +220,15 @@ public AbfsOutputStream(AbfsOutputStreamContext abfsOutputStreamContext)
createIngressHandler(serviceTypeAtInit,
abfsOutputStreamContext.getBlockFactory(), bufferSize, false, null);
try {
- md5 = MessageDigest.getInstance("MD5");
- fullBlobContentMd5 = MessageDigest.getInstance("MD5");
+ md5 = MessageDigest.getInstance(MD5);
+ fullBlobContentMd5 = MessageDigest.getInstance(MD5);
} catch (NoSuchAlgorithmException e) {
- throw new IOException("MD5 algorithm not available", e);
+ if (client.isChecksumValidationEnabled()) {
+ throw new IOException("MD5 algorithm not available", e);
+ }
}
}
- public int getBufferSize() {
- return bufferSize;
- }
-
/**
* Retrieves the current ingress handler.
*
@@ -454,7 +463,10 @@ public synchronized void write(final byte[] data, final int off, final int lengt
AbfsBlock block = createBlockIfNeeded(position);
int written = bufferData(block, data, off, length);
+// Update the incremental MD5 hash with the written data.
getMessageDigest().update(data, off, written);
+
+// Update the full blob MD5 hash with the written data.
getFullBlobContentMd5().update(data, off, written);
int remainingCapacity = block.remainingCapacity();
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AzureBlobBlockManager.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AzureBlobBlockManager.java
index 06739dd462cca..09cf8edaa1ef9 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AzureBlobBlockManager.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AzureBlobBlockManager.java
@@ -72,6 +72,15 @@ public AzureBlobBlockManager(AbfsOutputStream abfsOutputStream,
abfsOutputStream.getStreamID(), abfsOutputStream.getPath());
}
+ /**
+ * Retrieves the length of the block ID.
+ *
+ * @return the length of the block ID in bytes.
+ */
+ public int getBlockIdLength() {
+ return blockIdLength;
+ }
+
/**
* Creates a new block.
*
@@ -84,7 +93,7 @@ protected synchronized AbfsBlock createBlockInternal(long position)
throws IOException {
if (getActiveBlock() == null) {
setBlockCount(getBlockCount() + 1);
- AbfsBlock activeBlock = new AbfsBlobBlock(getAbfsOutputStream(), position, blockIdLength, getBlockCount());
+ AbfsBlock activeBlock = new AbfsBlobBlock(getAbfsOutputStream(), position, getBlockIdLength(), getBlockCount());
activeBlock.setBlockEntry(addNewEntry(activeBlock.getBlockId(), activeBlock.getOffset()));
getAbfsOutputStream().getMessageDigest().reset();
setActiveBlock(activeBlock);
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AzureDFSIngressHandler.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AzureDFSIngressHandler.java
index f27ce62af188c..aca475ad0a64a 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AzureDFSIngressHandler.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AzureDFSIngressHandler.java
@@ -193,8 +193,7 @@ protected synchronized AbfsRestOperation remoteFlush(final long offset,
if (digest != null && digest.length != 0) {
fullBlobMd5 = Base64.getEncoder().encodeToString(digest);
}
- LOG.trace("Flushing data at offset {} and path {}", offset,
- getAbfsOutputStream().getPath());
+ LOG.trace("Flushing data at offset {} and path {}", offset, getAbfsOutputStream().getPath());
AbfsRestOperation op;
try {
op = getClient()
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AzureIngressHandler.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AzureIngressHandler.java
index 135e2f5c86ca0..3072bdf5d04d6 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AzureIngressHandler.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AzureIngressHandler.java
@@ -19,7 +19,6 @@
package org.apache.hadoop.fs.azurebfs.services;
import java.io.IOException;
-import java.security.MessageDigest;
import java.util.Objects;
import org.slf4j.Logger;
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/RenameAtomicity.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/RenameAtomicity.java
index fdec073e4fdfc..9460a1a812b22 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/RenameAtomicity.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/RenameAtomicity.java
@@ -21,7 +21,6 @@
import java.io.IOException;
import java.nio.charset.Charset;
import java.nio.charset.StandardCharsets;
-import java.security.MessageDigest;
import java.util.Random;
import com.fasterxml.jackson.core.JsonProcessingException;
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsIntegrationTest.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsIntegrationTest.java
index b03e2cf56327b..207f0a8c7e39e 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsIntegrationTest.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsIntegrationTest.java
@@ -64,8 +64,10 @@
import org.apache.hadoop.io.IOUtils;
import static org.apache.hadoop.fs.azure.AzureBlobStorageTestAccount.WASB_ACCOUNT_NAME_DOMAIN_SUFFIX;
+import static org.apache.hadoop.fs.azure.NativeAzureFileSystem.APPEND_SUPPORT_ENABLE_PROPERTY_NAME;
import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.COLON;
import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.FORWARD_SLASH;
+import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.TRUE;
import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.*;
import static org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes.ABFS_BLOB_DOMAIN_NAME;
import static org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes.ABFS_DFS_DOMAIN_NAME;
@@ -210,7 +212,7 @@ public void setup() throws Exception {
if (rawConfig.get(keyProperty) == null) {
rawConfig.set(keyProperty, getAccountKey());
}
- rawConfig.set("fs.azure.enable.append.support", "true");
+ rawConfig.set(APPEND_SUPPORT_ENABLE_PROPERTY_NAME, TRUE);
azureNativeFileSystemStore.initialize(
wasbUri,
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemAppend.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemAppend.java
index 9fe50d945a121..890063f0f26c1 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemAppend.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemAppend.java
@@ -78,7 +78,6 @@
import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_INFINITE_LEASE_KEY;
import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_INGRESS_SERVICE_TYPE;
import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_LEASE_THREADS;
-import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.BLOCK_ID_LENGTH;
import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.ONE_MB;
import static org.apache.hadoop.fs.azurebfs.services.AbfsBlobClient.generateBlockListXml;
import static org.apache.hadoop.fs.store.DataBlocks.DATA_BLOCKS_BUFFER_ARRAY;
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemChecksum.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemChecksum.java
index 48bd369bd7eac..c9fac985d47aa 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemChecksum.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemChecksum.java
@@ -18,20 +18,18 @@
package org.apache.hadoop.fs.azurebfs;
-import java.io.IOException;
import java.nio.charset.StandardCharsets;
import java.security.MessageDigest;
import java.security.NoSuchAlgorithmException;
import java.security.SecureRandom;
import java.util.Arrays;
+import java.util.Base64;
import java.util.HashSet;
-import org.apache.commons.codec.binary.Base64;
import org.apache.hadoop.fs.azurebfs.contracts.services.AzureServiceErrorCode;
import org.assertj.core.api.Assertions;
import org.junit.Assume;
import org.junit.Test;
-import org.mockito.Mock;
import org.mockito.Mockito;
import org.apache.hadoop.conf.Configuration;
@@ -46,6 +44,7 @@
import org.apache.hadoop.fs.azurebfs.services.AbfsOutputStream;
import org.apache.hadoop.fs.impl.OpenFileParameters;
+import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.MD5;
import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_BUFFERED_PREAD_DISABLE;
import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.BLOCK_ID_LENGTH;
import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.ONE_MB;
@@ -65,14 +64,15 @@ public class ITestAzureBlobFileSystemChecksum extends AbstractAbfsIntegrationTes
private static final int MB_15 = 15 * ONE_MB;
private static final int MB_16 = 16 * ONE_MB;
private static final String INVALID_MD5_TEXT = "Text for Invalid MD5 Computation";
- private final MessageDigest md;
+ private MessageDigest md = null;
public ITestAzureBlobFileSystemChecksum() throws Exception {
super();
try {
- md = MessageDigest.getInstance("MD5");
+ md = MessageDigest.getInstance(MD5);
} catch (NoSuchAlgorithmException e) {
- throw new IOException("MD5 algorithm not available", e);
+ // MD5 algorithm not available; md will remain null
+ // Log this in production code if needed
}
}
@@ -90,7 +90,7 @@ public String getMd5(byte[] data, int off, int length) {
byte[] digest = md.digest();
String md5 = null;
if (digest.length != 0) {
- md5 = java.util.Base64.getEncoder().encodeToString(digest);
+ md5 = Base64.getEncoder().encodeToString(digest);
}
return md5;
}
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemMainOperation.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemMainOperation.java
index 9851d09b3609c..9b96246f51d5f 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemMainOperation.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemMainOperation.java
@@ -22,7 +22,6 @@
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.azurebfs.contract.ABFSContractTestBinding;
-import org.junit.Ignore;
import org.junit.jupiter.api.Disabled;
/**
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/AbfsClientTestUtil.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/AbfsClientTestUtil.java
index d15e343b32497..49aa9c922c9e2 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/AbfsClientTestUtil.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/AbfsClientTestUtil.java
@@ -141,20 +141,24 @@ public static void setMockAbfsRestOperationForListOperation(
* @throws Exception If an error occurs while setting up the mock operation.
*/
public static void setMockAbfsRestOperationForFlushOperation(
- final AbfsClient spiedClient, String eTag, String blockListXml, AbfsOutputStream os, FunctionRaisingIOE functionRaisingIOE)
+ final AbfsClient spiedClient,
+ String eTag,
+ String blockListXml,
+ AbfsOutputStream os,
+ FunctionRaisingIOE functionRaisingIOE)
throws Exception {
- List requestHeaders = ITestAbfsClient.getTestRequestHeaders(spiedClient);
+ List requestHeaders = ITestAbfsClient.getTestRequestHeaders(
+ spiedClient);
String blobMd5 = null;
MessageDigest blobDigest = os.getFullBlobContentMd5();
-
if (blobDigest != null) {
try {
MessageDigest clonedMd5 = (MessageDigest) blobDigest.clone();
byte[] digest = clonedMd5.digest();
- if (digest.length != 0) {
+ if (digest != null && digest.length != 0) {
blobMd5 = Base64.getEncoder().encodeToString(digest);
}
- } catch (CloneNotSupportedException e) {
+ } catch (CloneNotSupportedException ignored) {
}
}
byte[] buffer = blockListXml.getBytes(StandardCharsets.UTF_8);
From 6f249072c93f20820d68bd43f4e6f03390ffe785 Mon Sep 17 00:00:00 2001
From: Anmol Asrani
Date: Thu, 3 Jul 2025 01:45:42 -0700
Subject: [PATCH 06/25] fix build issue
---
.../azurebfs/services/AbfsOutputStream.java | 4 ++--
.../ITestAzureBlobFileSystemChecksum.java | 20 ++++++++++---------
2 files changed, 13 insertions(+), 11 deletions(-)
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java
index 1aa55aa46d944..b96376498e93e 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java
@@ -158,14 +158,14 @@ public class AbfsOutputStream extends OutputStream implements Syncable,
* The `MessageDigest` instance used for computing the incremental MD5 hash
* of the data written so far. This is updated as data is written to the stream.
*/
- private final MessageDigest md5;
+ private MessageDigest md5 = null;
/**
* The `MessageDigest` instance used for computing the MD5 hash
* of the full blob content. This is updated with all data written to the stream
* and represents the complete MD5 checksum of the blob.
*/
- private final MessageDigest fullBlobContentMd5;
+ private MessageDigest fullBlobContentMd5 = null;
public AbfsOutputStream(AbfsOutputStreamContext abfsOutputStreamContext)
throws IOException {
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemChecksum.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemChecksum.java
index c9fac985d47aa..ec2f422f73f61 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemChecksum.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemChecksum.java
@@ -23,9 +23,9 @@
import java.security.NoSuchAlgorithmException;
import java.security.SecureRandom;
import java.util.Arrays;
-import java.util.Base64;
import java.util.HashSet;
+import org.apache.commons.codec.binary.Base64;
import org.apache.hadoop.fs.azurebfs.contracts.services.AzureServiceErrorCode;
import org.assertj.core.api.Assertions;
import org.junit.Assume;
@@ -85,15 +85,17 @@ public ITestAzureBlobFileSystemChecksum() throws Exception {
* @return The Base64-encoded MD5 checksum of the specified data, or null if the digest is empty.
* @throws IllegalArgumentException If the offset or length is invalid for the given byte array.
*/
-public String getMd5(byte[] data, int off, int length) {
- md.update(data, off, length);
- byte[] digest = md.digest();
- String md5 = null;
- if (digest.length != 0) {
- md5 = Base64.getEncoder().encodeToString(digest);
+ public String getMd5(byte[] data, int off, int length) {
+ String md5 = null;
+ if (md != null) {
+ md.update(data, off, length);
+ byte[] digest = md.digest();
+ if (digest.length != 0) {
+ md5 = Base64.encodeBase64String(digest);
+ }
+ }
+ return md5;
}
- return md5;
-}
@Test
public void testWriteReadWithChecksum() throws Exception {
From 8292192be43adff87a99987e83f64455e9d42207 Mon Sep 17 00:00:00 2001
From: Anmol Asrani
Date: Thu, 3 Jul 2025 01:55:26 -0700
Subject: [PATCH 07/25] fix main
---
.../fs/azurebfs/ITestAzureBlobFileSystemMainOperation.java | 4 ++++
1 file changed, 4 insertions(+)
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemMainOperation.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemMainOperation.java
index 6956dbcade26a..749db790715d5 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemMainOperation.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemMainOperation.java
@@ -21,6 +21,8 @@
import org.apache.hadoop.fs.FSMainOperationsBaseTest;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.azurebfs.contract.ABFSContractTestBinding;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Disabled;
/**
@@ -41,12 +43,14 @@ public ITestAzureBlobFileSystemMainOperation () throws Exception {
binding = new ABFSContractTestBinding(false);
}
+ @BeforeEach
@Override
public void setUp() throws Exception {
binding.setup();
fSys = binding.getFileSystem();
}
+ @AfterEach
@Override
public void tearDown() throws Exception {
// Note: Because "tearDown()" is called during the testing,
From f15c5e6f0e12208af133bf72b52ddb4750e3bfe2 Mon Sep 17 00:00:00 2001
From: Anmol Asrani
Date: Thu, 3 Jul 2025 04:34:46 -0700
Subject: [PATCH 08/25] fix javadocs
---
.../services/AppendRequestParameters.java | 2 ++
.../fs/azurebfs/services/AbfsClient.java | 24 ++++++++++---------
2 files changed, 15 insertions(+), 11 deletions(-)
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/AppendRequestParameters.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/AppendRequestParameters.java
index 84fbb0e91b88f..51866e8fab274 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/AppendRequestParameters.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/AppendRequestParameters.java
@@ -49,6 +49,7 @@ public enum Mode {
* @param isAppendBlob true if the blob is append-blob
* @param leaseId leaseId of the blob to be appended
* @param isExpectHeaderEnabled true if the expect header is enabled
+ * @param md5 The Base64-encoded MD5 hash of the block for data integrity validation.
*/
public AppendRequestParameters(final long position,
final int offset,
@@ -80,6 +81,7 @@ public AppendRequestParameters(final long position,
* @param leaseId leaseId of the blob to be appended
* @param isExpectHeaderEnabled true if the expect header is enabled
* @param blobParams parameters specific to append operation on Blob Endpoint.
+ * @param md5 The Base64-encoded MD5 hash of the block for data integrity validation.
*/
public AppendRequestParameters(final long position,
final int offset,
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java
index beb5471aa603f..0ce84526eb422 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java
@@ -878,6 +878,7 @@ public boolean appendSuccessCheckOp(AbfsRestOperation op, final String path,
* @param leaseId if there is an active lease on the path.
* @param contextEncryptionAdapter to provide encryption context.
* @param tracingContext for tracing the server calls.
+ * @param blobMd5 The Base64-encoded MD5 hash of the blob for data integrity validation.
* @return executed rest operation containing response from server.
* @throws AzureBlobFileSystemException if rest operation fails.
*/
@@ -888,17 +889,18 @@ public abstract AbfsRestOperation flush(String path, long position,
throws AzureBlobFileSystemException;
/**
- * Flush previously uploaded data to a file.
- * @param buffer containing blockIds to be flushed.
- * @param path on which data has to be flushed.
- * @param isClose specify if this is the last flush to the file.
- * @param cachedSasToken to be used for the authenticating operation.
- * @param leaseId if there is an active lease on the path.
- * @param eTag to specify conditional headers.
- * @param contextEncryptionAdapter to provide encryption context.
- * @param tracingContext for tracing the server calls.
- * @return executed rest operation containing response from server.
- * @throws AzureBlobFileSystemException if rest operation fails.
+ * Flushes previously uploaded data to the specified path.
+ * @param buffer The buffer containing block IDs to be flushed.
+ * @param path The file path to which data should be flushed.
+ * @param isClose True if this is the final flush (i.e., the file is being closed).
+ * @param cachedSasToken SAS token used for authentication (if applicable).
+ * @param leaseId Lease ID, if a lease is active on the file.
+ * @param eTag ETag used for conditional request headers (e.g., If-Match).
+ * @param contextEncryptionAdapter Adapter to provide encryption context, if encryption is enabled.
+ * @param tracingContext Context for tracing the server calls.
+ * @param blobMd5 The Base64-encoded MD5 hash of the blob for data integrity validation.
+ * @return The executed {@link AbfsRestOperation} containing the server response.
+ * @throws AzureBlobFileSystemException if the flush operation fails.
*/
public abstract AbfsRestOperation flush(byte[] buffer,
String path,
From 7f8c4659804ecf9bdcf62212a531a627c1ee3560 Mon Sep 17 00:00:00 2001
From: Anmol Asrani
Date: Thu, 3 Jul 2025 04:40:40 -0700
Subject: [PATCH 09/25] checktsyle fixes
---
.../hadoop-azure/src/config/checkstyle-suppressions.xml | 2 ++
.../org/apache/hadoop/fs/azurebfs/services/AbfsBlobClient.java | 2 +-
.../hadoop/fs/azurebfs/services/AzureBlobBlockManager.java | 2 +-
3 files changed, 4 insertions(+), 2 deletions(-)
diff --git a/hadoop-tools/hadoop-azure/src/config/checkstyle-suppressions.xml b/hadoop-tools/hadoop-azure/src/config/checkstyle-suppressions.xml
index 1cedf0c54b040..f18a2a6a1bc18 100644
--- a/hadoop-tools/hadoop-azure/src/config/checkstyle-suppressions.xml
+++ b/hadoop-tools/hadoop-azure/src/config/checkstyle-suppressions.xml
@@ -48,6 +48,8 @@
files="org[\\/]apache[\\/]hadoop[\\/]fs[\\/]azurebfs[\\/]services[\\/]AbfsClient.java"/>
+
blockEntryList = new LinkedList<>();
- int blockIdLength = 0;
+ private int blockIdLength = 0;
/**
* Constructs an AzureBlobBlockManager.
From c42ac75b8c56363ea6b4601253af70fbd42a97a5 Mon Sep 17 00:00:00 2001
From: Anmol Asrani
Date: Thu, 3 Jul 2025 10:58:25 -0700
Subject: [PATCH 10/25] fix test
---
.../apache/hadoop/fs/azurebfs/ITestWasbAbfsCompatibility.java | 1 +
1 file changed, 1 insertion(+)
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestWasbAbfsCompatibility.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestWasbAbfsCompatibility.java
index 28f7cd1ca1520..69abb1b542825 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestWasbAbfsCompatibility.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestWasbAbfsCompatibility.java
@@ -165,6 +165,7 @@ public void testwriteFile() throws Exception {
Assume.assumeFalse("Namespace enabled account does not support this test",
getIsNamespaceEnabled(abfs));
assumeBlobServiceType();
+ Assume.assumeFalse("Not valid for APPEND BLOB", isAppendBlobEnabled());
NativeAzureFileSystem wasb = getWasbFileSystem();
Path testFile = path("/testReadFile");
From d0f2aea3c8d0328130269aa78312db53e5b8b727 Mon Sep 17 00:00:00 2001
From: Anmol Asrani
Date: Wed, 9 Jul 2025 04:36:47 -0700
Subject: [PATCH 11/25] PR comments
---
.../constants/FileSystemConfigurations.java | 32 ++++++++++-
.../fs/azurebfs/services/AbfsBlobBlock.java | 14 +++--
.../fs/azurebfs/services/AbfsBlobClient.java | 18 +++----
.../fs/azurebfs/services/AbfsClient.java | 13 +++++
.../services/AzureBlobIngressHandler.java | 13 +----
.../services/AzureDFSIngressHandler.java | 13 +----
.../services/AzureIngressHandler.java | 54 ++++++++++++++++---
.../fs/azurebfs/services/RenameAtomicity.java | 2 +-
.../ITestAzureBlobFileSystemAppend.java | 6 +--
.../ITestAzureBlobFileSystemChecksum.java | 3 +-
10 files changed, 116 insertions(+), 52 deletions(-)
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/FileSystemConfigurations.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/FileSystemConfigurations.java
index ea982f8a35f34..2603d35baf8d0 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/FileSystemConfigurations.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/FileSystemConfigurations.java
@@ -137,7 +137,7 @@ public final class FileSystemConfigurations {
public static final boolean DEFAULT_ENABLE_ABFS_LIST_ITERATOR = true;
public static final boolean DEFAULT_ENABLE_ABFS_RENAME_RESILIENCE = true;
public static final boolean DEFAULT_ENABLE_PAGINATED_DELETE = false;
- public static final boolean DEFAULT_ENABLE_ABFS_CHECKSUM_VALIDATION = false;
+ public static final boolean DEFAULT_ENABLE_ABFS_CHECKSUM_VALIDATION = true;
/**
* Limit of queued block upload operations before writes
@@ -150,6 +150,36 @@ public final class FileSystemConfigurations {
*/
public static final int BLOCK_ID_LENGTH = 60;
+ /**
+ * Format string for generating block IDs.
+ * Example: "%s-%06d" where %s is the stream ID and %06d is the block index.
+ */
+ public static final String BLOCK_ID_FORMAT = "%s-%06d";
+
+ /**
+ * Format string for padding block IDs.
+ * Example: "%-" specifies left alignment in the format string.
+ */
+ public static final String PADDING_FORMAT = "%-";
+
+ /**
+ * Suffix for string formatting.
+ * Example: "s" specifies the type as a string in the format string.
+ */
+ public static final String STRING_SUFFIX = "s";
+
+ /**
+ * Character used for padding spaces in block IDs.
+ * Example: ' ' represents a space character.
+ */
+ public static final char SPACE_CHARACTER = ' ';
+
+ /**
+ * Character used for padding block IDs.
+ * Example: '_' is used to replace spaces in padded block IDs.
+ */
+ public static final char PADDING_CHARACTER = '_';
+
/**
* Buffer blocks to disk.
* Capacity is limited to available disk space.
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsBlobBlock.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsBlobBlock.java
index 5939f049c11e1..5344bff865ea4 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsBlobBlock.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsBlobBlock.java
@@ -24,6 +24,12 @@
import org.apache.commons.codec.binary.Base64;
+import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.BLOCK_ID_FORMAT;
+import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.PADDING_CHARACTER;
+import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.PADDING_FORMAT;
+import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.SPACE_CHARACTER;
+import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.STRING_SUFFIX;
+
/**
* Represents a block in Azure Blob Storage used by Azure Data Lake Storage (ADLS).
*
@@ -45,7 +51,7 @@ public class AbfsBlobBlock extends AbfsBlock {
AbfsBlobBlock(AbfsOutputStream outputStream, long offset, int blockIdLength, long blockIndex) throws IOException {
super(outputStream, offset);
this.blockIndex = blockIndex;
- String streamId = getOutputStream().getStreamID();
+ String streamId = outputStream.getStreamID();
UUID streamIdGuid = UUID.nameUUIDFromBytes(streamId.getBytes(StandardCharsets.UTF_8));
this.blockId = generateBlockId(streamIdGuid, blockIdLength);
}
@@ -63,13 +69,13 @@ public class AbfsBlobBlock extends AbfsBlock {
* @return A Base64-encoded block ID string suitable for use in block-based storage APIs.
*/
private String generateBlockId(UUID streamId, int rawLength) {
- String rawBlockId = String.format("%s-%06d", streamId, blockIndex);
+ String rawBlockId = String.format(BLOCK_ID_FORMAT, streamId, blockIndex);
if (rawLength != 0) {
// Adjust to match expected decoded length
if (rawBlockId.length() < rawLength) {
- rawBlockId = String.format("%-" + rawLength + "s", rawBlockId)
- .replace(' ', '_');
+ rawBlockId = String.format(PADDING_FORMAT + rawLength + STRING_SUFFIX, rawBlockId)
+ .replace(SPACE_CHARACTER, PADDING_CHARACTER);
} else if (rawBlockId.length() > rawLength) {
rawBlockId = rawBlockId.substring(0, rawLength);
}
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsBlobClient.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsBlobClient.java
index 1a03f2d6278f5..1eb69a773ecc6 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsBlobClient.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsBlobClient.java
@@ -899,9 +899,7 @@ public AbfsRestOperation append(final String path,
requestHeaders.add(new AbfsHttpHeader(EXPECT, HUNDRED_CONTINUE));
}
if (isChecksumValidationEnabled()) {
- if (reqParams.getMd5() != null) {
- requestHeaders.add(new AbfsHttpHeader(CONTENT_MD5, reqParams.getMd5()));
- }
+ addCheckSumHeaderForWrite(requestHeaders, reqParams);
}
if (reqParams.isRetryDueToExpect()) {
String userAgentRetry = getUserAgent();
@@ -986,9 +984,7 @@ public AbfsRestOperation appendBlock(final String path,
requestHeaders.add(new AbfsHttpHeader(X_MS_LEASE_ID, requestParameters.getLeaseId()));
}
if (isChecksumValidationEnabled()) {
- if (requestParameters.getMd5() != null) {
- requestHeaders.add(new AbfsHttpHeader(CONTENT_MD5, requestParameters.getMd5()));
- }
+ addCheckSumHeaderForWrite(requestHeaders, requestParameters);
}
final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder();
abfsUriQueryBuilder.addQuery(QUERY_PARAM_COMP, APPEND_BLOCK);
@@ -1105,10 +1101,8 @@ public AbfsRestOperation flush(byte[] buffer,
AbfsRestOperation op1 = getPathStatus(path, true, tracingContext,
contextEncryptionAdapter);
String metadataMd5 = op1.getResult().getResponseHeader(CONTENT_MD5);
- if (blobMd5 != null) {
- if (!blobMd5.equals(metadataMd5)) {
- throw ex;
- }
+ if (blobMd5 != null && !blobMd5.equals(metadataMd5)) {
+ throw ex;
}
return op;
}
@@ -1924,6 +1918,8 @@ private List getMetadataHeadersList(final Hashtable requestHeaders,
+ final AppendRequestParameters reqParams) {
+ if (reqParams.getMd5() != null) {
+ requestHeaders.add(new AbfsHttpHeader(CONTENT_MD5, reqParams.getMd5()));
+ }
+ }
+
/**
* To verify the checksum information received from server for the data read.
* @param buffer stores the data received from server.
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AzureBlobIngressHandler.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AzureBlobIngressHandler.java
index 05cdf3aa47d25..60d2decae4f54 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AzureBlobIngressHandler.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AzureBlobIngressHandler.java
@@ -181,18 +181,7 @@ protected synchronized AbfsRestOperation remoteFlush(final long offset,
tracingContextFlush.setIngressHandler(BLOB_FLUSH);
tracingContextFlush.setPosition(String.valueOf(offset));
LOG.trace("Flushing data at offset {} for path {}", offset, getAbfsOutputStream().getPath());
- byte[] digest = null;
- String fullBlobMd5 = null;
- try {
- // Clone the MessageDigest to avoid resetting the original state
- MessageDigest clonedMd5 = (MessageDigest) getAbfsOutputStream().getFullBlobContentMd5().clone();
- digest = clonedMd5.digest();
- } catch (CloneNotSupportedException e) {
- LOG.warn("Failed to clone MessageDigest instance", e);
- }
- if (digest != null && digest.length != 0) {
- fullBlobMd5 = Base64.getEncoder().encodeToString(digest);
- }
+ String fullBlobMd5 = computeFullBlobMd5();
op = getClient().flush(blockListXml.getBytes(StandardCharsets.UTF_8),
getAbfsOutputStream().getPath(),
isClose, getAbfsOutputStream().getCachedSasTokenString(), leaseId,
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AzureDFSIngressHandler.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AzureDFSIngressHandler.java
index aca475ad0a64a..6a00157218ec0 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AzureDFSIngressHandler.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AzureDFSIngressHandler.java
@@ -181,18 +181,7 @@ protected synchronized AbfsRestOperation remoteFlush(final long offset,
tracingContextFlush.setIngressHandler(DFS_FLUSH);
tracingContextFlush.setPosition(String.valueOf(offset));
}
- byte[] digest = null;
- String fullBlobMd5 = null;
- try {
- // Clone the MessageDigest to avoid resetting the original state
- MessageDigest clonedMd5 = (MessageDigest) getAbfsOutputStream().getFullBlobContentMd5().clone();
- digest = clonedMd5.digest();
- } catch (CloneNotSupportedException e) {
- LOG.warn("Failed to clone MessageDigest instance", e);
- }
- if (digest != null && digest.length != 0) {
- fullBlobMd5 = Base64.getEncoder().encodeToString(digest);
- }
+ String fullBlobMd5 = computeFullBlobMd5();
LOG.trace("Flushing data at offset {} and path {}", offset, getAbfsOutputStream().getPath());
AbfsRestOperation op;
try {
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AzureIngressHandler.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AzureIngressHandler.java
index 3072bdf5d04d6..5d200cccb1647 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AzureIngressHandler.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AzureIngressHandler.java
@@ -19,6 +19,8 @@
package org.apache.hadoop.fs.azurebfs.services;
import java.io.IOException;
+import java.security.MessageDigest;
+import java.util.Base64;
import java.util.Objects;
import org.slf4j.Logger;
@@ -87,7 +89,9 @@ public void setAbfsOutputStream(final AbfsOutputStream abfsOutputStream) {
* @param data the data to buffer
* @param off the start offset in the data
* @param length the number of bytes to buffer
+ *
* @return the number of bytes buffered
+ *
* @throws IOException if an I/O error occurs
*/
protected abstract int bufferData(AbfsBlock block,
@@ -100,7 +104,9 @@ protected abstract int bufferData(AbfsBlock block,
* @param uploadData the data to upload
* @param reqParams the request parameters for the append operation
* @param tracingContext the tracing context
+ *
* @return the result of the REST operation
+ *
* @throws IOException if an I/O error occurs
*/
protected abstract AbfsRestOperation remoteWrite(AbfsBlock blockToUpload,
@@ -116,7 +122,9 @@ protected abstract AbfsRestOperation remoteWrite(AbfsBlock blockToUpload,
* @param isClose whether this is a close operation
* @param leaseId the lease ID
* @param tracingContext the tracing context
+ *
* @return the result of the REST operation
+ *
* @throws IOException if an I/O error occurs
*/
protected abstract AbfsRestOperation remoteFlush(long offset,
@@ -141,12 +149,14 @@ protected abstract void writeAppendBlobCurrentBufferToService()
* the data to be uploaded, the block of data, and additional parameters required for
* the append operation.
*
- * @param path The path of the append blob to which data is to be appended.
- * @param uploadData The data to be uploaded as part of the append operation.
- * @param block The block of data to append.
- * @param reqParams The additional parameters required for the append operation.
+ * @param path The path of the append blob to which data is to be appended.
+ * @param uploadData The data to be uploaded as part of the append operation.
+ * @param block The block of data to append.
+ * @param reqParams The additional parameters required for the append operation.
* @param tracingContext The tracing context for the operation.
+ *
* @return An {@link AbfsRestOperation} object representing the remote write operation.
+ *
* @throws IOException If an I/O error occurs during the append operation.
*/
protected abstract AbfsRestOperation remoteAppendBlobWrite(String path,
@@ -159,6 +169,7 @@ protected abstract AbfsRestOperation remoteAppendBlobWrite(String path,
* Determines if the ingress handler should be switched based on the given exception.
*
* @param ex the exception that occurred
+ *
* @return true if the ingress handler should be switched, false otherwise
*/
protected boolean shouldIngressHandlerBeSwitched(AbfsRestOperationException ex) {
@@ -168,8 +179,10 @@ protected boolean shouldIngressHandlerBeSwitched(AbfsRestOperationException ex)
String errorCode = ex.getErrorCode().getErrorCode();
if (errorCode != null) {
return ex.getStatusCode() == HTTP_CONFLICT
- && (Objects.equals(errorCode, AzureServiceErrorCode.BLOB_OPERATION_NOT_SUPPORTED.getErrorCode())
- || Objects.equals(errorCode, AzureServiceErrorCode.INVALID_APPEND_OPERATION.getErrorCode()));
+ && (Objects.equals(errorCode,
+ AzureServiceErrorCode.BLOB_OPERATION_NOT_SUPPORTED.getErrorCode())
+ || Objects.equals(errorCode,
+ AzureServiceErrorCode.INVALID_APPEND_OPERATION.getErrorCode()));
}
return false;
}
@@ -178,6 +191,7 @@ protected boolean shouldIngressHandlerBeSwitched(AbfsRestOperationException ex)
* Constructs an InvalidIngressServiceException that includes the current handler class name in the exception message.
*
* @param e the original AbfsRestOperationException that triggered this exception.
+ *
* @return an InvalidIngressServiceException with the status code, error code, original message, and handler class name.
*/
protected InvalidIngressServiceException getIngressHandlerSwitchException(
@@ -206,4 +220,32 @@ protected InvalidIngressServiceException getIngressHandlerSwitchException(
* @return the block manager
*/
public abstract AbfsClient getClient();
+
+ /**
+ * Computes the Base64-encoded MD5 hash of the full blob content.
+ *
+ * This method clones the current state of the {@link MessageDigest} instance
+ * associated with the blob content to avoid resetting its original state. It then
+ * calculates the MD5 digest and encodes it into a Base64 string.
+ *
+ * @return A Base64-encoded string representing the MD5 hash of the full blob content,
+ * or {@code null} if the digest could not be computed.
+ */
+ protected String computeFullBlobMd5() {
+ byte[] digest = null;
+ String fullBlobMd5 = null;
+ try {
+ // Clone the MessageDigest to avoid resetting the original state
+ MessageDigest clonedMd5
+ = (MessageDigest) getAbfsOutputStream().getFullBlobContentMd5()
+ .clone();
+ digest = clonedMd5.digest();
+ } catch (CloneNotSupportedException e) {
+ LOG.warn("Failed to clone MessageDigest instance", e);
+ }
+ if (digest != null && digest.length != 0) {
+ fullBlobMd5 = Base64.getEncoder().encodeToString(digest);
+ }
+ return fullBlobMd5;
+ }
}
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/RenameAtomicity.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/RenameAtomicity.java
index 9460a1a812b22..24f97238bde36 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/RenameAtomicity.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/RenameAtomicity.java
@@ -217,7 +217,7 @@ void createRenamePendingJson(Path path, byte[] bytes)
= new AppendRequestParameters(0, 0,
bytes.length, AppendRequestParameters.Mode.APPEND_MODE, false, null,
abfsClient.getAbfsConfiguration().isExpectHeaderEnabled(),
- new BlobAppendRequestParameters(blockId, eTag), computedMd5);
+ new BlobAppendRequestParameters(blockId, eTag), abfsClient.computeMD5Hash(bytes, 0, bytes.length));
abfsClient.append(path.toUri().getPath(), bytes,
appendRequestParameters, null, null, tracingContext);
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemAppend.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemAppend.java
index 890063f0f26c1..61f42c944a379 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemAppend.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemAppend.java
@@ -1130,7 +1130,7 @@ public void testFlushSuccessWithConnectionResetOnResponseValidMd5() throws Excep
Mockito.nullable(String.class),
Mockito.anyString(),
Mockito.nullable(ContextEncryptionAdapter.class),
- Mockito.any(TracingContext.class), Mockito.anyString()
+ Mockito.any(TracingContext.class), Mockito.nullable(String.class)
);
out.hsync();
@@ -1143,7 +1143,7 @@ public void testFlushSuccessWithConnectionResetOnResponseValidMd5() throws Excep
Mockito.nullable(String.class),
Mockito.anyString(),
Mockito.nullable(ContextEncryptionAdapter.class),
- Mockito.any(TracingContext.class), Mockito.anyString());
+ Mockito.any(TracingContext.class), Mockito.nullable(String.class));
}
}
@@ -1232,7 +1232,7 @@ public void testFlushSuccessWithConnectionResetOnResponseInvalidMd5() throws Exc
Mockito.nullable(String.class),
Mockito.anyString(),
Mockito.nullable(ContextEncryptionAdapter.class),
- Mockito.any(TracingContext.class), Mockito.anyString()
+ Mockito.any(TracingContext.class), Mockito.nullable(String.class)
);
FSDataOutputStream os1 = createMockedOutputStream(fs,
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemChecksum.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemChecksum.java
index ec2f422f73f61..6f5649eb606aa 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemChecksum.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemChecksum.java
@@ -72,7 +72,6 @@ public ITestAzureBlobFileSystemChecksum() throws Exception {
md = MessageDigest.getInstance(MD5);
} catch (NoSuchAlgorithmException e) {
// MD5 algorithm not available; md will remain null
- // Log this in production code if needed
}
}
@@ -115,7 +114,7 @@ public void testAppendWithChecksumAtDifferentOffsets() throws Exception {
byte[] data = generateRandomBytes(MB_4);
int pos = 0;
- pos += appendWithOffsetHelper(os, client, path, data, fs, pos, 0, getMd5(data, 0, data.length));
+ pos += appendWithOffsetHelper(os, client, path, data, fs, pos, 0, getMd5(data, 0, data.length));
pos += appendWithOffsetHelper(os, client, path, data, fs, pos, ONE_MB, getMd5(data, ONE_MB, data.length - ONE_MB));
pos += appendWithOffsetHelper(os, client, path, data, fs, pos, MB_2, getMd5(data, MB_2, data.length-MB_2));
appendWithOffsetHelper(os, client, path, data, fs, pos, MB_4 - 1, getMd5(data, MB_4 - 1, data.length - (MB_4 - 1)));
From 07c396cd54775d71627f210357b378b5938013f9 Mon Sep 17 00:00:00 2001
From: Anmol Asrani
Date: Wed, 9 Jul 2025 04:39:33 -0700
Subject: [PATCH 12/25] revert config change
---
.../hadoop/fs/azurebfs/constants/FileSystemConfigurations.java | 2 +-
1 file changed, 1 insertion(+), 1 deletion(-)
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/FileSystemConfigurations.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/FileSystemConfigurations.java
index 2603d35baf8d0..f7ccc0e9175b7 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/FileSystemConfigurations.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/FileSystemConfigurations.java
@@ -137,7 +137,7 @@ public final class FileSystemConfigurations {
public static final boolean DEFAULT_ENABLE_ABFS_LIST_ITERATOR = true;
public static final boolean DEFAULT_ENABLE_ABFS_RENAME_RESILIENCE = true;
public static final boolean DEFAULT_ENABLE_PAGINATED_DELETE = false;
- public static final boolean DEFAULT_ENABLE_ABFS_CHECKSUM_VALIDATION = true;
+ public static final boolean DEFAULT_ENABLE_ABFS_CHECKSUM_VALIDATION = false;
/**
* Limit of queued block upload operations before writes
From 5e6298a6dc7328fb1de1aa8e6a6202102d4d64a0 Mon Sep 17 00:00:00 2001
From: Anmol Asrani
Date: Wed, 9 Jul 2025 04:42:33 -0700
Subject: [PATCH 13/25] remove unintended changes
---
.../services/AzureIngressHandler.java | 24 +++++--------------
1 file changed, 6 insertions(+), 18 deletions(-)
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AzureIngressHandler.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AzureIngressHandler.java
index 5d200cccb1647..81007e1c3dd1d 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AzureIngressHandler.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AzureIngressHandler.java
@@ -89,9 +89,7 @@ public void setAbfsOutputStream(final AbfsOutputStream abfsOutputStream) {
* @param data the data to buffer
* @param off the start offset in the data
* @param length the number of bytes to buffer
- *
* @return the number of bytes buffered
- *
* @throws IOException if an I/O error occurs
*/
protected abstract int bufferData(AbfsBlock block,
@@ -104,9 +102,7 @@ protected abstract int bufferData(AbfsBlock block,
* @param uploadData the data to upload
* @param reqParams the request parameters for the append operation
* @param tracingContext the tracing context
- *
* @return the result of the REST operation
- *
* @throws IOException if an I/O error occurs
*/
protected abstract AbfsRestOperation remoteWrite(AbfsBlock blockToUpload,
@@ -122,9 +118,7 @@ protected abstract AbfsRestOperation remoteWrite(AbfsBlock blockToUpload,
* @param isClose whether this is a close operation
* @param leaseId the lease ID
* @param tracingContext the tracing context
- *
* @return the result of the REST operation
- *
* @throws IOException if an I/O error occurs
*/
protected abstract AbfsRestOperation remoteFlush(long offset,
@@ -149,14 +143,12 @@ protected abstract void writeAppendBlobCurrentBufferToService()
* the data to be uploaded, the block of data, and additional parameters required for
* the append operation.
*
- * @param path The path of the append blob to which data is to be appended.
- * @param uploadData The data to be uploaded as part of the append operation.
- * @param block The block of data to append.
- * @param reqParams The additional parameters required for the append operation.
+ * @param path The path of the append blob to which data is to be appended.
+ * @param uploadData The data to be uploaded as part of the append operation.
+ * @param block The block of data to append.
+ * @param reqParams The additional parameters required for the append operation.
* @param tracingContext The tracing context for the operation.
- *
* @return An {@link AbfsRestOperation} object representing the remote write operation.
- *
* @throws IOException If an I/O error occurs during the append operation.
*/
protected abstract AbfsRestOperation remoteAppendBlobWrite(String path,
@@ -169,7 +161,6 @@ protected abstract AbfsRestOperation remoteAppendBlobWrite(String path,
* Determines if the ingress handler should be switched based on the given exception.
*
* @param ex the exception that occurred
- *
* @return true if the ingress handler should be switched, false otherwise
*/
protected boolean shouldIngressHandlerBeSwitched(AbfsRestOperationException ex) {
@@ -179,10 +170,8 @@ protected boolean shouldIngressHandlerBeSwitched(AbfsRestOperationException ex)
String errorCode = ex.getErrorCode().getErrorCode();
if (errorCode != null) {
return ex.getStatusCode() == HTTP_CONFLICT
- && (Objects.equals(errorCode,
- AzureServiceErrorCode.BLOB_OPERATION_NOT_SUPPORTED.getErrorCode())
- || Objects.equals(errorCode,
- AzureServiceErrorCode.INVALID_APPEND_OPERATION.getErrorCode()));
+ && (Objects.equals(errorCode, AzureServiceErrorCode.BLOB_OPERATION_NOT_SUPPORTED.getErrorCode())
+ || Objects.equals(errorCode, AzureServiceErrorCode.INVALID_APPEND_OPERATION.getErrorCode()));
}
return false;
}
@@ -191,7 +180,6 @@ protected boolean shouldIngressHandlerBeSwitched(AbfsRestOperationException ex)
* Constructs an InvalidIngressServiceException that includes the current handler class name in the exception message.
*
* @param e the original AbfsRestOperationException that triggered this exception.
- *
* @return an InvalidIngressServiceException with the status code, error code, original message, and handler class name.
*/
protected InvalidIngressServiceException getIngressHandlerSwitchException(
From d951cc081529a99a69e4fcfb9c9523bbd700c21d Mon Sep 17 00:00:00 2001
From: Anmol Asrani
Date: Wed, 9 Jul 2025 04:47:37 -0700
Subject: [PATCH 14/25] fix comments
---
.../org/apache/hadoop/fs/azurebfs/services/AbfsDfsClient.java | 4 +---
.../apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java | 4 ++--
2 files changed, 3 insertions(+), 5 deletions(-)
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsDfsClient.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsDfsClient.java
index d9f5a63796137..8a96d22212a92 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsDfsClient.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsDfsClient.java
@@ -764,9 +764,7 @@ public AbfsRestOperation append(final String path,
// Add MD5 Hash of request content as request header if feature is enabled
if (isChecksumValidationEnabled()) {
- if (reqParams.getMd5() != null) {
- requestHeaders.add(new AbfsHttpHeader(CONTENT_MD5, reqParams.getMd5()));
- }
+ addCheckSumHeaderForWrite(requestHeaders, reqParams);
}
// AbfsInputStream/AbfsOutputStream reuse SAS tokens for better performance
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java
index b96376498e93e..10e7d9910c884 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java
@@ -463,10 +463,10 @@ public synchronized void write(final byte[] data, final int off, final int lengt
AbfsBlock block = createBlockIfNeeded(position);
int written = bufferData(block, data, off, length);
-// Update the incremental MD5 hash with the written data.
+ // Update the incremental MD5 hash with the written data.
getMessageDigest().update(data, off, written);
-// Update the full blob MD5 hash with the written data.
+ // Update the full blob MD5 hash with the written data.
getFullBlobContentMd5().update(data, off, written);
int remainingCapacity = block.remainingCapacity();
From 9ae0198739af5ba345b4fd85ecf8b43f1efb9222 Mon Sep 17 00:00:00 2001
From: Anmol Asrani
Date: Wed, 9 Jul 2025 10:25:15 -0700
Subject: [PATCH 15/25] PR comments
---
.../services/AppendRequestParameters.java | 2 +-
.../fs/azurebfs/services/AbfsBlobBlock.java | 19 +++++++++---------
.../fs/azurebfs/services/AbfsBlobClient.java | 4 +++-
.../fs/azurebfs/services/AbfsClient.java | 20 ++++++++++---------
.../fs/azurebfs/services/AbfsDfsClient.java | 16 +++++++++++++++
.../services/AzureBlobIngressHandler.java | 2 --
.../services/AzureDFSIngressHandler.java | 2 --
7 files changed, 41 insertions(+), 24 deletions(-)
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/AppendRequestParameters.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/AppendRequestParameters.java
index 51866e8fab274..b4b1afaa266f4 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/AppendRequestParameters.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/AppendRequestParameters.java
@@ -81,7 +81,7 @@ public AppendRequestParameters(final long position,
* @param leaseId leaseId of the blob to be appended
* @param isExpectHeaderEnabled true if the expect header is enabled
* @param blobParams parameters specific to append operation on Blob Endpoint.
- * @param md5 The Base64-encoded MD5 hash of the block for data integrity validation.
+ * @param md5 The Base64-encoded MD5 hash of the block for data integrity validation.
*/
public AppendRequestParameters(final long position,
final int offset,
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsBlobBlock.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsBlobBlock.java
index 5344bff865ea4..91a8a9a4dbf2e 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsBlobBlock.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsBlobBlock.java
@@ -46,6 +46,8 @@ public class AbfsBlobBlock extends AbfsBlock {
*
* @param outputStream AbfsOutputStream Instance.
* @param offset Used to generate blockId based on offset.
+ * @param blockIdLength the expected length of the generated block ID.
+ * @param blockIndex the index of the block; used in block ID generation.
* @throws IOException exception is thrown.
*/
AbfsBlobBlock(AbfsOutputStream outputStream, long offset, int blockIdLength, long blockIndex) throws IOException {
@@ -57,16 +59,15 @@ public class AbfsBlobBlock extends AbfsBlock {
}
/**
- * Generates a Base64-encoded block ID string based on the given position, stream ID, and desired raw length.
- * The block ID is composed using the stream UUID and the block index, which is derived from
- * the given position divided by the output stream's buffer size. The resulting string is
- * optionally adjusted to match the specified raw length, padded or trimmed as needed, and
- * then Base64-encoded.
+ * Generates a Base64-encoded block ID string using the given stream UUID and block index.
+ * The block ID is first created as a raw string using a format with the stream ID and block index.
+ * If a non-zero rawLength is provided, the raw block ID is padded or trimmed to match the length.
+ * The final string is then Base64-encoded and returned.
*
- * @param streamId The UUID representing the stream, used as a prefix in the block ID.
- * @param rawLength The desired length of the raw block ID string before Base64 encoding.
- * If 0, no length adjustment is made.
- * @return A Base64-encoded block ID string suitable for use in block-based storage APIs.
+ * @param streamId the UUID of the stream used to generate the block ID.
+ * @param rawLength the desired length of the raw block ID string before encoding.
+ * If 0, no length adjustment is done.
+ * @return the Base64-encoded block ID string.
*/
private String generateBlockId(UUID streamId, int rawLength) {
String rawBlockId = String.format(BLOCK_ID_FORMAT, streamId, blockIndex);
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsBlobClient.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsBlobClient.java
index 1eb69a773ecc6..9e8a792763858 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsBlobClient.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsBlobClient.java
@@ -984,7 +984,7 @@ public AbfsRestOperation appendBlock(final String path,
requestHeaders.add(new AbfsHttpHeader(X_MS_LEASE_ID, requestParameters.getLeaseId()));
}
if (isChecksumValidationEnabled()) {
- addCheckSumHeaderForWrite(requestHeaders, requestParameters);
+ addCheckSumHeaderForWrite(requestHeaders, requestParameters);
}
final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder();
abfsUriQueryBuilder.addQuery(QUERY_PARAM_COMP, APPEND_BLOCK);
@@ -1025,6 +1025,7 @@ public AbfsRestOperation appendBlock(final String path,
* @param leaseId if there is an active lease on the path.
* @param contextEncryptionAdapter to provide encryption context.
* @param tracingContext for tracing the server calls.
+ * @param blobMd5 the MD5 hash of the blob for integrity verification.
* @return exception as this operation is not supported on Blob Endpoint.
* @throws UnsupportedOperationException always.
*/
@@ -1053,6 +1054,7 @@ public AbfsRestOperation flush(final String path,
* @param eTag The etag of the blob.
* @param contextEncryptionAdapter to provide encryption context.
* @param tracingContext for tracing the service call.
+ * @param blobMd5 the MD5 hash of the blob for integrity verification.
* @return executed rest operation containing response from server.
* @throws AzureBlobFileSystemException if rest operation fails.
*/
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java
index ce7d9232c9eca..1e1ce504bb389 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java
@@ -890,16 +890,18 @@ public abstract AbfsRestOperation flush(String path, long position,
/**
* Flushes previously uploaded data to the specified path.
- * @param buffer The buffer containing block IDs to be flushed.
- * @param path The file path to which data should be flushed.
- * @param isClose True if this is the final flush (i.e., the file is being closed).
- * @param cachedSasToken SAS token used for authentication (if applicable).
- * @param leaseId Lease ID, if a lease is active on the file.
- * @param eTag ETag used for conditional request headers (e.g., If-Match).
+ *
+ * @param buffer The buffer containing block IDs to be flushed.
+ * @param path The file path to which data should be flushed.
+ * @param isClose True if this is the final flush (i.e., the file is being closed).
+ * @param cachedSasToken SAS token used for authentication (if applicable).
+ * @param leaseId Lease ID, if a lease is active on the file.
+ * @param eTag ETag used for conditional request headers (e.g., If-Match).
* @param contextEncryptionAdapter Adapter to provide encryption context, if encryption is enabled.
- * @param tracingContext Context for tracing the server calls.
- * @param blobMd5 The Base64-encoded MD5 hash of the blob for data integrity validation.
- * @return The executed {@link AbfsRestOperation} containing the server response.
+ * @param tracingContext Context for tracing the server calls.
+ * @param blobMd5 The Base64-encoded MD5 hash of the blob for data integrity validation.
+ * @return The executed {@link AbfsRestOperation} containing the server response.
+ *
* @throws AzureBlobFileSystemException if the flush operation fails.
*/
public abstract AbfsRestOperation flush(byte[] buffer,
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsDfsClient.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsDfsClient.java
index 8a96d22212a92..4335652e2528b 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsDfsClient.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsDfsClient.java
@@ -846,6 +846,7 @@ && appendSuccessCheckOp(op, path,
* @param leaseId if there is an active lease on the path.
* @param contextEncryptionAdapter to provide encryption context.
* @param tracingContext for tracing the server calls.
+ * @param blobMd5 the MD5 hash of the blob for integrity verification.
* @return executed rest operation containing response from server.
* @throws AzureBlobFileSystemException if rest operation fails.
*/
@@ -891,6 +892,21 @@ public AbfsRestOperation flush(final String path,
return op;
}
+ /**
+ * Flushes data to a file at the specified path, using the provided buffer and other parameters.
+ * This operation is not supported on the DFS endpoint and will throw an {@link UnsupportedOperationException}.
+ *
+ * @param buffer the byte array containing the data to be flushed to the file.
+ * @param path the path where the data has to be flushed.
+ * @param isClose whether this is the last flush operation to the file.
+ * @param cachedSasToken the SAS token to authenticate the operation.
+ * @param leaseId the lease ID, if an active lease exists on the path.
+ * @param eTag the ETag for concurrency control to ensure the flush is applied to the correct file version.
+ * @param contextEncryptionAdapter the adapter providing the encryption context.
+ * @param tracingContext the tracing context for tracking server calls.
+ * @param blobMd5 the MD5 hash of the blob for integrity verification.
+ * @throws UnsupportedOperationException if flush with blockIds is called on a DFS endpoint.
+ */
@Override
public AbfsRestOperation flush(byte[] buffer,
final String path,
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AzureBlobIngressHandler.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AzureBlobIngressHandler.java
index 60d2decae4f54..48dc04bdb32ab 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AzureBlobIngressHandler.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AzureBlobIngressHandler.java
@@ -20,8 +20,6 @@
import java.io.IOException;
import java.nio.charset.StandardCharsets;
-import java.security.MessageDigest;
-import java.util.Base64;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AzureDFSIngressHandler.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AzureDFSIngressHandler.java
index 6a00157218ec0..faf293b9242ce 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AzureDFSIngressHandler.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AzureDFSIngressHandler.java
@@ -19,8 +19,6 @@
package org.apache.hadoop.fs.azurebfs.services;
import java.io.IOException;
-import java.security.MessageDigest;
-import java.util.Base64;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
From 5f48139ca271a716f66d8b358a8eb07de79ff0c0 Mon Sep 17 00:00:00 2001
From: Anmol Asrani
Date: Wed, 9 Jul 2025 22:27:19 -0700
Subject: [PATCH 16/25] unused import
---
.../org/apache/hadoop/fs/azurebfs/services/AbfsDfsClient.java | 1 -
1 file changed, 1 deletion(-)
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsDfsClient.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsDfsClient.java
index 4335652e2528b..70e69fc1f7089 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsDfsClient.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsDfsClient.java
@@ -110,7 +110,6 @@
import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.TRUE;
import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.XMS_PROPERTIES_ENCODING_ASCII;
import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.ACCEPT;
-import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.CONTENT_MD5;
import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.EXPECT;
import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.IF_MATCH;
import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.IF_NONE_MATCH;
From 4b4b7a4db2deaa11362e04e490dc02961bc0cfeb Mon Sep 17 00:00:00 2001
From: Anmol Asrani
Date: Fri, 18 Jul 2025 00:26:07 -0700
Subject: [PATCH 17/25] PR review comments
---
.../services/AzureBlobIngressHandler.java | 1 +
.../azurebfs/services/AzureBlockManager.java | 2 +-
.../services/AzureDFSBlockManager.java | 2 +-
.../ITestAzureBlobFileSystemAppend.java | 20 +-
.../ITestAzureBlobFileSystemAttributes.java | 16 +-
.../ITestAzureBlobFileSystemChecksum.java | 38 +-
.../azurebfs/ITestWasbAbfsCompatibility.java | 405 ++++++++++++------
.../services/ITestAbfsOutputStream.java | 58 +++
.../services/TestAbfsOutputStream.java | 7 +-
9 files changed, 361 insertions(+), 188 deletions(-)
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AzureBlobIngressHandler.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AzureBlobIngressHandler.java
index 48dc04bdb32ab..eaa4ca41c781e 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AzureBlobIngressHandler.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AzureBlobIngressHandler.java
@@ -165,6 +165,7 @@ protected synchronized AbfsRestOperation remoteFlush(final long offset,
TracingContext tracingContext)
throws IOException {
AbfsRestOperation op;
+ AzureBlobBlockManager blobBlockManager = (AzureBlobBlockManager) getBlockManager();
if (getAbfsOutputStream().isAppendBlob()) {
return null;
}
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AzureBlockManager.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AzureBlockManager.java
index 73a65ab03ba63..3ebc09da0b4a1 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AzureBlockManager.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AzureBlockManager.java
@@ -89,7 +89,7 @@ protected abstract AbfsBlock createBlockInternal(long position)
*
* @return the active block
*/
- protected synchronized AbfsBlock getActiveBlock() {
+ public synchronized AbfsBlock getActiveBlock() {
return activeBlock;
}
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AzureDFSBlockManager.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AzureDFSBlockManager.java
index c2fc28afcae03..c58f1d1b28461 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AzureDFSBlockManager.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AzureDFSBlockManager.java
@@ -74,7 +74,7 @@ protected synchronized AbfsBlock createBlockInternal(long position)
* @return the active block
*/
@Override
- protected synchronized AbfsBlock getActiveBlock() {
+ public synchronized AbfsBlock getActiveBlock() {
return super.getActiveBlock();
}
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemAppend.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemAppend.java
index 61f42c944a379..9b7152233c15e 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemAppend.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemAppend.java
@@ -1039,18 +1039,6 @@ public void testWriteAsyncOpFailedAfterCloseCalled() throws Exception {
}
}
- /**
- * Helper method that generates blockId.
- * @return String representing the block ID generated.
- */
- private String generateBlockId(AbfsOutputStream os) {
- String streamId = os.getStreamID();
- UUID streamIdGuid = UUID.nameUUIDFromBytes(streamId.getBytes(StandardCharsets.UTF_8));
- long blockIndex = os.getBlockManager().getBlockCount();
- String rawBlockId = String.format("%s-%06d", streamIdGuid, blockIndex);
- return Base64.encodeBase64String(rawBlockId.getBytes(StandardCharsets.UTF_8));
- }
-
/**
* Test to simulate a successful flush operation followed by a connection reset
* on the response, triggering a retry.
@@ -1087,11 +1075,11 @@ public void testFlushSuccessWithConnectionResetOnResponseValidMd5() throws Excep
new Path("/test/file"), blobClient);
AbfsOutputStream out = (AbfsOutputStream) os.getWrappedStream();
String eTag = out.getIngressHandler().getETag();
- byte[] bytes = new byte[1024 * 1024 * 8];
+ byte[] bytes = new byte[1024 * 1024 * 4];
new Random().nextBytes(bytes);
// Write some bytes and attempt to flush, which should retry
out.write(bytes);
- String blockId = generateBlockId(out);
+ String blockId = out.getBlockManager().getActiveBlock().getBlockId();
String blockListXml = generateBlockListXml(blockId);
Mockito.doAnswer(answer -> {
@@ -1184,11 +1172,11 @@ public void testFlushSuccessWithConnectionResetOnResponseInvalidMd5() throws Exc
new Path("/test/file"), blobClient);
AbfsOutputStream out = (AbfsOutputStream) os.getWrappedStream();
String eTag = out.getIngressHandler().getETag();
- byte[] bytes = new byte[1024 * 1024 * 8];
+ byte[] bytes = new byte[1024 * 1024 * 4];
new Random().nextBytes(bytes);
// Write some bytes and attempt to flush, which should retry
out.write(bytes);
- String blockId = generateBlockId(out);
+ String blockId = out.getBlockManager().getActiveBlock().getBlockId();
String blockListXml = generateBlockListXml(blockId);
Mockito.doAnswer(answer -> {
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemAttributes.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemAttributes.java
index 3e0959d5dd6a0..b3f99651de4f4 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemAttributes.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemAttributes.java
@@ -67,7 +67,7 @@ public void testSetGetXAttrCreateReplace() throws Exception {
// after creating a file, it must be possible to create a new xAttr
touch(testFile);
fs.setXAttr(testFile, attributeName, attributeValue, CREATE_FLAG);
- assertAttributeEqual(fs.getXAttr(testFile, attributeName), attributeValue, decodedAttributeValue);
+ assertAttributeEqual(fs, fs.getXAttr(testFile, attributeName), attributeValue, decodedAttributeValue);
// however after the xAttr is created, creating it again must fail
intercept(IOException.class, () -> fs.setXAttr(testFile, attributeName, attributeValue, CREATE_FLAG));
@@ -92,7 +92,7 @@ public void testSetGetXAttrReplace() throws Exception {
// however after the xAttr is created, replacing it must succeed
fs.setXAttr(testFile, attributeName, attributeValue1, CREATE_FLAG);
fs.setXAttr(testFile, attributeName, attributeValue2, REPLACE_FLAG);
- assertAttributeEqual(fs.getXAttr(testFile, attributeName), attributeValue2,
+ assertAttributeEqual(fs, fs.getXAttr(testFile, attributeName), attributeValue2,
decodedAttribute2);
}
@@ -176,7 +176,7 @@ public void testSetXAttrMultipleOperations() throws Exception {
// Check if the attribute is retrievable
byte[] rv = fs.getXAttr(path, attributeName);
- assertAttributeEqual(rv, attributeValue, decodedAttributeValue);
+ assertAttributeEqual(fs, rv, attributeValue, decodedAttributeValue);
}
/**
@@ -220,7 +220,7 @@ private void testGetSetXAttrHelper(final AzureBlobFileSystem fs,
// Check if the attribute is retrievable
fs.setListenerOperation(FSOperationType.GET_ATTR);
byte[] rv = fs.getXAttr(testPath, attributeName1);
- assertAttributeEqual(rv, attributeValue1, decodedAttributeValue1);
+ assertAttributeEqual(fs, rv, attributeValue1, decodedAttributeValue1);
fs.registerListener(null);
// Set the second Attribute
@@ -228,10 +228,10 @@ private void testGetSetXAttrHelper(final AzureBlobFileSystem fs,
// Check all the attributes present and previous Attribute not overridden
rv = fs.getXAttr(testPath, attributeName1);
- assertAttributeEqual(rv, attributeValue1, decodedAttributeValue1);
+ assertAttributeEqual(fs, rv, attributeValue1, decodedAttributeValue1);
rv = fs.getXAttr(testPath, attributeName2);
- assertAttributeEqual(rv, attributeValue2, decodedAttributeValue2);
+ assertAttributeEqual(fs, rv, attributeValue2, decodedAttributeValue2);
}
private void assertAttributeNull(byte[] rv) {
@@ -240,12 +240,12 @@ private void assertAttributeNull(byte[] rv) {
.isNull();
}
- private void assertAttributeEqual(byte[] rv, byte[] attributeValue,
+ public static void assertAttributeEqual(AzureBlobFileSystem fs, byte[] rv, byte[] attributeValue,
String decodedAttributeValue) throws Exception {
Assertions.assertThat(rv)
.describedAs("Retrieved Attribute Does not Matches in Encoded Form")
.containsExactly(attributeValue);
- Assertions.assertThat(getFileSystem().getAbfsStore().decodeAttribute(rv))
+ Assertions.assertThat(fs.getAbfsStore().decodeAttribute(rv))
.describedAs("Retrieved Attribute Does not Matches in Decoded Form")
.isEqualTo(decodedAttributeValue);
}
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemChecksum.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemChecksum.java
index 6f5649eb606aa..41aaeaf37ef30 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemChecksum.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemChecksum.java
@@ -19,8 +19,6 @@
package org.apache.hadoop.fs.azurebfs;
import java.nio.charset.StandardCharsets;
-import java.security.MessageDigest;
-import java.security.NoSuchAlgorithmException;
import java.security.SecureRandom;
import java.util.Arrays;
import java.util.HashSet;
@@ -44,7 +42,6 @@
import org.apache.hadoop.fs.azurebfs.services.AbfsOutputStream;
import org.apache.hadoop.fs.impl.OpenFileParameters;
-import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.MD5;
import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_BUFFERED_PREAD_DISABLE;
import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.BLOCK_ID_LENGTH;
import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.ONE_MB;
@@ -64,36 +61,9 @@ public class ITestAzureBlobFileSystemChecksum extends AbstractAbfsIntegrationTes
private static final int MB_15 = 15 * ONE_MB;
private static final int MB_16 = 16 * ONE_MB;
private static final String INVALID_MD5_TEXT = "Text for Invalid MD5 Computation";
- private MessageDigest md = null;
public ITestAzureBlobFileSystemChecksum() throws Exception {
super();
- try {
- md = MessageDigest.getInstance(MD5);
- } catch (NoSuchAlgorithmException e) {
- // MD5 algorithm not available; md will remain null
- }
- }
-
- /**
- * Computes the MD5 checksum of a specified portion of the input byte array.
- *
- * @param data The byte array containing the data to compute the MD5 checksum for.
- * @param off The starting offset in the byte array.
- * @param length The number of bytes to include in the checksum computation.
- * @return The Base64-encoded MD5 checksum of the specified data, or null if the digest is empty.
- * @throws IllegalArgumentException If the offset or length is invalid for the given byte array.
- */
- public String getMd5(byte[] data, int off, int length) {
- String md5 = null;
- if (md != null) {
- md.update(data, off, length);
- byte[] digest = md.digest();
- if (digest.length != 0) {
- md5 = Base64.encodeBase64String(digest);
- }
- }
- return md5;
}
@Test
@@ -114,10 +84,10 @@ public void testAppendWithChecksumAtDifferentOffsets() throws Exception {
byte[] data = generateRandomBytes(MB_4);
int pos = 0;
- pos += appendWithOffsetHelper(os, client, path, data, fs, pos, 0, getMd5(data, 0, data.length));
- pos += appendWithOffsetHelper(os, client, path, data, fs, pos, ONE_MB, getMd5(data, ONE_MB, data.length - ONE_MB));
- pos += appendWithOffsetHelper(os, client, path, data, fs, pos, MB_2, getMd5(data, MB_2, data.length-MB_2));
- appendWithOffsetHelper(os, client, path, data, fs, pos, MB_4 - 1, getMd5(data, MB_4 - 1, data.length - (MB_4 - 1)));
+ pos += appendWithOffsetHelper(os, client, path, data, fs, pos, 0, client.computeMD5Hash(data, 0, data.length));
+ pos += appendWithOffsetHelper(os, client, path, data, fs, pos, ONE_MB, client.computeMD5Hash(data, ONE_MB, data.length - ONE_MB));
+ pos += appendWithOffsetHelper(os, client, path, data, fs, pos, MB_2, client.computeMD5Hash(data, MB_2, data.length-MB_2));
+ appendWithOffsetHelper(os, client, path, data, fs, pos, MB_4 - 1, client.computeMD5Hash(data, MB_4 - 1, data.length - (MB_4 - 1)));
fs.close();
}
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestWasbAbfsCompatibility.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestWasbAbfsCompatibility.java
index 69abb1b542825..3d9264e4bc018 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestWasbAbfsCompatibility.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestWasbAbfsCompatibility.java
@@ -21,7 +21,6 @@
import java.io.IOException;
import java.io.InputStreamReader;
import java.nio.charset.StandardCharsets;
-import java.util.Arrays;
import java.util.EnumSet;
import java.util.UUID;
@@ -39,8 +38,10 @@
import org.apache.hadoop.fs.RemoteIterator;
import org.apache.hadoop.fs.XAttrSetFlag;
import org.apache.hadoop.fs.azure.NativeAzureFileSystem;
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException;
import org.apache.hadoop.fs.contract.ContractTestUtils;
+import static java.net.HttpURLConnection.HTTP_CONFLICT;
import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_ABFS_ENABLE_CHECKSUM_VALIDATION;
import static org.apache.hadoop.fs.contract.ContractTestUtils.assertDeleted;
import static org.apache.hadoop.fs.contract.ContractTestUtils.assertIsDirectory;
@@ -51,28 +52,18 @@
* Test compatibility between ABFS client and WASB client.
*/
public class ITestWasbAbfsCompatibility extends AbstractAbfsIntegrationTest {
-
private static final String WASB_TEST_CONTEXT = "wasb test file";
-
private static final String ABFS_TEST_CONTEXT = "abfs test file";
-
private static final String TEST_CONTEXT = "THIS IS FOR TEST";
-
private static final String TEST_CONTEXT1 = "THIS IS FOR TEST1";
-
private static final byte[] ATTRIBUTE_VALUE_1 = "one".getBytes(
StandardCharsets.UTF_8);
-
private static final byte[] ATTRIBUTE_VALUE_2 = "two".getBytes(
StandardCharsets.UTF_8);
-
private static final String ATTRIBUTE_NAME_1 = "user_someAttribute";
-
private static final String ATTRIBUTE_NAME_2 = "user_someAttribute1";
-
private static final EnumSet CREATE_FLAG = EnumSet.of(
XAttrSetFlag.CREATE);
-
private static final Logger LOG =
LoggerFactory.getLogger(ITestWasbAbfsCompatibility.class);
@@ -158,7 +149,10 @@ public void testReadFile() throws Exception {
}
}
- // Create and write a file using WASB then read and write the same file using ABFS.
+ /**
+ * Flow: Create and write a file using WASB, then read and append to it using ABFS. Finally, delete the file via ABFS after verifying content consistency.
+ * Expected: WASB successfully creates the file and writes content. ABFS reads, appends, and deletes the file without data loss or errors.
+ */
@Test
public void testwriteFile() throws Exception {
AzureBlobFileSystem abfs = getFileSystem();
@@ -186,16 +180,20 @@ public void testwriteFile() throws Exception {
assertEquals("Wrong text from " + abfs,
TEST_CONTEXT, line);
}
- try (FSDataOutputStream nativeFsStream = abfs.append(path)) {
- nativeFsStream.write(TEST_CONTEXT.getBytes());
- nativeFsStream.flush();
- nativeFsStream.hsync();
+ try (FSDataOutputStream abfsOutputStream = abfs.append(path)) {
+ abfsOutputStream.write(TEST_CONTEXT.getBytes());
+ abfsOutputStream.flush();
+ abfsOutputStream.hsync();
}
// Remove file
assertDeleted(abfs, path, true);
}
- // Create and write a file using ABFS, then append the file using wasb and then write the file again using ABFS.
+ /**
+ * Flow: Create and write a file using ABFS, append to the file using WASB, then write again using ABFS.
+ * Expected: File is created and written correctly by ABFS, appended by WASB, and final ABFS write reflects all updates without errors.
+ */
+
@Test
public void testwriteFile1() throws Exception {
AzureBlobFileSystem abfs = getFileSystem();
@@ -232,7 +230,10 @@ public void testwriteFile1() throws Exception {
assertDeleted(abfs, path, true);
}
- // Create file using azcopy and append it using ABFS.
+ /**
+ * Flow: Create the file using AzCopy, then append to the file using ABFS.
+ * Expected: ABFS append succeeds and final file reflects both AzCopy and appended data correctly.
+ */
@Test
public void testazcopywasbcompatibility() throws Exception {
AzureBlobFileSystem abfs = getFileSystem();
@@ -328,7 +329,10 @@ public void testSetWorkingDirectory() throws Exception {
// Scenario wise testing
- //Scenario 1: - Create and write via WASB, read via ABFS
+ /**
+ * Scenario 1: Create and write a file using WASB, then read the file using ABFS.
+ * Expected Outcome: ABFS should correctly read the content written by WASB.
+ */
@Test
public void testScenario1() throws Exception {
AzureBlobFileSystem abfs = getFileSystem();
@@ -359,7 +363,10 @@ public void testScenario1() throws Exception {
assertDeleted(abfs, path, true);
}
- //Scenario 2: - Create and write via WASB, read via ABFS and then write the same file via ABFS
+ /**
+ * Scenario 2: Create and write a file using WASB, read it using ABFS, then write to the same file using ABFS.
+ * Expected Outcome: ABFS should read the WASB-written content and successfully write new content to the same file.
+ */
@Test
public void testScenario2() throws Exception {
AzureBlobFileSystem abfs = getFileSystem();
@@ -399,7 +406,10 @@ public void testScenario2() throws Exception {
assertDeleted(abfs, path, true);
}
- //Scenario 3: - Create and write via ABFS and the read via WASB
+ /**
+ * Scenario 3: Create and write a file using ABFS, then read it using WASB.
+ * Expected Outcome: WASB should be able to read the content written by ABFS without any data mismatch or error.
+ */
@Test
public void testScenario3() throws Exception {
AzureBlobFileSystem abfs = getFileSystem();
@@ -431,7 +441,10 @@ public void testScenario3() throws Exception {
assertDeleted(abfs, path, true);
}
- //Scenario 4:- Create via WASB, write via ABFS and then write via WASB
+ /**
+ * Scenario 4: Create a file using WASB, write to it using ABFS, and then write again using WASB.
+ * Expected Outcome: All writes should succeed and the final content should reflect changes from both ABFS and WASB.
+ */
@Test
public void testScenario4() throws Exception {
AzureBlobFileSystem abfs = getFileSystem();
@@ -464,7 +477,10 @@ public void testScenario4() throws Exception {
assertDeleted(abfs, path, true);
}
- //Scenario 5:- Create via ABFS, write via WASB, read via ABFS (Checksum validation disabled)
+ /**
+ * Scenario 5: Create a file using ABFS, write to it using WASB, and read it back using ABFS with checksum validation disabled.
+ * Expected Outcome: The read operation should succeed and reflect the data written via WASB despite checksum validation being off.
+ */
@Test
public void testScenario5() throws Exception {
Configuration conf = getRawConfiguration();
@@ -502,7 +518,10 @@ public void testScenario5() throws Exception {
assertDeleted(abfs, path, true);
}
- //Scenario 6: - Create via ABFS, write via WASB, read via ABFS (Checksum validation enabled)
+ /**
+ * Scenario 6: Create a file using ABFS, write to it using WASB, and read it via ABFS with checksum validation enabled.
+ * Expected Outcome: Read should fail due to checksum mismatch caused by WASB write, verifying integrity enforcement.
+ */
@Test
public void testScenario6() throws Exception {
Configuration conf = getRawConfiguration();
@@ -540,7 +559,10 @@ public void testScenario6() throws Exception {
assertDeleted(abfs, path, true);
}
- // Scenario 7 :- Create via WASB and then create overwrite true using ABFS
+ /**
+ * Scenario 7: Create a file using WASB and then overwrite it using ABFS with overwrite=true.
+ * Expected Outcome: ABFS should successfully overwrite the existing file created by WASB without error.
+ */
@Test
public void testScenario7() throws Exception {
Configuration conf = getRawConfiguration();
@@ -571,12 +593,19 @@ public void testScenario7() throws Exception {
TEST_CONTEXT, line);
}
abfs.create(path, true);
+ FileStatus fileStatus = abfs.getFileStatus(path);
+ Assertions.assertThat(fileStatus.getLen())
+ .as("Expected file length to be 0 after overwrite")
+ .isEqualTo(0L);
// Remove file
assertDeleted(abfs, path, true);
}
- // Scenario 8 :- Create via WASB and then create overwrite false using ABFS
+ /**
+ * Scenario 8: Create a file using WASB and then attempt to create the same file using ABFS with overwrite=false.
+ * Expected Outcome: ABFS should fail to create the file due to the file already existing.
+ */
@Test
public void testScenario8() throws Exception {
Configuration conf = getRawConfiguration();
@@ -608,15 +637,26 @@ public void testScenario8() throws Exception {
}
try {
abfs.create(path, false);
- } catch (Exception e) {
- assertTrue(e.getMessage().contains("AlreadyExists"));
+ } catch (IOException e) {
+ AbfsRestOperationException restEx = (AbfsRestOperationException) e.getCause();
+ if (restEx != null) {
+ Assertions.assertThat(restEx.getStatusCode())
+ .as("Expected HTTP status code 409 (Conflict) when file already exists")
+ .isEqualTo(HTTP_CONFLICT);
+ }
+ Assertions.assertThat(e.getMessage())
+ .as("Expected error message to contain 'AlreadyExists'")
+ .contains("AlreadyExists");
}
// Remove file
assertDeleted(abfs, path, true);
}
- // Scenario 9 :- Create via ABFS and then create overwrite true using WASB
+ /**
+ * Scenario 9: Create a file using ABFS and then attempt to create the same file using WASB with overwrite=true.
+ * Expected Outcome: WASB should successfully overwrite the existing file.
+ */
@Test
public void testScenario9() throws Exception {
Configuration conf = getRawConfiguration();
@@ -648,12 +688,20 @@ public void testScenario9() throws Exception {
TEST_CONTEXT, line);
}
wasb.create(path, true);
+ FileStatus fileStatus = abfs.getFileStatus(path);
+ Assertions.assertThat(fileStatus.getLen())
+ .as("Expected file length to be 0 after overwrite")
+ .isEqualTo(0L);
// Remove file
assertDeleted(abfs, path, true);
}
- // Scenario 10 :- Create via ABFS and then create overwrite false using WASB
+ /**
+ * Scenario 10: Create a file using ABFS and then attempt to create the same file using WASB with overwrite=false.
+ * Expected Outcome: WASB should fail to create the file as it already exists. The exception should indicate
+ * an "AlreadyExists" error with HTTP status code 409 (Conflict).
+ */
@Test
public void testScenario10() throws Exception {
Configuration conf = getRawConfiguration();
@@ -684,15 +732,27 @@ public void testScenario10() throws Exception {
}
try {
wasb.create(path, false);
- } catch (Exception e) {
- assertTrue(e.getMessage().toLowerCase().contains("exists"));
+ } catch (IOException e) {
+ AbfsRestOperationException restEx
+ = (AbfsRestOperationException) e.getCause();
+ if (restEx != null) {
+ Assertions.assertThat(restEx.getStatusCode())
+ .as("Expected HTTP status code 409 (Conflict) when file already exists")
+ .isEqualTo(HTTP_CONFLICT);
+ }
+ Assertions.assertThat(e.getMessage())
+ .as("Expected error message to contain 'exists'")
+ .contains("exists");
}
-
// Remove file
assertDeleted(abfs, path, true);
}
- // Scenario 11 :- Create via ABFS and then write via WASB and delete via ABFS
+ /**
+ * Scenario 11: Create a file using ABFS, write data to it using WASB, and then delete the file using ABFS.
+ * Expected Outcome: File should be created via ABFS and writable by WASB.
+ * ABFS delete should succeed, and the file should no longer exist.
+ */
@Test
public void testScenario11() throws Exception {
Configuration conf = getRawConfiguration();
@@ -728,7 +788,11 @@ public void testScenario11() throws Exception {
abfs.delete(path, true);
}
- // Scenario 12 :- Create and write via ABFS and delete via WASB
+ /**
+ * Scenario 12: Create and write a file using ABFS, and then delete the same file using WASB.
+ * Expected Outcome: File should be created and written successfully via ABFS.
+ * WASB should be able to delete the file without errors.
+ */
@Test
public void testScenario12() throws Exception {
Configuration conf = getRawConfiguration();
@@ -743,10 +807,10 @@ public void testScenario12() throws Exception {
Path path = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
// Write
- try (FSDataOutputStream nativeFsStream = abfs.create(path, true)) {
- nativeFsStream.write(TEST_CONTEXT.getBytes());
- nativeFsStream.flush();
- nativeFsStream.hsync();
+ try (FSDataOutputStream abfsOutputStream = abfs.create(path, true)) {
+ abfsOutputStream.write(TEST_CONTEXT.getBytes());
+ abfsOutputStream.flush();
+ abfsOutputStream.hsync();
}
// Check file status
@@ -761,7 +825,10 @@ public void testScenario12() throws Exception {
wasb.delete(path, true);
}
- // Scenario 13:- Create via ABFS, write via WASB, and read via wasb
+ /**
+ * Scenario 13: Create a file using ABFS, write data to it using WASB, and then read the file using WASB.
+ * Expected Outcome: The read operation via WASB should return the correct content written via WASB.
+ */
@Test
public void testScenario13() throws Exception {
Configuration conf = getRawConfiguration();
@@ -797,7 +864,10 @@ public void testScenario13() throws Exception {
abfs.delete(path, true);
}
- // Scenario 14:- Create via ABFS, write via WASB, and delete via wasb
+ /**
+ * Scenario 14: Create a file using ABFS, write data to it using WASB, and delete the file using WASB.
+ * Expected Outcome: Write via WASB should succeed and data should be persisted; delete via WASB should succeed without errors.
+ */
@Test
public void testScenario14() throws Exception {
Configuration conf = getRawConfiguration();
@@ -833,7 +903,10 @@ public void testScenario14() throws Exception {
wasb.delete(path, true);
}
- // Scenario 15 :- Create and write via WASB and delete via ABFS
+ /**
+ * Scenario 15: Create and write a file using WASB, then delete the file using ABFS.
+ * Expected Outcome: Write via WASB should succeed and data should be persisted; delete via ABFS should succeed without errors.
+ */
@Test
public void testScenario15() throws Exception {
Configuration conf = getRawConfiguration();
@@ -866,7 +939,10 @@ public void testScenario15() throws Exception {
abfs.delete(path, true);
}
- // Scenario 16: Create via WASB, write via ABFS, and delete via WASB
+ /**
+ * Scenario 16: Create a file using WASB, write data to it using ABFS, and then delete the file using WASB.
+ * Expected Outcome: Write via ABFS should succeed and persist data; delete via WASB should succeed without errors.
+ */
@Test
public void testScenario16() throws Exception {
Configuration conf = getRawConfiguration();
@@ -902,7 +978,10 @@ public void testScenario16() throws Exception {
wasb.delete(path, true);
}
- // Scenario 17: Create, setXAttr and getXAttr via ABFS
+ /**
+ * Scenario 17: Create a file using ABFS, set attribute (xAttr), and retrieve it using ABFS.
+ * Expected Outcome: setXAttr and getXAttr operations via ABFS should succeed and return the correct value.
+ */
@Test
public void testScenario17() throws Exception {
Configuration conf = getRawConfiguration();
@@ -911,8 +990,6 @@ public void testScenario17() throws Exception {
AzureBlobFileSystem abfs = (AzureBlobFileSystem) fileSystem;
Assume.assumeFalse("Namespace enabled account does not support this test",
getIsNamespaceEnabled(abfs));
- NativeAzureFileSystem wasb = getWasbFileSystem();
-
Path testFile = path("/testReadFile");
Path path = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
@@ -929,21 +1006,24 @@ public void testScenario17() throws Exception {
// --- SET XATTR #1 ---
abfs.setXAttr(path, ATTRIBUTE_NAME_1, ATTRIBUTE_VALUE_1);
byte[] readValue = abfs.getXAttr(path, ATTRIBUTE_NAME_1);
- assertAttributeEqual(readValue, ATTRIBUTE_VALUE_1, "one");
+ ITestAzureBlobFileSystemAttributes.assertAttributeEqual(abfs, readValue, ATTRIBUTE_VALUE_1, "one");
// --- SET XATTR #2 ---
abfs.setXAttr(path, ATTRIBUTE_NAME_2, ATTRIBUTE_VALUE_2);
readValue = abfs.getXAttr(path, ATTRIBUTE_NAME_2);
- assertAttributeEqual(readValue, ATTRIBUTE_VALUE_2, "two");
+ ITestAzureBlobFileSystemAttributes.assertAttributeEqual(abfs, readValue, ATTRIBUTE_VALUE_2, "two");
// --- VERIFY XATTR #1 AGAIN ---
readValue = abfs.getXAttr(path, ATTRIBUTE_NAME_1);
- assertAttributeEqual(readValue, ATTRIBUTE_VALUE_1, "one");
+ ITestAzureBlobFileSystemAttributes.assertAttributeEqual(abfs, readValue, ATTRIBUTE_VALUE_1, "one");
abfs.delete(path, true);
}
- // Scenario 17: Create, setXAttr and getXAttr via WASB
+ /**
+ * Scenario 18: Create a file using WASB, set an attribute (xAttr), and retrieve it using WASB.
+ * Expected Outcome: setXAttr and getXAttr operations via WASB should succeed and return the correct value.
+ */
@Test
public void testScenario18() throws Exception {
Configuration conf = getRawConfiguration();
@@ -970,21 +1050,24 @@ public void testScenario18() throws Exception {
// --- SET XATTR #1 ---
wasb.setXAttr(path, ATTRIBUTE_NAME_1, ATTRIBUTE_VALUE_1);
byte[] readValue = wasb.getXAttr(path, ATTRIBUTE_NAME_1);
- assertAttributeEqual(readValue, ATTRIBUTE_VALUE_1, "one");
+ ITestAzureBlobFileSystemAttributes.assertAttributeEqual(abfs, readValue, ATTRIBUTE_VALUE_1, "one");
// --- SET XATTR #2 ---
wasb.setXAttr(path, ATTRIBUTE_NAME_2, ATTRIBUTE_VALUE_2);
readValue = wasb.getXAttr(path, ATTRIBUTE_NAME_2);
- assertAttributeEqual(readValue, ATTRIBUTE_VALUE_2, "two");
+ ITestAzureBlobFileSystemAttributes.assertAttributeEqual(abfs, readValue, ATTRIBUTE_VALUE_2, "two");
// --- VERIFY XATTR #1 AGAIN ---
readValue = wasb.getXAttr(path, ATTRIBUTE_NAME_1);
- assertAttributeEqual(readValue, ATTRIBUTE_VALUE_1, "one");
+ ITestAzureBlobFileSystemAttributes.assertAttributeEqual(abfs, readValue, ATTRIBUTE_VALUE_1, "one");
wasb.delete(path, true);
}
- // Scenario 19: Create, setXAttr via wasb and getXAttr via ABFS
+ /**
+ * Scenario 19: Create a file using WASB, set an attribute using WASB, and retrieve it using ABFS.
+ * Expected Outcome: Attribute set via WASB should be retrievable via ABFS and should match the original value.
+ */
@Test
public void testScenario19() throws Exception {
Configuration conf = getRawConfiguration();
@@ -1011,21 +1094,26 @@ public void testScenario19() throws Exception {
// --- SET XATTR #1 ---
wasb.setXAttr(path, ATTRIBUTE_NAME_1, ATTRIBUTE_VALUE_1);
byte[] readValue = abfs.getXAttr(path, ATTRIBUTE_NAME_1);
- assertAttributeEqual(readValue, ATTRIBUTE_VALUE_1, "one");
+ ITestAzureBlobFileSystemAttributes.assertAttributeEqual(abfs, readValue, ATTRIBUTE_VALUE_1, "one");
// --- SET XATTR #2 ---
wasb.setXAttr(path, ATTRIBUTE_NAME_2, ATTRIBUTE_VALUE_2);
readValue = abfs.getXAttr(path, ATTRIBUTE_NAME_2);
- assertAttributeEqual(readValue, ATTRIBUTE_VALUE_2, "two");
+ ITestAzureBlobFileSystemAttributes.assertAttributeEqual(abfs, readValue, ATTRIBUTE_VALUE_2, "two");
// --- VERIFY XATTR #1 AGAIN ---
readValue = abfs.getXAttr(path, ATTRIBUTE_NAME_1);
- assertAttributeEqual(readValue, ATTRIBUTE_VALUE_1, "one");
+ ITestAzureBlobFileSystemAttributes.assertAttributeEqual(abfs, readValue, ATTRIBUTE_VALUE_1, "one");
wasb.delete(path, true);
}
- // Scenario 20: Create, setXAttr via wasb and getXAttr via ABFS and create overwrite via ABFS
+ /**
+ * Scenario 20: Create a file using WASB, set an attribute via WASB, retrieve the attribute via ABFS,
+ * and then create the file again using ABFS with overwrite=true.
+ * Expected Outcome: Attribute set via WASB should be retrievable via ABFS before overwrite.
+ * After overwrite via ABFS, the attribute should no longer exist.
+ */
@Test
public void testScenario20() throws Exception {
Configuration conf = getRawConfiguration();
@@ -1053,22 +1141,31 @@ public void testScenario20() throws Exception {
// --- SET XATTR #1 ---
wasb.setXAttr(path, ATTRIBUTE_NAME_1, ATTRIBUTE_VALUE_1);
byte[] readValue = abfs.getXAttr(path, ATTRIBUTE_NAME_1);
- assertAttributeEqual(readValue, ATTRIBUTE_VALUE_1, "one");
+ ITestAzureBlobFileSystemAttributes.assertAttributeEqual(abfs, readValue, ATTRIBUTE_VALUE_1, "one");
// --- SET XATTR #2 ---
wasb.setXAttr(path, ATTRIBUTE_NAME_2, ATTRIBUTE_VALUE_2);
readValue = abfs.getXAttr(path, ATTRIBUTE_NAME_2);
- assertAttributeEqual(readValue, ATTRIBUTE_VALUE_2, "two");
+ ITestAzureBlobFileSystemAttributes.assertAttributeEqual(abfs, readValue, ATTRIBUTE_VALUE_2, "two");
// --- VERIFY XATTR #1 AGAIN ---
readValue = abfs.getXAttr(path, ATTRIBUTE_NAME_1);
- assertAttributeEqual(readValue, ATTRIBUTE_VALUE_1, "one");
+ ITestAzureBlobFileSystemAttributes.assertAttributeEqual(abfs, readValue, ATTRIBUTE_VALUE_1, "one");
abfs.create(path, true);
+ FileStatus fileStatus = abfs.getFileStatus(path);
+ Assertions.assertThat(fileStatus.getLen())
+ .as("Expected file length to be 0 after overwrite")
+ .isEqualTo(0L);
wasb.delete(path, true);
}
- // Scenario 21: Create, setXAttr ABFS, getXAttr WASB and create overwrite via WASB
+ /**
+ * Scenario 21: Create a file using ABFS, set an attribute via ABFS, retrieve the attribute via WASB,
+ * and then create the file again using WASB with overwrite=true.
+ * Expected Outcome: Attribute set via ABFS should be retrievable via WASB before overwrite.
+ * After overwrite via WASB, the attribute should no longer exist.
+ */
@Test
public void testScenario21() throws Exception {
Configuration conf = getRawConfiguration();
@@ -1097,22 +1194,31 @@ public void testScenario21() throws Exception {
// --- SET XATTR #1 ---
abfs.setXAttr(path, ATTRIBUTE_NAME_1, ATTRIBUTE_VALUE_1);
byte[] readValue = wasb.getXAttr(path, ATTRIBUTE_NAME_1);
- assertAttributeEqual(readValue, ATTRIBUTE_VALUE_1, "one");
+ ITestAzureBlobFileSystemAttributes.assertAttributeEqual(abfs, readValue, ATTRIBUTE_VALUE_1, "one");
// --- SET XATTR #2 ---
abfs.setXAttr(path, ATTRIBUTE_NAME_2, ATTRIBUTE_VALUE_2);
readValue = wasb.getXAttr(path, ATTRIBUTE_NAME_2);
- assertAttributeEqual(readValue, ATTRIBUTE_VALUE_2, "two");
+ ITestAzureBlobFileSystemAttributes.assertAttributeEqual(abfs, readValue, ATTRIBUTE_VALUE_2, "two");
// --- VERIFY XATTR #1 AGAIN ---
readValue = wasb.getXAttr(path, ATTRIBUTE_NAME_1);
- assertAttributeEqual(readValue, ATTRIBUTE_VALUE_1, "one");
+ ITestAzureBlobFileSystemAttributes.assertAttributeEqual(abfs, readValue, ATTRIBUTE_VALUE_1, "one");
wasb.create(path, true);
+ FileStatus fileStatus = abfs.getFileStatus(path);
+ Assertions.assertThat(fileStatus.getLen())
+ .as("Expected file length to be 0 after overwrite")
+ .isEqualTo(0L);
wasb.delete(path, true);
}
- // Scenario 22: Create via WASB, setXAttr ABFS, getXAttr wasb and create overwrite via WASB
+ /**
+ * Scenario 22: Create a file using WASB, set an attribute via ABFS,
+ * retrieve the attribute via WASB, and then create the file again using WASB with overwrite=true.
+ * Expected Outcome: Attribute set via ABFS should be retrievable via WASB before overwrite.
+ * After overwrite via WASB, the attribute should be removed.
+ */
@Test
public void testScenario22() throws Exception {
Configuration conf = getRawConfiguration();
@@ -1139,22 +1245,31 @@ public void testScenario22() throws Exception {
// --- SET XATTR #1 ---
abfs.setXAttr(path, ATTRIBUTE_NAME_1, ATTRIBUTE_VALUE_1);
byte[] readValue = wasb.getXAttr(path, ATTRIBUTE_NAME_1);
- assertAttributeEqual(readValue, ATTRIBUTE_VALUE_1, "one");
+ ITestAzureBlobFileSystemAttributes.assertAttributeEqual(abfs, readValue, ATTRIBUTE_VALUE_1, "one");
// --- SET XATTR #2 ---
abfs.setXAttr(path, ATTRIBUTE_NAME_2, ATTRIBUTE_VALUE_2);
readValue = wasb.getXAttr(path, ATTRIBUTE_NAME_2);
- assertAttributeEqual(readValue, ATTRIBUTE_VALUE_2, "two");
+ ITestAzureBlobFileSystemAttributes.assertAttributeEqual(abfs, readValue, ATTRIBUTE_VALUE_2, "two");
// --- VERIFY XATTR #1 AGAIN ---
readValue = wasb.getXAttr(path, ATTRIBUTE_NAME_1);
- assertAttributeEqual(readValue, ATTRIBUTE_VALUE_1, "one");
+ ITestAzureBlobFileSystemAttributes.assertAttributeEqual(abfs, readValue, ATTRIBUTE_VALUE_1, "one");
wasb.create(path, true);
+ FileStatus fileStatus = abfs.getFileStatus(path);
+ Assertions.assertThat(fileStatus.getLen())
+ .as("Expected file length to be 0 after overwrite")
+ .isEqualTo(0L);
wasb.delete(path, true);
}
- // Scenario 23: Create via WASB, setXAttr ABFS, then setXAttr via WASB and getXAttr via ABFS
+ /**
+ * Scenario 23: Create a file using WASB, set an attribute via ABFS,
+ * then set another attribute via WASB, and retrieve attributes via ABFS.
+ * Expected Outcome: Both attributes should be retrievable via ABFS,
+ * confirming that updates from both ABFS and WASB are visible.
+ */
@Test
public void testScenario23() throws Exception {
Configuration conf = getRawConfiguration();
@@ -1180,21 +1295,26 @@ public void testScenario23() throws Exception {
// --- SET XATTR #1 ---
abfs.setXAttr(path, ATTRIBUTE_NAME_1, ATTRIBUTE_VALUE_1);
byte[] readValue = abfs.getXAttr(path, ATTRIBUTE_NAME_1);
- assertAttributeEqual(readValue, ATTRIBUTE_VALUE_1, "one");
+ ITestAzureBlobFileSystemAttributes.assertAttributeEqual(abfs, readValue, ATTRIBUTE_VALUE_1, "one");
// --- SET XATTR #2 ---
wasb.setXAttr(path, ATTRIBUTE_NAME_2, ATTRIBUTE_VALUE_2, CREATE_FLAG);
readValue = abfs.getXAttr(path, ATTRIBUTE_NAME_2);
- assertAttributeEqual(readValue, ATTRIBUTE_VALUE_2, "two");
+ ITestAzureBlobFileSystemAttributes.assertAttributeEqual(abfs, readValue, ATTRIBUTE_VALUE_2, "two");
// --- VERIFY XATTR #1 AGAIN ---
readValue = abfs.getXAttr(path, ATTRIBUTE_NAME_1);
- assertAttributeEqual(readValue, ATTRIBUTE_VALUE_1, "one");
+ ITestAzureBlobFileSystemAttributes.assertAttributeEqual(abfs, readValue, ATTRIBUTE_VALUE_1, "one");
wasb.delete(path, true);
}
- // Scenario 24: Create via ABFS, then setXAttr via WASB and getXAttr via ABFS
+ /**
+ * Scenario 24: Create a file using ABFS, then set an attribute via WASB,
+ * and retrieve the attribute via ABFS.
+ * Expected Outcome: Attribute set via WASB should be retrievable via ABFS,
+ * verifying cross-compatibility of attribute operations.
+ */
@Test
public void testScenario24() throws Exception {
Configuration conf = getRawConfiguration();
@@ -1222,21 +1342,26 @@ public void testScenario24() throws Exception {
// --- SET XATTR #1 ---
wasb.setXAttr(path, ATTRIBUTE_NAME_1, ATTRIBUTE_VALUE_1);
byte[] readValue = abfs.getXAttr(path, ATTRIBUTE_NAME_1);
- assertAttributeEqual(readValue, ATTRIBUTE_VALUE_1, "one");
+ ITestAzureBlobFileSystemAttributes.assertAttributeEqual(abfs, readValue, ATTRIBUTE_VALUE_1, "one");
// --- SET XATTR #2 ---
wasb.setXAttr(path, ATTRIBUTE_NAME_2, ATTRIBUTE_VALUE_2, CREATE_FLAG);
readValue = abfs.getXAttr(path, ATTRIBUTE_NAME_2);
- assertAttributeEqual(readValue, ATTRIBUTE_VALUE_2, "two");
+ ITestAzureBlobFileSystemAttributes.assertAttributeEqual(abfs, readValue, ATTRIBUTE_VALUE_2, "two");
// --- VERIFY XATTR #1 AGAIN ---
readValue = abfs.getXAttr(path, ATTRIBUTE_NAME_1);
- assertAttributeEqual(readValue, ATTRIBUTE_VALUE_1, "one");
+ ITestAzureBlobFileSystemAttributes.assertAttributeEqual(abfs, readValue, ATTRIBUTE_VALUE_1, "one");
wasb.delete(path, true);
}
- // Scenario 24: Create via WASB, then setXAttr getXAttr via ABFS and delete via WASB
+ /**
+ * Scenario 25: Create a file using WASB, then set and retrieve an attribute via ABFS,
+ * and finally delete the file using WASB.
+ * Expected Outcome: Attribute set via ABFS should be retrievable via ABFS,
+ * and file deletion via WASB should succeed.
+ */
@Test
public void testScenario25() throws Exception {
Configuration conf = getRawConfiguration();
@@ -1263,21 +1388,26 @@ public void testScenario25() throws Exception {
// --- SET XATTR #1 ---
abfs.setXAttr(path, ATTRIBUTE_NAME_1, ATTRIBUTE_VALUE_1);
byte[] readValue = abfs.getXAttr(path, ATTRIBUTE_NAME_1);
- assertAttributeEqual(readValue, ATTRIBUTE_VALUE_1, "one");
+ ITestAzureBlobFileSystemAttributes.assertAttributeEqual(abfs, readValue, ATTRIBUTE_VALUE_1, "one");
// --- SET XATTR #2 ---
abfs.setXAttr(path, ATTRIBUTE_NAME_2, ATTRIBUTE_VALUE_2, CREATE_FLAG);
readValue = abfs.getXAttr(path, ATTRIBUTE_NAME_2);
- assertAttributeEqual(readValue, ATTRIBUTE_VALUE_2, "two");
+ ITestAzureBlobFileSystemAttributes.assertAttributeEqual(abfs, readValue, ATTRIBUTE_VALUE_2, "two");
// --- VERIFY XATTR #1 AGAIN ---
readValue = abfs.getXAttr(path, ATTRIBUTE_NAME_1);
- assertAttributeEqual(readValue, ATTRIBUTE_VALUE_1, "one");
+ ITestAzureBlobFileSystemAttributes.assertAttributeEqual(abfs, readValue, ATTRIBUTE_VALUE_1, "one");
wasb.delete(path, true);
}
- // Scenario 26: Create via ABFS, then setXAttr getXAttr via WASB and delete via WASB
+ /**
+ * Scenario 26: Create a file using ABFS, then set and retrieve an attribute via WASB,
+ * and finally delete the file using WASB.
+ * Expected Outcome: Attribute set via WASB should be retrievable via WASB,
+ * and file deletion via WASB should succeed.
+ */
@Test
public void testScenario26() throws Exception {
Configuration conf = getRawConfiguration();
@@ -1305,21 +1435,24 @@ public void testScenario26() throws Exception {
// --- SET XATTR #1 ---
wasb.setXAttr(path, ATTRIBUTE_NAME_1, ATTRIBUTE_VALUE_1);
byte[] readValue = wasb.getXAttr(path, ATTRIBUTE_NAME_1);
- assertAttributeEqual(readValue, ATTRIBUTE_VALUE_1, "one");
+ ITestAzureBlobFileSystemAttributes.assertAttributeEqual(abfs, readValue, ATTRIBUTE_VALUE_1, "one");
// --- SET XATTR #2 ---
wasb.setXAttr(path, ATTRIBUTE_NAME_2, ATTRIBUTE_VALUE_2, CREATE_FLAG);
readValue = wasb.getXAttr(path, ATTRIBUTE_NAME_2);
- assertAttributeEqual(readValue, ATTRIBUTE_VALUE_2, "two");
+ ITestAzureBlobFileSystemAttributes.assertAttributeEqual(abfs, readValue, ATTRIBUTE_VALUE_2, "two");
// --- VERIFY XATTR #1 AGAIN ---
readValue = wasb.getXAttr(path, ATTRIBUTE_NAME_1);
- assertAttributeEqual(readValue, ATTRIBUTE_VALUE_1, "one");
+ ITestAzureBlobFileSystemAttributes.assertAttributeEqual(abfs, readValue, ATTRIBUTE_VALUE_1, "one");
wasb.delete(path, true);
}
- // Scenario 27: Create and write via ABFS, rename via wasb
+ /**
+ * Scenario 27: Create and write a file using ABFS, then rename the file using WASB.
+ * Expected Outcome: WASB should successfully rename the file created and written by ABFS.
+ */
@Test
public void testScenario27() throws Exception {
Configuration conf = getRawConfiguration();
@@ -1366,7 +1499,10 @@ public void testScenario27() throws Exception {
wasb.delete(testPath2, true);
}
- // Scenario 28: Create and write via WASB, rename via ABFS, list via ABFS
+ /**
+ * Scenario 28: Create and write a file using WASB, rename the file using ABFS, and list files using ABFS.
+ * Expected Outcome: ABFS should successfully rename the file created by WASB, and the renamed file should appear in listings.
+ */
@Test
public void testScenario28() throws Exception {
Configuration conf = getRawConfiguration();
@@ -1412,7 +1548,10 @@ public void testScenario28() throws Exception {
wasb.delete(testPath2, true);
}
- // Scenario 29: Create via WASB and write via ABFS, rename via ABFS, list via ABFS
+ /**
+ * Scenario 29: Create a file using WASB, write data to it via ABFS, rename the file using ABFS, and list files using ABFS.
+ * Expected Outcome: ABFS should successfully rename the file and list the renamed file accurately.
+ */
@Test
public void testScenario29() throws Exception {
Configuration conf = getRawConfiguration();
@@ -1461,7 +1600,10 @@ public void testScenario29() throws Exception {
wasb.delete(testPath2, true);
}
- //Scenario 30: Create and write via WASB, rename via WASB, rename via ABFS, list via ABFS
+ /**
+ * Scenario 30: Create and write a file using WASB, rename it via WASB, rename again via ABFS, and list files using ABFS.
+ * Expected Outcome: Both renames should succeed, and ABFS listing should reflect the latest filename.
+ */
@Test
public void testScenario30() throws Exception {
Configuration conf = getRawConfiguration();
@@ -1514,7 +1656,10 @@ public void testScenario30() throws Exception {
wasb.delete(testPath3, true);
}
- //Scenario 31: Create and write via WASB, delete via WASB, rename via ABFS -> should fail
+ /**
+ * Scenario 31: Create and write a file using WASB, delete it via WASB, then attempt to rename the deleted file via ABFS.
+ * Expected Outcome: Rename should fail since the file was deleted, ensuring proper error handling.
+ */
@Test
public void testScenario31() throws Exception {
Configuration conf = getRawConfiguration();
@@ -1554,7 +1699,10 @@ public void testScenario31() throws Exception {
.isFalse();
}
- //Scenario 32 :Create Dir & File via WASB → Rename Dir via ABFS → List Files via ABFS
+ /**
+ * Scenario 32: Create a directory and file using WASB, rename the directory using ABFS, and list files using ABFS.
+ * Expected Outcome: ABFS should successfully rename the directory, and listing should reflect the updated directory name.
+ */
@Test
public void testScenario32() throws Exception {
Configuration conf = getRawConfiguration();
@@ -1602,7 +1750,10 @@ public void testScenario32() throws Exception {
.isEqualTo(5);
}
- //Scenario 33 :Create Dir & File via ABFS → Rename Dir via WASB → List Files via WASB
+ /**
+ * Scenario 33: Create a directory and file using ABFS, rename the directory using WASB, and list files using WASB.
+ * Expected Outcome: WASB should successfully rename the directory, and listing should reflect the updated directory name.
+ */
@Test
public void testScenario33() throws Exception {
Configuration conf = getRawConfiguration();
@@ -1651,7 +1802,10 @@ public void testScenario33() throws Exception {
.isEqualTo(5);
}
- //Scenario 34: Create Dir via ABFS → Rename File inside Dir via WASB → List via ABFS
+ /**
+ * Scenario 34: Create a directory via ABFS, rename a file inside the directory using WASB, and list files via ABFS.
+ * Expected Outcome: WASB should successfully rename the file, and ABFS listing should reflect the updated filename.
+ */
@Test
public void testScenario34() throws Exception {
Configuration conf = getRawConfiguration();
@@ -1697,7 +1851,10 @@ public void testScenario34() throws Exception {
.isEqualTo(4);
}
- //Scenario 35: Create Dir via WASB → Rename File inside Dir via ABFS → List via WASB
+ /**
+ * Scenario 35: Create a directory via WASB, rename a file inside the directory using ABFS, and list files via WASB.
+ * Expected Outcome: ABFS should successfully rename the file, and WASB listing should reflect the updated filename.
+ */
@Test
public void testScenario35() throws Exception {
Configuration conf = getRawConfiguration();
@@ -1743,7 +1900,11 @@ public void testScenario35() throws Exception {
.isEqualTo(4);
}
- //Scenario 36: Create via WASB → Rename to existing name via ABFS → List via WASB
+ /**
+ * Scenario 36: Create a file via WASB, attempt to rename it to an existing filename using ABFS, and list files via WASB.
+ * Expected Outcome: Rename should fail due to existing target name, and WASB listing should remain unchanged.
+ */
+
@Test
public void testScenario36() throws Exception {
Configuration conf = getRawConfiguration();
@@ -1783,7 +1944,10 @@ public void testScenario36() throws Exception {
.isFalse();
}
- //Scenario 37: Rename a non-existent file via WASB
+ /**
+ * Scenario 37: Attempt to rename a non-existent file using WASB.
+ * Expected Outcome: Rename operation should fail with an appropriate error indicating the file does not exist.
+ */
@Test
public void testScenario37() throws Exception {
Configuration conf = getRawConfiguration();
@@ -1824,7 +1988,10 @@ public void testScenario37() throws Exception {
.isFalse();
}
- // Scenario 38: Create via WASB, setXAttr and getXAttr WASB and create overwrite via WASB
+ /**
+ * Scenario 38: Create a file using WASB, set and get an attribute via WASB, then create the file again with overwrite=true using WASB.
+ * Expected Outcome: Attribute operations should succeed before overwrite, and after overwrite, the file should be replaced with no prior attributes.
+ */
@Test
public void testScenario38() throws Exception {
Configuration conf = getRawConfiguration();
@@ -1851,22 +2018,29 @@ public void testScenario38() throws Exception {
// --- SET XATTR #1 ---
wasb.setXAttr(path, ATTRIBUTE_NAME_1, ATTRIBUTE_VALUE_1);
byte[] readValue = wasb.getXAttr(path, ATTRIBUTE_NAME_1);
- assertAttributeEqual(readValue, ATTRIBUTE_VALUE_1, "one");
+ ITestAzureBlobFileSystemAttributes.assertAttributeEqual(abfs, readValue, ATTRIBUTE_VALUE_1, "one");
// --- SET XATTR #2 ---
wasb.setXAttr(path, ATTRIBUTE_NAME_2, ATTRIBUTE_VALUE_2);
readValue = wasb.getXAttr(path, ATTRIBUTE_NAME_2);
- assertAttributeEqual(readValue, ATTRIBUTE_VALUE_2, "two");
+ ITestAzureBlobFileSystemAttributes.assertAttributeEqual(abfs, readValue, ATTRIBUTE_VALUE_2, "two");
// --- VERIFY XATTR #1 AGAIN ---
readValue = wasb.getXAttr(path, ATTRIBUTE_NAME_1);
- assertAttributeEqual(readValue, ATTRIBUTE_VALUE_1, "one");
+ ITestAzureBlobFileSystemAttributes.assertAttributeEqual(abfs, readValue, ATTRIBUTE_VALUE_1, "one");
wasb.create(path, true);
+ FileStatus fileStatus = abfs.getFileStatus(path);
+ Assertions.assertThat(fileStatus.getLen())
+ .as("Expected file length to be 0 after overwrite")
+ .isEqualTo(0L);
wasb.delete(path, true);
}
- // Scenario 39: Create and write via WASB, rename via wasb and list via WASB
+ /**
+ * Scenario 39: Create and write a file using WASB, rename the file using WASB, and list files using WASB.
+ * Expected Outcome: WASB should successfully rename the file, and the renamed file should appear in the listing.
+ */
@Test
public void testScenario39() throws Exception {
Configuration conf = getRawConfiguration();
@@ -1946,29 +2120,12 @@ public static int listAllFilesAndDirs(FileSystem fs, Path path) throws IOExcepti
* @throws AssertionError If the path is a directory or a symlink.
*/
private static void assertIsFile(Path path, FileStatus status) {
- if (status.isDirectory()) {
- throw new AssertionError("File claims to be a directory: " + path + " " + status);
- }
- if (status.isSymlink()) {
- throw new AssertionError("File claims to be a symlink: " + path + " " + status);
- }
- }
+ Assertions.assertThat(status.isDirectory())
+ .as("Expected a regular file, but was a directory: %s %s", path, status)
+ .isFalse();
- /**
- * Checks that the actual byte array matches the expected byte array and decoded string.
- *
- * @param actual The actual byte array.
- * @param expected The expected byte array.
- * @param expectedDecoded The expected decoded string.
- * @throws AssertionError If the bytes or decoded string don't match.
- */
- private static void assertAttributeEqual(byte[] actual, byte[] expected, String expectedDecoded) {
- if (!Arrays.equals(actual, expected)) {
- throw new AssertionError("Encoded attribute does not match expected bytes");
- }
- String decoded = new String(actual, StandardCharsets.UTF_8);
- if (!decoded.equals(expectedDecoded)) {
- throw new AssertionError("Decoded attribute does not match. Got: " + decoded + ", Expected: " + expectedDecoded);
- }
+ Assertions.assertThat(status.isSymlink())
+ .as("Expected a regular file, but was a symlink: %s %s", path, status)
+ .isFalse();
}
}
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsOutputStream.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsOutputStream.java
index e3f040a1bedb7..b2b2c697b593e 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsOutputStream.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsOutputStream.java
@@ -25,7 +25,9 @@
import java.net.URI;
import java.net.URISyntaxException;
import java.net.URL;
+import java.security.MessageDigest;
import java.util.Arrays;
+import java.util.Collections;
import org.assertj.core.api.Assertions;
import org.junit.Assume;
@@ -48,10 +50,12 @@
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.security.ContextEncryptionAdapter;
import org.apache.hadoop.fs.azurebfs.utils.TracingContext;
import org.apache.http.HttpResponse;
import static java.net.HttpURLConnection.HTTP_CONFLICT;
+import static java.net.HttpURLConnection.HTTP_UNAVAILABLE;
import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.EXPECT_100_JDK_ERROR;
import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_ACCOUNT_IS_EXPECT_HEADER_ENABLED;
import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.EXPECT;
@@ -426,4 +430,58 @@ public void testNoNetworkCallsForSecondFlush() throws Exception {
flush(Mockito.any(byte[].class), Mockito.anyString(), Mockito.anyBoolean(), Mockito.any(), Mockito.any(), Mockito.anyString(), Mockito.any(),
Mockito.any(TracingContext.class), Mockito.anyString());
}
+
+ /**
+ * Tests that the message digest is reset when an exception occurs during remote flush.
+ * Simulates a failure in the flush operation and verifies reset is called on MessageDigest.
+ */
+ @Test
+ public void testResetCalledOnExceptionInRemoteFlush() throws Exception {
+ AzureBlobFileSystem fs = Mockito.spy(getFileSystem());
+ Assume.assumeTrue(!getIsNamespaceEnabled(fs));
+ AzureBlobFileSystemStore store = Mockito.spy(fs.getAbfsStore());
+ assumeBlobServiceType();
+ Assume.assumeFalse("Not valid for APPEND BLOB", isAppendBlobEnabled());
+
+ // Create a file and spy on AbfsOutputStream
+ Path path = new Path("/testFile");
+ AbfsOutputStream realOs = (AbfsOutputStream) fs.create(path).getWrappedStream();
+ AbfsOutputStream os = Mockito.spy(realOs);
+ AzureIngressHandler ingressHandler = Mockito.spy(os.getIngressHandler());
+ Mockito.doReturn(ingressHandler).when(os).getIngressHandler();
+ AbfsClient spiedClient = Mockito.spy(ingressHandler.getClient());
+ Mockito.doReturn(spiedClient).when(ingressHandler).getClient();
+ AzureBlobBlockManager blockManager = Mockito.spy((AzureBlobBlockManager) os.getBlockManager());
+ Mockito.doReturn(blockManager).when(ingressHandler).getBlockManager();
+ Mockito.doReturn(true).when(blockManager).hasBlocksToCommit();
+ Mockito.doReturn("dummy-block-id").when(blockManager).getBlockIdToCommit();
+
+ MessageDigest mockMessageDigest = Mockito.mock(MessageDigest.class);
+ Mockito.doReturn(mockMessageDigest).when(os).getFullBlobContentMd5();
+ Mockito.doReturn(os).when(ingressHandler).getAbfsOutputStream();
+ Mockito.doReturn("dummyMd5").when(ingressHandler).computeFullBlobMd5();
+
+ // Simulating the exception in client flush call
+ Mockito.doThrow(
+ new AbfsRestOperationException(HTTP_UNAVAILABLE, "", "", new Exception()))
+ .when(spiedClient).flush(
+ Mockito.any(byte[].class),
+ Mockito.anyString(),
+ Mockito.anyBoolean(),
+ Mockito.nullable(String.class),
+ Mockito.nullable(String.class),
+ Mockito.anyString(),
+ Mockito.nullable(ContextEncryptionAdapter.class),
+ Mockito.any(TracingContext.class), Mockito.nullable(String.class));
+
+ // Triggering the flush to simulate exception
+ try {
+ ingressHandler.remoteFlush(0, false, false, null,
+ getTestTracingContext(fs, true));
+ } catch (AzureBlobFileSystemException e) {
+ //expected exception
+ }
+ // Verify that reset was called on the message digest
+ Mockito.verify(mockMessageDigest, Mockito.times(1)).reset();
+ }
}
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsOutputStream.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsOutputStream.java
index fb045b0ab6606..a4eefce0cb876 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsOutputStream.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsOutputStream.java
@@ -118,7 +118,7 @@ public void verifyShortWriteRequest() throws Exception {
AbfsConfiguration abfsConf;
final Configuration conf = new Configuration();
conf.set(accountKey1, accountValue1);
- abfsConf = Mockito.spy(new AbfsConfiguration(conf, accountName1));
+ abfsConf = new AbfsConfiguration(conf, accountName1);
AbfsPerfTracker tracker = new AbfsPerfTracker("test", accountName1, abfsConf);
when(client.getAbfsPerfTracker()).thenReturn(tracker);
when(client.getAbfsConfiguration()).thenReturn(abfsConf);
@@ -130,8 +130,7 @@ public void verifyShortWriteRequest() throws Exception {
isNull(), any(), any(TracingContext.class), anyString())).thenReturn(op);
when(clientHandler.getClient(any())).thenReturn(client);
when(clientHandler.getDfsClient()).thenReturn(client);
- AbfsOutputStream out;
- out = Mockito.spy(new AbfsOutputStream(
+ AbfsOutputStream out = Mockito.spy(new AbfsOutputStream(
populateAbfsOutputStreamContext(
BUFFER_SIZE,
true,
@@ -186,7 +185,7 @@ public void verifyWriteRequest() throws Exception {
AbfsConfiguration abfsConf;
final Configuration conf = new Configuration();
conf.set(accountKey1, accountValue1);
- abfsConf = Mockito.spy(new AbfsConfiguration(conf, accountName1));
+ abfsConf = new AbfsConfiguration(conf, accountName1);
AbfsPerfTracker tracker = new AbfsPerfTracker("test", accountName1, abfsConf);
when(client.getAbfsPerfTracker()).thenReturn(tracker);
when(client.getAbfsConfiguration()).thenReturn(abfsConf);
From 513511a9a9812854846e0331e700618eac81de57 Mon Sep 17 00:00:00 2001
From: Anmol Asrani
Date: Fri, 18 Jul 2025 03:53:55 -0700
Subject: [PATCH 18/25] checkstyle
---
.../hadoop/fs/azurebfs/ITestAzureBlobFileSystemAppend.java | 3 ---
.../hadoop/fs/azurebfs/services/ITestAbfsOutputStream.java | 1 -
2 files changed, 4 deletions(-)
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemAppend.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemAppend.java
index 9b7152233c15e..414f830aa2c3d 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemAppend.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemAppend.java
@@ -22,14 +22,12 @@
import java.io.IOException;
import java.io.OutputStream;
import java.lang.reflect.Field;
-import java.nio.charset.StandardCharsets;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.HashSet;
import java.util.List;
import java.util.Random;
import java.util.Set;
-import java.util.UUID;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.Future;
@@ -40,7 +38,6 @@
import org.junit.Test;
import org.mockito.Mockito;
-import org.apache.commons.codec.binary.Base64;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FSDataOutputStream;
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsOutputStream.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsOutputStream.java
index b2b2c697b593e..0b7cbb38db76e 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsOutputStream.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsOutputStream.java
@@ -27,7 +27,6 @@
import java.net.URL;
import java.security.MessageDigest;
import java.util.Arrays;
-import java.util.Collections;
import org.assertj.core.api.Assertions;
import org.junit.Assume;
From dbb743f569c6528fca6376a2582f78184c4099f1 Mon Sep 17 00:00:00 2001
From: Anmol Asrani
Date: Sun, 20 Jul 2025 23:50:22 -0700
Subject: [PATCH 19/25] Variable name correction
---
.../azurebfs/ITestWasbAbfsCompatibility.java | 104 +++++++++---------
1 file changed, 52 insertions(+), 52 deletions(-)
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestWasbAbfsCompatibility.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestWasbAbfsCompatibility.java
index 3d9264e4bc018..6f7d37d992951 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestWasbAbfsCompatibility.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestWasbAbfsCompatibility.java
@@ -422,10 +422,10 @@ public void testScenario3() throws Exception {
Path path = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
// Write
- try (FSDataOutputStream nativeFsStream = abfs.create(path, true)) {
- nativeFsStream.write(TEST_CONTEXT.getBytes());
- nativeFsStream.flush();
- nativeFsStream.hsync();
+ try (FSDataOutputStream abfsOutputStream = abfs.create(path, true)) {
+ abfsOutputStream.write(TEST_CONTEXT.getBytes());
+ abfsOutputStream.flush();
+ abfsOutputStream.hsync();
}
// Check file status
@@ -672,10 +672,10 @@ public void testScenario9() throws Exception {
Path testFile = path("/testReadFile");
Path path = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
- try (FSDataOutputStream nativeFsStream = abfs.create(path, true)) {
- nativeFsStream.write(TEST_CONTEXT.getBytes());
- nativeFsStream.flush();
- nativeFsStream.hsync();
+ try (FSDataOutputStream abfsOutputStream = abfs.create(path, true)) {
+ abfsOutputStream.write(TEST_CONTEXT.getBytes());
+ abfsOutputStream.flush();
+ abfsOutputStream.hsync();
}
// Check file status
@@ -715,10 +715,10 @@ public void testScenario10() throws Exception {
Path testFile = path("/testReadFile");
Path path = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
- try (FSDataOutputStream nativeFsStream = abfs.create(path, true)) {
- nativeFsStream.write(TEST_CONTEXT.getBytes());
- nativeFsStream.flush();
- nativeFsStream.hsync();
+ try (FSDataOutputStream abfsOutputStream = abfs.create(path, true)) {
+ abfsOutputStream.write(TEST_CONTEXT.getBytes());
+ abfsOutputStream.flush();
+ abfsOutputStream.hsync();
}
// Check file status
@@ -960,10 +960,10 @@ public void testScenario16() throws Exception {
// Write
wasb.create(path, true);
- try (FSDataOutputStream nativeFsStream = abfs.append(path)) {
- nativeFsStream.write(TEST_CONTEXT.getBytes());
- nativeFsStream.flush();
- nativeFsStream.hsync();
+ try (FSDataOutputStream abfsOutputStream = abfs.append(path)) {
+ abfsOutputStream.write(TEST_CONTEXT.getBytes());
+ abfsOutputStream.flush();
+ abfsOutputStream.hsync();
}
// Check file status
@@ -994,10 +994,10 @@ public void testScenario17() throws Exception {
Path path = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
// Write
- try (FSDataOutputStream nativeFsStream = abfs.create(path, true)) {
- nativeFsStream.write(TEST_CONTEXT.getBytes());
- nativeFsStream.flush();
- nativeFsStream.hsync();
+ try (FSDataOutputStream abfsOutputStream = abfs.create(path, true)) {
+ abfsOutputStream.write(TEST_CONTEXT.getBytes());
+ abfsOutputStream.flush();
+ abfsOutputStream.hsync();
}
// --- VALIDATE FILE ---
FileStatus status = abfs.getFileStatus(path);
@@ -1182,10 +1182,10 @@ public void testScenario21() throws Exception {
Path path = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
// Write
- try (FSDataOutputStream nativeFsStream = abfs.create(path, true)) {
- nativeFsStream.write(TEST_CONTEXT.getBytes());
- nativeFsStream.flush();
- nativeFsStream.hsync();
+ try (FSDataOutputStream abfsOutputStream = abfs.create(path, true)) {
+ abfsOutputStream.write(TEST_CONTEXT.getBytes());
+ abfsOutputStream.flush();
+ abfsOutputStream.hsync();
}
// --- VALIDATE FILE ---
FileStatus status = wasb.getFileStatus(path);
@@ -1330,10 +1330,10 @@ public void testScenario24() throws Exception {
Path path = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
// Write
- try (FSDataOutputStream nativeFsStream = abfs.create(path, true)) {
- nativeFsStream.write(TEST_CONTEXT.getBytes());
- nativeFsStream.flush();
- nativeFsStream.hsync();
+ try (FSDataOutputStream abfsOutputStream = abfs.create(path, true)) {
+ abfsOutputStream.write(TEST_CONTEXT.getBytes());
+ abfsOutputStream.flush();
+ abfsOutputStream.hsync();
}
// --- VALIDATE FILE ---
FileStatus status = wasb.getFileStatus(path);
@@ -1423,10 +1423,10 @@ public void testScenario26() throws Exception {
Path path = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
// Write
- try (FSDataOutputStream nativeFsStream = abfs.create(path, true)) {
- nativeFsStream.write(TEST_CONTEXT.getBytes());
- nativeFsStream.flush();
- nativeFsStream.hsync();
+ try (FSDataOutputStream abfsOutputStream = abfs.create(path, true)) {
+ abfsOutputStream.write(TEST_CONTEXT.getBytes());
+ abfsOutputStream.flush();
+ abfsOutputStream.hsync();
}
// --- VALIDATE FILE ---
FileStatus status = abfs.getFileStatus(path);
@@ -1469,10 +1469,10 @@ public void testScenario27() throws Exception {
Path testPath2 = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
// Write
- try (FSDataOutputStream nativeFsStream = abfs.create(testPath1, true)) {
- nativeFsStream.write(TEST_CONTEXT.getBytes());
- nativeFsStream.flush();
- nativeFsStream.hsync();
+ try (FSDataOutputStream abfsOutputStream = abfs.create(testPath1, true)) {
+ abfsOutputStream.write(TEST_CONTEXT.getBytes());
+ abfsOutputStream.flush();
+ abfsOutputStream.hsync();
}
// Check file status
@@ -1570,10 +1570,10 @@ public void testScenario29() throws Exception {
// Write
wasb.create(testPath1, true);
- try (FSDataOutputStream nativeFsStream = abfs.append(testPath1)) {
- nativeFsStream.write(TEST_CONTEXT.getBytes());
- nativeFsStream.flush();
- nativeFsStream.hsync();
+ try (FSDataOutputStream abfsOutputStream = abfs.append(testPath1)) {
+ abfsOutputStream.write(TEST_CONTEXT.getBytes());
+ abfsOutputStream.flush();
+ abfsOutputStream.hsync();
}
// Check file status
@@ -1773,10 +1773,10 @@ public void testScenario33() throws Exception {
// Write
abfs.mkdirs(testFile);
- try (FSDataOutputStream nativeFsStream = abfs.create(testPath1, true)) {
- nativeFsStream.write(TEST_CONTEXT.getBytes());
- nativeFsStream.flush();
- nativeFsStream.hsync();
+ try (FSDataOutputStream abfsOutputStream = abfs.create(testPath1, true)) {
+ abfsOutputStream.write(TEST_CONTEXT.getBytes());
+ abfsOutputStream.flush();
+ abfsOutputStream.hsync();
}
abfs.create(testPath2, true);
abfs.create(testPath3, true);
@@ -1823,10 +1823,10 @@ public void testScenario34() throws Exception {
// Write
abfs.mkdirs(testFile);
- try (FSDataOutputStream nativeFsStream = abfs.create(testPath1, true)) {
- nativeFsStream.write(TEST_CONTEXT.getBytes());
- nativeFsStream.flush();
- nativeFsStream.hsync();
+ try (FSDataOutputStream abfsOutputStream = abfs.create(testPath1, true)) {
+ abfsOutputStream.write(TEST_CONTEXT.getBytes());
+ abfsOutputStream.flush();
+ abfsOutputStream.hsync();
}
abfs.create(testPath3, true);
@@ -1965,10 +1965,10 @@ public void testScenario37() throws Exception {
// Write
abfs.mkdirs(testFile);
- try (FSDataOutputStream nativeFsStream = abfs.create(testPath1, true)) {
- nativeFsStream.write(TEST_CONTEXT.getBytes());
- nativeFsStream.flush();
- nativeFsStream.hsync();
+ try (FSDataOutputStream abfsOutputStream = abfs.create(testPath1, true)) {
+ abfsOutputStream.write(TEST_CONTEXT.getBytes());
+ abfsOutputStream.flush();
+ abfsOutputStream.hsync();
}
abfs.create(testPath3, true);
From 3c249d92770fde82e4e1d0036dd1780740223cbe Mon Sep 17 00:00:00 2001
From: Anmol Asrani
Date: Fri, 1 Aug 2025 04:47:59 -0700
Subject: [PATCH 20/25] md5 config changes
---
.../hadoop/fs/azurebfs/AbfsConfiguration.java | 8 +++++
.../azurebfs/constants/ConfigurationKeys.java | 3 ++
.../constants/FileSystemConfigurations.java | 1 +
.../fs/azurebfs/services/AbfsBlobClient.java | 23 +++++++++++--
.../fs/azurebfs/services/AbfsClient.java | 11 +++++++
.../fs/azurebfs/services/AbfsDfsClient.java | 3 +-
.../azurebfs/services/AbfsOutputStream.java | 32 ++++++++++++++++---
.../services/AzureBlobBlockManager.java | 4 ++-
.../services/AzureBlobIngressHandler.java | 11 +++++--
.../services/AzureDFSBlockManager.java | 4 ++-
.../services/AzureDFSIngressHandler.java | 9 ++++--
.../fs/azurebfs/services/RenameAtomicity.java | 9 ++++--
.../ITestAzureBlobFileSystemRename.java | 2 +-
.../azurebfs/ITestWasbAbfsCompatibility.java | 11 +++++--
.../azurebfs/services/AbfsClientTestUtil.java | 7 +++-
.../services/ITestAbfsOutputStream.java | 7 ++--
16 files changed, 120 insertions(+), 25 deletions(-)
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java
index 1242122f03015..09daae0b6f1f7 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java
@@ -433,6 +433,10 @@ public class AbfsConfiguration{
FS_AZURE_ABFS_ENABLE_CHECKSUM_VALIDATION, DefaultValue = DEFAULT_ENABLE_ABFS_CHECKSUM_VALIDATION)
private boolean isChecksumValidationEnabled;
+ @BooleanConfigurationValidatorAnnotation(ConfigurationKey =
+ FS_AZURE_ABFS_ENABLE_FULL_BLOB_CHECKSUM_VALIDATION, DefaultValue = DEFAULT_ENABLE_FULL_BLOB_ABFS_CHECKSUM_VALIDATION)
+ private boolean isFullBlobChecksumValidationEnabled;
+
@BooleanConfigurationValidatorAnnotation(ConfigurationKey =
FS_AZURE_ENABLE_PAGINATED_DELETE, DefaultValue = DEFAULT_ENABLE_PAGINATED_DELETE)
private boolean isPaginatedDeleteEnabled;
@@ -1692,6 +1696,10 @@ public void setIsChecksumValidationEnabled(boolean isChecksumValidationEnabled)
this.isChecksumValidationEnabled = isChecksumValidationEnabled;
}
+ public boolean isFullBlobChecksumValidationEnabled() {
+ return isFullBlobChecksumValidationEnabled;
+ }
+
public long getBlobCopyProgressPollWaitMillis() {
return blobCopyProgressPollWaitMillis;
}
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/ConfigurationKeys.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/ConfigurationKeys.java
index 50a88ab4e4587..e523a1c6fc626 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/ConfigurationKeys.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/ConfigurationKeys.java
@@ -356,6 +356,9 @@ public final class ConfigurationKeys {
/** Add extra layer of verification of the integrity of the request content during transport: {@value}. */
public static final String FS_AZURE_ABFS_ENABLE_CHECKSUM_VALIDATION = "fs.azure.enable.checksum.validation";
+ /** Add extra layer of verification of the integrity of the full blob request content during transport: {@value}. */
+ public static final String FS_AZURE_ABFS_ENABLE_FULL_BLOB_CHECKSUM_VALIDATION = "fs.azure.enable.full.blob.checksum.validation";
+
public static String accountProperty(String property, String account) {
return property + DOT + account;
}
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/FileSystemConfigurations.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/FileSystemConfigurations.java
index 824a4c9701ea0..b0c28da441861 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/FileSystemConfigurations.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/FileSystemConfigurations.java
@@ -146,6 +146,7 @@ public final class FileSystemConfigurations {
public static final boolean DEFAULT_ENABLE_ABFS_RENAME_RESILIENCE = true;
public static final boolean DEFAULT_ENABLE_PAGINATED_DELETE = false;
public static final boolean DEFAULT_ENABLE_ABFS_CHECKSUM_VALIDATION = false;
+ public static final boolean DEFAULT_ENABLE_FULL_BLOB_ABFS_CHECKSUM_VALIDATION = false;
/**
* Limit of queued block upload operations before writes
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsBlobClient.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsBlobClient.java
index 9e8a792763858..c10335b3541dc 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsBlobClient.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsBlobClient.java
@@ -1073,11 +1073,15 @@ public AbfsRestOperation flush(byte[] buffer,
requestHeaders.add(new AbfsHttpHeader(CONTENT_LENGTH, String.valueOf(buffer.length)));
requestHeaders.add(new AbfsHttpHeader(CONTENT_TYPE, APPLICATION_XML));
requestHeaders.add(new AbfsHttpHeader(IF_MATCH, eTag));
+ String md5Hash = null;
if (leaseId != null) {
requestHeaders.add(new AbfsHttpHeader(X_MS_LEASE_ID, leaseId));
}
- if (blobMd5 != null) {
+ if (isFullBlobChecksumValidationEnabled() && blobMd5 != null) {
requestHeaders.add(new AbfsHttpHeader(X_MS_BLOB_CONTENT_MD5, blobMd5));
+ } else {
+ md5Hash = computeMD5Hash(buffer, 0, buffer.length);
+ requestHeaders.add(new AbfsHttpHeader(X_MS_BLOB_CONTENT_MD5, md5Hash));
}
final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder();
abfsUriQueryBuilder.addQuery(QUERY_PARAM_COMP, BLOCKLIST);
@@ -1103,8 +1107,21 @@ public AbfsRestOperation flush(byte[] buffer,
AbfsRestOperation op1 = getPathStatus(path, true, tracingContext,
contextEncryptionAdapter);
String metadataMd5 = op1.getResult().getResponseHeader(CONTENT_MD5);
- if (blobMd5 != null && !blobMd5.equals(metadataMd5)) {
- throw ex;
+ /*
+ * Validate the response by comparing the server's MD5 metadata against either:
+ * 1. The full blob content MD5 (if full blob checksum validation is enabled), or
+ * 2. The full block ID buffer MD5 (fallback if blob checksum validation is disabled)
+ */
+ if (getAbfsConfiguration().isFullBlobChecksumValidationEnabled() && blobMd5 != null) {
+ // Full blob content MD5 mismatch — integrity check failed
+ if ( !blobMd5.equals(metadataMd5)) {
+ throw ex;
+ }
+ } else {
+ // Block ID buffer MD5 mismatch — integrity check failed
+ if (md5Hash != null && !md5Hash.equals(metadataMd5)) {
+ throw ex;
+ }
}
return op;
}
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java
index faf6bf8aadc8f..5c2cc6c6b183a 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java
@@ -1423,6 +1423,17 @@ protected boolean isChecksumValidationEnabled() {
return getAbfsConfiguration().getIsChecksumValidationEnabled();
}
+ /**
+ * Conditions check for allowing checksum support for write operation.
+ * Server will support this if client sends the MD5 Hash as a request header.
+ * For azure stoage service documentation and more details refer to
+ * Path - Update Azure Rest API.
+ * @return true if full blob checksum validation enabled.
+ */
+ protected boolean isFullBlobChecksumValidationEnabled() {
+ return getAbfsConfiguration().isFullBlobChecksumValidationEnabled();
+ }
+
/**
* Compute MD5Hash of the given byte array starting from given offset up to given length.
* @param data byte array from which data is fetched to compute MD5 Hash.
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsDfsClient.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsDfsClient.java
index 197744e79fe6c..f88df14c3423a 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsDfsClient.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsDfsClient.java
@@ -867,10 +867,9 @@ public AbfsRestOperation flush(final String path,
if (leaseId != null) {
requestHeaders.add(new AbfsHttpHeader(X_MS_LEASE_ID, leaseId));
}
- if (isChecksumValidationEnabled() && blobMd5 != null) {
+ if (isFullBlobChecksumValidationEnabled() && blobMd5 != null) {
requestHeaders.add(new AbfsHttpHeader(X_MS_BLOB_CONTENT_MD5, blobMd5));
}
-
final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder();
abfsUriQueryBuilder.addQuery(QUERY_PARAM_ACTION, FLUSH_ACTION);
abfsUriQueryBuilder.addQuery(QUERY_PARAM_POSITION, Long.toString(position));
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java
index 10e7d9910c884..90805078ea98b 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java
@@ -223,7 +223,7 @@ public AbfsOutputStream(AbfsOutputStreamContext abfsOutputStreamContext)
md5 = MessageDigest.getInstance(MD5);
fullBlobContentMd5 = MessageDigest.getInstance(MD5);
} catch (NoSuchAlgorithmException e) {
- if (client.isChecksumValidationEnabled()) {
+ if (isChecksumValidationEnabled()) {
throw new IOException("MD5 algorithm not available", e);
}
}
@@ -464,10 +464,13 @@ public synchronized void write(final byte[] data, final int off, final int lengt
AbfsBlock block = createBlockIfNeeded(position);
int written = bufferData(block, data, off, length);
// Update the incremental MD5 hash with the written data.
- getMessageDigest().update(data, off, written);
-
+ if (isChecksumValidationEnabled()) {
+ getMessageDigest().update(data, off, written);
+ }
// Update the full blob MD5 hash with the written data.
- getFullBlobContentMd5().update(data, off, written);
+ if (isFullBlobChecksumValidationEnabled()) {
+ getFullBlobContentMd5().update(data, off, written);
+ }
int remainingCapacity = block.remainingCapacity();
if (written < length) {
@@ -544,7 +547,12 @@ private void uploadBlockAsync(AbfsBlock blockToUpload,
outputStreamStatistics.bytesToUpload(bytesLength);
outputStreamStatistics.writeCurrentBuffer();
DataBlocks.BlockUploadData blockUploadData = blockToUpload.startUpload();
- String md5Hash = getMd5();
+ String md5Hash;
+ if (getClient().getAbfsConfiguration().getIsChecksumValidationEnabled()) {
+ md5Hash = getMd5();
+ } else {
+ md5Hash = null;
+ }
final Future job =
executorService.submit(() -> {
AbfsPerfTracker tracker =
@@ -1222,6 +1230,20 @@ public MessageDigest getFullBlobContentMd5() {
return fullBlobContentMd5;
}
+ /**
+ * @return true if checksum validation is enabled.
+ */
+ public boolean isChecksumValidationEnabled() {
+ return getClient().isChecksumValidationEnabled();
+ }
+
+ /**
+ * @return true if full blob checksum validation is enabled.
+ */
+ public boolean isFullBlobChecksumValidationEnabled() {
+ return getClient().isFullBlobChecksumValidationEnabled();
+ }
+
/**
* Returns the Base64-encoded MD5 checksum based on the current digest state.
* This finalizes the digest calculation. Returns null if the digest is empty.
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AzureBlobBlockManager.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AzureBlobBlockManager.java
index 4e3e20f00246c..befcf26f37a73 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AzureBlobBlockManager.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AzureBlobBlockManager.java
@@ -95,7 +95,9 @@ protected synchronized AbfsBlock createBlockInternal(long position)
setBlockCount(getBlockCount() + 1);
AbfsBlock activeBlock = new AbfsBlobBlock(getAbfsOutputStream(), position, getBlockIdLength(), getBlockCount());
activeBlock.setBlockEntry(addNewEntry(activeBlock.getBlockId(), activeBlock.getOffset()));
- getAbfsOutputStream().getMessageDigest().reset();
+ if (getAbfsOutputStream().isChecksumValidationEnabled()) {
+ getAbfsOutputStream().getMessageDigest().reset();
+ }
setActiveBlock(activeBlock);
}
return getActiveBlock();
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AzureBlobIngressHandler.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AzureBlobIngressHandler.java
index eaa4ca41c781e..8610adde30b26 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AzureBlobIngressHandler.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AzureBlobIngressHandler.java
@@ -180,7 +180,10 @@ protected synchronized AbfsRestOperation remoteFlush(final long offset,
tracingContextFlush.setIngressHandler(BLOB_FLUSH);
tracingContextFlush.setPosition(String.valueOf(offset));
LOG.trace("Flushing data at offset {} for path {}", offset, getAbfsOutputStream().getPath());
- String fullBlobMd5 = computeFullBlobMd5();
+ String fullBlobMd5 = null;
+ if (getClient().isFullBlobChecksumValidationEnabled()) {
+ fullBlobMd5 = computeFullBlobMd5();
+ }
op = getClient().flush(blockListXml.getBytes(StandardCharsets.UTF_8),
getAbfsOutputStream().getPath(),
isClose, getAbfsOutputStream().getCachedSasTokenString(), leaseId,
@@ -194,7 +197,9 @@ isClose, getAbfsOutputStream().getCachedSasTokenString(), leaseId,
LOG.error("Error in remote flush for path {} and offset {}", getAbfsOutputStream().getPath(), offset, ex);
throw ex;
} finally {
- getAbfsOutputStream().getFullBlobContentMd5().reset();
+ if (getClient().isFullBlobChecksumValidationEnabled()) {
+ getAbfsOutputStream().getFullBlobContentMd5().reset();
+ }
}
return op;
}
@@ -221,7 +226,7 @@ protected AbfsRestOperation remoteAppendBlobWrite(String path,
AppendRequestParameters reqParams,
TracingContext tracingContext) throws IOException {
// Perform the remote append operation using the blob client.
- AbfsRestOperation op = null;
+ AbfsRestOperation op;
try {
op = blobClient.appendBlock(path, reqParams, uploadData.toByteArray(), tracingContext);
} catch (AbfsRestOperationException ex) {
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AzureDFSBlockManager.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AzureDFSBlockManager.java
index c58f1d1b28461..dc0104a9ae223 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AzureDFSBlockManager.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AzureDFSBlockManager.java
@@ -62,7 +62,9 @@ protected synchronized AbfsBlock createBlockInternal(long position)
if (getActiveBlock() == null) {
setBlockCount(getBlockCount() + 1);
AbfsBlock activeBlock = new AbfsBlock(getAbfsOutputStream(), position);
- getAbfsOutputStream().getMessageDigest().reset();
+ if (getAbfsOutputStream().isChecksumValidationEnabled()) {
+ getAbfsOutputStream().getMessageDigest().reset();
+ }
setActiveBlock(activeBlock);
}
return getActiveBlock();
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AzureDFSIngressHandler.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AzureDFSIngressHandler.java
index faf293b9242ce..96a8f07cc3d50 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AzureDFSIngressHandler.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AzureDFSIngressHandler.java
@@ -179,7 +179,10 @@ protected synchronized AbfsRestOperation remoteFlush(final long offset,
tracingContextFlush.setIngressHandler(DFS_FLUSH);
tracingContextFlush.setPosition(String.valueOf(offset));
}
- String fullBlobMd5 = computeFullBlobMd5();
+ String fullBlobMd5 = null;
+ if (getClient().isFullBlobChecksumValidationEnabled()) {
+ fullBlobMd5 = computeFullBlobMd5();
+ }
LOG.trace("Flushing data at offset {} and path {}", offset, getAbfsOutputStream().getPath());
AbfsRestOperation op;
try {
@@ -194,7 +197,9 @@ protected synchronized AbfsRestOperation remoteFlush(final long offset,
getAbfsOutputStream().getPath(), offset, ex);
throw ex;
} finally {
- getAbfsOutputStream().getFullBlobContentMd5().reset();
+ if (getClient().isFullBlobChecksumValidationEnabled()) {
+ getAbfsOutputStream().getFullBlobContentMd5().reset();
+ }
}
return op;
}
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/RenameAtomicity.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/RenameAtomicity.java
index 24f97238bde36..43157c6d730a4 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/RenameAtomicity.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/RenameAtomicity.java
@@ -211,13 +211,18 @@ void createRenamePendingJson(Path path, byte[] bytes)
String blockId = generateBlockId();
String blockList = generateBlockListXml(blockId);
byte[] buffer = blockList.getBytes(StandardCharsets.UTF_8);
- String computedMd5 = abfsClient.computeMD5Hash(buffer, 0, buffer.length);
+ String computedMd5 = null;
+ if (abfsClient.isFullBlobChecksumValidationEnabled()) {
+ computedMd5 = abfsClient.computeMD5Hash(buffer, 0, buffer.length);
+ }
AppendRequestParameters appendRequestParameters
= new AppendRequestParameters(0, 0,
bytes.length, AppendRequestParameters.Mode.APPEND_MODE, false, null,
abfsClient.getAbfsConfiguration().isExpectHeaderEnabled(),
- new BlobAppendRequestParameters(blockId, eTag), abfsClient.computeMD5Hash(bytes, 0, bytes.length));
+ new BlobAppendRequestParameters(blockId, eTag),
+ abfsClient.isChecksumValidationEnabled() ? abfsClient.computeMD5Hash(
+ bytes, 0, bytes.length) : null);
abfsClient.append(path.toUri().getPath(), bytes,
appendRequestParameters, null, null, tracingContext);
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemRename.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemRename.java
index 14f2a14b2b64d..e4a2d0eed4d66 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemRename.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemRename.java
@@ -2128,7 +2128,7 @@ private void testRenamePreRenameFailureResolution(final AzureBlobFileSystem fs)
Mockito.anyBoolean(), Mockito.nullable(String.class),
Mockito.nullable(String.class), Mockito.anyString(),
Mockito.nullable(ContextEncryptionAdapter.class),
- Mockito.any(TracingContext.class), Mockito.anyString());
+ Mockito.any(TracingContext.class), Mockito.nullable(String.class));
return createAnswer.callRealMethod();
};
RenameAtomicityTestUtils.addCreatePathMock(client,
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestWasbAbfsCompatibility.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestWasbAbfsCompatibility.java
index 6f7d37d992951..223d4273cff5d 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestWasbAbfsCompatibility.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestWasbAbfsCompatibility.java
@@ -43,6 +43,7 @@
import static java.net.HttpURLConnection.HTTP_CONFLICT;
import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_ABFS_ENABLE_CHECKSUM_VALIDATION;
+import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_ABFS_ENABLE_FULL_BLOB_CHECKSUM_VALIDATION;
import static org.apache.hadoop.fs.contract.ContractTestUtils.assertDeleted;
import static org.apache.hadoop.fs.contract.ContractTestUtils.assertIsDirectory;
import static org.apache.hadoop.fs.contract.ContractTestUtils.assertMkdirs;
@@ -113,7 +114,10 @@ public void testReadFile() throws Exception {
boolean[] createFileWithAbfs = new boolean[]{false, true, false, true};
boolean[] readFileWithAbfs = new boolean[]{false, true, true, false};
- AzureBlobFileSystem abfs = getFileSystem();
+ Configuration conf = getRawConfiguration();
+ conf.setBoolean(FS_AZURE_ABFS_ENABLE_FULL_BLOB_CHECKSUM_VALIDATION, true);
+ FileSystem fileSystem = FileSystem.newInstance(conf);
+ AzureBlobFileSystem abfs = (AzureBlobFileSystem) fileSystem;
// test only valid for non-namespace enabled account
Assume.assumeFalse("Namespace enabled account does not support this test",
getIsNamespaceEnabled(abfs));
@@ -412,7 +416,10 @@ public void testScenario2() throws Exception {
*/
@Test
public void testScenario3() throws Exception {
- AzureBlobFileSystem abfs = getFileSystem();
+ Configuration conf = getRawConfiguration();
+ conf.setBoolean(FS_AZURE_ABFS_ENABLE_FULL_BLOB_CHECKSUM_VALIDATION, true);
+ FileSystem fileSystem = FileSystem.newInstance(conf);
+ AzureBlobFileSystem abfs = (AzureBlobFileSystem) fileSystem;
Assume.assumeFalse("Namespace enabled account does not support this test",
getIsNamespaceEnabled(abfs));
Assume.assumeFalse("Not valid for APPEND BLOB", isAppendBlobEnabled());
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/AbfsClientTestUtil.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/AbfsClientTestUtil.java
index 4ab5f68bb3ca5..bdeee28d9d54e 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/AbfsClientTestUtil.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/AbfsClientTestUtil.java
@@ -166,7 +166,12 @@ public static void setMockAbfsRestOperationForFlushOperation(
requestHeaders.add(new AbfsHttpHeader(CONTENT_LENGTH, String.valueOf(buffer.length)));
requestHeaders.add(new AbfsHttpHeader(CONTENT_TYPE, APPLICATION_XML));
requestHeaders.add(new AbfsHttpHeader(IF_MATCH, eTag));
- requestHeaders.add(new AbfsHttpHeader(X_MS_BLOB_CONTENT_MD5, blobMd5));
+ if (spiedClient.isFullBlobChecksumValidationEnabled()) {
+ requestHeaders.add(new AbfsHttpHeader(X_MS_BLOB_CONTENT_MD5, blobMd5));
+ } else {
+ requestHeaders.add(new AbfsHttpHeader(X_MS_BLOB_CONTENT_MD5,
+ spiedClient.computeMD5Hash(buffer, 0, buffer.length)));
+ }
final AbfsUriQueryBuilder abfsUriQueryBuilder = spiedClient.createDefaultUriQueryBuilder();
abfsUriQueryBuilder.addQuery(QUERY_PARAM_COMP, BLOCKLIST);
abfsUriQueryBuilder.addQuery(QUERY_PARAM_CLOSE, String.valueOf(false));
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsOutputStream.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsOutputStream.java
index 0b7cbb38db76e..4ec1090617fd9 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsOutputStream.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsOutputStream.java
@@ -33,6 +33,7 @@
import org.junit.Test;
import org.junit.runner.RunWith;
import org.junit.runners.Parameterized;
+import org.mockito.Mock;
import org.mockito.Mockito;
import org.apache.hadoop.conf.Configuration;
@@ -427,7 +428,7 @@ public void testNoNetworkCallsForSecondFlush() throws Exception {
Mockito.any(TracingContext.class));
Mockito.verify(blobClient, Mockito.times(1)).
flush(Mockito.any(byte[].class), Mockito.anyString(), Mockito.anyBoolean(), Mockito.any(), Mockito.any(), Mockito.anyString(), Mockito.any(),
- Mockito.any(TracingContext.class), Mockito.anyString());
+ Mockito.any(TracingContext.class), Mockito.nullable(String.class));
}
/**
@@ -481,6 +482,8 @@ public void testResetCalledOnExceptionInRemoteFlush() throws Exception {
//expected exception
}
// Verify that reset was called on the message digest
- Mockito.verify(mockMessageDigest, Mockito.times(1)).reset();
+ if (spiedClient.isChecksumValidationEnabled()) {
+ Mockito.verify(mockMessageDigest, Mockito.times(1)).reset();
+ }
}
}
From 4c24330ce3c53ad29b5aa575b079cea7bf892aaa Mon Sep 17 00:00:00 2001
From: Anmol Asrani
Date: Tue, 5 Aug 2025 00:01:28 -0700
Subject: [PATCH 21/25] checkstyle fixes
---
.../org/apache/hadoop/fs/azurebfs/services/AbfsBlobClient.java | 2 +-
.../hadoop/fs/azurebfs/services/ITestAbfsOutputStream.java | 1 -
2 files changed, 1 insertion(+), 2 deletions(-)
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsBlobClient.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsBlobClient.java
index c10335b3541dc..ec0aaf206fdc3 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsBlobClient.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsBlobClient.java
@@ -1114,7 +1114,7 @@ public AbfsRestOperation flush(byte[] buffer,
*/
if (getAbfsConfiguration().isFullBlobChecksumValidationEnabled() && blobMd5 != null) {
// Full blob content MD5 mismatch — integrity check failed
- if ( !blobMd5.equals(metadataMd5)) {
+ if (!blobMd5.equals(metadataMd5)) {
throw ex;
}
} else {
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsOutputStream.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsOutputStream.java
index 4ec1090617fd9..c2f90332007c1 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsOutputStream.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsOutputStream.java
@@ -33,7 +33,6 @@
import org.junit.Test;
import org.junit.runner.RunWith;
import org.junit.runners.Parameterized;
-import org.mockito.Mock;
import org.mockito.Mockito;
import org.apache.hadoop.conf.Configuration;
From 144ba1ada896dbf3d2fb8d4a3e0a5173c1339f51 Mon Sep 17 00:00:00 2001
From: Anmol Asrani
Date: Fri, 8 Aug 2025 04:56:02 -0700
Subject: [PATCH 22/25] PR commnets
---
.../hadoop/fs/azurebfs/AbfsConfiguration.java | 2 +-
.../azurebfs/constants/ConfigurationKeys.java | 2 +-
.../fs/azurebfs/services/AbfsBlobClient.java | 25 +-
.../fs/azurebfs/services/AbfsClient.java | 4 +-
.../azurebfs/services/AbfsOutputStream.java | 7 +-
.../azurebfs/ITestWasbAbfsCompatibility.java | 2937 +++++++++--------
.../azurebfs/services/AbfsClientTestUtil.java | 12 +-
.../services/ITestAbfsOutputStream.java | 102 +-
8 files changed, 1633 insertions(+), 1458 deletions(-)
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java
index 09daae0b6f1f7..bb757d61bee24 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java
@@ -434,7 +434,7 @@ public class AbfsConfiguration{
private boolean isChecksumValidationEnabled;
@BooleanConfigurationValidatorAnnotation(ConfigurationKey =
- FS_AZURE_ABFS_ENABLE_FULL_BLOB_CHECKSUM_VALIDATION, DefaultValue = DEFAULT_ENABLE_FULL_BLOB_ABFS_CHECKSUM_VALIDATION)
+ FS_AZURE_ENABLE_FULL_BLOB_CHECKSUM_VALIDATION, DefaultValue = DEFAULT_ENABLE_FULL_BLOB_ABFS_CHECKSUM_VALIDATION)
private boolean isFullBlobChecksumValidationEnabled;
@BooleanConfigurationValidatorAnnotation(ConfigurationKey =
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/ConfigurationKeys.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/ConfigurationKeys.java
index e523a1c6fc626..ef39fb11b2d19 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/ConfigurationKeys.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/ConfigurationKeys.java
@@ -357,7 +357,7 @@ public final class ConfigurationKeys {
public static final String FS_AZURE_ABFS_ENABLE_CHECKSUM_VALIDATION = "fs.azure.enable.checksum.validation";
/** Add extra layer of verification of the integrity of the full blob request content during transport: {@value}. */
- public static final String FS_AZURE_ABFS_ENABLE_FULL_BLOB_CHECKSUM_VALIDATION = "fs.azure.enable.full.blob.checksum.validation";
+ public static final String FS_AZURE_ENABLE_FULL_BLOB_CHECKSUM_VALIDATION = "fs.azure.enable.full.blob.checksum.validation";
public static String accountProperty(String property, String account) {
return property + DOT + account;
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsBlobClient.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsBlobClient.java
index a20fae4932893..6d372a570dfde 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsBlobClient.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsBlobClient.java
@@ -1073,16 +1073,13 @@ public AbfsRestOperation flush(byte[] buffer,
requestHeaders.add(new AbfsHttpHeader(CONTENT_LENGTH, String.valueOf(buffer.length)));
requestHeaders.add(new AbfsHttpHeader(CONTENT_TYPE, APPLICATION_XML));
requestHeaders.add(new AbfsHttpHeader(IF_MATCH, eTag));
- String md5Hash = null;
if (leaseId != null) {
requestHeaders.add(new AbfsHttpHeader(X_MS_LEASE_ID, leaseId));
}
- if (isFullBlobChecksumValidationEnabled() && blobMd5 != null) {
- requestHeaders.add(new AbfsHttpHeader(X_MS_BLOB_CONTENT_MD5, blobMd5));
- } else {
- md5Hash = computeMD5Hash(buffer, 0, buffer.length);
- requestHeaders.add(new AbfsHttpHeader(X_MS_BLOB_CONTENT_MD5, md5Hash));
- }
+ String md5Value = isFullBlobChecksumValidationEnabled()
+ ? blobMd5
+ : computeMD5Hash(buffer, 0, buffer.length);
+ requestHeaders.add(new AbfsHttpHeader(X_MS_BLOB_CONTENT_MD5, md5Value));
final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder();
abfsUriQueryBuilder.addQuery(QUERY_PARAM_COMP, BLOCKLIST);
abfsUriQueryBuilder.addQuery(QUERY_PARAM_CLOSE, String.valueOf(isClose));
@@ -1110,18 +1107,10 @@ public AbfsRestOperation flush(byte[] buffer,
/*
* Validate the response by comparing the server's MD5 metadata against either:
* 1. The full blob content MD5 (if full blob checksum validation is enabled), or
- * 2. The full block ID buffer MD5 (fallback if blob checksum validation is disabled)
+ * 2. The full block ID list buffer MD5 (fallback if blob checksum validation is disabled)
*/
- if (getAbfsConfiguration().isFullBlobChecksumValidationEnabled() && blobMd5 != null) {
- // Full blob content MD5 mismatch — integrity check failed
- if (!blobMd5.equals(metadataMd5)) {
- throw ex;
- }
- } else {
- // Block ID buffer MD5 mismatch — integrity check failed
- if (md5Hash != null && !md5Hash.equals(metadataMd5)) {
- throw ex;
- }
+ if (md5Value != null && !md5Value.equals(metadataMd5)) {
+ throw ex;
}
return op;
}
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java
index 5c2cc6c6b183a..107558bb6e421 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java
@@ -1415,7 +1415,7 @@ protected boolean isChecksumValidationEnabled(List requestHeader
/**
* Conditions check for allowing checksum support for write operation.
* Server will support this if client sends the MD5 Hash as a request header.
- * For azure stoage service documentation and more details refer to
+ * For azure storage service documentation and more details refer to
* Path - Update Azure Rest API.
* @return true if checksum validation enabled.
*/
@@ -1426,7 +1426,7 @@ protected boolean isChecksumValidationEnabled() {
/**
* Conditions check for allowing checksum support for write operation.
* Server will support this if client sends the MD5 Hash as a request header.
- * For azure stoage service documentation and more details refer to
+ * For azure storage service documentation and more details refer to
* Path - Update Azure Rest API.
* @return true if full blob checksum validation enabled.
*/
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java
index 90805078ea98b..3abf21c778793 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java
@@ -547,12 +547,7 @@ private void uploadBlockAsync(AbfsBlock blockToUpload,
outputStreamStatistics.bytesToUpload(bytesLength);
outputStreamStatistics.writeCurrentBuffer();
DataBlocks.BlockUploadData blockUploadData = blockToUpload.startUpload();
- String md5Hash;
- if (getClient().getAbfsConfiguration().getIsChecksumValidationEnabled()) {
- md5Hash = getMd5();
- } else {
- md5Hash = null;
- }
+ String md5Hash = getClient().isChecksumValidationEnabled() ? getMd5() : null;
final Future job =
executorService.submit(() -> {
AbfsPerfTracker tracker =
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestWasbAbfsCompatibility.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestWasbAbfsCompatibility.java
index 223d4273cff5d..28382dde7329f 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestWasbAbfsCompatibility.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestWasbAbfsCompatibility.java
@@ -43,7 +43,7 @@
import static java.net.HttpURLConnection.HTTP_CONFLICT;
import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_ABFS_ENABLE_CHECKSUM_VALIDATION;
-import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_ABFS_ENABLE_FULL_BLOB_CHECKSUM_VALIDATION;
+import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_ENABLE_FULL_BLOB_CHECKSUM_VALIDATION;
import static org.apache.hadoop.fs.contract.ContractTestUtils.assertDeleted;
import static org.apache.hadoop.fs.contract.ContractTestUtils.assertIsDirectory;
import static org.apache.hadoop.fs.contract.ContractTestUtils.assertMkdirs;
@@ -115,82 +115,85 @@ public void testReadFile() throws Exception {
boolean[] readFileWithAbfs = new boolean[]{false, true, true, false};
Configuration conf = getRawConfiguration();
- conf.setBoolean(FS_AZURE_ABFS_ENABLE_FULL_BLOB_CHECKSUM_VALIDATION, true);
+ conf.setBoolean(FS_AZURE_ENABLE_FULL_BLOB_CHECKSUM_VALIDATION, true);
FileSystem fileSystem = FileSystem.newInstance(conf);
- AzureBlobFileSystem abfs = (AzureBlobFileSystem) fileSystem;
- // test only valid for non-namespace enabled account
- Assume.assumeFalse("Namespace enabled account does not support this test",
- getIsNamespaceEnabled(abfs));
- Assume.assumeFalse("Not valid for APPEND BLOB", isAppendBlobEnabled());
-
- NativeAzureFileSystem wasb = getWasbFileSystem();
+ try (AzureBlobFileSystem abfs = (AzureBlobFileSystem) fileSystem) {
+ // test only valid for non-namespace enabled account
+ Assume.assumeFalse("Namespace enabled account does not support this test",
+ getIsNamespaceEnabled(abfs));
+ Assume.assumeFalse("Not valid for APPEND BLOB", isAppendBlobEnabled());
+
+ NativeAzureFileSystem wasb = getWasbFileSystem();
+
+ Path testFile = path("/testReadFile");
+ for (int i = 0; i < 4; i++) {
+ Path path = new Path(testFile + "/~12/!008/testfile" + i);
+ final FileSystem createFs = createFileWithAbfs[i] ? abfs : wasb;
+ // Read
+ final FileSystem readFs = readFileWithAbfs[i] ? abfs : wasb;
+ // Write
+ try (FSDataOutputStream nativeFsStream = createFs.create(path, true)) {
+ nativeFsStream.write(TEST_CONTEXT.getBytes());
+ nativeFsStream.flush();
+ nativeFsStream.hsync();
+ }
+
+ // Check file status
+ ContractTestUtils.assertIsFile(createFs, path);
+
+ try (BufferedReader br = new BufferedReader(
+ new InputStreamReader(readFs.open(path)))) {
+ String line = br.readLine();
+ assertEquals("Wrong text from " + readFs,
+ TEST_CONTEXT, line);
+ }
+
+ // Remove file
+ assertDeleted(readFs, path, true);
+ }
+ }
+ }
- Path testFile = path("/testReadFile");
- for (int i = 0; i < 4; i++) {
- Path path = new Path(testFile + "/~12/!008/testfile" + i);
- final FileSystem createFs = createFileWithAbfs[i] ? abfs : wasb;
- // Read
- final FileSystem readFs = readFileWithAbfs[i] ? abfs : wasb;
+ /**
+ * Flow: Create and write a file using WASB, then read and append to it using ABFS. Finally, delete the file via ABFS after verifying content consistency.
+ * Expected: WASB successfully creates the file and writes content. ABFS reads, appends, and deletes the file without data loss or errors.
+ */
+ @Test
+ public void testwriteFile() throws Exception {
+ try (AzureBlobFileSystem abfs = getFileSystem()) {
+ Assume.assumeFalse("Namespace enabled account does not support this test",
+ getIsNamespaceEnabled(abfs));
+ assumeBlobServiceType();
+ Assume.assumeFalse("Not valid for APPEND BLOB", isAppendBlobEnabled());
+ NativeAzureFileSystem wasb = getWasbFileSystem();
+
+ Path testFile = path("/testReadFile");
+ Path path = new Path(
+ testFile + "/~12/!008/testfile_" + UUID.randomUUID());
// Write
- try (FSDataOutputStream nativeFsStream = createFs.create(path, true)) {
+ try (FSDataOutputStream nativeFsStream = wasb.create(path, true)) {
nativeFsStream.write(TEST_CONTEXT.getBytes());
nativeFsStream.flush();
nativeFsStream.hsync();
}
// Check file status
- ContractTestUtils.assertIsFile(createFs, path);
+ ContractTestUtils.assertIsFile(wasb, path);
try (BufferedReader br = new BufferedReader(
- new InputStreamReader(readFs.open(path)))) {
+ new InputStreamReader(abfs.open(path)))) {
String line = br.readLine();
- assertEquals("Wrong text from " + readFs,
+ assertEquals("Wrong text from " + abfs,
TEST_CONTEXT, line);
}
-
+ try (FSDataOutputStream abfsOutputStream = abfs.append(path)) {
+ abfsOutputStream.write(TEST_CONTEXT.getBytes());
+ abfsOutputStream.flush();
+ abfsOutputStream.hsync();
+ }
// Remove file
- assertDeleted(readFs, path, true);
- }
- }
-
- /**
- * Flow: Create and write a file using WASB, then read and append to it using ABFS. Finally, delete the file via ABFS after verifying content consistency.
- * Expected: WASB successfully creates the file and writes content. ABFS reads, appends, and deletes the file without data loss or errors.
- */
- @Test
- public void testwriteFile() throws Exception {
- AzureBlobFileSystem abfs = getFileSystem();
- Assume.assumeFalse("Namespace enabled account does not support this test",
- getIsNamespaceEnabled(abfs));
- assumeBlobServiceType();
- Assume.assumeFalse("Not valid for APPEND BLOB", isAppendBlobEnabled());
- NativeAzureFileSystem wasb = getWasbFileSystem();
-
- Path testFile = path("/testReadFile");
- Path path = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
- // Write
- try (FSDataOutputStream nativeFsStream = wasb.create(path, true)) {
- nativeFsStream.write(TEST_CONTEXT.getBytes());
- nativeFsStream.flush();
- nativeFsStream.hsync();
- }
-
- // Check file status
- ContractTestUtils.assertIsFile(wasb, path);
-
- try (BufferedReader br = new BufferedReader(
- new InputStreamReader(abfs.open(path)))) {
- String line = br.readLine();
- assertEquals("Wrong text from " + abfs,
- TEST_CONTEXT, line);
- }
- try (FSDataOutputStream abfsOutputStream = abfs.append(path)) {
- abfsOutputStream.write(TEST_CONTEXT.getBytes());
- abfsOutputStream.flush();
- abfsOutputStream.hsync();
+ assertDeleted(abfs, path, true);
}
- // Remove file
- assertDeleted(abfs, path, true);
}
/**
@@ -200,38 +203,40 @@ public void testwriteFile() throws Exception {
@Test
public void testwriteFile1() throws Exception {
- AzureBlobFileSystem abfs = getFileSystem();
- Assume.assumeFalse("Namespace enabled account does not support this test",
- getIsNamespaceEnabled(abfs));
- assumeBlobServiceType();
- Assume.assumeFalse("Not valid for APPEND BLOB", isAppendBlobEnabled());
- NativeAzureFileSystem wasb = getWasbFileSystem();
-
- Path testFile = path("/testReadFile");
- Path path = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
- // Write
- try (FSDataOutputStream nativeFsStream = abfs.create(path, true)) {
- nativeFsStream.write(TEST_CONTEXT.getBytes());
- nativeFsStream.flush();
- nativeFsStream.hsync();
- }
+ try (AzureBlobFileSystem abfs = getFileSystem()) {
+ Assume.assumeFalse("Namespace enabled account does not support this test",
+ getIsNamespaceEnabled(abfs));
+ assumeBlobServiceType();
+ Assume.assumeFalse("Not valid for APPEND BLOB", isAppendBlobEnabled());
+ NativeAzureFileSystem wasb = getWasbFileSystem();
+
+ Path testFile = path("/testReadFile");
+ Path path = new Path(
+ testFile + "/~12/!008/testfile_" + UUID.randomUUID());
+ // Write
+ try (FSDataOutputStream nativeFsStream = abfs.create(path, true)) {
+ nativeFsStream.write(TEST_CONTEXT.getBytes());
+ nativeFsStream.flush();
+ nativeFsStream.hsync();
+ }
- // Check file status
- ContractTestUtils.assertIsFile(abfs, path);
+ // Check file status
+ ContractTestUtils.assertIsFile(abfs, path);
- try (FSDataOutputStream nativeFsStream = wasb.append(path)) {
- nativeFsStream.write(TEST_CONTEXT.getBytes());
- nativeFsStream.flush();
- nativeFsStream.hsync();
- }
+ try (FSDataOutputStream nativeFsStream = wasb.append(path)) {
+ nativeFsStream.write(TEST_CONTEXT.getBytes());
+ nativeFsStream.flush();
+ nativeFsStream.hsync();
+ }
- try (FSDataOutputStream nativeFsStream = abfs.append(path)) {
- nativeFsStream.write(TEST_CONTEXT1.getBytes());
- nativeFsStream.flush();
- nativeFsStream.hsync();
+ try (FSDataOutputStream nativeFsStream = abfs.append(path)) {
+ nativeFsStream.write(TEST_CONTEXT1.getBytes());
+ nativeFsStream.flush();
+ nativeFsStream.hsync();
+ }
+ // Remove file
+ assertDeleted(abfs, path, true);
}
- // Remove file
- assertDeleted(abfs, path, true);
}
/**
@@ -240,22 +245,24 @@ public void testwriteFile1() throws Exception {
*/
@Test
public void testazcopywasbcompatibility() throws Exception {
- AzureBlobFileSystem abfs = getFileSystem();
- Assume.assumeFalse("Namespace enabled account does not support this test",
- getIsNamespaceEnabled(abfs));
- Assume.assumeFalse("Not valid for APPEND BLOB", isAppendBlobEnabled());
-
- Path testFile = path("/testReadFile");
- Path path = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
- createAzCopyFile(path);
-
- try (FSDataOutputStream nativeFsStream = abfs.append(path)) {
- nativeFsStream.write(TEST_CONTEXT1.getBytes());
- nativeFsStream.flush();
- nativeFsStream.hsync();
+ try (AzureBlobFileSystem abfs = getFileSystem()) {
+ Assume.assumeFalse("Namespace enabled account does not support this test",
+ getIsNamespaceEnabled(abfs));
+ Assume.assumeFalse("Not valid for APPEND BLOB", isAppendBlobEnabled());
+
+ Path testFile = path("/testReadFile");
+ Path path = new Path(
+ testFile + "/~12/!008/testfile_" + UUID.randomUUID());
+ createAzCopyFile(path);
+
+ try (FSDataOutputStream nativeFsStream = abfs.append(path)) {
+ nativeFsStream.write(TEST_CONTEXT1.getBytes());
+ nativeFsStream.flush();
+ nativeFsStream.hsync();
+ }
+ // Remove file
+ assertDeleted(abfs, path, true);
}
- // Remove file
- assertDeleted(abfs, path, true);
}
@@ -264,28 +271,29 @@ public void testDir() throws Exception {
boolean[] createDirWithAbfs = new boolean[]{false, true, false, true};
boolean[] readDirWithAbfs = new boolean[]{false, true, true, false};
- AzureBlobFileSystem abfs = getFileSystem();
- // test only valid for non-namespace enabled account
- Assume.assumeFalse("Namespace enabled account does not support this test",
- getIsNamespaceEnabled(abfs));
-
- NativeAzureFileSystem wasb = getWasbFileSystem();
-
- Path testDir = path("/testDir");
- for (int i = 0; i < 4; i++) {
- Path path = new Path(testDir + "/t" + i);
- //create
- final FileSystem createFs = createDirWithAbfs[i] ? abfs : wasb;
- assertTrue(createFs.mkdirs(path));
- //check
- assertPathExists(createFs, "Created dir not found with " + createFs,
- path);
- //read
- final FileSystem readFs = readDirWithAbfs[i] ? abfs : wasb;
- assertPathExists(readFs, "Created dir not found with " + readFs,
- path);
- assertIsDirectory(readFs, path);
- assertDeleted(readFs, path, true);
+ try (AzureBlobFileSystem abfs = getFileSystem()) {
+ // test only valid for non-namespace enabled account
+ Assume.assumeFalse("Namespace enabled account does not support this test",
+ getIsNamespaceEnabled(abfs));
+
+ NativeAzureFileSystem wasb = getWasbFileSystem();
+
+ Path testDir = path("/testDir");
+ for (int i = 0; i < 4; i++) {
+ Path path = new Path(testDir + "/t" + i);
+ //create
+ final FileSystem createFs = createDirWithAbfs[i] ? abfs : wasb;
+ assertTrue(createFs.mkdirs(path));
+ //check
+ assertPathExists(createFs, "Created dir not found with " + createFs,
+ path);
+ //read
+ final FileSystem readFs = readDirWithAbfs[i] ? abfs : wasb;
+ assertPathExists(readFs, "Created dir not found with " + readFs,
+ path);
+ assertIsDirectory(readFs, path);
+ assertDeleted(readFs, path, true);
+ }
}
}
@@ -303,32 +311,34 @@ public void testUrlConversion() {
@Test
public void testSetWorkingDirectory() throws Exception {
//create folders
- AzureBlobFileSystem abfs = getFileSystem();
- // test only valid for non-namespace enabled account
- Assume.assumeFalse("Namespace enabled account does not support this test",
- getIsNamespaceEnabled(abfs));
-
- NativeAzureFileSystem wasb = getWasbFileSystem();
-
- Path d1 = path("/d1");
- Path d1d4 = new Path(d1 + "/d2/d3/d4");
- assertMkdirs(abfs, d1d4);
-
- //set working directory to path1
- Path path1 = new Path(d1 + "/d2");
- wasb.setWorkingDirectory(path1);
- abfs.setWorkingDirectory(path1);
- assertEquals(path1, wasb.getWorkingDirectory());
- assertEquals(path1, abfs.getWorkingDirectory());
-
- //set working directory to path2
- Path path2 = new Path("d3/d4");
- wasb.setWorkingDirectory(path2);
- abfs.setWorkingDirectory(path2);
-
- Path path3 = d1d4;
- assertEquals(path3, wasb.getWorkingDirectory());
- assertEquals(path3, abfs.getWorkingDirectory());
+ try (AzureBlobFileSystem abfs = getFileSystem()) {
+ // test only valid for non-namespace enabled account
+ Assume.assumeFalse("Namespace enabled account does not support this test",
+ getIsNamespaceEnabled(abfs));
+
+ try (NativeAzureFileSystem wasb = getWasbFileSystem()) {
+
+ Path d1 = path("/d1");
+ Path d1d4 = new Path(d1 + "/d2/d3/d4");
+ assertMkdirs(abfs, d1d4);
+
+ //set working directory to path1
+ Path path1 = new Path(d1 + "/d2");
+ wasb.setWorkingDirectory(path1);
+ abfs.setWorkingDirectory(path1);
+ assertEquals(path1, wasb.getWorkingDirectory());
+ assertEquals(path1, abfs.getWorkingDirectory());
+
+ //set working directory to path2
+ Path path2 = new Path("d3/d4");
+ wasb.setWorkingDirectory(path2);
+ abfs.setWorkingDirectory(path2);
+
+ Path path3 = d1d4;
+ assertEquals(path3, wasb.getWorkingDirectory());
+ assertEquals(path3, abfs.getWorkingDirectory());
+ }
+ }
}
// Scenario wise testing
@@ -339,32 +349,35 @@ public void testSetWorkingDirectory() throws Exception {
*/
@Test
public void testScenario1() throws Exception {
- AzureBlobFileSystem abfs = getFileSystem();
- Assume.assumeFalse("Namespace enabled account does not support this test",
- getIsNamespaceEnabled(abfs));
- NativeAzureFileSystem wasb = getWasbFileSystem();
-
- Path testFile = path("/testReadFile");
- Path path = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
-
- // Write
- try (FSDataOutputStream nativeFsStream = wasb.create(path, true)) {
- nativeFsStream.write(TEST_CONTEXT.getBytes());
- nativeFsStream.flush();
- nativeFsStream.hsync();
- }
- // Check file status
- ContractTestUtils.assertIsFile(wasb, path);
-
- try (BufferedReader br = new BufferedReader(
- new InputStreamReader(abfs.open(path)))) {
- String line = br.readLine();
- assertEquals("Wrong text from " + abfs,
- TEST_CONTEXT, line);
+ try (AzureBlobFileSystem abfs = getFileSystem()) {
+ Assume.assumeFalse("Namespace enabled account does not support this test",
+ getIsNamespaceEnabled(abfs));
+ try (NativeAzureFileSystem wasb = getWasbFileSystem()) {
+
+ Path testFile = path("/testReadFile");
+ Path path = new Path(
+ testFile + "/~12/!008/testfile_" + UUID.randomUUID());
+
+ // Write
+ try (FSDataOutputStream nativeFsStream = wasb.create(path, true)) {
+ nativeFsStream.write(TEST_CONTEXT.getBytes());
+ nativeFsStream.flush();
+ nativeFsStream.hsync();
+ }
+ // Check file status
+ ContractTestUtils.assertIsFile(wasb, path);
+
+ try (BufferedReader br = new BufferedReader(
+ new InputStreamReader(abfs.open(path)))) {
+ String line = br.readLine();
+ assertEquals("Wrong text from " + abfs,
+ TEST_CONTEXT, line);
+ }
+
+ // Remove file
+ assertDeleted(abfs, path, true);
+ }
}
-
- // Remove file
- assertDeleted(abfs, path, true);
}
/**
@@ -373,41 +386,44 @@ public void testScenario1() throws Exception {
*/
@Test
public void testScenario2() throws Exception {
- AzureBlobFileSystem abfs = getFileSystem();
- Assume.assumeFalse("Namespace enabled account does not support this test",
- getIsNamespaceEnabled(abfs));
- assumeBlobServiceType();
- Assume.assumeFalse("Not valid for APPEND BLOB", isAppendBlobEnabled());
- NativeAzureFileSystem wasb = getWasbFileSystem();
-
- Path testFile = path("/testReadFile");
- Path path = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
-
- // Write
- try (FSDataOutputStream nativeFsStream = wasb.create(path, true)) {
- nativeFsStream.write(TEST_CONTEXT.getBytes());
- nativeFsStream.flush();
- nativeFsStream.hsync();
- }
- // Check file status
- ContractTestUtils.assertIsFile(wasb, path);
-
- try (BufferedReader br = new BufferedReader(
- new InputStreamReader(abfs.open(path)))) {
- String line = br.readLine();
- assertEquals("Wrong text from " + abfs,
- TEST_CONTEXT, line);
- }
-
- // Write
- try (FSDataOutputStream abfsOutputStream = abfs.append(path)) {
- abfsOutputStream.write(TEST_CONTEXT1.getBytes());
- abfsOutputStream.flush();
- abfsOutputStream.hsync();
+ try (AzureBlobFileSystem abfs = getFileSystem()) {
+ Assume.assumeFalse("Namespace enabled account does not support this test",
+ getIsNamespaceEnabled(abfs));
+ assumeBlobServiceType();
+ Assume.assumeFalse("Not valid for APPEND BLOB", isAppendBlobEnabled());
+ try (NativeAzureFileSystem wasb = getWasbFileSystem()) {
+
+ Path testFile = path("/testReadFile");
+ Path path = new Path(
+ testFile + "/~12/!008/testfile_" + UUID.randomUUID());
+
+ // Write
+ try (FSDataOutputStream nativeFsStream = wasb.create(path, true)) {
+ nativeFsStream.write(TEST_CONTEXT.getBytes());
+ nativeFsStream.flush();
+ nativeFsStream.hsync();
+ }
+ // Check file status
+ ContractTestUtils.assertIsFile(wasb, path);
+
+ try (BufferedReader br = new BufferedReader(
+ new InputStreamReader(abfs.open(path)))) {
+ String line = br.readLine();
+ assertEquals("Wrong text from " + abfs,
+ TEST_CONTEXT, line);
+ }
+
+ // Write
+ try (FSDataOutputStream abfsOutputStream = abfs.append(path)) {
+ abfsOutputStream.write(TEST_CONTEXT1.getBytes());
+ abfsOutputStream.flush();
+ abfsOutputStream.hsync();
+ }
+
+ // Remove file
+ assertDeleted(abfs, path, true);
+ }
}
-
- // Remove file
- assertDeleted(abfs, path, true);
}
/**
@@ -417,35 +433,38 @@ public void testScenario2() throws Exception {
@Test
public void testScenario3() throws Exception {
Configuration conf = getRawConfiguration();
- conf.setBoolean(FS_AZURE_ABFS_ENABLE_FULL_BLOB_CHECKSUM_VALIDATION, true);
+ conf.setBoolean(FS_AZURE_ENABLE_FULL_BLOB_CHECKSUM_VALIDATION, true);
FileSystem fileSystem = FileSystem.newInstance(conf);
- AzureBlobFileSystem abfs = (AzureBlobFileSystem) fileSystem;
- Assume.assumeFalse("Namespace enabled account does not support this test",
- getIsNamespaceEnabled(abfs));
- Assume.assumeFalse("Not valid for APPEND BLOB", isAppendBlobEnabled());
- NativeAzureFileSystem wasb = getWasbFileSystem();
-
- Path testFile = path("/testReadFile");
- Path path = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
-
- // Write
- try (FSDataOutputStream abfsOutputStream = abfs.create(path, true)) {
- abfsOutputStream.write(TEST_CONTEXT.getBytes());
- abfsOutputStream.flush();
- abfsOutputStream.hsync();
- }
-
- // Check file status
- ContractTestUtils.assertIsFile(abfs, path);
-
- try (BufferedReader br = new BufferedReader(
- new InputStreamReader(wasb.open(path)))) {
- String line = br.readLine();
- assertEquals("Wrong text from " + wasb,
- TEST_CONTEXT, line);
+ try (AzureBlobFileSystem abfs = (AzureBlobFileSystem) fileSystem) {
+ Assume.assumeFalse("Namespace enabled account does not support this test",
+ getIsNamespaceEnabled(abfs));
+ Assume.assumeFalse("Not valid for APPEND BLOB", isAppendBlobEnabled());
+ try (NativeAzureFileSystem wasb = getWasbFileSystem()) {
+
+ Path testFile = path("/testReadFile");
+ Path path = new Path(
+ testFile + "/~12/!008/testfile_" + UUID.randomUUID());
+
+ // Write
+ try (FSDataOutputStream abfsOutputStream = abfs.create(path, true)) {
+ abfsOutputStream.write(TEST_CONTEXT.getBytes());
+ abfsOutputStream.flush();
+ abfsOutputStream.hsync();
+ }
+
+ // Check file status
+ ContractTestUtils.assertIsFile(abfs, path);
+
+ try (BufferedReader br = new BufferedReader(
+ new InputStreamReader(wasb.open(path)))) {
+ String line = br.readLine();
+ assertEquals("Wrong text from " + wasb,
+ TEST_CONTEXT, line);
+ }
+ // Remove file
+ assertDeleted(abfs, path, true);
+ }
}
- // Remove file
- assertDeleted(abfs, path, true);
}
/**
@@ -454,34 +473,37 @@ public void testScenario3() throws Exception {
*/
@Test
public void testScenario4() throws Exception {
- AzureBlobFileSystem abfs = getFileSystem();
- Assume.assumeFalse("Namespace enabled account does not support this test",
- getIsNamespaceEnabled(abfs));
- assumeBlobServiceType();
- Assume.assumeFalse("Not valid for APPEND BLOB", isAppendBlobEnabled());
- NativeAzureFileSystem wasb = getWasbFileSystem();
-
- Path testFile = path("/testReadFile");
- Path path = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
-
- // Write
- wasb.create(path, true);
- try (FSDataOutputStream abfsOutputStream = abfs.append(path)) {
- abfsOutputStream.write(TEST_CONTEXT.getBytes());
- abfsOutputStream.flush();
- abfsOutputStream.hsync();
- }
-
- try (FSDataOutputStream nativeFsStream = wasb.append(path)) {
- nativeFsStream.write(TEST_CONTEXT1.getBytes());
- nativeFsStream.flush();
- nativeFsStream.hsync();
+ try (AzureBlobFileSystem abfs = getFileSystem()) {
+ Assume.assumeFalse("Namespace enabled account does not support this test",
+ getIsNamespaceEnabled(abfs));
+ assumeBlobServiceType();
+ Assume.assumeFalse("Not valid for APPEND BLOB", isAppendBlobEnabled());
+ try (NativeAzureFileSystem wasb = getWasbFileSystem()) {
+
+ Path testFile = path("/testReadFile");
+ Path path = new Path(
+ testFile + "/~12/!008/testfile_" + UUID.randomUUID());
+
+ // Write
+ wasb.create(path, true);
+ try (FSDataOutputStream abfsOutputStream = abfs.append(path)) {
+ abfsOutputStream.write(TEST_CONTEXT.getBytes());
+ abfsOutputStream.flush();
+ abfsOutputStream.hsync();
+ }
+
+ try (FSDataOutputStream nativeFsStream = wasb.append(path)) {
+ nativeFsStream.write(TEST_CONTEXT1.getBytes());
+ nativeFsStream.flush();
+ nativeFsStream.hsync();
+ }
+
+ // Check file status
+ ContractTestUtils.assertIsFile(abfs, path);
+ // Remove file
+ assertDeleted(abfs, path, true);
+ }
}
-
- // Check file status
- ContractTestUtils.assertIsFile(abfs, path);
- // Remove file
- assertDeleted(abfs, path, true);
}
/**
@@ -493,36 +515,38 @@ public void testScenario5() throws Exception {
Configuration conf = getRawConfiguration();
conf.setBoolean(FS_AZURE_ABFS_ENABLE_CHECKSUM_VALIDATION, false);
FileSystem fileSystem = FileSystem.newInstance(conf);
- AzureBlobFileSystem abfs = (AzureBlobFileSystem) fileSystem;
- Assume.assumeFalse("Namespace enabled account does not support this test",
- getIsNamespaceEnabled(abfs));
- assumeBlobServiceType();
- Assume.assumeFalse("Not valid for APPEND BLOB", isAppendBlobEnabled());
- NativeAzureFileSystem wasb = getWasbFileSystem();
-
- Path testFile = path("/testReadFile");
- Path path = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
-
- // Write
- abfs.create(path, true);
- try (FSDataOutputStream nativeFsStream = wasb.append(path)) {
- nativeFsStream.write(TEST_CONTEXT.getBytes());
- nativeFsStream.flush();
- nativeFsStream.hsync();
- }
-
- // Check file status
- ContractTestUtils.assertIsFile(abfs, path);
-
- try (BufferedReader br = new BufferedReader(
- new InputStreamReader(abfs.open(path)))) {
- String line = br.readLine();
- assertEquals("Wrong text from " + abfs,
- TEST_CONTEXT, line);
+ try (AzureBlobFileSystem abfs = (AzureBlobFileSystem) fileSystem) {
+ Assume.assumeFalse("Namespace enabled account does not support this test",
+ getIsNamespaceEnabled(abfs));
+ assumeBlobServiceType();
+ Assume.assumeFalse("Not valid for APPEND BLOB", isAppendBlobEnabled());
+ try (NativeAzureFileSystem wasb = getWasbFileSystem()) {
+
+ Path testFile = path("/testReadFile");
+ Path path = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
+
+ // Write
+ abfs.create(path, true);
+ try (FSDataOutputStream nativeFsStream = wasb.append(path)) {
+ nativeFsStream.write(TEST_CONTEXT.getBytes());
+ nativeFsStream.flush();
+ nativeFsStream.hsync();
+ }
+
+ // Check file status
+ ContractTestUtils.assertIsFile(abfs, path);
+
+ try (BufferedReader br = new BufferedReader(
+ new InputStreamReader(abfs.open(path)))) {
+ String line = br.readLine();
+ assertEquals("Wrong text from " + abfs,
+ TEST_CONTEXT, line);
+ }
+
+ // Remove file
+ assertDeleted(abfs, path, true);
+ }
}
-
- // Remove file
- assertDeleted(abfs, path, true);
}
/**
@@ -534,36 +558,38 @@ public void testScenario6() throws Exception {
Configuration conf = getRawConfiguration();
conf.setBoolean(FS_AZURE_ABFS_ENABLE_CHECKSUM_VALIDATION, true);
FileSystem fileSystem = FileSystem.newInstance(conf);
- AzureBlobFileSystem abfs = (AzureBlobFileSystem) fileSystem;
- Assume.assumeFalse("Namespace enabled account does not support this test",
- getIsNamespaceEnabled(abfs));
- Assume.assumeFalse("Not valid for APPEND BLOB", isAppendBlobEnabled());
- assumeBlobServiceType();
- NativeAzureFileSystem wasb = getWasbFileSystem();
-
- Path testFile = path("/testReadFile");
- Path path = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
-
- // Write
- abfs.create(path, true);
- try (FSDataOutputStream nativeFsStream = wasb.append(path)) {
- nativeFsStream.write(TEST_CONTEXT.getBytes());
- nativeFsStream.flush();
- nativeFsStream.hsync();
- }
-
- // Check file status
- ContractTestUtils.assertIsFile(abfs, path);
-
- try (BufferedReader br = new BufferedReader(
- new InputStreamReader(abfs.open(path)))) {
- String line = br.readLine();
- assertEquals("Wrong text from " + abfs,
- TEST_CONTEXT, line);
+ try (AzureBlobFileSystem abfs = (AzureBlobFileSystem) fileSystem) {
+ Assume.assumeFalse("Namespace enabled account does not support this test",
+ getIsNamespaceEnabled(abfs));
+ Assume.assumeFalse("Not valid for APPEND BLOB", isAppendBlobEnabled());
+ assumeBlobServiceType();
+ try (NativeAzureFileSystem wasb = getWasbFileSystem()) {
+
+ Path testFile = path("/testReadFile");
+ Path path = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
+
+ // Write
+ abfs.create(path, true);
+ try (FSDataOutputStream nativeFsStream = wasb.append(path)) {
+ nativeFsStream.write(TEST_CONTEXT.getBytes());
+ nativeFsStream.flush();
+ nativeFsStream.hsync();
+ }
+
+ // Check file status
+ ContractTestUtils.assertIsFile(abfs, path);
+
+ try (BufferedReader br = new BufferedReader(
+ new InputStreamReader(abfs.open(path)))) {
+ String line = br.readLine();
+ assertEquals("Wrong text from " + abfs,
+ TEST_CONTEXT, line);
+ }
+
+ // Remove file
+ assertDeleted(abfs, path, true);
+ }
}
-
- // Remove file
- assertDeleted(abfs, path, true);
}
/**
@@ -575,38 +601,40 @@ public void testScenario7() throws Exception {
Configuration conf = getRawConfiguration();
conf.setBoolean(FS_AZURE_ABFS_ENABLE_CHECKSUM_VALIDATION, true);
FileSystem fileSystem = FileSystem.newInstance(conf);
- AzureBlobFileSystem abfs = (AzureBlobFileSystem) fileSystem;
- Assume.assumeFalse("Namespace enabled account does not support this test",
- getIsNamespaceEnabled(abfs));
- Assume.assumeFalse("Not valid for APPEND BLOB", isAppendBlobEnabled());
- NativeAzureFileSystem wasb = getWasbFileSystem();
-
- Path testFile = path("/testReadFile");
- Path path = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
-
- try (FSDataOutputStream nativeFsStream = wasb.create(path, true)) {
- nativeFsStream.write(TEST_CONTEXT.getBytes());
- nativeFsStream.flush();
- nativeFsStream.hsync();
- }
-
- // Check file status
- ContractTestUtils.assertIsFile(abfs, path);
-
- try (BufferedReader br = new BufferedReader(
- new InputStreamReader(abfs.open(path)))) {
- String line = br.readLine();
- assertEquals("Wrong text from " + abfs,
- TEST_CONTEXT, line);
+ try (AzureBlobFileSystem abfs = (AzureBlobFileSystem) fileSystem) {
+ Assume.assumeFalse("Namespace enabled account does not support this test",
+ getIsNamespaceEnabled(abfs));
+ Assume.assumeFalse("Not valid for APPEND BLOB", isAppendBlobEnabled());
+ try (NativeAzureFileSystem wasb = getWasbFileSystem()) {
+
+ Path testFile = path("/testReadFile");
+ Path path = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
+
+ try (FSDataOutputStream nativeFsStream = wasb.create(path, true)) {
+ nativeFsStream.write(TEST_CONTEXT.getBytes());
+ nativeFsStream.flush();
+ nativeFsStream.hsync();
+ }
+
+ // Check file status
+ ContractTestUtils.assertIsFile(abfs, path);
+
+ try (BufferedReader br = new BufferedReader(
+ new InputStreamReader(abfs.open(path)))) {
+ String line = br.readLine();
+ assertEquals("Wrong text from " + abfs,
+ TEST_CONTEXT, line);
+ }
+ abfs.create(path, true);
+ FileStatus fileStatus = abfs.getFileStatus(path);
+ Assertions.assertThat(fileStatus.getLen())
+ .as("Expected file length to be 0 after overwrite")
+ .isEqualTo(0L);
+
+ // Remove file
+ assertDeleted(abfs, path, true);
+ }
}
- abfs.create(path, true);
- FileStatus fileStatus = abfs.getFileStatus(path);
- Assertions.assertThat(fileStatus.getLen())
- .as("Expected file length to be 0 after overwrite")
- .isEqualTo(0L);
-
- // Remove file
- assertDeleted(abfs, path, true);
}
/**
@@ -618,46 +646,48 @@ public void testScenario8() throws Exception {
Configuration conf = getRawConfiguration();
conf.setBoolean(FS_AZURE_ABFS_ENABLE_CHECKSUM_VALIDATION, true);
FileSystem fileSystem = FileSystem.newInstance(conf);
- AzureBlobFileSystem abfs = (AzureBlobFileSystem) fileSystem;
- Assume.assumeFalse("Namespace enabled account does not support this test",
- getIsNamespaceEnabled(abfs));
- Assume.assumeFalse("Not valid for APPEND BLOB", isAppendBlobEnabled());
- NativeAzureFileSystem wasb = getWasbFileSystem();
-
- Path testFile = path("/testReadFile");
- Path path = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
-
- try (FSDataOutputStream nativeFsStream = wasb.create(path, true)) {
- nativeFsStream.write(TEST_CONTEXT.getBytes());
- nativeFsStream.flush();
- nativeFsStream.hsync();
- }
-
- // Check file status
- ContractTestUtils.assertIsFile(abfs, path);
-
- try (BufferedReader br = new BufferedReader(
- new InputStreamReader(abfs.open(path)))) {
- String line = br.readLine();
- assertEquals("Wrong text from " + abfs,
- TEST_CONTEXT, line);
- }
- try {
- abfs.create(path, false);
- } catch (IOException e) {
- AbfsRestOperationException restEx = (AbfsRestOperationException) e.getCause();
- if (restEx != null) {
- Assertions.assertThat(restEx.getStatusCode())
- .as("Expected HTTP status code 409 (Conflict) when file already exists")
- .isEqualTo(HTTP_CONFLICT);
+ try (AzureBlobFileSystem abfs = (AzureBlobFileSystem) fileSystem) {
+ Assume.assumeFalse("Namespace enabled account does not support this test",
+ getIsNamespaceEnabled(abfs));
+ Assume.assumeFalse("Not valid for APPEND BLOB", isAppendBlobEnabled());
+ try (NativeAzureFileSystem wasb = getWasbFileSystem()) {
+
+ Path testFile = path("/testReadFile");
+ Path path = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
+
+ try (FSDataOutputStream nativeFsStream = wasb.create(path, true)) {
+ nativeFsStream.write(TEST_CONTEXT.getBytes());
+ nativeFsStream.flush();
+ nativeFsStream.hsync();
+ }
+
+ // Check file status
+ ContractTestUtils.assertIsFile(abfs, path);
+
+ try (BufferedReader br = new BufferedReader(
+ new InputStreamReader(abfs.open(path)))) {
+ String line = br.readLine();
+ assertEquals("Wrong text from " + abfs,
+ TEST_CONTEXT, line);
+ }
+ try {
+ abfs.create(path, false);
+ } catch (IOException e) {
+ AbfsRestOperationException restEx = (AbfsRestOperationException) e.getCause();
+ if (restEx != null) {
+ Assertions.assertThat(restEx.getStatusCode())
+ .as("Expected HTTP status code 409 (Conflict) when file already exists")
+ .isEqualTo(HTTP_CONFLICT);
+ }
+ Assertions.assertThat(e.getMessage())
+ .as("Expected error message to contain 'AlreadyExists'")
+ .contains("AlreadyExists");
+ }
+
+ // Remove file
+ assertDeleted(abfs, path, true);
}
- Assertions.assertThat(e.getMessage())
- .as("Expected error message to contain 'AlreadyExists'")
- .contains("AlreadyExists");
}
-
- // Remove file
- assertDeleted(abfs, path, true);
}
/**
@@ -669,39 +699,41 @@ public void testScenario9() throws Exception {
Configuration conf = getRawConfiguration();
conf.setBoolean(FS_AZURE_ABFS_ENABLE_CHECKSUM_VALIDATION, true);
FileSystem fileSystem = FileSystem.newInstance(conf);
- AzureBlobFileSystem abfs = (AzureBlobFileSystem) fileSystem;
- Assume.assumeFalse("Namespace enabled account does not support this test",
- getIsNamespaceEnabled(abfs));
- assumeBlobServiceType();
- Assume.assumeFalse("Not valid for APPEND BLOB", isAppendBlobEnabled());
- NativeAzureFileSystem wasb = getWasbFileSystem();
-
- Path testFile = path("/testReadFile");
- Path path = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
-
- try (FSDataOutputStream abfsOutputStream = abfs.create(path, true)) {
- abfsOutputStream.write(TEST_CONTEXT.getBytes());
- abfsOutputStream.flush();
- abfsOutputStream.hsync();
- }
-
- // Check file status
- ContractTestUtils.assertIsFile(abfs, path);
-
- try (BufferedReader br = new BufferedReader(
- new InputStreamReader(abfs.open(path)))) {
- String line = br.readLine();
- assertEquals("Wrong text from " + abfs,
- TEST_CONTEXT, line);
+ try (AzureBlobFileSystem abfs = (AzureBlobFileSystem) fileSystem) {
+ Assume.assumeFalse("Namespace enabled account does not support this test",
+ getIsNamespaceEnabled(abfs));
+ assumeBlobServiceType();
+ Assume.assumeFalse("Not valid for APPEND BLOB", isAppendBlobEnabled());
+ try (NativeAzureFileSystem wasb = getWasbFileSystem()) {
+
+ Path testFile = path("/testReadFile");
+ Path path = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
+
+ try (FSDataOutputStream abfsOutputStream = abfs.create(path, true)) {
+ abfsOutputStream.write(TEST_CONTEXT.getBytes());
+ abfsOutputStream.flush();
+ abfsOutputStream.hsync();
+ }
+
+ // Check file status
+ ContractTestUtils.assertIsFile(abfs, path);
+
+ try (BufferedReader br = new BufferedReader(
+ new InputStreamReader(abfs.open(path)))) {
+ String line = br.readLine();
+ assertEquals("Wrong text from " + abfs,
+ TEST_CONTEXT, line);
+ }
+ wasb.create(path, true);
+ FileStatus fileStatus = abfs.getFileStatus(path);
+ Assertions.assertThat(fileStatus.getLen())
+ .as("Expected file length to be 0 after overwrite")
+ .isEqualTo(0L);
+
+ // Remove file
+ assertDeleted(abfs, path, true);
+ }
}
- wasb.create(path, true);
- FileStatus fileStatus = abfs.getFileStatus(path);
- Assertions.assertThat(fileStatus.getLen())
- .as("Expected file length to be 0 after overwrite")
- .isEqualTo(0L);
-
- // Remove file
- assertDeleted(abfs, path, true);
}
/**
@@ -714,45 +746,47 @@ public void testScenario10() throws Exception {
Configuration conf = getRawConfiguration();
conf.setBoolean(FS_AZURE_ABFS_ENABLE_CHECKSUM_VALIDATION, true);
FileSystem fileSystem = FileSystem.newInstance(conf);
- AzureBlobFileSystem abfs = (AzureBlobFileSystem) fileSystem;
- Assume.assumeFalse("Namespace enabled account does not support this test",
- getIsNamespaceEnabled(abfs));
- NativeAzureFileSystem wasb = getWasbFileSystem();
-
- Path testFile = path("/testReadFile");
- Path path = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
-
- try (FSDataOutputStream abfsOutputStream = abfs.create(path, true)) {
- abfsOutputStream.write(TEST_CONTEXT.getBytes());
- abfsOutputStream.flush();
- abfsOutputStream.hsync();
- }
-
- // Check file status
- ContractTestUtils.assertIsFile(abfs, path);
-
- try (BufferedReader br = new BufferedReader(
- new InputStreamReader(abfs.open(path)))) {
- String line = br.readLine();
- assertEquals("Wrong text from " + abfs,
- TEST_CONTEXT, line);
- }
- try {
- wasb.create(path, false);
- } catch (IOException e) {
- AbfsRestOperationException restEx
- = (AbfsRestOperationException) e.getCause();
- if (restEx != null) {
- Assertions.assertThat(restEx.getStatusCode())
- .as("Expected HTTP status code 409 (Conflict) when file already exists")
- .isEqualTo(HTTP_CONFLICT);
+ try (AzureBlobFileSystem abfs = (AzureBlobFileSystem) fileSystem) {
+ Assume.assumeFalse("Namespace enabled account does not support this test",
+ getIsNamespaceEnabled(abfs));
+ try (NativeAzureFileSystem wasb = getWasbFileSystem()) {
+
+ Path testFile = path("/testReadFile");
+ Path path = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
+
+ try (FSDataOutputStream abfsOutputStream = abfs.create(path, true)) {
+ abfsOutputStream.write(TEST_CONTEXT.getBytes());
+ abfsOutputStream.flush();
+ abfsOutputStream.hsync();
+ }
+
+ // Check file status
+ ContractTestUtils.assertIsFile(abfs, path);
+
+ try (BufferedReader br = new BufferedReader(
+ new InputStreamReader(abfs.open(path)))) {
+ String line = br.readLine();
+ assertEquals("Wrong text from " + abfs,
+ TEST_CONTEXT, line);
+ }
+ try {
+ wasb.create(path, false);
+ } catch (IOException e) {
+ AbfsRestOperationException restEx
+ = (AbfsRestOperationException) e.getCause();
+ if (restEx != null) {
+ Assertions.assertThat(restEx.getStatusCode())
+ .as("Expected HTTP status code 409 (Conflict) when file already exists")
+ .isEqualTo(HTTP_CONFLICT);
+ }
+ Assertions.assertThat(e.getMessage())
+ .as("Expected error message to contain 'exists'")
+ .contains("exists");
+ }
+ // Remove file
+ assertDeleted(abfs, path, true);
}
- Assertions.assertThat(e.getMessage())
- .as("Expected error message to contain 'exists'")
- .contains("exists");
}
- // Remove file
- assertDeleted(abfs, path, true);
}
/**
@@ -765,34 +799,36 @@ public void testScenario11() throws Exception {
Configuration conf = getRawConfiguration();
conf.setBoolean(FS_AZURE_ABFS_ENABLE_CHECKSUM_VALIDATION, true);
FileSystem fileSystem = FileSystem.newInstance(conf);
- AzureBlobFileSystem abfs = (AzureBlobFileSystem) fileSystem;
- Assume.assumeFalse("Namespace enabled account does not support this test",
- getIsNamespaceEnabled(abfs));
- assumeBlobServiceType();
- Assume.assumeFalse("Not valid for APPEND BLOB", isAppendBlobEnabled());
- NativeAzureFileSystem wasb = getWasbFileSystem();
-
- Path testFile = path("/testReadFile");
- Path path = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
-
- // Write
- abfs.create(path, true);
- try (FSDataOutputStream nativeFsStream = wasb.append(path)) {
- nativeFsStream.write(TEST_CONTEXT.getBytes());
- nativeFsStream.flush();
- nativeFsStream.hsync();
- }
-
- // Check file status
- ContractTestUtils.assertIsFile(abfs, path);
-
- try (BufferedReader br = new BufferedReader(
- new InputStreamReader(abfs.open(path)))) {
- String line = br.readLine();
- assertEquals("Wrong text from " + abfs,
- TEST_CONTEXT, line);
+ try (AzureBlobFileSystem abfs = (AzureBlobFileSystem) fileSystem) {
+ Assume.assumeFalse("Namespace enabled account does not support this test",
+ getIsNamespaceEnabled(abfs));
+ assumeBlobServiceType();
+ Assume.assumeFalse("Not valid for APPEND BLOB", isAppendBlobEnabled());
+ try (NativeAzureFileSystem wasb = getWasbFileSystem()) {
+
+ Path testFile = path("/testReadFile");
+ Path path = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
+
+ // Write
+ abfs.create(path, true);
+ try (FSDataOutputStream nativeFsStream = wasb.append(path)) {
+ nativeFsStream.write(TEST_CONTEXT.getBytes());
+ nativeFsStream.flush();
+ nativeFsStream.hsync();
+ }
+
+ // Check file status
+ ContractTestUtils.assertIsFile(abfs, path);
+
+ try (BufferedReader br = new BufferedReader(
+ new InputStreamReader(abfs.open(path)))) {
+ String line = br.readLine();
+ assertEquals("Wrong text from " + abfs,
+ TEST_CONTEXT, line);
+ }
+ abfs.delete(path, true);
+ }
}
- abfs.delete(path, true);
}
/**
@@ -805,31 +841,33 @@ public void testScenario12() throws Exception {
Configuration conf = getRawConfiguration();
conf.setBoolean(FS_AZURE_ABFS_ENABLE_CHECKSUM_VALIDATION, true);
FileSystem fileSystem = FileSystem.newInstance(conf);
- AzureBlobFileSystem abfs = (AzureBlobFileSystem) fileSystem;
- Assume.assumeFalse("Namespace enabled account does not support this test",
- getIsNamespaceEnabled(abfs));
- NativeAzureFileSystem wasb = getWasbFileSystem();
-
- Path testFile = path("/testReadFile");
- Path path = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
-
- // Write
- try (FSDataOutputStream abfsOutputStream = abfs.create(path, true)) {
- abfsOutputStream.write(TEST_CONTEXT.getBytes());
- abfsOutputStream.flush();
- abfsOutputStream.hsync();
- }
-
- // Check file status
- ContractTestUtils.assertIsFile(abfs, path);
-
- try (BufferedReader br = new BufferedReader(
- new InputStreamReader(abfs.open(path)))) {
- String line = br.readLine();
- assertEquals("Wrong text from " + abfs,
- TEST_CONTEXT, line);
+ try (AzureBlobFileSystem abfs = (AzureBlobFileSystem) fileSystem) {
+ Assume.assumeFalse("Namespace enabled account does not support this test",
+ getIsNamespaceEnabled(abfs));
+ try (NativeAzureFileSystem wasb = getWasbFileSystem()) {
+
+ Path testFile = path("/testReadFile");
+ Path path = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
+
+ // Write
+ try (FSDataOutputStream abfsOutputStream = abfs.create(path, true)) {
+ abfsOutputStream.write(TEST_CONTEXT.getBytes());
+ abfsOutputStream.flush();
+ abfsOutputStream.hsync();
+ }
+
+ // Check file status
+ ContractTestUtils.assertIsFile(abfs, path);
+
+ try (BufferedReader br = new BufferedReader(
+ new InputStreamReader(abfs.open(path)))) {
+ String line = br.readLine();
+ assertEquals("Wrong text from " + abfs,
+ TEST_CONTEXT, line);
+ }
+ wasb.delete(path, true);
+ }
}
- wasb.delete(path, true);
}
/**
@@ -841,34 +879,36 @@ public void testScenario13() throws Exception {
Configuration conf = getRawConfiguration();
conf.setBoolean(FS_AZURE_ABFS_ENABLE_CHECKSUM_VALIDATION, true);
FileSystem fileSystem = FileSystem.newInstance(conf);
- AzureBlobFileSystem abfs = (AzureBlobFileSystem) fileSystem;
- Assume.assumeFalse("Namespace enabled account does not support this test",
- getIsNamespaceEnabled(abfs));
- assumeBlobServiceType();
- Assume.assumeFalse("Not valid for APPEND BLOB", isAppendBlobEnabled());
- NativeAzureFileSystem wasb = getWasbFileSystem();
-
- Path testFile = path("/testReadFile");
- Path path = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
-
- // Write
- abfs.create(path, true);
- try (FSDataOutputStream nativeFsStream = wasb.append(path)) {
- nativeFsStream.write(TEST_CONTEXT.getBytes());
- nativeFsStream.flush();
- nativeFsStream.hsync();
- }
-
- // Check file status
- ContractTestUtils.assertIsFile(abfs, path);
-
- try (BufferedReader br = new BufferedReader(
- new InputStreamReader(wasb.open(path)))) {
- String line = br.readLine();
- assertEquals("Wrong text from " + wasb,
- TEST_CONTEXT, line);
+ try (AzureBlobFileSystem abfs = (AzureBlobFileSystem) fileSystem) {
+ Assume.assumeFalse("Namespace enabled account does not support this test",
+ getIsNamespaceEnabled(abfs));
+ assumeBlobServiceType();
+ Assume.assumeFalse("Not valid for APPEND BLOB", isAppendBlobEnabled());
+ try (NativeAzureFileSystem wasb = getWasbFileSystem()) {
+
+ Path testFile = path("/testReadFile");
+ Path path = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
+
+ // Write
+ abfs.create(path, true);
+ try (FSDataOutputStream nativeFsStream = wasb.append(path)) {
+ nativeFsStream.write(TEST_CONTEXT.getBytes());
+ nativeFsStream.flush();
+ nativeFsStream.hsync();
+ }
+
+ // Check file status
+ ContractTestUtils.assertIsFile(abfs, path);
+
+ try (BufferedReader br = new BufferedReader(
+ new InputStreamReader(wasb.open(path)))) {
+ String line = br.readLine();
+ assertEquals("Wrong text from " + wasb,
+ TEST_CONTEXT, line);
+ }
+ abfs.delete(path, true);
+ }
}
- abfs.delete(path, true);
}
/**
@@ -880,34 +920,36 @@ public void testScenario14() throws Exception {
Configuration conf = getRawConfiguration();
conf.setBoolean(FS_AZURE_ABFS_ENABLE_CHECKSUM_VALIDATION, true);
FileSystem fileSystem = FileSystem.newInstance(conf);
- AzureBlobFileSystem abfs = (AzureBlobFileSystem) fileSystem;
- Assume.assumeFalse("Namespace enabled account does not support this test",
- getIsNamespaceEnabled(abfs));
- assumeBlobServiceType();
- Assume.assumeFalse("Not valid for APPEND BLOB", isAppendBlobEnabled());
- NativeAzureFileSystem wasb = getWasbFileSystem();
-
- Path testFile = path("/testReadFile");
- Path path = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
-
- // Write
- abfs.create(path, true);
- try (FSDataOutputStream nativeFsStream = wasb.append(path)) {
- nativeFsStream.write(TEST_CONTEXT.getBytes());
- nativeFsStream.flush();
- nativeFsStream.hsync();
- }
-
- // Check file status
- ContractTestUtils.assertIsFile(abfs, path);
-
- try (BufferedReader br = new BufferedReader(
- new InputStreamReader(wasb.open(path)))) {
- String line = br.readLine();
- assertEquals("Wrong text from " + wasb,
- TEST_CONTEXT, line);
+ try (AzureBlobFileSystem abfs = (AzureBlobFileSystem) fileSystem) {
+ Assume.assumeFalse("Namespace enabled account does not support this test",
+ getIsNamespaceEnabled(abfs));
+ assumeBlobServiceType();
+ Assume.assumeFalse("Not valid for APPEND BLOB", isAppendBlobEnabled());
+ try (NativeAzureFileSystem wasb = getWasbFileSystem()) {
+
+ Path testFile = path("/testReadFile");
+ Path path = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
+
+ // Write
+ abfs.create(path, true);
+ try (FSDataOutputStream nativeFsStream = wasb.append(path)) {
+ nativeFsStream.write(TEST_CONTEXT.getBytes());
+ nativeFsStream.flush();
+ nativeFsStream.hsync();
+ }
+
+ // Check file status
+ ContractTestUtils.assertIsFile(abfs, path);
+
+ try (BufferedReader br = new BufferedReader(
+ new InputStreamReader(wasb.open(path)))) {
+ String line = br.readLine();
+ assertEquals("Wrong text from " + wasb,
+ TEST_CONTEXT, line);
+ }
+ wasb.delete(path, true);
+ }
}
- wasb.delete(path, true);
}
/**
@@ -919,70 +961,74 @@ public void testScenario15() throws Exception {
Configuration conf = getRawConfiguration();
conf.setBoolean(FS_AZURE_ABFS_ENABLE_CHECKSUM_VALIDATION, true);
FileSystem fileSystem = FileSystem.newInstance(conf);
- AzureBlobFileSystem abfs = (AzureBlobFileSystem) fileSystem;
- Assume.assumeFalse("Namespace enabled account does not support this test",
- getIsNamespaceEnabled(abfs));
- NativeAzureFileSystem wasb = getWasbFileSystem();
-
- Path testFile = path("/testReadFile");
- Path path = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
-
- // Write
- try (FSDataOutputStream nativeFsStream = wasb.create(path, true)) {
- nativeFsStream.write(TEST_CONTEXT.getBytes());
- nativeFsStream.flush();
- nativeFsStream.hsync();
- }
-
- // Check file status
- ContractTestUtils.assertIsFile(abfs, path);
-
- try (BufferedReader br = new BufferedReader(
- new InputStreamReader(wasb.open(path)))) {
- String line = br.readLine();
- assertEquals("Wrong text from " + wasb,
- TEST_CONTEXT, line);
+ try (AzureBlobFileSystem abfs = (AzureBlobFileSystem) fileSystem) {
+ Assume.assumeFalse("Namespace enabled account does not support this test",
+ getIsNamespaceEnabled(abfs));
+ try (NativeAzureFileSystem wasb = getWasbFileSystem()) {
+
+ Path testFile = path("/testReadFile");
+ Path path = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
+
+ // Write
+ try (FSDataOutputStream nativeFsStream = wasb.create(path, true)) {
+ nativeFsStream.write(TEST_CONTEXT.getBytes());
+ nativeFsStream.flush();
+ nativeFsStream.hsync();
+ }
+
+ // Check file status
+ ContractTestUtils.assertIsFile(abfs, path);
+
+ try (BufferedReader br = new BufferedReader(
+ new InputStreamReader(wasb.open(path)))) {
+ String line = br.readLine();
+ assertEquals("Wrong text from " + wasb,
+ TEST_CONTEXT, line);
+ }
+ abfs.delete(path, true);
+ }
}
- abfs.delete(path, true);
}
/**
* Scenario 16: Create a file using WASB, write data to it using ABFS, and then delete the file using WASB.
- * Expected Outcome: Write via ABFS should succeed and persist data; delete via WASB should succeed without errors.
- */
- @Test
- public void testScenario16() throws Exception {
- Configuration conf = getRawConfiguration();
- conf.setBoolean(FS_AZURE_ABFS_ENABLE_CHECKSUM_VALIDATION, true);
- FileSystem fileSystem = FileSystem.newInstance(conf);
- AzureBlobFileSystem abfs = (AzureBlobFileSystem) fileSystem;
- Assume.assumeFalse("Namespace enabled account does not support this test",
- getIsNamespaceEnabled(abfs));
- assumeBlobServiceType();
- Assume.assumeFalse("Not valid for APPEND BLOB", isAppendBlobEnabled());
- NativeAzureFileSystem wasb = getWasbFileSystem();
-
- Path testFile = path("/testReadFile");
- Path path = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
-
- // Write
- wasb.create(path, true);
- try (FSDataOutputStream abfsOutputStream = abfs.append(path)) {
- abfsOutputStream.write(TEST_CONTEXT.getBytes());
- abfsOutputStream.flush();
- abfsOutputStream.hsync();
- }
-
- // Check file status
- ContractTestUtils.assertIsFile(abfs, path);
-
- try (BufferedReader br = new BufferedReader(
- new InputStreamReader(abfs.open(path)))) {
- String line = br.readLine();
- assertEquals("Wrong text from " + abfs,
- TEST_CONTEXT, line);
+ * Expected Outcome: Write via ABFS should succeed and persist data; delete via WASB should succeed without errors.
+ */
+ @Test
+ public void testScenario16() throws Exception {
+ Configuration conf = getRawConfiguration();
+ conf.setBoolean(FS_AZURE_ABFS_ENABLE_CHECKSUM_VALIDATION, true);
+ FileSystem fileSystem = FileSystem.newInstance(conf);
+ try (AzureBlobFileSystem abfs = (AzureBlobFileSystem) fileSystem) {
+ Assume.assumeFalse("Namespace enabled account does not support this test",
+ getIsNamespaceEnabled(abfs));
+ assumeBlobServiceType();
+ Assume.assumeFalse("Not valid for APPEND BLOB", isAppendBlobEnabled());
+ try (NativeAzureFileSystem wasb = getWasbFileSystem()) {
+
+ Path testFile = path("/testReadFile");
+ Path path = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
+
+ // Write
+ wasb.create(path, true);
+ try (FSDataOutputStream abfsOutputStream = abfs.append(path)) {
+ abfsOutputStream.write(TEST_CONTEXT.getBytes());
+ abfsOutputStream.flush();
+ abfsOutputStream.hsync();
+ }
+
+ // Check file status
+ ContractTestUtils.assertIsFile(abfs, path);
+
+ try (BufferedReader br = new BufferedReader(
+ new InputStreamReader(abfs.open(path)))) {
+ String line = br.readLine();
+ assertEquals("Wrong text from " + abfs,
+ TEST_CONTEXT, line);
+ }
+ wasb.delete(path, true);
+ }
}
- wasb.delete(path, true);
}
/**
@@ -994,37 +1040,38 @@ public void testScenario17() throws Exception {
Configuration conf = getRawConfiguration();
conf.setBoolean(FS_AZURE_ABFS_ENABLE_CHECKSUM_VALIDATION, true);
FileSystem fileSystem = FileSystem.newInstance(conf);
- AzureBlobFileSystem abfs = (AzureBlobFileSystem) fileSystem;
- Assume.assumeFalse("Namespace enabled account does not support this test",
- getIsNamespaceEnabled(abfs));
- Path testFile = path("/testReadFile");
- Path path = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
-
- // Write
- try (FSDataOutputStream abfsOutputStream = abfs.create(path, true)) {
- abfsOutputStream.write(TEST_CONTEXT.getBytes());
- abfsOutputStream.flush();
- abfsOutputStream.hsync();
- }
- // --- VALIDATE FILE ---
- FileStatus status = abfs.getFileStatus(path);
- assertIsFile(path, status);
+ try (AzureBlobFileSystem abfs = (AzureBlobFileSystem) fileSystem) {
+ Assume.assumeFalse("Namespace enabled account does not support this test",
+ getIsNamespaceEnabled(abfs));
+ Path testFile = path("/testReadFile");
+ Path path = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
+
+ // Write
+ try (FSDataOutputStream abfsOutputStream = abfs.create(path, true)) {
+ abfsOutputStream.write(TEST_CONTEXT.getBytes());
+ abfsOutputStream.flush();
+ abfsOutputStream.hsync();
+ }
+ // --- VALIDATE FILE ---
+ FileStatus status = abfs.getFileStatus(path);
+ assertIsFile(path, status);
- // --- SET XATTR #1 ---
- abfs.setXAttr(path, ATTRIBUTE_NAME_1, ATTRIBUTE_VALUE_1);
- byte[] readValue = abfs.getXAttr(path, ATTRIBUTE_NAME_1);
- ITestAzureBlobFileSystemAttributes.assertAttributeEqual(abfs, readValue, ATTRIBUTE_VALUE_1, "one");
+ // --- SET XATTR #1 ---
+ abfs.setXAttr(path, ATTRIBUTE_NAME_1, ATTRIBUTE_VALUE_1);
+ byte[] readValue = abfs.getXAttr(path, ATTRIBUTE_NAME_1);
+ ITestAzureBlobFileSystemAttributes.assertAttributeEqual(abfs, readValue, ATTRIBUTE_VALUE_1, "one");
- // --- SET XATTR #2 ---
- abfs.setXAttr(path, ATTRIBUTE_NAME_2, ATTRIBUTE_VALUE_2);
- readValue = abfs.getXAttr(path, ATTRIBUTE_NAME_2);
- ITestAzureBlobFileSystemAttributes.assertAttributeEqual(abfs, readValue, ATTRIBUTE_VALUE_2, "two");
+ // --- SET XATTR #2 ---
+ abfs.setXAttr(path, ATTRIBUTE_NAME_2, ATTRIBUTE_VALUE_2);
+ readValue = abfs.getXAttr(path, ATTRIBUTE_NAME_2);
+ ITestAzureBlobFileSystemAttributes.assertAttributeEqual(abfs, readValue, ATTRIBUTE_VALUE_2, "two");
- // --- VERIFY XATTR #1 AGAIN ---
- readValue = abfs.getXAttr(path, ATTRIBUTE_NAME_1);
- ITestAzureBlobFileSystemAttributes.assertAttributeEqual(abfs, readValue, ATTRIBUTE_VALUE_1, "one");
+ // --- VERIFY XATTR #1 AGAIN ---
+ readValue = abfs.getXAttr(path, ATTRIBUTE_NAME_1);
+ ITestAzureBlobFileSystemAttributes.assertAttributeEqual(abfs, readValue, ATTRIBUTE_VALUE_1, "one");
- abfs.delete(path, true);
+ abfs.delete(path, true);
+ }
}
/**
@@ -1036,39 +1083,41 @@ public void testScenario18() throws Exception {
Configuration conf = getRawConfiguration();
conf.setBoolean(FS_AZURE_ABFS_ENABLE_CHECKSUM_VALIDATION, true);
FileSystem fileSystem = FileSystem.newInstance(conf);
- AzureBlobFileSystem abfs = (AzureBlobFileSystem) fileSystem;
- Assume.assumeFalse("Namespace enabled account does not support this test",
- getIsNamespaceEnabled(abfs));
- NativeAzureFileSystem wasb = getWasbFileSystem();
-
- Path testFile = path("/testReadFile");
- Path path = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
-
- // Write
- try (FSDataOutputStream nativeFsStream = wasb.create(path, true)) {
- nativeFsStream.write(TEST_CONTEXT.getBytes());
- nativeFsStream.flush();
- nativeFsStream.hsync();
+ try (AzureBlobFileSystem abfs = (AzureBlobFileSystem) fileSystem) {
+ Assume.assumeFalse("Namespace enabled account does not support this test",
+ getIsNamespaceEnabled(abfs));
+ try (NativeAzureFileSystem wasb = getWasbFileSystem()) {
+
+ Path testFile = path("/testReadFile");
+ Path path = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
+
+ // Write
+ try (FSDataOutputStream nativeFsStream = wasb.create(path, true)) {
+ nativeFsStream.write(TEST_CONTEXT.getBytes());
+ nativeFsStream.flush();
+ nativeFsStream.hsync();
+ }
+ // --- VALIDATE FILE ---
+ FileStatus status = wasb.getFileStatus(path);
+ assertIsFile(path, status);
+
+ // --- SET XATTR #1 ---
+ wasb.setXAttr(path, ATTRIBUTE_NAME_1, ATTRIBUTE_VALUE_1);
+ byte[] readValue = wasb.getXAttr(path, ATTRIBUTE_NAME_1);
+ ITestAzureBlobFileSystemAttributes.assertAttributeEqual(abfs, readValue, ATTRIBUTE_VALUE_1, "one");
+
+ // --- SET XATTR #2 ---
+ wasb.setXAttr(path, ATTRIBUTE_NAME_2, ATTRIBUTE_VALUE_2);
+ readValue = wasb.getXAttr(path, ATTRIBUTE_NAME_2);
+ ITestAzureBlobFileSystemAttributes.assertAttributeEqual(abfs, readValue, ATTRIBUTE_VALUE_2, "two");
+
+ // --- VERIFY XATTR #1 AGAIN ---
+ readValue = wasb.getXAttr(path, ATTRIBUTE_NAME_1);
+ ITestAzureBlobFileSystemAttributes.assertAttributeEqual(abfs, readValue, ATTRIBUTE_VALUE_1, "one");
+
+ wasb.delete(path, true);
+ }
}
- // --- VALIDATE FILE ---
- FileStatus status = wasb.getFileStatus(path);
- assertIsFile(path, status);
-
- // --- SET XATTR #1 ---
- wasb.setXAttr(path, ATTRIBUTE_NAME_1, ATTRIBUTE_VALUE_1);
- byte[] readValue = wasb.getXAttr(path, ATTRIBUTE_NAME_1);
- ITestAzureBlobFileSystemAttributes.assertAttributeEqual(abfs, readValue, ATTRIBUTE_VALUE_1, "one");
-
- // --- SET XATTR #2 ---
- wasb.setXAttr(path, ATTRIBUTE_NAME_2, ATTRIBUTE_VALUE_2);
- readValue = wasb.getXAttr(path, ATTRIBUTE_NAME_2);
- ITestAzureBlobFileSystemAttributes.assertAttributeEqual(abfs, readValue, ATTRIBUTE_VALUE_2, "two");
-
- // --- VERIFY XATTR #1 AGAIN ---
- readValue = wasb.getXAttr(path, ATTRIBUTE_NAME_1);
- ITestAzureBlobFileSystemAttributes.assertAttributeEqual(abfs, readValue, ATTRIBUTE_VALUE_1, "one");
-
- wasb.delete(path, true);
}
/**
@@ -1080,39 +1129,41 @@ public void testScenario19() throws Exception {
Configuration conf = getRawConfiguration();
conf.setBoolean(FS_AZURE_ABFS_ENABLE_CHECKSUM_VALIDATION, true);
FileSystem fileSystem = FileSystem.newInstance(conf);
- AzureBlobFileSystem abfs = (AzureBlobFileSystem) fileSystem;
- Assume.assumeFalse("Namespace enabled account does not support this test",
- getIsNamespaceEnabled(abfs));
- NativeAzureFileSystem wasb = getWasbFileSystem();
-
- Path testFile = path("/testReadFile");
- Path path = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
-
- // Write
- try (FSDataOutputStream nativeFsStream = wasb.create(path, true)) {
- nativeFsStream.write(TEST_CONTEXT.getBytes());
- nativeFsStream.flush();
- nativeFsStream.hsync();
+ try (AzureBlobFileSystem abfs = (AzureBlobFileSystem) fileSystem) {
+ Assume.assumeFalse("Namespace enabled account does not support this test",
+ getIsNamespaceEnabled(abfs));
+ try (NativeAzureFileSystem wasb = getWasbFileSystem()) {
+
+ Path testFile = path("/testReadFile");
+ Path path = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
+
+ // Write
+ try (FSDataOutputStream nativeFsStream = wasb.create(path, true)) {
+ nativeFsStream.write(TEST_CONTEXT.getBytes());
+ nativeFsStream.flush();
+ nativeFsStream.hsync();
+ }
+ // --- VALIDATE FILE ---
+ FileStatus status = wasb.getFileStatus(path);
+ assertIsFile(path, status);
+
+ // --- SET XATTR #1 ---
+ wasb.setXAttr(path, ATTRIBUTE_NAME_1, ATTRIBUTE_VALUE_1);
+ byte[] readValue = abfs.getXAttr(path, ATTRIBUTE_NAME_1);
+ ITestAzureBlobFileSystemAttributes.assertAttributeEqual(abfs, readValue, ATTRIBUTE_VALUE_1, "one");
+
+ // --- SET XATTR #2 ---
+ wasb.setXAttr(path, ATTRIBUTE_NAME_2, ATTRIBUTE_VALUE_2);
+ readValue = abfs.getXAttr(path, ATTRIBUTE_NAME_2);
+ ITestAzureBlobFileSystemAttributes.assertAttributeEqual(abfs, readValue, ATTRIBUTE_VALUE_2, "two");
+
+ // --- VERIFY XATTR #1 AGAIN ---
+ readValue = abfs.getXAttr(path, ATTRIBUTE_NAME_1);
+ ITestAzureBlobFileSystemAttributes.assertAttributeEqual(abfs, readValue, ATTRIBUTE_VALUE_1, "one");
+
+ wasb.delete(path, true);
+ }
}
- // --- VALIDATE FILE ---
- FileStatus status = wasb.getFileStatus(path);
- assertIsFile(path, status);
-
- // --- SET XATTR #1 ---
- wasb.setXAttr(path, ATTRIBUTE_NAME_1, ATTRIBUTE_VALUE_1);
- byte[] readValue = abfs.getXAttr(path, ATTRIBUTE_NAME_1);
- ITestAzureBlobFileSystemAttributes.assertAttributeEqual(abfs, readValue, ATTRIBUTE_VALUE_1, "one");
-
- // --- SET XATTR #2 ---
- wasb.setXAttr(path, ATTRIBUTE_NAME_2, ATTRIBUTE_VALUE_2);
- readValue = abfs.getXAttr(path, ATTRIBUTE_NAME_2);
- ITestAzureBlobFileSystemAttributes.assertAttributeEqual(abfs, readValue, ATTRIBUTE_VALUE_2, "two");
-
- // --- VERIFY XATTR #1 AGAIN ---
- readValue = abfs.getXAttr(path, ATTRIBUTE_NAME_1);
- ITestAzureBlobFileSystemAttributes.assertAttributeEqual(abfs, readValue, ATTRIBUTE_VALUE_1, "one");
-
- wasb.delete(path, true);
}
/**
@@ -1126,45 +1177,47 @@ public void testScenario20() throws Exception {
Configuration conf = getRawConfiguration();
conf.setBoolean(FS_AZURE_ABFS_ENABLE_CHECKSUM_VALIDATION, true);
FileSystem fileSystem = FileSystem.newInstance(conf);
- AzureBlobFileSystem abfs = (AzureBlobFileSystem) fileSystem;
- Assume.assumeFalse("Namespace enabled account does not support this test",
- getIsNamespaceEnabled(abfs));
- Assume.assumeFalse("Not valid for APPEND BLOB", isAppendBlobEnabled());
- NativeAzureFileSystem wasb = getWasbFileSystem();
-
- Path testFile = path("/testReadFile");
- Path path = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
-
- // Write
- try (FSDataOutputStream nativeFsStream = wasb.create(path, true)) {
- nativeFsStream.write(TEST_CONTEXT.getBytes());
- nativeFsStream.flush();
- nativeFsStream.hsync();
+ try (AzureBlobFileSystem abfs = (AzureBlobFileSystem) fileSystem) {
+ Assume.assumeFalse("Namespace enabled account does not support this test",
+ getIsNamespaceEnabled(abfs));
+ Assume.assumeFalse("Not valid for APPEND BLOB", isAppendBlobEnabled());
+ try (NativeAzureFileSystem wasb = getWasbFileSystem()) {
+
+ Path testFile = path("/testReadFile");
+ Path path = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
+
+ // Write
+ try (FSDataOutputStream nativeFsStream = wasb.create(path, true)) {
+ nativeFsStream.write(TEST_CONTEXT.getBytes());
+ nativeFsStream.flush();
+ nativeFsStream.hsync();
+ }
+ // --- VALIDATE FILE ---
+ FileStatus status = wasb.getFileStatus(path);
+ assertIsFile(path, status);
+
+ // --- SET XATTR #1 ---
+ wasb.setXAttr(path, ATTRIBUTE_NAME_1, ATTRIBUTE_VALUE_1);
+ byte[] readValue = abfs.getXAttr(path, ATTRIBUTE_NAME_1);
+ ITestAzureBlobFileSystemAttributes.assertAttributeEqual(abfs, readValue, ATTRIBUTE_VALUE_1, "one");
+
+ // --- SET XATTR #2 ---
+ wasb.setXAttr(path, ATTRIBUTE_NAME_2, ATTRIBUTE_VALUE_2);
+ readValue = abfs.getXAttr(path, ATTRIBUTE_NAME_2);
+ ITestAzureBlobFileSystemAttributes.assertAttributeEqual(abfs, readValue, ATTRIBUTE_VALUE_2, "two");
+
+ // --- VERIFY XATTR #1 AGAIN ---
+ readValue = abfs.getXAttr(path, ATTRIBUTE_NAME_1);
+ ITestAzureBlobFileSystemAttributes.assertAttributeEqual(abfs, readValue, ATTRIBUTE_VALUE_1, "one");
+
+ abfs.create(path, true);
+ FileStatus fileStatus = abfs.getFileStatus(path);
+ Assertions.assertThat(fileStatus.getLen())
+ .as("Expected file length to be 0 after overwrite")
+ .isEqualTo(0L);
+ wasb.delete(path, true);
+ }
}
- // --- VALIDATE FILE ---
- FileStatus status = wasb.getFileStatus(path);
- assertIsFile(path, status);
-
- // --- SET XATTR #1 ---
- wasb.setXAttr(path, ATTRIBUTE_NAME_1, ATTRIBUTE_VALUE_1);
- byte[] readValue = abfs.getXAttr(path, ATTRIBUTE_NAME_1);
- ITestAzureBlobFileSystemAttributes.assertAttributeEqual(abfs, readValue, ATTRIBUTE_VALUE_1, "one");
-
- // --- SET XATTR #2 ---
- wasb.setXAttr(path, ATTRIBUTE_NAME_2, ATTRIBUTE_VALUE_2);
- readValue = abfs.getXAttr(path, ATTRIBUTE_NAME_2);
- ITestAzureBlobFileSystemAttributes.assertAttributeEqual(abfs, readValue, ATTRIBUTE_VALUE_2, "two");
-
- // --- VERIFY XATTR #1 AGAIN ---
- readValue = abfs.getXAttr(path, ATTRIBUTE_NAME_1);
- ITestAzureBlobFileSystemAttributes.assertAttributeEqual(abfs, readValue, ATTRIBUTE_VALUE_1, "one");
-
- abfs.create(path, true);
- FileStatus fileStatus = abfs.getFileStatus(path);
- Assertions.assertThat(fileStatus.getLen())
- .as("Expected file length to be 0 after overwrite")
- .isEqualTo(0L);
- wasb.delete(path, true);
}
/**
@@ -1178,46 +1231,48 @@ public void testScenario21() throws Exception {
Configuration conf = getRawConfiguration();
conf.setBoolean(FS_AZURE_ABFS_ENABLE_CHECKSUM_VALIDATION, true);
FileSystem fileSystem = FileSystem.newInstance(conf);
- AzureBlobFileSystem abfs = (AzureBlobFileSystem) fileSystem;
- Assume.assumeFalse("Namespace enabled account does not support this test",
- getIsNamespaceEnabled(abfs));
- assumeBlobServiceType();
- Assume.assumeFalse("Not valid for APPEND BLOB", isAppendBlobEnabled());
- NativeAzureFileSystem wasb = getWasbFileSystem();
-
- Path testFile = path("/testReadFile");
- Path path = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
-
- // Write
- try (FSDataOutputStream abfsOutputStream = abfs.create(path, true)) {
- abfsOutputStream.write(TEST_CONTEXT.getBytes());
- abfsOutputStream.flush();
- abfsOutputStream.hsync();
+ try (AzureBlobFileSystem abfs = (AzureBlobFileSystem) fileSystem) {
+ Assume.assumeFalse("Namespace enabled account does not support this test",
+ getIsNamespaceEnabled(abfs));
+ assumeBlobServiceType();
+ Assume.assumeFalse("Not valid for APPEND BLOB", isAppendBlobEnabled());
+ try (NativeAzureFileSystem wasb = getWasbFileSystem()) {
+
+ Path testFile = path("/testReadFile");
+ Path path = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
+
+ // Write
+ try (FSDataOutputStream abfsOutputStream = abfs.create(path, true)) {
+ abfsOutputStream.write(TEST_CONTEXT.getBytes());
+ abfsOutputStream.flush();
+ abfsOutputStream.hsync();
+ }
+ // --- VALIDATE FILE ---
+ FileStatus status = wasb.getFileStatus(path);
+ assertIsFile(path, status);
+
+ // --- SET XATTR #1 ---
+ abfs.setXAttr(path, ATTRIBUTE_NAME_1, ATTRIBUTE_VALUE_1);
+ byte[] readValue = wasb.getXAttr(path, ATTRIBUTE_NAME_1);
+ ITestAzureBlobFileSystemAttributes.assertAttributeEqual(abfs, readValue, ATTRIBUTE_VALUE_1, "one");
+
+ // --- SET XATTR #2 ---
+ abfs.setXAttr(path, ATTRIBUTE_NAME_2, ATTRIBUTE_VALUE_2);
+ readValue = wasb.getXAttr(path, ATTRIBUTE_NAME_2);
+ ITestAzureBlobFileSystemAttributes.assertAttributeEqual(abfs, readValue, ATTRIBUTE_VALUE_2, "two");
+
+ // --- VERIFY XATTR #1 AGAIN ---
+ readValue = wasb.getXAttr(path, ATTRIBUTE_NAME_1);
+ ITestAzureBlobFileSystemAttributes.assertAttributeEqual(abfs, readValue, ATTRIBUTE_VALUE_1, "one");
+
+ wasb.create(path, true);
+ FileStatus fileStatus = abfs.getFileStatus(path);
+ Assertions.assertThat(fileStatus.getLen())
+ .as("Expected file length to be 0 after overwrite")
+ .isEqualTo(0L);
+ wasb.delete(path, true);
+ }
}
- // --- VALIDATE FILE ---
- FileStatus status = wasb.getFileStatus(path);
- assertIsFile(path, status);
-
- // --- SET XATTR #1 ---
- abfs.setXAttr(path, ATTRIBUTE_NAME_1, ATTRIBUTE_VALUE_1);
- byte[] readValue = wasb.getXAttr(path, ATTRIBUTE_NAME_1);
- ITestAzureBlobFileSystemAttributes.assertAttributeEqual(abfs, readValue, ATTRIBUTE_VALUE_1, "one");
-
- // --- SET XATTR #2 ---
- abfs.setXAttr(path, ATTRIBUTE_NAME_2, ATTRIBUTE_VALUE_2);
- readValue = wasb.getXAttr(path, ATTRIBUTE_NAME_2);
- ITestAzureBlobFileSystemAttributes.assertAttributeEqual(abfs, readValue, ATTRIBUTE_VALUE_2, "two");
-
- // --- VERIFY XATTR #1 AGAIN ---
- readValue = wasb.getXAttr(path, ATTRIBUTE_NAME_1);
- ITestAzureBlobFileSystemAttributes.assertAttributeEqual(abfs, readValue, ATTRIBUTE_VALUE_1, "one");
-
- wasb.create(path, true);
- FileStatus fileStatus = abfs.getFileStatus(path);
- Assertions.assertThat(fileStatus.getLen())
- .as("Expected file length to be 0 after overwrite")
- .isEqualTo(0L);
- wasb.delete(path, true);
}
/**
@@ -1231,44 +1286,46 @@ public void testScenario22() throws Exception {
Configuration conf = getRawConfiguration();
conf.setBoolean(FS_AZURE_ABFS_ENABLE_CHECKSUM_VALIDATION, true);
FileSystem fileSystem = FileSystem.newInstance(conf);
- AzureBlobFileSystem abfs = (AzureBlobFileSystem) fileSystem;
- Assume.assumeFalse("Namespace enabled account does not support this test",
- getIsNamespaceEnabled(abfs));
- NativeAzureFileSystem wasb = getWasbFileSystem();
-
- Path testFile = path("/testReadFile");
- Path path = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
-
- // Write
- try (FSDataOutputStream nativeFsStream = wasb.create(path, true)) {
- nativeFsStream.write(TEST_CONTEXT.getBytes());
- nativeFsStream.flush();
- nativeFsStream.hsync();
+ try (AzureBlobFileSystem abfs = (AzureBlobFileSystem) fileSystem) {
+ Assume.assumeFalse("Namespace enabled account does not support this test",
+ getIsNamespaceEnabled(abfs));
+ try (NativeAzureFileSystem wasb = getWasbFileSystem()) {
+
+ Path testFile = path("/testReadFile");
+ Path path = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
+
+ // Write
+ try (FSDataOutputStream nativeFsStream = wasb.create(path, true)) {
+ nativeFsStream.write(TEST_CONTEXT.getBytes());
+ nativeFsStream.flush();
+ nativeFsStream.hsync();
+ }
+ // --- VALIDATE FILE ---
+ FileStatus status = wasb.getFileStatus(path);
+ assertIsFile(path, status);
+
+ // --- SET XATTR #1 ---
+ abfs.setXAttr(path, ATTRIBUTE_NAME_1, ATTRIBUTE_VALUE_1);
+ byte[] readValue = wasb.getXAttr(path, ATTRIBUTE_NAME_1);
+ ITestAzureBlobFileSystemAttributes.assertAttributeEqual(abfs, readValue, ATTRIBUTE_VALUE_1, "one");
+
+ // --- SET XATTR #2 ---
+ abfs.setXAttr(path, ATTRIBUTE_NAME_2, ATTRIBUTE_VALUE_2);
+ readValue = wasb.getXAttr(path, ATTRIBUTE_NAME_2);
+ ITestAzureBlobFileSystemAttributes.assertAttributeEqual(abfs, readValue, ATTRIBUTE_VALUE_2, "two");
+
+ // --- VERIFY XATTR #1 AGAIN ---
+ readValue = wasb.getXAttr(path, ATTRIBUTE_NAME_1);
+ ITestAzureBlobFileSystemAttributes.assertAttributeEqual(abfs, readValue, ATTRIBUTE_VALUE_1, "one");
+
+ wasb.create(path, true);
+ FileStatus fileStatus = abfs.getFileStatus(path);
+ Assertions.assertThat(fileStatus.getLen())
+ .as("Expected file length to be 0 after overwrite")
+ .isEqualTo(0L);
+ wasb.delete(path, true);
+ }
}
- // --- VALIDATE FILE ---
- FileStatus status = wasb.getFileStatus(path);
- assertIsFile(path, status);
-
- // --- SET XATTR #1 ---
- abfs.setXAttr(path, ATTRIBUTE_NAME_1, ATTRIBUTE_VALUE_1);
- byte[] readValue = wasb.getXAttr(path, ATTRIBUTE_NAME_1);
- ITestAzureBlobFileSystemAttributes.assertAttributeEqual(abfs, readValue, ATTRIBUTE_VALUE_1, "one");
-
- // --- SET XATTR #2 ---
- abfs.setXAttr(path, ATTRIBUTE_NAME_2, ATTRIBUTE_VALUE_2);
- readValue = wasb.getXAttr(path, ATTRIBUTE_NAME_2);
- ITestAzureBlobFileSystemAttributes.assertAttributeEqual(abfs, readValue, ATTRIBUTE_VALUE_2, "two");
-
- // --- VERIFY XATTR #1 AGAIN ---
- readValue = wasb.getXAttr(path, ATTRIBUTE_NAME_1);
- ITestAzureBlobFileSystemAttributes.assertAttributeEqual(abfs, readValue, ATTRIBUTE_VALUE_1, "one");
-
- wasb.create(path, true);
- FileStatus fileStatus = abfs.getFileStatus(path);
- Assertions.assertThat(fileStatus.getLen())
- .as("Expected file length to be 0 after overwrite")
- .isEqualTo(0L);
- wasb.delete(path, true);
}
/**
@@ -1282,38 +1339,40 @@ public void testScenario23() throws Exception {
Configuration conf = getRawConfiguration();
conf.setBoolean(FS_AZURE_ABFS_ENABLE_CHECKSUM_VALIDATION, true);
FileSystem fileSystem = FileSystem.newInstance(conf);
- AzureBlobFileSystem abfs = (AzureBlobFileSystem) fileSystem;
- Assume.assumeFalse("Namespace enabled account does not support this test",
- getIsNamespaceEnabled(abfs));
- NativeAzureFileSystem wasb = getWasbFileSystem();
-
- Path testFile = path("/testReadFile");
- Path path = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
- // Write
- try (FSDataOutputStream nativeFsStream = wasb.create(path, true)) {
- nativeFsStream.write(TEST_CONTEXT.getBytes());
- nativeFsStream.flush();
- nativeFsStream.hsync();
+ try (AzureBlobFileSystem abfs = (AzureBlobFileSystem) fileSystem) {
+ Assume.assumeFalse("Namespace enabled account does not support this test",
+ getIsNamespaceEnabled(abfs));
+ try (NativeAzureFileSystem wasb = getWasbFileSystem()) {
+
+ Path testFile = path("/testReadFile");
+ Path path = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
+ // Write
+ try (FSDataOutputStream nativeFsStream = wasb.create(path, true)) {
+ nativeFsStream.write(TEST_CONTEXT.getBytes());
+ nativeFsStream.flush();
+ nativeFsStream.hsync();
+ }
+ // --- VALIDATE FILE ---
+ FileStatus status = wasb.getFileStatus(path);
+ assertIsFile(path, status);
+
+ // --- SET XATTR #1 ---
+ abfs.setXAttr(path, ATTRIBUTE_NAME_1, ATTRIBUTE_VALUE_1);
+ byte[] readValue = abfs.getXAttr(path, ATTRIBUTE_NAME_1);
+ ITestAzureBlobFileSystemAttributes.assertAttributeEqual(abfs, readValue, ATTRIBUTE_VALUE_1, "one");
+
+ // --- SET XATTR #2 ---
+ wasb.setXAttr(path, ATTRIBUTE_NAME_2, ATTRIBUTE_VALUE_2, CREATE_FLAG);
+ readValue = abfs.getXAttr(path, ATTRIBUTE_NAME_2);
+ ITestAzureBlobFileSystemAttributes.assertAttributeEqual(abfs, readValue, ATTRIBUTE_VALUE_2, "two");
+
+ // --- VERIFY XATTR #1 AGAIN ---
+ readValue = abfs.getXAttr(path, ATTRIBUTE_NAME_1);
+ ITestAzureBlobFileSystemAttributes.assertAttributeEqual(abfs, readValue, ATTRIBUTE_VALUE_1, "one");
+
+ wasb.delete(path, true);
+ }
}
- // --- VALIDATE FILE ---
- FileStatus status = wasb.getFileStatus(path);
- assertIsFile(path, status);
-
- // --- SET XATTR #1 ---
- abfs.setXAttr(path, ATTRIBUTE_NAME_1, ATTRIBUTE_VALUE_1);
- byte[] readValue = abfs.getXAttr(path, ATTRIBUTE_NAME_1);
- ITestAzureBlobFileSystemAttributes.assertAttributeEqual(abfs, readValue, ATTRIBUTE_VALUE_1, "one");
-
- // --- SET XATTR #2 ---
- wasb.setXAttr(path, ATTRIBUTE_NAME_2, ATTRIBUTE_VALUE_2, CREATE_FLAG);
- readValue = abfs.getXAttr(path, ATTRIBUTE_NAME_2);
- ITestAzureBlobFileSystemAttributes.assertAttributeEqual(abfs, readValue, ATTRIBUTE_VALUE_2, "two");
-
- // --- VERIFY XATTR #1 AGAIN ---
- readValue = abfs.getXAttr(path, ATTRIBUTE_NAME_1);
- ITestAzureBlobFileSystemAttributes.assertAttributeEqual(abfs, readValue, ATTRIBUTE_VALUE_1, "one");
-
- wasb.delete(path, true);
}
/**
@@ -1327,40 +1386,42 @@ public void testScenario24() throws Exception {
Configuration conf = getRawConfiguration();
conf.setBoolean(FS_AZURE_ABFS_ENABLE_CHECKSUM_VALIDATION, true);
FileSystem fileSystem = FileSystem.newInstance(conf);
- AzureBlobFileSystem abfs = (AzureBlobFileSystem) fileSystem;
- Assume.assumeFalse("Namespace enabled account does not support this test",
- getIsNamespaceEnabled(abfs));
- Assume.assumeFalse("Not valid for APPEND BLOB", isAppendBlobEnabled());
- NativeAzureFileSystem wasb = getWasbFileSystem();
-
- Path testFile = path("/testReadFile");
- Path path = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
-
- // Write
- try (FSDataOutputStream abfsOutputStream = abfs.create(path, true)) {
- abfsOutputStream.write(TEST_CONTEXT.getBytes());
- abfsOutputStream.flush();
- abfsOutputStream.hsync();
+ try (AzureBlobFileSystem abfs = (AzureBlobFileSystem) fileSystem) {
+ Assume.assumeFalse("Namespace enabled account does not support this test",
+ getIsNamespaceEnabled(abfs));
+ Assume.assumeFalse("Not valid for APPEND BLOB", isAppendBlobEnabled());
+ try (NativeAzureFileSystem wasb = getWasbFileSystem()) {
+
+ Path testFile = path("/testReadFile");
+ Path path = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
+
+ // Write
+ try (FSDataOutputStream abfsOutputStream = abfs.create(path, true)) {
+ abfsOutputStream.write(TEST_CONTEXT.getBytes());
+ abfsOutputStream.flush();
+ abfsOutputStream.hsync();
+ }
+ // --- VALIDATE FILE ---
+ FileStatus status = wasb.getFileStatus(path);
+ assertIsFile(path, status);
+
+ // --- SET XATTR #1 ---
+ wasb.setXAttr(path, ATTRIBUTE_NAME_1, ATTRIBUTE_VALUE_1);
+ byte[] readValue = abfs.getXAttr(path, ATTRIBUTE_NAME_1);
+ ITestAzureBlobFileSystemAttributes.assertAttributeEqual(abfs, readValue, ATTRIBUTE_VALUE_1, "one");
+
+ // --- SET XATTR #2 ---
+ wasb.setXAttr(path, ATTRIBUTE_NAME_2, ATTRIBUTE_VALUE_2, CREATE_FLAG);
+ readValue = abfs.getXAttr(path, ATTRIBUTE_NAME_2);
+ ITestAzureBlobFileSystemAttributes.assertAttributeEqual(abfs, readValue, ATTRIBUTE_VALUE_2, "two");
+
+ // --- VERIFY XATTR #1 AGAIN ---
+ readValue = abfs.getXAttr(path, ATTRIBUTE_NAME_1);
+ ITestAzureBlobFileSystemAttributes.assertAttributeEqual(abfs, readValue, ATTRIBUTE_VALUE_1, "one");
+
+ wasb.delete(path, true);
+ }
}
- // --- VALIDATE FILE ---
- FileStatus status = wasb.getFileStatus(path);
- assertIsFile(path, status);
-
- // --- SET XATTR #1 ---
- wasb.setXAttr(path, ATTRIBUTE_NAME_1, ATTRIBUTE_VALUE_1);
- byte[] readValue = abfs.getXAttr(path, ATTRIBUTE_NAME_1);
- ITestAzureBlobFileSystemAttributes.assertAttributeEqual(abfs, readValue, ATTRIBUTE_VALUE_1, "one");
-
- // --- SET XATTR #2 ---
- wasb.setXAttr(path, ATTRIBUTE_NAME_2, ATTRIBUTE_VALUE_2, CREATE_FLAG);
- readValue = abfs.getXAttr(path, ATTRIBUTE_NAME_2);
- ITestAzureBlobFileSystemAttributes.assertAttributeEqual(abfs, readValue, ATTRIBUTE_VALUE_2, "two");
-
- // --- VERIFY XATTR #1 AGAIN ---
- readValue = abfs.getXAttr(path, ATTRIBUTE_NAME_1);
- ITestAzureBlobFileSystemAttributes.assertAttributeEqual(abfs, readValue, ATTRIBUTE_VALUE_1, "one");
-
- wasb.delete(path, true);
}
/**
@@ -1374,39 +1435,41 @@ public void testScenario25() throws Exception {
Configuration conf = getRawConfiguration();
conf.setBoolean(FS_AZURE_ABFS_ENABLE_CHECKSUM_VALIDATION, true);
FileSystem fileSystem = FileSystem.newInstance(conf);
- AzureBlobFileSystem abfs = (AzureBlobFileSystem) fileSystem;
- Assume.assumeFalse("Namespace enabled account does not support this test",
- getIsNamespaceEnabled(abfs));
- NativeAzureFileSystem wasb = getWasbFileSystem();
-
- Path testFile = path("/testReadFile");
- Path path = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
-
- // Write
- try (FSDataOutputStream nativeFsStream = wasb.create(path, true)) {
- nativeFsStream.write(TEST_CONTEXT.getBytes());
- nativeFsStream.flush();
- nativeFsStream.hsync();
+ try (AzureBlobFileSystem abfs = (AzureBlobFileSystem) fileSystem) {
+ Assume.assumeFalse("Namespace enabled account does not support this test",
+ getIsNamespaceEnabled(abfs));
+ try (NativeAzureFileSystem wasb = getWasbFileSystem()) {
+
+ Path testFile = path("/testReadFile");
+ Path path = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
+
+ // Write
+ try (FSDataOutputStream nativeFsStream = wasb.create(path, true)) {
+ nativeFsStream.write(TEST_CONTEXT.getBytes());
+ nativeFsStream.flush();
+ nativeFsStream.hsync();
+ }
+ // --- VALIDATE FILE ---
+ FileStatus status = wasb.getFileStatus(path);
+ assertIsFile(path, status);
+
+ // --- SET XATTR #1 ---
+ abfs.setXAttr(path, ATTRIBUTE_NAME_1, ATTRIBUTE_VALUE_1);
+ byte[] readValue = abfs.getXAttr(path, ATTRIBUTE_NAME_1);
+ ITestAzureBlobFileSystemAttributes.assertAttributeEqual(abfs, readValue, ATTRIBUTE_VALUE_1, "one");
+
+ // --- SET XATTR #2 ---
+ abfs.setXAttr(path, ATTRIBUTE_NAME_2, ATTRIBUTE_VALUE_2, CREATE_FLAG);
+ readValue = abfs.getXAttr(path, ATTRIBUTE_NAME_2);
+ ITestAzureBlobFileSystemAttributes.assertAttributeEqual(abfs, readValue, ATTRIBUTE_VALUE_2, "two");
+
+ // --- VERIFY XATTR #1 AGAIN ---
+ readValue = abfs.getXAttr(path, ATTRIBUTE_NAME_1);
+ ITestAzureBlobFileSystemAttributes.assertAttributeEqual(abfs, readValue, ATTRIBUTE_VALUE_1, "one");
+
+ wasb.delete(path, true);
+ }
}
- // --- VALIDATE FILE ---
- FileStatus status = wasb.getFileStatus(path);
- assertIsFile(path, status);
-
- // --- SET XATTR #1 ---
- abfs.setXAttr(path, ATTRIBUTE_NAME_1, ATTRIBUTE_VALUE_1);
- byte[] readValue = abfs.getXAttr(path, ATTRIBUTE_NAME_1);
- ITestAzureBlobFileSystemAttributes.assertAttributeEqual(abfs, readValue, ATTRIBUTE_VALUE_1, "one");
-
- // --- SET XATTR #2 ---
- abfs.setXAttr(path, ATTRIBUTE_NAME_2, ATTRIBUTE_VALUE_2, CREATE_FLAG);
- readValue = abfs.getXAttr(path, ATTRIBUTE_NAME_2);
- ITestAzureBlobFileSystemAttributes.assertAttributeEqual(abfs, readValue, ATTRIBUTE_VALUE_2, "two");
-
- // --- VERIFY XATTR #1 AGAIN ---
- readValue = abfs.getXAttr(path, ATTRIBUTE_NAME_1);
- ITestAzureBlobFileSystemAttributes.assertAttributeEqual(abfs, readValue, ATTRIBUTE_VALUE_1, "one");
-
- wasb.delete(path, true);
}
/**
@@ -1420,40 +1483,42 @@ public void testScenario26() throws Exception {
Configuration conf = getRawConfiguration();
conf.setBoolean(FS_AZURE_ABFS_ENABLE_CHECKSUM_VALIDATION, true);
FileSystem fileSystem = FileSystem.newInstance(conf);
- AzureBlobFileSystem abfs = (AzureBlobFileSystem) fileSystem;
- Assume.assumeFalse("Namespace enabled account does not support this test",
- getIsNamespaceEnabled(abfs));
- Assume.assumeFalse("Not valid for APPEND BLOB", isAppendBlobEnabled());
- NativeAzureFileSystem wasb = getWasbFileSystem();
-
- Path testFile = path("/testReadFile");
- Path path = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
-
- // Write
- try (FSDataOutputStream abfsOutputStream = abfs.create(path, true)) {
- abfsOutputStream.write(TEST_CONTEXT.getBytes());
- abfsOutputStream.flush();
- abfsOutputStream.hsync();
+ try (AzureBlobFileSystem abfs = (AzureBlobFileSystem) fileSystem) {
+ Assume.assumeFalse("Namespace enabled account does not support this test",
+ getIsNamespaceEnabled(abfs));
+ Assume.assumeFalse("Not valid for APPEND BLOB", isAppendBlobEnabled());
+ try (NativeAzureFileSystem wasb = getWasbFileSystem()) {
+
+ Path testFile = path("/testReadFile");
+ Path path = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
+
+ // Write
+ try (FSDataOutputStream abfsOutputStream = abfs.create(path, true)) {
+ abfsOutputStream.write(TEST_CONTEXT.getBytes());
+ abfsOutputStream.flush();
+ abfsOutputStream.hsync();
+ }
+ // --- VALIDATE FILE ---
+ FileStatus status = abfs.getFileStatus(path);
+ assertIsFile(path, status);
+
+ // --- SET XATTR #1 ---
+ wasb.setXAttr(path, ATTRIBUTE_NAME_1, ATTRIBUTE_VALUE_1);
+ byte[] readValue = wasb.getXAttr(path, ATTRIBUTE_NAME_1);
+ ITestAzureBlobFileSystemAttributes.assertAttributeEqual(abfs, readValue, ATTRIBUTE_VALUE_1, "one");
+
+ // --- SET XATTR #2 ---
+ wasb.setXAttr(path, ATTRIBUTE_NAME_2, ATTRIBUTE_VALUE_2, CREATE_FLAG);
+ readValue = wasb.getXAttr(path, ATTRIBUTE_NAME_2);
+ ITestAzureBlobFileSystemAttributes.assertAttributeEqual(abfs, readValue, ATTRIBUTE_VALUE_2, "two");
+
+ // --- VERIFY XATTR #1 AGAIN ---
+ readValue = wasb.getXAttr(path, ATTRIBUTE_NAME_1);
+ ITestAzureBlobFileSystemAttributes.assertAttributeEqual(abfs, readValue, ATTRIBUTE_VALUE_1, "one");
+
+ wasb.delete(path, true);
+ }
}
- // --- VALIDATE FILE ---
- FileStatus status = abfs.getFileStatus(path);
- assertIsFile(path, status);
-
- // --- SET XATTR #1 ---
- wasb.setXAttr(path, ATTRIBUTE_NAME_1, ATTRIBUTE_VALUE_1);
- byte[] readValue = wasb.getXAttr(path, ATTRIBUTE_NAME_1);
- ITestAzureBlobFileSystemAttributes.assertAttributeEqual(abfs, readValue, ATTRIBUTE_VALUE_1, "one");
-
- // --- SET XATTR #2 ---
- wasb.setXAttr(path, ATTRIBUTE_NAME_2, ATTRIBUTE_VALUE_2, CREATE_FLAG);
- readValue = wasb.getXAttr(path, ATTRIBUTE_NAME_2);
- ITestAzureBlobFileSystemAttributes.assertAttributeEqual(abfs, readValue, ATTRIBUTE_VALUE_2, "two");
-
- // --- VERIFY XATTR #1 AGAIN ---
- readValue = wasb.getXAttr(path, ATTRIBUTE_NAME_1);
- ITestAzureBlobFileSystemAttributes.assertAttributeEqual(abfs, readValue, ATTRIBUTE_VALUE_1, "one");
-
- wasb.delete(path, true);
}
/**
@@ -1465,45 +1530,47 @@ public void testScenario27() throws Exception {
Configuration conf = getRawConfiguration();
conf.setBoolean(FS_AZURE_ABFS_ENABLE_CHECKSUM_VALIDATION, true);
FileSystem fileSystem = FileSystem.newInstance(conf);
- AzureBlobFileSystem abfs = (AzureBlobFileSystem) fileSystem;
- Assume.assumeFalse("Namespace enabled account does not support this test",
- getIsNamespaceEnabled(abfs));
- Assume.assumeFalse("Not valid for APPEND BLOB", isAppendBlobEnabled());
- NativeAzureFileSystem wasb = getWasbFileSystem();
-
- Path testFile = path("/testReadFile");
- Path testPath1 = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
- Path testPath2 = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
-
- // Write
- try (FSDataOutputStream abfsOutputStream = abfs.create(testPath1, true)) {
- abfsOutputStream.write(TEST_CONTEXT.getBytes());
- abfsOutputStream.flush();
- abfsOutputStream.hsync();
- }
-
- // Check file status
- ContractTestUtils.assertIsFile(abfs, testPath1);
-
- try (BufferedReader br = new BufferedReader(
- new InputStreamReader(abfs.open(testPath1)))) {
- String line = br.readLine();
- assertEquals("Wrong text from " + abfs,
- TEST_CONTEXT, line);
+ try (AzureBlobFileSystem abfs = (AzureBlobFileSystem) fileSystem) {
+ Assume.assumeFalse("Namespace enabled account does not support this test",
+ getIsNamespaceEnabled(abfs));
+ Assume.assumeFalse("Not valid for APPEND BLOB", isAppendBlobEnabled());
+ try (NativeAzureFileSystem wasb = getWasbFileSystem()) {
+
+ Path testFile = path("/testReadFile");
+ Path testPath1 = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
+ Path testPath2 = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
+
+ // Write
+ try (FSDataOutputStream abfsOutputStream = abfs.create(testPath1, true)) {
+ abfsOutputStream.write(TEST_CONTEXT.getBytes());
+ abfsOutputStream.flush();
+ abfsOutputStream.hsync();
+ }
+
+ // Check file status
+ ContractTestUtils.assertIsFile(abfs, testPath1);
+
+ try (BufferedReader br = new BufferedReader(
+ new InputStreamReader(abfs.open(testPath1)))) {
+ String line = br.readLine();
+ assertEquals("Wrong text from " + abfs,
+ TEST_CONTEXT, line);
+ }
+ // --- RENAME FILE ---
+ boolean renamed = wasb.rename(testPath1, testPath2);
+ Assertions.assertThat(renamed)
+ .as("Rename failed")
+ .isTrue();
+
+ // --- LIST FILES IN DIRECTORY ---
+ Path parentDir = new Path(testFile + "/~12/!008");
+ int noOfFiles = listAllFilesAndDirs(wasb, parentDir);
+ Assertions.assertThat(noOfFiles)
+ .as("Expected only 1 file or directory under path: %s", parentDir)
+ .isEqualTo(1);
+ wasb.delete(testPath2, true);
+ }
}
- // --- RENAME FILE ---
- boolean renamed = wasb.rename(testPath1, testPath2);
- Assertions.assertThat(renamed)
- .as("Rename failed")
- .isTrue();
-
- // --- LIST FILES IN DIRECTORY ---
- Path parentDir = new Path(testFile + "/~12/!008");
- int noOfFiles = listAllFilesAndDirs(wasb, parentDir);
- Assertions.assertThat(noOfFiles)
- .as("Expected only 1 file or directory under path: %s", parentDir)
- .isEqualTo(1);
- wasb.delete(testPath2, true);
}
/**
@@ -1515,44 +1582,46 @@ public void testScenario28() throws Exception {
Configuration conf = getRawConfiguration();
conf.setBoolean(FS_AZURE_ABFS_ENABLE_CHECKSUM_VALIDATION, true);
FileSystem fileSystem = FileSystem.newInstance(conf);
- AzureBlobFileSystem abfs = (AzureBlobFileSystem) fileSystem;
- Assume.assumeFalse("Namespace enabled account does not support this test",
- getIsNamespaceEnabled(abfs));
- NativeAzureFileSystem wasb = getWasbFileSystem();
-
- Path testFile = path("/testReadFile");
- Path testPath1 = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
- Path testPath2 = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
-
- // Write
- try (FSDataOutputStream nativeFsStream = wasb.create(testPath1, true)) {
- nativeFsStream.write(TEST_CONTEXT.getBytes());
- nativeFsStream.flush();
- nativeFsStream.hsync();
- }
-
- // Check file status
- ContractTestUtils.assertIsFile(abfs, testPath1);
-
- try (BufferedReader br = new BufferedReader(
- new InputStreamReader(abfs.open(testPath1)))) {
- String line = br.readLine();
- assertEquals("Wrong text from " + abfs,
- TEST_CONTEXT, line);
+ try (AzureBlobFileSystem abfs = (AzureBlobFileSystem) fileSystem) {
+ Assume.assumeFalse("Namespace enabled account does not support this test",
+ getIsNamespaceEnabled(abfs));
+ try (NativeAzureFileSystem wasb = getWasbFileSystem()) {
+
+ Path testFile = path("/testReadFile");
+ Path testPath1 = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
+ Path testPath2 = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
+
+ // Write
+ try (FSDataOutputStream nativeFsStream = wasb.create(testPath1, true)) {
+ nativeFsStream.write(TEST_CONTEXT.getBytes());
+ nativeFsStream.flush();
+ nativeFsStream.hsync();
+ }
+
+ // Check file status
+ ContractTestUtils.assertIsFile(abfs, testPath1);
+
+ try (BufferedReader br = new BufferedReader(
+ new InputStreamReader(abfs.open(testPath1)))) {
+ String line = br.readLine();
+ assertEquals("Wrong text from " + abfs,
+ TEST_CONTEXT, line);
+ }
+ // --- RENAME FILE ---
+ boolean renamed = abfs.rename(testPath1, testPath2);
+ Assertions.assertThat(renamed)
+ .as("Rename failed")
+ .isTrue();
+
+ // --- LIST FILES IN DIRECTORY ---
+ Path parentDir = new Path(testFile + "/~12/!008");
+ int noOfFiles = listAllFilesAndDirs(abfs, parentDir);
+ Assertions.assertThat(noOfFiles)
+ .as("Expected only 1 file or directory under path: %s", parentDir)
+ .isEqualTo(1);
+ wasb.delete(testPath2, true);
+ }
}
- // --- RENAME FILE ---
- boolean renamed = abfs.rename(testPath1, testPath2);
- Assertions.assertThat(renamed)
- .as("Rename failed")
- .isTrue();
-
- // --- LIST FILES IN DIRECTORY ---
- Path parentDir = new Path(testFile + "/~12/!008");
- int noOfFiles = listAllFilesAndDirs(abfs, parentDir);
- Assertions.assertThat(noOfFiles)
- .as("Expected only 1 file or directory under path: %s", parentDir)
- .isEqualTo(1);
- wasb.delete(testPath2, true);
}
/**
@@ -1564,47 +1633,49 @@ public void testScenario29() throws Exception {
Configuration conf = getRawConfiguration();
conf.setBoolean(FS_AZURE_ABFS_ENABLE_CHECKSUM_VALIDATION, true);
FileSystem fileSystem = FileSystem.newInstance(conf);
- AzureBlobFileSystem abfs = (AzureBlobFileSystem) fileSystem;
- Assume.assumeFalse("Namespace enabled account does not support this test",
- getIsNamespaceEnabled(abfs));
- assumeBlobServiceType();
- Assume.assumeFalse("Not valid for APPEND BLOB", isAppendBlobEnabled());
- NativeAzureFileSystem wasb = getWasbFileSystem();
-
- Path testFile = path("/testReadFile");
- Path testPath1 = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
- Path testPath2 = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
-
- // Write
- wasb.create(testPath1, true);
- try (FSDataOutputStream abfsOutputStream = abfs.append(testPath1)) {
- abfsOutputStream.write(TEST_CONTEXT.getBytes());
- abfsOutputStream.flush();
- abfsOutputStream.hsync();
- }
-
- // Check file status
- ContractTestUtils.assertIsFile(abfs, testPath1);
-
- try (BufferedReader br = new BufferedReader(
- new InputStreamReader(abfs.open(testPath1)))) {
- String line = br.readLine();
- assertEquals("Wrong text from " + abfs,
- TEST_CONTEXT, line);
+ try (AzureBlobFileSystem abfs = (AzureBlobFileSystem) fileSystem) {
+ Assume.assumeFalse("Namespace enabled account does not support this test",
+ getIsNamespaceEnabled(abfs));
+ assumeBlobServiceType();
+ Assume.assumeFalse("Not valid for APPEND BLOB", isAppendBlobEnabled());
+ try (NativeAzureFileSystem wasb = getWasbFileSystem()) {
+
+ Path testFile = path("/testReadFile");
+ Path testPath1 = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
+ Path testPath2 = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
+
+ // Write
+ wasb.create(testPath1, true);
+ try (FSDataOutputStream abfsOutputStream = abfs.append(testPath1)) {
+ abfsOutputStream.write(TEST_CONTEXT.getBytes());
+ abfsOutputStream.flush();
+ abfsOutputStream.hsync();
+ }
+
+ // Check file status
+ ContractTestUtils.assertIsFile(abfs, testPath1);
+
+ try (BufferedReader br = new BufferedReader(
+ new InputStreamReader(abfs.open(testPath1)))) {
+ String line = br.readLine();
+ assertEquals("Wrong text from " + abfs,
+ TEST_CONTEXT, line);
+ }
+ // --- RENAME FILE ---
+ boolean renamed = abfs.rename(testPath1, testPath2);
+ Assertions.assertThat(renamed)
+ .as("Rename failed")
+ .isTrue();
+
+ // --- LIST FILES IN DIRECTORY ---
+ Path parentDir = new Path(testFile + "/~12/!008");
+ int noOfFiles = listAllFilesAndDirs(abfs, parentDir);
+ Assertions.assertThat(noOfFiles)
+ .as("Expected only 1 file or directory under path: %s", parentDir)
+ .isEqualTo(1);
+ wasb.delete(testPath2, true);
+ }
}
- // --- RENAME FILE ---
- boolean renamed = abfs.rename(testPath1, testPath2);
- Assertions.assertThat(renamed)
- .as("Rename failed")
- .isTrue();
-
- // --- LIST FILES IN DIRECTORY ---
- Path parentDir = new Path(testFile + "/~12/!008");
- int noOfFiles = listAllFilesAndDirs(abfs, parentDir);
- Assertions.assertThat(noOfFiles)
- .as("Expected only 1 file or directory under path: %s", parentDir)
- .isEqualTo(1);
- wasb.delete(testPath2, true);
}
/**
@@ -1616,51 +1687,53 @@ public void testScenario30() throws Exception {
Configuration conf = getRawConfiguration();
conf.setBoolean(FS_AZURE_ABFS_ENABLE_CHECKSUM_VALIDATION, true);
FileSystem fileSystem = FileSystem.newInstance(conf);
- AzureBlobFileSystem abfs = (AzureBlobFileSystem) fileSystem;
+ try (AzureBlobFileSystem abfs = (AzureBlobFileSystem) fileSystem) {
Assume.assumeFalse("Namespace enabled account does not support this test",
getIsNamespaceEnabled(abfs));
- NativeAzureFileSystem wasb = getWasbFileSystem();
+ try (NativeAzureFileSystem wasb = getWasbFileSystem()) {
- Path testFile = path("/testReadFile");
- Path testPath1 = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
- Path testPath2 = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
- Path testPath3 = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
+ Path testFile = path("/testReadFile");
+ Path testPath1 = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
+ Path testPath2 = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
+ Path testPath3 = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
- // Write
- try (FSDataOutputStream nativeFsStream = wasb.create(testPath1, true)) {
- nativeFsStream.write(TEST_CONTEXT.getBytes());
- nativeFsStream.flush();
- nativeFsStream.hsync();
- }
+ // Write
+ try (FSDataOutputStream nativeFsStream = wasb.create(testPath1, true)) {
+ nativeFsStream.write(TEST_CONTEXT.getBytes());
+ nativeFsStream.flush();
+ nativeFsStream.hsync();
+ }
- // Check file status
- ContractTestUtils.assertIsFile(abfs, testPath1);
+ // Check file status
+ ContractTestUtils.assertIsFile(abfs, testPath1);
- try (BufferedReader br = new BufferedReader(
- new InputStreamReader(abfs.open(testPath1)))) {
- String line = br.readLine();
- assertEquals("Wrong text from " + abfs,
- TEST_CONTEXT, line);
+ try (BufferedReader br = new BufferedReader(
+ new InputStreamReader(abfs.open(testPath1)))) {
+ String line = br.readLine();
+ assertEquals("Wrong text from " + abfs,
+ TEST_CONTEXT, line);
+ }
+ // --- RENAME FILE ---
+ boolean renamed = wasb.rename(testPath1, testPath2);
+ Assertions.assertThat(renamed)
+ .as("Rename failed")
+ .isTrue();
+
+ // --- RENAME FILE ---
+ boolean renamed1 = abfs.rename(testPath2, testPath3);
+ Assertions.assertThat(renamed1)
+ .as("Rename failed")
+ .isTrue();
+
+ // --- LIST FILES IN DIRECTORY ---
+ Path parentDir = new Path(testFile + "/~12/!008");
+ int noOfFiles = listAllFilesAndDirs(abfs, parentDir);
+ Assertions.assertThat(noOfFiles)
+ .as("Expected only 1 file or directory under path: %s", parentDir)
+ .isEqualTo(1);
+ wasb.delete(testPath3, true);
+ }
}
- // --- RENAME FILE ---
- boolean renamed = wasb.rename(testPath1, testPath2);
- Assertions.assertThat(renamed)
- .as("Rename failed")
- .isTrue();
-
- // --- RENAME FILE ---
- boolean renamed1 = abfs.rename(testPath2, testPath3);
- Assertions.assertThat(renamed1)
- .as("Rename failed")
- .isTrue();
-
- // --- LIST FILES IN DIRECTORY ---
- Path parentDir = new Path(testFile + "/~12/!008");
- int noOfFiles = listAllFilesAndDirs(abfs, parentDir);
- Assertions.assertThat(noOfFiles)
- .as("Expected only 1 file or directory under path: %s", parentDir)
- .isEqualTo(1);
- wasb.delete(testPath3, true);
}
/**
@@ -1672,38 +1745,40 @@ public void testScenario31() throws Exception {
Configuration conf = getRawConfiguration();
conf.setBoolean(FS_AZURE_ABFS_ENABLE_CHECKSUM_VALIDATION, true);
FileSystem fileSystem = FileSystem.newInstance(conf);
- AzureBlobFileSystem abfs = (AzureBlobFileSystem) fileSystem;
+ try (AzureBlobFileSystem abfs = (AzureBlobFileSystem) fileSystem) {
Assume.assumeFalse("Namespace enabled account does not support this test",
getIsNamespaceEnabled(abfs));
- NativeAzureFileSystem wasb = getWasbFileSystem();
+ try (NativeAzureFileSystem wasb = getWasbFileSystem()) {
- Path testFile = path("/testReadFile");
- Path testPath1 = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
- Path testPath2 = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
+ Path testFile = path("/testReadFile");
+ Path testPath1 = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
+ Path testPath2 = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
- // Write
- try (FSDataOutputStream nativeFsStream = wasb.create(testPath1, true)) {
- nativeFsStream.write(TEST_CONTEXT.getBytes());
- nativeFsStream.flush();
- nativeFsStream.hsync();
- }
+ // Write
+ try (FSDataOutputStream nativeFsStream = wasb.create(testPath1, true)) {
+ nativeFsStream.write(TEST_CONTEXT.getBytes());
+ nativeFsStream.flush();
+ nativeFsStream.hsync();
+ }
- // Check file status
- ContractTestUtils.assertIsFile(abfs, testPath1);
+ // Check file status
+ ContractTestUtils.assertIsFile(abfs, testPath1);
- try (BufferedReader br = new BufferedReader(
- new InputStreamReader(abfs.open(testPath1)))) {
- String line = br.readLine();
- assertEquals("Wrong text from " + abfs,
- TEST_CONTEXT, line);
- }
- wasb.delete(testPath1, true);
+ try (BufferedReader br = new BufferedReader(
+ new InputStreamReader(abfs.open(testPath1)))) {
+ String line = br.readLine();
+ assertEquals("Wrong text from " + abfs,
+ TEST_CONTEXT, line);
+ }
+ wasb.delete(testPath1, true);
- // --- RENAME FILE ---
- boolean renamed = abfs.rename(testPath1, testPath2);
- Assertions.assertThat(renamed)
- .as("Rename operation should have failed but returned true")
- .isFalse();
+ // --- RENAME FILE ---
+ boolean renamed = abfs.rename(testPath1, testPath2);
+ Assertions.assertThat(renamed)
+ .as("Rename operation should have failed but returned true")
+ .isFalse();
+ }
+ }
}
/**
@@ -1715,46 +1790,48 @@ public void testScenario32() throws Exception {
Configuration conf = getRawConfiguration();
conf.setBoolean(FS_AZURE_ABFS_ENABLE_CHECKSUM_VALIDATION, true);
FileSystem fileSystem = FileSystem.newInstance(conf);
- AzureBlobFileSystem abfs = (AzureBlobFileSystem) fileSystem;
- Assume.assumeFalse("Namespace enabled account does not support this test",
- getIsNamespaceEnabled(abfs));
- NativeAzureFileSystem wasb = getWasbFileSystem();
-
- Path testFile = path("/testReadFile");
- Path testFile1 = path("/testReadFile1");
- Path testPath1 = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
- Path testPath2 = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
- Path testPath3 = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
-
- // Write
- wasb.mkdirs(testFile);
- try (FSDataOutputStream nativeFsStream = wasb.create(testPath1, true)) {
- nativeFsStream.write(TEST_CONTEXT.getBytes());
- nativeFsStream.flush();
- nativeFsStream.hsync();
- }
- wasb.create(testPath2, true);
- wasb.create(testPath3, true);
-
- // Check file status
- ContractTestUtils.assertIsFile(abfs, testPath1);
-
- try (BufferedReader br = new BufferedReader(
- new InputStreamReader(abfs.open(testPath1)))) {
- String line = br.readLine();
- assertEquals("Wrong text from " + abfs,
- TEST_CONTEXT, line);
+ try (AzureBlobFileSystem abfs = (AzureBlobFileSystem) fileSystem) {
+ Assume.assumeFalse("Namespace enabled account does not support this test",
+ getIsNamespaceEnabled(abfs));
+ try (NativeAzureFileSystem wasb = getWasbFileSystem()) {
+
+ Path testFile = path("/testReadFile");
+ Path testFile1 = path("/testReadFile1");
+ Path testPath1 = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
+ Path testPath2 = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
+ Path testPath3 = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
+
+ // Write
+ wasb.mkdirs(testFile);
+ try (FSDataOutputStream nativeFsStream = wasb.create(testPath1, true)) {
+ nativeFsStream.write(TEST_CONTEXT.getBytes());
+ nativeFsStream.flush();
+ nativeFsStream.hsync();
+ }
+ wasb.create(testPath2, true);
+ wasb.create(testPath3, true);
+
+ // Check file status
+ ContractTestUtils.assertIsFile(abfs, testPath1);
+
+ try (BufferedReader br = new BufferedReader(
+ new InputStreamReader(abfs.open(testPath1)))) {
+ String line = br.readLine();
+ assertEquals("Wrong text from " + abfs,
+ TEST_CONTEXT, line);
+ }
+ // --- RENAME DIR ---
+ boolean renamed = abfs.rename(testFile, testFile1);
+ Assertions.assertThat(renamed)
+ .as("Rename failed")
+ .isTrue();
+ // --- LIST FILES IN DIRECTORY ---
+ int listResult = listAllFilesAndDirs(abfs, testFile1);
+ Assertions.assertThat(listResult)
+ .as("Expected only 5 entries under path: %s", testFile1)
+ .isEqualTo(5);
+ }
}
- // --- RENAME DIR ---
- boolean renamed = abfs.rename(testFile, testFile1);
- Assertions.assertThat(renamed)
- .as("Rename failed")
- .isTrue();
- // --- LIST FILES IN DIRECTORY ---
- int listResult = listAllFilesAndDirs(abfs, testFile1);
- Assertions.assertThat(listResult)
- .as("Expected only 5 entries under path: %s", testFile1)
- .isEqualTo(5);
}
/**
@@ -1766,47 +1843,49 @@ public void testScenario33() throws Exception {
Configuration conf = getRawConfiguration();
conf.setBoolean(FS_AZURE_ABFS_ENABLE_CHECKSUM_VALIDATION, true);
FileSystem fileSystem = FileSystem.newInstance(conf);
- AzureBlobFileSystem abfs = (AzureBlobFileSystem) fileSystem;
- Assume.assumeFalse("Namespace enabled account does not support this test",
- getIsNamespaceEnabled(abfs));
- Assume.assumeFalse("Not valid for APPEND BLOB", isAppendBlobEnabled());
- NativeAzureFileSystem wasb = getWasbFileSystem();
-
- Path testFile = path("/testReadFile");
- Path testFile1 = path("/testReadFile1");
- Path testPath1 = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
- Path testPath2 = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
- Path testPath3 = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
-
- // Write
- abfs.mkdirs(testFile);
- try (FSDataOutputStream abfsOutputStream = abfs.create(testPath1, true)) {
- abfsOutputStream.write(TEST_CONTEXT.getBytes());
- abfsOutputStream.flush();
- abfsOutputStream.hsync();
- }
- abfs.create(testPath2, true);
- abfs.create(testPath3, true);
-
- // Check file status
- ContractTestUtils.assertIsFile(abfs, testPath1);
-
- try (BufferedReader br = new BufferedReader(
- new InputStreamReader(abfs.open(testPath1)))) {
- String line = br.readLine();
- assertEquals("Wrong text from " + abfs,
- TEST_CONTEXT, line);
+ try (AzureBlobFileSystem abfs = (AzureBlobFileSystem) fileSystem) {
+ Assume.assumeFalse("Namespace enabled account does not support this test",
+ getIsNamespaceEnabled(abfs));
+ Assume.assumeFalse("Not valid for APPEND BLOB", isAppendBlobEnabled());
+ try (NativeAzureFileSystem wasb = getWasbFileSystem()) {
+
+ Path testFile = path("/testReadFile");
+ Path testFile1 = path("/testReadFile1");
+ Path testPath1 = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
+ Path testPath2 = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
+ Path testPath3 = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
+
+ // Write
+ abfs.mkdirs(testFile);
+ try (FSDataOutputStream abfsOutputStream = abfs.create(testPath1, true)) {
+ abfsOutputStream.write(TEST_CONTEXT.getBytes());
+ abfsOutputStream.flush();
+ abfsOutputStream.hsync();
+ }
+ abfs.create(testPath2, true);
+ abfs.create(testPath3, true);
+
+ // Check file status
+ ContractTestUtils.assertIsFile(abfs, testPath1);
+
+ try (BufferedReader br = new BufferedReader(
+ new InputStreamReader(abfs.open(testPath1)))) {
+ String line = br.readLine();
+ assertEquals("Wrong text from " + abfs,
+ TEST_CONTEXT, line);
+ }
+ // --- RENAME DIR ---
+ boolean renamed = wasb.rename(testFile, testFile1);
+ Assertions.assertThat(renamed)
+ .as("Rename failed")
+ .isTrue();
+ // --- LIST FILES IN DIRECTORY ---
+ int listResult = listAllFilesAndDirs(wasb, testFile1);
+ Assertions.assertThat(listResult)
+ .as("Expected only 5 entries under path: %s", testFile1)
+ .isEqualTo(5);
+ }
}
- // --- RENAME DIR ---
- boolean renamed = wasb.rename(testFile, testFile1);
- Assertions.assertThat(renamed)
- .as("Rename failed")
- .isTrue();
- // --- LIST FILES IN DIRECTORY ---
- int listResult = listAllFilesAndDirs(wasb, testFile1);
- Assertions.assertThat(listResult)
- .as("Expected only 5 entries under path: %s", testFile1)
- .isEqualTo(5);
}
/**
@@ -1818,44 +1897,46 @@ public void testScenario34() throws Exception {
Configuration conf = getRawConfiguration();
conf.setBoolean(FS_AZURE_ABFS_ENABLE_CHECKSUM_VALIDATION, true);
FileSystem fileSystem = FileSystem.newInstance(conf);
- AzureBlobFileSystem abfs = (AzureBlobFileSystem) fileSystem;
- Assume.assumeFalse("Namespace enabled account does not support this test",
- getIsNamespaceEnabled(abfs));
- NativeAzureFileSystem wasb = getWasbFileSystem();
-
- Path testFile = path("/testReadFile");
- Path testPath1 = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
- Path testPath2 = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
- Path testPath3 = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
-
- // Write
- abfs.mkdirs(testFile);
- try (FSDataOutputStream abfsOutputStream = abfs.create(testPath1, true)) {
- abfsOutputStream.write(TEST_CONTEXT.getBytes());
- abfsOutputStream.flush();
- abfsOutputStream.hsync();
- }
- abfs.create(testPath3, true);
-
- // Check file status
- ContractTestUtils.assertIsFile(abfs, testPath1);
-
- try (BufferedReader br = new BufferedReader(
- new InputStreamReader(abfs.open(testPath1)))) {
- String line = br.readLine();
- assertEquals("Wrong text from " + abfs,
- TEST_CONTEXT, line);
+ try (AzureBlobFileSystem abfs = (AzureBlobFileSystem) fileSystem) {
+ Assume.assumeFalse("Namespace enabled account does not support this test",
+ getIsNamespaceEnabled(abfs));
+ try (NativeAzureFileSystem wasb = getWasbFileSystem()) {
+
+ Path testFile = path("/testReadFile");
+ Path testPath1 = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
+ Path testPath2 = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
+ Path testPath3 = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
+
+ // Write
+ abfs.mkdirs(testFile);
+ try (FSDataOutputStream abfsOutputStream = abfs.create(testPath1, true)) {
+ abfsOutputStream.write(TEST_CONTEXT.getBytes());
+ abfsOutputStream.flush();
+ abfsOutputStream.hsync();
+ }
+ abfs.create(testPath3, true);
+
+ // Check file status
+ ContractTestUtils.assertIsFile(abfs, testPath1);
+
+ try (BufferedReader br = new BufferedReader(
+ new InputStreamReader(abfs.open(testPath1)))) {
+ String line = br.readLine();
+ assertEquals("Wrong text from " + abfs,
+ TEST_CONTEXT, line);
+ }
+ // --- RENAME DIR ---
+ boolean renamed = wasb.rename(testPath1, testPath2);
+ Assertions.assertThat(renamed)
+ .as("Rename failed")
+ .isTrue();
+ // --- LIST FILES IN DIRECTORY ---
+ int listResult = listAllFilesAndDirs(abfs, testFile);
+ Assertions.assertThat(listResult)
+ .as("Expected only 4 entries under path: %s", testFile)
+ .isEqualTo(4);
+ }
}
- // --- RENAME DIR ---
- boolean renamed = wasb.rename(testPath1, testPath2);
- Assertions.assertThat(renamed)
- .as("Rename failed")
- .isTrue();
- // --- LIST FILES IN DIRECTORY ---
- int listResult = listAllFilesAndDirs(abfs, testFile);
- Assertions.assertThat(listResult)
- .as("Expected only 4 entries under path: %s", testFile)
- .isEqualTo(4);
}
/**
@@ -1867,44 +1948,46 @@ public void testScenario35() throws Exception {
Configuration conf = getRawConfiguration();
conf.setBoolean(FS_AZURE_ABFS_ENABLE_CHECKSUM_VALIDATION, true);
FileSystem fileSystem = FileSystem.newInstance(conf);
- AzureBlobFileSystem abfs = (AzureBlobFileSystem) fileSystem;
- Assume.assumeFalse("Namespace enabled account does not support this test",
- getIsNamespaceEnabled(abfs));
- NativeAzureFileSystem wasb = getWasbFileSystem();
-
- Path testFile = path("/testReadFile");
- Path testPath1 = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
- Path testPath2 = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
- Path testPath3 = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
-
- // Write
- wasb.mkdirs(testFile);
- try (FSDataOutputStream nativeFsStream = wasb.create(testPath1, true)) {
- nativeFsStream.write(TEST_CONTEXT.getBytes());
- nativeFsStream.flush();
- nativeFsStream.hsync();
- }
- wasb.create(testPath3, true);
-
- // Check file status
- ContractTestUtils.assertIsFile(abfs, testPath1);
-
- try (BufferedReader br = new BufferedReader(
- new InputStreamReader(abfs.open(testPath1)))) {
- String line = br.readLine();
- assertEquals("Wrong text from " + abfs,
- TEST_CONTEXT, line);
+ try (AzureBlobFileSystem abfs = (AzureBlobFileSystem) fileSystem) {
+ Assume.assumeFalse("Namespace enabled account does not support this test",
+ getIsNamespaceEnabled(abfs));
+ try (NativeAzureFileSystem wasb = getWasbFileSystem()) {
+
+ Path testFile = path("/testReadFile");
+ Path testPath1 = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
+ Path testPath2 = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
+ Path testPath3 = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
+
+ // Write
+ wasb.mkdirs(testFile);
+ try (FSDataOutputStream nativeFsStream = wasb.create(testPath1, true)) {
+ nativeFsStream.write(TEST_CONTEXT.getBytes());
+ nativeFsStream.flush();
+ nativeFsStream.hsync();
+ }
+ wasb.create(testPath3, true);
+
+ // Check file status
+ ContractTestUtils.assertIsFile(abfs, testPath1);
+
+ try (BufferedReader br = new BufferedReader(
+ new InputStreamReader(abfs.open(testPath1)))) {
+ String line = br.readLine();
+ assertEquals("Wrong text from " + abfs,
+ TEST_CONTEXT, line);
+ }
+ // --- RENAME DIR ---
+ boolean renamed = abfs.rename(testPath1, testPath2);
+ Assertions.assertThat(renamed)
+ .as("Rename failed")
+ .isTrue();
+ // --- LIST FILES IN DIRECTORY ---
+ int listResult = listAllFilesAndDirs(wasb, testFile);
+ Assertions.assertThat(listResult)
+ .as("Expected only 4 entries under path: %s", testFile)
+ .isEqualTo(4);
+ }
}
- // --- RENAME DIR ---
- boolean renamed = abfs.rename(testPath1, testPath2);
- Assertions.assertThat(renamed)
- .as("Rename failed")
- .isTrue();
- // --- LIST FILES IN DIRECTORY ---
- int listResult = listAllFilesAndDirs(wasb, testFile);
- Assertions.assertThat(listResult)
- .as("Expected only 4 entries under path: %s", testFile)
- .isEqualTo(4);
}
/**
@@ -1917,38 +2000,40 @@ public void testScenario36() throws Exception {
Configuration conf = getRawConfiguration();
conf.setBoolean(FS_AZURE_ABFS_ENABLE_CHECKSUM_VALIDATION, true);
FileSystem fileSystem = FileSystem.newInstance(conf);
- AzureBlobFileSystem abfs = (AzureBlobFileSystem) fileSystem;
- Assume.assumeFalse("Namespace enabled account does not support this test",
- getIsNamespaceEnabled(abfs));
- NativeAzureFileSystem wasb = getWasbFileSystem();
-
- Path testFile = path("/testReadFile");
- Path testPath1 = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
- Path testPath3 = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
-
- // Write
- wasb.mkdirs(testFile);
- try (FSDataOutputStream nativeFsStream = wasb.create(testPath1, true)) {
- nativeFsStream.write(TEST_CONTEXT.getBytes());
- nativeFsStream.flush();
- nativeFsStream.hsync();
- }
- wasb.create(testPath3, true);
-
- // Check file status
- ContractTestUtils.assertIsFile(abfs, testPath1);
-
- try (BufferedReader br = new BufferedReader(
- new InputStreamReader(abfs.open(testPath1)))) {
- String line = br.readLine();
- assertEquals("Wrong text from " + abfs,
- TEST_CONTEXT, line);
+ try (AzureBlobFileSystem abfs = (AzureBlobFileSystem) fileSystem) {
+ Assume.assumeFalse("Namespace enabled account does not support this test",
+ getIsNamespaceEnabled(abfs));
+ try (NativeAzureFileSystem wasb = getWasbFileSystem()) {
+
+ Path testFile = path("/testReadFile");
+ Path testPath1 = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
+ Path testPath3 = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
+
+ // Write
+ wasb.mkdirs(testFile);
+ try (FSDataOutputStream nativeFsStream = wasb.create(testPath1, true)) {
+ nativeFsStream.write(TEST_CONTEXT.getBytes());
+ nativeFsStream.flush();
+ nativeFsStream.hsync();
+ }
+ wasb.create(testPath3, true);
+
+ // Check file status
+ ContractTestUtils.assertIsFile(abfs, testPath1);
+
+ try (BufferedReader br = new BufferedReader(
+ new InputStreamReader(abfs.open(testPath1)))) {
+ String line = br.readLine();
+ assertEquals("Wrong text from " + abfs,
+ TEST_CONTEXT, line);
+ }
+ // --- RENAME DIR ---
+ boolean renamed = abfs.rename(testFile, testFile);
+ Assertions.assertThat(renamed)
+ .as("Rename operation should have failed but returned true")
+ .isFalse();
+ }
}
- // --- RENAME DIR ---
- boolean renamed = abfs.rename(testFile, testFile);
- Assertions.assertThat(renamed)
- .as("Rename operation should have failed but returned true")
- .isFalse();
}
/**
@@ -1960,39 +2045,41 @@ public void testScenario37() throws Exception {
Configuration conf = getRawConfiguration();
conf.setBoolean(FS_AZURE_ABFS_ENABLE_CHECKSUM_VALIDATION, true);
FileSystem fileSystem = FileSystem.newInstance(conf);
- AzureBlobFileSystem abfs = (AzureBlobFileSystem) fileSystem;
- Assume.assumeFalse("Namespace enabled account does not support this test",
- getIsNamespaceEnabled(abfs));
- NativeAzureFileSystem wasb = getWasbFileSystem();
-
- Path testFile = path("/testReadFile");
- Path testPath1 = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
- Path testPath2 = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
- Path testPath3 = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
-
- // Write
- abfs.mkdirs(testFile);
- try (FSDataOutputStream abfsOutputStream = abfs.create(testPath1, true)) {
- abfsOutputStream.write(TEST_CONTEXT.getBytes());
- abfsOutputStream.flush();
- abfsOutputStream.hsync();
- }
- abfs.create(testPath3, true);
-
- // Check file status
- ContractTestUtils.assertIsFile(abfs, testPath1);
-
- try (BufferedReader br = new BufferedReader(
- new InputStreamReader(abfs.open(testPath1)))) {
- String line = br.readLine();
- assertEquals("Wrong text from " + abfs,
- TEST_CONTEXT, line);
+ try (AzureBlobFileSystem abfs = (AzureBlobFileSystem) fileSystem) {
+ Assume.assumeFalse("Namespace enabled account does not support this test",
+ getIsNamespaceEnabled(abfs));
+ try (NativeAzureFileSystem wasb = getWasbFileSystem()) {
+
+ Path testFile = path("/testReadFile");
+ Path testPath1 = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
+ Path testPath2 = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
+ Path testPath3 = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
+
+ // Write
+ abfs.mkdirs(testFile);
+ try (FSDataOutputStream abfsOutputStream = abfs.create(testPath1, true)) {
+ abfsOutputStream.write(TEST_CONTEXT.getBytes());
+ abfsOutputStream.flush();
+ abfsOutputStream.hsync();
+ }
+ abfs.create(testPath3, true);
+
+ // Check file status
+ ContractTestUtils.assertIsFile(abfs, testPath1);
+
+ try (BufferedReader br = new BufferedReader(
+ new InputStreamReader(abfs.open(testPath1)))) {
+ String line = br.readLine();
+ assertEquals("Wrong text from " + abfs,
+ TEST_CONTEXT, line);
+ }
+ // --- RENAME NON EXISTENT FILE ---
+ boolean renamed = wasb.rename(testPath2, testPath3);
+ Assertions.assertThat(renamed)
+ .as("Rename operation should have failed but returned true")
+ .isFalse();
+ }
}
- // --- RENAME NON EXISTENT FILE ---
- boolean renamed = wasb.rename(testPath2, testPath3);
- Assertions.assertThat(renamed)
- .as("Rename operation should have failed but returned true")
- .isFalse();
}
/**
@@ -2004,44 +2091,46 @@ public void testScenario38() throws Exception {
Configuration conf = getRawConfiguration();
conf.setBoolean(FS_AZURE_ABFS_ENABLE_CHECKSUM_VALIDATION, true);
FileSystem fileSystem = FileSystem.newInstance(conf);
- AzureBlobFileSystem abfs = (AzureBlobFileSystem) fileSystem;
- Assume.assumeFalse("Namespace enabled account does not support this test",
- getIsNamespaceEnabled(abfs));
- NativeAzureFileSystem wasb = getWasbFileSystem();
-
- Path testFile = path("/testReadFile");
- Path path = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
-
- // Write
- try (FSDataOutputStream nativeFsStream = wasb.create(path, true)) {
- nativeFsStream.write(TEST_CONTEXT.getBytes());
- nativeFsStream.flush();
- nativeFsStream.hsync();
+ try (AzureBlobFileSystem abfs = (AzureBlobFileSystem) fileSystem) {
+ Assume.assumeFalse("Namespace enabled account does not support this test",
+ getIsNamespaceEnabled(abfs));
+ try (NativeAzureFileSystem wasb = getWasbFileSystem()) {
+
+ Path testFile = path("/testReadFile");
+ Path path = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
+
+ // Write
+ try (FSDataOutputStream nativeFsStream = wasb.create(path, true)) {
+ nativeFsStream.write(TEST_CONTEXT.getBytes());
+ nativeFsStream.flush();
+ nativeFsStream.hsync();
+ }
+ // --- VALIDATE FILE ---
+ FileStatus status = wasb.getFileStatus(path);
+ assertIsFile(path, status);
+
+ // --- SET XATTR #1 ---
+ wasb.setXAttr(path, ATTRIBUTE_NAME_1, ATTRIBUTE_VALUE_1);
+ byte[] readValue = wasb.getXAttr(path, ATTRIBUTE_NAME_1);
+ ITestAzureBlobFileSystemAttributes.assertAttributeEqual(abfs, readValue, ATTRIBUTE_VALUE_1, "one");
+
+ // --- SET XATTR #2 ---
+ wasb.setXAttr(path, ATTRIBUTE_NAME_2, ATTRIBUTE_VALUE_2);
+ readValue = wasb.getXAttr(path, ATTRIBUTE_NAME_2);
+ ITestAzureBlobFileSystemAttributes.assertAttributeEqual(abfs, readValue, ATTRIBUTE_VALUE_2, "two");
+
+ // --- VERIFY XATTR #1 AGAIN ---
+ readValue = wasb.getXAttr(path, ATTRIBUTE_NAME_1);
+ ITestAzureBlobFileSystemAttributes.assertAttributeEqual(abfs, readValue, ATTRIBUTE_VALUE_1, "one");
+
+ wasb.create(path, true);
+ FileStatus fileStatus = abfs.getFileStatus(path);
+ Assertions.assertThat(fileStatus.getLen())
+ .as("Expected file length to be 0 after overwrite")
+ .isEqualTo(0L);
+ wasb.delete(path, true);
+ }
}
- // --- VALIDATE FILE ---
- FileStatus status = wasb.getFileStatus(path);
- assertIsFile(path, status);
-
- // --- SET XATTR #1 ---
- wasb.setXAttr(path, ATTRIBUTE_NAME_1, ATTRIBUTE_VALUE_1);
- byte[] readValue = wasb.getXAttr(path, ATTRIBUTE_NAME_1);
- ITestAzureBlobFileSystemAttributes.assertAttributeEqual(abfs, readValue, ATTRIBUTE_VALUE_1, "one");
-
- // --- SET XATTR #2 ---
- wasb.setXAttr(path, ATTRIBUTE_NAME_2, ATTRIBUTE_VALUE_2);
- readValue = wasb.getXAttr(path, ATTRIBUTE_NAME_2);
- ITestAzureBlobFileSystemAttributes.assertAttributeEqual(abfs, readValue, ATTRIBUTE_VALUE_2, "two");
-
- // --- VERIFY XATTR #1 AGAIN ---
- readValue = wasb.getXAttr(path, ATTRIBUTE_NAME_1);
- ITestAzureBlobFileSystemAttributes.assertAttributeEqual(abfs, readValue, ATTRIBUTE_VALUE_1, "one");
-
- wasb.create(path, true);
- FileStatus fileStatus = abfs.getFileStatus(path);
- Assertions.assertThat(fileStatus.getLen())
- .as("Expected file length to be 0 after overwrite")
- .isEqualTo(0L);
- wasb.delete(path, true);
}
/**
@@ -2053,46 +2142,48 @@ public void testScenario39() throws Exception {
Configuration conf = getRawConfiguration();
conf.setBoolean(FS_AZURE_ABFS_ENABLE_CHECKSUM_VALIDATION, true);
FileSystem fileSystem = FileSystem.newInstance(conf);
- AzureBlobFileSystem abfs = (AzureBlobFileSystem) fileSystem;
+ try (AzureBlobFileSystem abfs = (AzureBlobFileSystem) fileSystem) {
Assume.assumeFalse("Namespace enabled account does not support this test",
getIsNamespaceEnabled(abfs));
- NativeAzureFileSystem wasb = getWasbFileSystem();
+ try (NativeAzureFileSystem wasb = getWasbFileSystem()) {
- String testRunId = UUID.randomUUID().toString();
- Path baseDir = path("/testScenario39_" + testRunId);
- Path testFile = new Path(baseDir, "testReadFile");
- Path testPath1 = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
- Path testPath2 = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
- Path testPath3 = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
-
- // Write
- wasb.mkdirs(testFile);
- try (FSDataOutputStream nativeFsStream = wasb.create(testPath1, true)) {
- nativeFsStream.write(TEST_CONTEXT.getBytes());
- nativeFsStream.flush();
- nativeFsStream.hsync();
- }
- wasb.create(testPath3, true);
+ String testRunId = UUID.randomUUID().toString();
+ Path baseDir = path("/testScenario39_" + testRunId);
+ Path testFile = new Path(baseDir, "testReadFile");
+ Path testPath1 = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
+ Path testPath2 = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
+ Path testPath3 = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
+
+ // Write
+ wasb.mkdirs(testFile);
+ try (FSDataOutputStream nativeFsStream = wasb.create(testPath1, true)) {
+ nativeFsStream.write(TEST_CONTEXT.getBytes());
+ nativeFsStream.flush();
+ nativeFsStream.hsync();
+ }
+ wasb.create(testPath3, true);
- // Check file status
- ContractTestUtils.assertIsFile(abfs, testPath1);
+ // Check file status
+ ContractTestUtils.assertIsFile(abfs, testPath1);
- try (BufferedReader br = new BufferedReader(
- new InputStreamReader(abfs.open(testPath1)))) {
- String line = br.readLine();
- assertEquals("Wrong text from " + abfs,
- TEST_CONTEXT, line);
+ try (BufferedReader br = new BufferedReader(
+ new InputStreamReader(abfs.open(testPath1)))) {
+ String line = br.readLine();
+ assertEquals("Wrong text from " + abfs,
+ TEST_CONTEXT, line);
+ }
+ // --- RENAME DIR ---
+ boolean renamed = wasb.rename(testPath1, testPath2);
+ Assertions.assertThat(renamed)
+ .as("Rename failed")
+ .isTrue();
+ // --- LIST FILES IN DIRECTORY ---
+ int listResult = listAllFilesAndDirs(wasb, testFile);
+ Assertions.assertThat(listResult)
+ .as("Expected only 4 entries under path: %s", testFile)
+ .isEqualTo(4);
+ }
}
- // --- RENAME DIR ---
- boolean renamed = wasb.rename(testPath1, testPath2);
- Assertions.assertThat(renamed)
- .as("Rename failed")
- .isTrue();
- // --- LIST FILES IN DIRECTORY ---
- int listResult = listAllFilesAndDirs(wasb, testFile);
- Assertions.assertThat(listResult)
- .as("Expected only 4 entries under path: %s", testFile)
- .isEqualTo(4);
}
/**
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/AbfsClientTestUtil.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/AbfsClientTestUtil.java
index bdeee28d9d54e..27679ed6dc7ac 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/AbfsClientTestUtil.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/AbfsClientTestUtil.java
@@ -166,12 +166,12 @@ public static void setMockAbfsRestOperationForFlushOperation(
requestHeaders.add(new AbfsHttpHeader(CONTENT_LENGTH, String.valueOf(buffer.length)));
requestHeaders.add(new AbfsHttpHeader(CONTENT_TYPE, APPLICATION_XML));
requestHeaders.add(new AbfsHttpHeader(IF_MATCH, eTag));
- if (spiedClient.isFullBlobChecksumValidationEnabled()) {
- requestHeaders.add(new AbfsHttpHeader(X_MS_BLOB_CONTENT_MD5, blobMd5));
- } else {
- requestHeaders.add(new AbfsHttpHeader(X_MS_BLOB_CONTENT_MD5,
- spiedClient.computeMD5Hash(buffer, 0, buffer.length)));
- }
+ requestHeaders.add(new AbfsHttpHeader(
+ X_MS_BLOB_CONTENT_MD5,
+ spiedClient.isFullBlobChecksumValidationEnabled()
+ ? blobMd5
+ : spiedClient.computeMD5Hash(buffer, 0, buffer.length)
+ ));
final AbfsUriQueryBuilder abfsUriQueryBuilder = spiedClient.createDefaultUriQueryBuilder();
abfsUriQueryBuilder.addQuery(QUERY_PARAM_COMP, BLOCKLIST);
abfsUriQueryBuilder.addQuery(QUERY_PARAM_CLOSE, String.valueOf(false));
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsOutputStream.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsOutputStream.java
index c2f90332007c1..9ff11bae800aa 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsOutputStream.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsOutputStream.java
@@ -27,6 +27,8 @@
import java.net.URL;
import java.security.MessageDigest;
import java.util.Arrays;
+import java.util.Random;
+import java.util.concurrent.atomic.AtomicInteger;
import org.assertj.core.api.Assertions;
import org.junit.Assume;
@@ -56,9 +58,12 @@
import static java.net.HttpURLConnection.HTTP_CONFLICT;
import static java.net.HttpURLConnection.HTTP_UNAVAILABLE;
import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.EXPECT_100_JDK_ERROR;
+import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_ABFS_ENABLE_CHECKSUM_VALIDATION;
import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_ACCOUNT_IS_EXPECT_HEADER_ENABLED;
+import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_ENABLE_FULL_BLOB_CHECKSUM_VALIDATION;
import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.EXPECT;
import static org.apache.hadoop.test.LambdaTestUtils.intercept;
+import static org.assertj.core.api.AssertionsForClassTypes.assertThatCode;
/**
* Test create operation.
@@ -482,7 +487,102 @@ public void testResetCalledOnExceptionInRemoteFlush() throws Exception {
}
// Verify that reset was called on the message digest
if (spiedClient.isChecksumValidationEnabled()) {
- Mockito.verify(mockMessageDigest, Mockito.times(1)).reset();
+ Assertions.assertThat(Mockito.mockingDetails(mockMessageDigest).getInvocations()
+ .stream()
+ .filter(i -> i.getMethod().getName().equals("reset"))
+ .count())
+ .as("Expected MessageDigest.reset() to be called exactly once when checksum validation is enabled")
+ .isEqualTo(1);
}
}
+
+ /**
+ * Tests that the message digest is reset when an exception occurs during remote flush.
+ * Simulates a failure in the flush operation and verifies reset is called on MessageDigest.
+ */
+ @Test
+ public void testNoChecksumComputedWhenConfigFalse() throws Exception {
+ Configuration conf = getRawConfiguration();
+ conf.setBoolean(FS_AZURE_ABFS_ENABLE_CHECKSUM_VALIDATION, false);
+ FileSystem fileSystem = FileSystem.newInstance(conf);
+ AzureBlobFileSystem fs = (AzureBlobFileSystem) fileSystem;
+ Assume.assumeTrue(!getIsNamespaceEnabled(fs));
+ AzureBlobFileSystemStore store = Mockito.spy(fs.getAbfsStore());
+ assumeBlobServiceType();
+ Assume.assumeFalse("Not valid for APPEND BLOB", isAppendBlobEnabled());
+
+ // Create spies for the client handler and blob client
+ AbfsClientHandler clientHandler = Mockito.spy(store.getClientHandler());
+ AbfsBlobClient blobClient = Mockito.spy(clientHandler.getBlobClient());
+
+ // Set up the spies to return the mocked objects
+ Mockito.doReturn(clientHandler).when(store).getClientHandler();
+ Mockito.doReturn(blobClient).when(clientHandler).getBlobClient();
+ Mockito.doReturn(blobClient).when(clientHandler).getIngressClient();
+ AbfsOutputStream abfsOutputStream = Mockito.spy(
+ (AbfsOutputStream) fs.create(new Path("/test/file")).getWrappedStream());
+ AzureIngressHandler ingressHandler = Mockito.spy(
+ abfsOutputStream.getIngressHandler());
+ Mockito.doReturn(ingressHandler).when(abfsOutputStream).getIngressHandler();
+ Mockito.doReturn(blobClient).when(ingressHandler).getClient();
+ FSDataOutputStream os = Mockito.spy(
+ new FSDataOutputStream(abfsOutputStream, null));
+ AbfsOutputStream out = (AbfsOutputStream) os.getWrappedStream();
+ byte[] bytes = new byte[1024 * 1024 * 4];
+ new Random().nextBytes(bytes);
+ // Write some bytes and attempt to flush, which should retry
+ out.write(bytes);
+ out.hsync();
+ Assertions.assertThat(Mockito.mockingDetails(blobClient).getInvocations()
+ .stream()
+ .filter(i -> i.getMethod().getName().equals("addCheckSumHeaderForWrite"))
+ .count())
+ .as("Expected addCheckSumHeaderForWrite() to be called exactly 0 times")
+ .isZero();
+ }
+
+ /**
+ * Tests that the message digest is reset when an exception occurs during remote flush.
+ * Simulates a failure in the flush operation and verifies reset is called on MessageDigest.
+ */
+ @Test
+ public void testChecksumComputedWhenConfigTrue() throws Exception {
+ Configuration conf = getRawConfiguration();
+ conf.setBoolean(FS_AZURE_ABFS_ENABLE_CHECKSUM_VALIDATION, true);
+ FileSystem fileSystem = FileSystem.newInstance(conf);
+ AzureBlobFileSystem fs = (AzureBlobFileSystem) fileSystem;
+ Assume.assumeTrue(!getIsNamespaceEnabled(fs));
+ AzureBlobFileSystemStore store = Mockito.spy(fs.getAbfsStore());
+ assumeBlobServiceType();
+ Assume.assumeFalse("Not valid for APPEND BLOB", isAppendBlobEnabled());
+
+ // Create spies for the client handler and blob client
+ AbfsClientHandler clientHandler = Mockito.spy(store.getClientHandler());
+ AbfsBlobClient blobClient = Mockito.spy(clientHandler.getBlobClient());
+
+ // Set up the spies to return the mocked objects
+ Mockito.doReturn(clientHandler).when(store).getClientHandler();
+ Mockito.doReturn(blobClient).when(clientHandler).getBlobClient();
+ Mockito.doReturn(blobClient).when(clientHandler).getIngressClient();
+ AbfsOutputStream abfsOutputStream = Mockito.spy(
+ (AbfsOutputStream) fs.create(new Path("/test/file")).getWrappedStream());
+ AzureIngressHandler ingressHandler = Mockito.spy(
+ abfsOutputStream.getIngressHandler());
+ Mockito.doReturn(ingressHandler).when(abfsOutputStream).getIngressHandler();
+ Mockito.doReturn(blobClient).when(ingressHandler).getClient();
+ FSDataOutputStream os = Mockito.spy(
+ new FSDataOutputStream(abfsOutputStream, null));
+ AbfsOutputStream out = (AbfsOutputStream) os.getWrappedStream();
+ byte[] bytes = new byte[1024 * 1024 * 4];
+ new Random().nextBytes(bytes);
+ // Write some bytes and attempt to flush, which should retry
+ out.write(bytes);
+ out.hsync();
+ Assertions.assertThat(Mockito.mockingDetails(blobClient).getInvocations()
+ .stream()
+ .filter(i -> i.getMethod().getName().equals("addCheckSumHeaderForWrite"))
+ .count())
+ .as("Expected addCheckSumHeaderForWrite() to be called exactly once")
+ .isEqualTo(1);
+ }
}
From 20f358218ab04b08474e0829fe8dc653d813132b Mon Sep 17 00:00:00 2001
From: Anmol Asrani
Date: Fri, 8 Aug 2025 05:19:53 -0700
Subject: [PATCH 23/25] null checks
---
.../apache/hadoop/fs/azurebfs/services/AbfsBlobClient.java | 2 +-
.../hadoop/fs/azurebfs/services/AbfsClientTestUtil.java | 2 +-
.../hadoop/fs/azurebfs/services/ITestAbfsOutputStream.java | 6 +-----
3 files changed, 3 insertions(+), 7 deletions(-)
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsBlobClient.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsBlobClient.java
index 6d372a570dfde..bb46a97835fec 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsBlobClient.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsBlobClient.java
@@ -1076,7 +1076,7 @@ public AbfsRestOperation flush(byte[] buffer,
if (leaseId != null) {
requestHeaders.add(new AbfsHttpHeader(X_MS_LEASE_ID, leaseId));
}
- String md5Value = isFullBlobChecksumValidationEnabled()
+ String md5Value = (isFullBlobChecksumValidationEnabled() && blobMd5 != null)
? blobMd5
: computeMD5Hash(buffer, 0, buffer.length);
requestHeaders.add(new AbfsHttpHeader(X_MS_BLOB_CONTENT_MD5, md5Value));
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/AbfsClientTestUtil.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/AbfsClientTestUtil.java
index 27679ed6dc7ac..b9dcefc35e239 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/AbfsClientTestUtil.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/AbfsClientTestUtil.java
@@ -168,7 +168,7 @@ public static void setMockAbfsRestOperationForFlushOperation(
requestHeaders.add(new AbfsHttpHeader(IF_MATCH, eTag));
requestHeaders.add(new AbfsHttpHeader(
X_MS_BLOB_CONTENT_MD5,
- spiedClient.isFullBlobChecksumValidationEnabled()
+ (spiedClient.isFullBlobChecksumValidationEnabled() && blobMd5 != null)
? blobMd5
: spiedClient.computeMD5Hash(buffer, 0, buffer.length)
));
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsOutputStream.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsOutputStream.java
index 9ff11bae800aa..f1341fa902299 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsOutputStream.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsOutputStream.java
@@ -28,7 +28,6 @@
import java.security.MessageDigest;
import java.util.Arrays;
import java.util.Random;
-import java.util.concurrent.atomic.AtomicInteger;
import org.assertj.core.api.Assertions;
import org.junit.Assume;
@@ -60,10 +59,8 @@
import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.EXPECT_100_JDK_ERROR;
import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_ABFS_ENABLE_CHECKSUM_VALIDATION;
import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_ACCOUNT_IS_EXPECT_HEADER_ENABLED;
-import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_ENABLE_FULL_BLOB_CHECKSUM_VALIDATION;
import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.EXPECT;
import static org.apache.hadoop.test.LambdaTestUtils.intercept;
-import static org.assertj.core.api.AssertionsForClassTypes.assertThatCode;
/**
* Test create operation.
@@ -443,7 +440,6 @@ public void testNoNetworkCallsForSecondFlush() throws Exception {
public void testResetCalledOnExceptionInRemoteFlush() throws Exception {
AzureBlobFileSystem fs = Mockito.spy(getFileSystem());
Assume.assumeTrue(!getIsNamespaceEnabled(fs));
- AzureBlobFileSystemStore store = Mockito.spy(fs.getAbfsStore());
assumeBlobServiceType();
Assume.assumeFalse("Not valid for APPEND BLOB", isAppendBlobEnabled());
@@ -486,7 +482,7 @@ public void testResetCalledOnExceptionInRemoteFlush() throws Exception {
//expected exception
}
// Verify that reset was called on the message digest
- if (spiedClient.isChecksumValidationEnabled()) {
+ if (spiedClient.isFullBlobChecksumValidationEnabled()) {
Assertions.assertThat(Mockito.mockingDetails(mockMessageDigest).getInvocations()
.stream()
.filter(i -> i.getMethod().getName().equals("reset"))
From c5f32be28205b5294fc2e94fe13256359a6afd52 Mon Sep 17 00:00:00 2001
From: Anmol Asrani
Date: Wed, 20 Aug 2025 05:54:30 -0700
Subject: [PATCH 24/25] PR comments
---
.../azurebfs/ITestWasbAbfsCompatibility.java | 2187 ++++++++---------
.../services/ITestAbfsOutputStream.java | 151 +-
2 files changed, 1094 insertions(+), 1244 deletions(-)
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestWasbAbfsCompatibility.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestWasbAbfsCompatibility.java
index f87b02f56eba2..ec63c3dcda72f 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestWasbAbfsCompatibility.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestWasbAbfsCompatibility.java
@@ -71,18 +71,15 @@ public class ITestWasbAbfsCompatibility extends AbstractAbfsIntegrationTest {
public ITestWasbAbfsCompatibility() throws Exception {
assumeThat(isIPAddress()).as("Emulator is not supported").isFalse();
+ assumeHnsDisabled();
+ assumeBlobServiceType();
}
@Test
public void testListFileStatus() throws Exception {
// crate file using abfs
- AzureBlobFileSystem fs = getFileSystem();
- // test only valid for non-namespace enabled account
- assumeThat(getIsNamespaceEnabled(fs))
- .as("Namespace enabled account does not support this test")
- .isFalse();
assumeThat(isAppendBlobEnabled()).as("Not valid for APPEND BLOB").isFalse();
-
+ AzureBlobFileSystem fs = getFileSystem();
NativeAzureFileSystem wasb = getWasbFileSystem();
Path testFiles = path("/testfiles");
@@ -113,6 +110,7 @@ public void testListFileStatus() throws Exception {
@Test
public void testReadFile() throws Exception {
+ Assume.assumeFalse("Not valid for APPEND BLOB", isAppendBlobEnabled());
boolean[] createFileWithAbfs = new boolean[]{false, true, false, true};
boolean[] readFileWithAbfs = new boolean[]{false, true, true, false};
@@ -120,11 +118,6 @@ public void testReadFile() throws Exception {
conf.setBoolean(FS_AZURE_ENABLE_FULL_BLOB_CHECKSUM_VALIDATION, true);
FileSystem fileSystem = FileSystem.newInstance(conf);
try (AzureBlobFileSystem abfs = (AzureBlobFileSystem) fileSystem) {
- // test only valid for non-namespace enabled account
- Assume.assumeFalse("Namespace enabled account does not support this test",
- getIsNamespaceEnabled(abfs));
- Assume.assumeFalse("Not valid for APPEND BLOB", isAppendBlobEnabled());
-
NativeAzureFileSystem wasb = getWasbFileSystem();
Path testFile = path("/testReadFile");
@@ -162,40 +155,36 @@ public void testReadFile() throws Exception {
*/
@Test
public void testwriteFile() throws Exception {
- try (AzureBlobFileSystem abfs = getFileSystem()) {
- Assume.assumeFalse("Namespace enabled account does not support this test",
- getIsNamespaceEnabled(abfs));
- assumeBlobServiceType();
- Assume.assumeFalse("Not valid for APPEND BLOB", isAppendBlobEnabled());
- NativeAzureFileSystem wasb = getWasbFileSystem();
+ Assume.assumeFalse("Not valid for APPEND BLOB", isAppendBlobEnabled());
+ AzureBlobFileSystem abfs = getFileSystem();
+ NativeAzureFileSystem wasb = getWasbFileSystem();
- Path testFile = path("/testReadFile");
- Path path = new Path(
- testFile + "/~12/!008/testfile_" + UUID.randomUUID());
- // Write
- try (FSDataOutputStream nativeFsStream = wasb.create(path, true)) {
- nativeFsStream.write(TEST_CONTEXT.getBytes());
- nativeFsStream.flush();
- nativeFsStream.hsync();
- }
+ Path testFile = path("/testReadFile");
+ Path path = new Path(
+ testFile + "/~12/!008/testfile_" + UUID.randomUUID());
+ // Write
+ try (FSDataOutputStream nativeFsStream = wasb.create(path, true)) {
+ nativeFsStream.write(TEST_CONTEXT.getBytes());
+ nativeFsStream.flush();
+ nativeFsStream.hsync();
+ }
- // Check file status
- ContractTestUtils.assertIsFile(wasb, path);
+ // Check file status
+ ContractTestUtils.assertIsFile(wasb, path);
- try (BufferedReader br = new BufferedReader(
- new InputStreamReader(abfs.open(path)))) {
- String line = br.readLine();
- assertEquals("Wrong text from " + abfs,
- TEST_CONTEXT, line);
- }
- try (FSDataOutputStream abfsOutputStream = abfs.append(path)) {
- abfsOutputStream.write(TEST_CONTEXT.getBytes());
- abfsOutputStream.flush();
- abfsOutputStream.hsync();
- }
- // Remove file
- assertDeleted(abfs, path, true);
+ try (BufferedReader br = new BufferedReader(
+ new InputStreamReader(abfs.open(path)))) {
+ String line = br.readLine();
+ assertEquals("Wrong text from " + abfs,
+ TEST_CONTEXT, line);
+ }
+ try (FSDataOutputStream abfsOutputStream = abfs.append(path)) {
+ abfsOutputStream.write(TEST_CONTEXT.getBytes());
+ abfsOutputStream.flush();
+ abfsOutputStream.hsync();
}
+ // Remove file
+ assertDeleted(abfs, path, true);
}
/**
@@ -205,40 +194,36 @@ public void testwriteFile() throws Exception {
@Test
public void testwriteFile1() throws Exception {
- try (AzureBlobFileSystem abfs = getFileSystem()) {
- Assume.assumeFalse("Namespace enabled account does not support this test",
- getIsNamespaceEnabled(abfs));
- assumeBlobServiceType();
- Assume.assumeFalse("Not valid for APPEND BLOB", isAppendBlobEnabled());
- NativeAzureFileSystem wasb = getWasbFileSystem();
+ Assume.assumeFalse("Not valid for APPEND BLOB", isAppendBlobEnabled());
+ AzureBlobFileSystem abfs = getFileSystem();
+ NativeAzureFileSystem wasb = getWasbFileSystem();
- Path testFile = path("/testReadFile");
- Path path = new Path(
- testFile + "/~12/!008/testfile_" + UUID.randomUUID());
- // Write
- try (FSDataOutputStream nativeFsStream = abfs.create(path, true)) {
- nativeFsStream.write(TEST_CONTEXT.getBytes());
- nativeFsStream.flush();
- nativeFsStream.hsync();
- }
+ Path testFile = path("/testReadFile");
+ Path path = new Path(
+ testFile + "/~12/!008/testfile_" + UUID.randomUUID());
+ // Write
+ try (FSDataOutputStream nativeFsStream = abfs.create(path, true)) {
+ nativeFsStream.write(TEST_CONTEXT.getBytes());
+ nativeFsStream.flush();
+ nativeFsStream.hsync();
+ }
- // Check file status
- ContractTestUtils.assertIsFile(abfs, path);
+ // Check file status
+ ContractTestUtils.assertIsFile(abfs, path);
- try (FSDataOutputStream nativeFsStream = wasb.append(path)) {
- nativeFsStream.write(TEST_CONTEXT.getBytes());
- nativeFsStream.flush();
- nativeFsStream.hsync();
- }
+ try (FSDataOutputStream nativeFsStream = wasb.append(path)) {
+ nativeFsStream.write(TEST_CONTEXT.getBytes());
+ nativeFsStream.flush();
+ nativeFsStream.hsync();
+ }
- try (FSDataOutputStream nativeFsStream = abfs.append(path)) {
- nativeFsStream.write(TEST_CONTEXT1.getBytes());
- nativeFsStream.flush();
- nativeFsStream.hsync();
- }
- // Remove file
- assertDeleted(abfs, path, true);
+ try (FSDataOutputStream nativeFsStream = abfs.append(path)) {
+ nativeFsStream.write(TEST_CONTEXT1.getBytes());
+ nativeFsStream.flush();
+ nativeFsStream.hsync();
}
+ // Remove file
+ assertDeleted(abfs, path, true);
}
/**
@@ -247,24 +232,20 @@ public void testwriteFile1() throws Exception {
*/
@Test
public void testazcopywasbcompatibility() throws Exception {
- try (AzureBlobFileSystem abfs = getFileSystem()) {
- Assume.assumeFalse("Namespace enabled account does not support this test",
- getIsNamespaceEnabled(abfs));
- Assume.assumeFalse("Not valid for APPEND BLOB", isAppendBlobEnabled());
-
- Path testFile = path("/testReadFile");
- Path path = new Path(
- testFile + "/~12/!008/testfile_" + UUID.randomUUID());
- createAzCopyFile(path);
-
- try (FSDataOutputStream nativeFsStream = abfs.append(path)) {
- nativeFsStream.write(TEST_CONTEXT1.getBytes());
- nativeFsStream.flush();
- nativeFsStream.hsync();
- }
- // Remove file
- assertDeleted(abfs, path, true);
+ Assume.assumeFalse("Not valid for APPEND BLOB", isAppendBlobEnabled());
+ AzureBlobFileSystem abfs = getFileSystem();
+ Path testFile = path("/testReadFile");
+ Path path = new Path(
+ testFile + "/~12/!008/testfile_" + UUID.randomUUID());
+ createAzCopyFile(path);
+
+ try (FSDataOutputStream nativeFsStream = abfs.append(path)) {
+ nativeFsStream.write(TEST_CONTEXT1.getBytes());
+ nativeFsStream.flush();
+ nativeFsStream.hsync();
}
+ // Remove file
+ assertDeleted(abfs, path, true);
}
@@ -273,29 +254,24 @@ public void testDir() throws Exception {
boolean[] createDirWithAbfs = new boolean[]{false, true, false, true};
boolean[] readDirWithAbfs = new boolean[]{false, true, true, false};
- try (AzureBlobFileSystem abfs = getFileSystem()) {
- // test only valid for non-namespace enabled account
- Assume.assumeFalse("Namespace enabled account does not support this test",
- getIsNamespaceEnabled(abfs));
-
- NativeAzureFileSystem wasb = getWasbFileSystem();
+ AzureBlobFileSystem abfs = getFileSystem();
+ NativeAzureFileSystem wasb = getWasbFileSystem();
- Path testDir = path("/testDir");
- for (int i = 0; i < 4; i++) {
- Path path = new Path(testDir + "/t" + i);
- //create
- final FileSystem createFs = createDirWithAbfs[i] ? abfs : wasb;
- assertTrue(createFs.mkdirs(path));
- //check
- assertPathExists(createFs, "Created dir not found with " + createFs,
- path);
- //read
- final FileSystem readFs = readDirWithAbfs[i] ? abfs : wasb;
- assertPathExists(readFs, "Created dir not found with " + readFs,
- path);
- assertIsDirectory(readFs, path);
- assertDeleted(readFs, path, true);
- }
+ Path testDir = path("/testDir");
+ for (int i = 0; i < 4; i++) {
+ Path path = new Path(testDir + "/t" + i);
+ //create
+ final FileSystem createFs = createDirWithAbfs[i] ? abfs : wasb;
+ assertTrue(createFs.mkdirs(path));
+ //check
+ assertPathExists(createFs, "Created dir not found with " + createFs,
+ path);
+ //read
+ final FileSystem readFs = readDirWithAbfs[i] ? abfs : wasb;
+ assertPathExists(readFs, "Created dir not found with " + readFs,
+ path);
+ assertIsDirectory(readFs, path);
+ assertDeleted(readFs, path, true);
}
}
@@ -313,34 +289,28 @@ public void testUrlConversion() {
@Test
public void testSetWorkingDirectory() throws Exception {
//create folders
- try (AzureBlobFileSystem abfs = getFileSystem()) {
- // test only valid for non-namespace enabled account
- Assume.assumeFalse("Namespace enabled account does not support this test",
- getIsNamespaceEnabled(abfs));
-
- try (NativeAzureFileSystem wasb = getWasbFileSystem()) {
-
- Path d1 = path("/d1");
- Path d1d4 = new Path(d1 + "/d2/d3/d4");
- assertMkdirs(abfs, d1d4);
-
- //set working directory to path1
- Path path1 = new Path(d1 + "/d2");
- wasb.setWorkingDirectory(path1);
- abfs.setWorkingDirectory(path1);
- assertEquals(path1, wasb.getWorkingDirectory());
- assertEquals(path1, abfs.getWorkingDirectory());
-
- //set working directory to path2
- Path path2 = new Path("d3/d4");
- wasb.setWorkingDirectory(path2);
- abfs.setWorkingDirectory(path2);
+ AzureBlobFileSystem abfs = getFileSystem();
+ NativeAzureFileSystem wasb = getWasbFileSystem();
- Path path3 = d1d4;
- assertEquals(path3, wasb.getWorkingDirectory());
- assertEquals(path3, abfs.getWorkingDirectory());
- }
- }
+ Path d1 = path("/d1");
+ Path d1d4 = new Path(d1 + "/d2/d3/d4");
+ assertMkdirs(abfs, d1d4);
+
+ //set working directory to path1
+ Path path1 = new Path(d1 + "/d2");
+ wasb.setWorkingDirectory(path1);
+ abfs.setWorkingDirectory(path1);
+ assertEquals(path1, wasb.getWorkingDirectory());
+ assertEquals(path1, abfs.getWorkingDirectory());
+
+ //set working directory to path2
+ Path path2 = new Path("d3/d4");
+ wasb.setWorkingDirectory(path2);
+ abfs.setWorkingDirectory(path2);
+
+ Path path3 = d1d4;
+ assertEquals(path3, wasb.getWorkingDirectory());
+ assertEquals(path3, abfs.getWorkingDirectory());
}
// Scenario wise testing
@@ -351,35 +321,31 @@ public void testSetWorkingDirectory() throws Exception {
*/
@Test
public void testScenario1() throws Exception {
- try (AzureBlobFileSystem abfs = getFileSystem()) {
- Assume.assumeFalse("Namespace enabled account does not support this test",
- getIsNamespaceEnabled(abfs));
- try (NativeAzureFileSystem wasb = getWasbFileSystem()) {
-
- Path testFile = path("/testReadFile");
- Path path = new Path(
- testFile + "/~12/!008/testfile_" + UUID.randomUUID());
-
- // Write
- try (FSDataOutputStream nativeFsStream = wasb.create(path, true)) {
- nativeFsStream.write(TEST_CONTEXT.getBytes());
- nativeFsStream.flush();
- nativeFsStream.hsync();
- }
- // Check file status
- ContractTestUtils.assertIsFile(wasb, path);
+ AzureBlobFileSystem abfs = getFileSystem();
+ NativeAzureFileSystem wasb = getWasbFileSystem();
- try (BufferedReader br = new BufferedReader(
- new InputStreamReader(abfs.open(path)))) {
- String line = br.readLine();
- assertEquals("Wrong text from " + abfs,
- TEST_CONTEXT, line);
- }
+ Path testFile = path("/testReadFile");
+ Path path = new Path(
+ testFile + "/~12/!008/testfile_" + UUID.randomUUID());
- // Remove file
- assertDeleted(abfs, path, true);
- }
+ // Write
+ try (FSDataOutputStream nativeFsStream = wasb.create(path, true)) {
+ nativeFsStream.write(TEST_CONTEXT.getBytes());
+ nativeFsStream.flush();
+ nativeFsStream.hsync();
}
+ // Check file status
+ ContractTestUtils.assertIsFile(wasb, path);
+
+ try (BufferedReader br = new BufferedReader(
+ new InputStreamReader(abfs.open(path)))) {
+ String line = br.readLine();
+ assertEquals("Wrong text from " + abfs,
+ TEST_CONTEXT, line);
+ }
+
+ // Remove file
+ assertDeleted(abfs, path, true);
}
/**
@@ -388,44 +354,39 @@ public void testScenario1() throws Exception {
*/
@Test
public void testScenario2() throws Exception {
- try (AzureBlobFileSystem abfs = getFileSystem()) {
- Assume.assumeFalse("Namespace enabled account does not support this test",
- getIsNamespaceEnabled(abfs));
- assumeBlobServiceType();
- Assume.assumeFalse("Not valid for APPEND BLOB", isAppendBlobEnabled());
- try (NativeAzureFileSystem wasb = getWasbFileSystem()) {
-
- Path testFile = path("/testReadFile");
- Path path = new Path(
- testFile + "/~12/!008/testfile_" + UUID.randomUUID());
-
- // Write
- try (FSDataOutputStream nativeFsStream = wasb.create(path, true)) {
- nativeFsStream.write(TEST_CONTEXT.getBytes());
- nativeFsStream.flush();
- nativeFsStream.hsync();
- }
- // Check file status
- ContractTestUtils.assertIsFile(wasb, path);
+ Assume.assumeFalse("Not valid for APPEND BLOB", isAppendBlobEnabled());
+ AzureBlobFileSystem abfs = getFileSystem();
+ NativeAzureFileSystem wasb = getWasbFileSystem();
- try (BufferedReader br = new BufferedReader(
- new InputStreamReader(abfs.open(path)))) {
- String line = br.readLine();
- assertEquals("Wrong text from " + abfs,
- TEST_CONTEXT, line);
- }
+ Path testFile = path("/testReadFile");
+ Path path = new Path(
+ testFile + "/~12/!008/testfile_" + UUID.randomUUID());
- // Write
- try (FSDataOutputStream abfsOutputStream = abfs.append(path)) {
- abfsOutputStream.write(TEST_CONTEXT1.getBytes());
- abfsOutputStream.flush();
- abfsOutputStream.hsync();
- }
+ // Write
+ try (FSDataOutputStream nativeFsStream = wasb.create(path, true)) {
+ nativeFsStream.write(TEST_CONTEXT.getBytes());
+ nativeFsStream.flush();
+ nativeFsStream.hsync();
+ }
+ // Check file status
+ ContractTestUtils.assertIsFile(wasb, path);
+
+ try (BufferedReader br = new BufferedReader(
+ new InputStreamReader(abfs.open(path)))) {
+ String line = br.readLine();
+ assertEquals("Wrong text from " + abfs,
+ TEST_CONTEXT, line);
+ }
- // Remove file
- assertDeleted(abfs, path, true);
- }
+ // Write
+ try (FSDataOutputStream abfsOutputStream = abfs.append(path)) {
+ abfsOutputStream.write(TEST_CONTEXT1.getBytes());
+ abfsOutputStream.flush();
+ abfsOutputStream.hsync();
}
+
+ // Remove file
+ assertDeleted(abfs, path, true);
}
/**
@@ -434,38 +395,35 @@ public void testScenario2() throws Exception {
*/
@Test
public void testScenario3() throws Exception {
+ Assume.assumeFalse("Not valid for APPEND BLOB", isAppendBlobEnabled());
Configuration conf = getRawConfiguration();
conf.setBoolean(FS_AZURE_ENABLE_FULL_BLOB_CHECKSUM_VALIDATION, true);
FileSystem fileSystem = FileSystem.newInstance(conf);
try (AzureBlobFileSystem abfs = (AzureBlobFileSystem) fileSystem) {
- Assume.assumeFalse("Namespace enabled account does not support this test",
- getIsNamespaceEnabled(abfs));
- Assume.assumeFalse("Not valid for APPEND BLOB", isAppendBlobEnabled());
- try (NativeAzureFileSystem wasb = getWasbFileSystem()) {
+ NativeAzureFileSystem wasb = getWasbFileSystem();
- Path testFile = path("/testReadFile");
- Path path = new Path(
- testFile + "/~12/!008/testfile_" + UUID.randomUUID());
+ Path testFile = path("/testReadFile");
+ Path path = new Path(
+ testFile + "/~12/!008/testfile_" + UUID.randomUUID());
- // Write
- try (FSDataOutputStream abfsOutputStream = abfs.create(path, true)) {
- abfsOutputStream.write(TEST_CONTEXT.getBytes());
- abfsOutputStream.flush();
- abfsOutputStream.hsync();
- }
+ // Write
+ try (FSDataOutputStream abfsOutputStream = abfs.create(path, true)) {
+ abfsOutputStream.write(TEST_CONTEXT.getBytes());
+ abfsOutputStream.flush();
+ abfsOutputStream.hsync();
+ }
- // Check file status
- ContractTestUtils.assertIsFile(abfs, path);
+ // Check file status
+ ContractTestUtils.assertIsFile(abfs, path);
- try (BufferedReader br = new BufferedReader(
- new InputStreamReader(wasb.open(path)))) {
- String line = br.readLine();
- assertEquals("Wrong text from " + wasb,
- TEST_CONTEXT, line);
- }
- // Remove file
- assertDeleted(abfs, path, true);
+ try (BufferedReader br = new BufferedReader(
+ new InputStreamReader(wasb.open(path)))) {
+ String line = br.readLine();
+ assertEquals("Wrong text from " + wasb,
+ TEST_CONTEXT, line);
}
+ // Remove file
+ assertDeleted(abfs, path, true);
}
}
@@ -475,37 +433,32 @@ public void testScenario3() throws Exception {
*/
@Test
public void testScenario4() throws Exception {
- try (AzureBlobFileSystem abfs = getFileSystem()) {
- Assume.assumeFalse("Namespace enabled account does not support this test",
- getIsNamespaceEnabled(abfs));
- assumeBlobServiceType();
- Assume.assumeFalse("Not valid for APPEND BLOB", isAppendBlobEnabled());
- try (NativeAzureFileSystem wasb = getWasbFileSystem()) {
-
- Path testFile = path("/testReadFile");
- Path path = new Path(
- testFile + "/~12/!008/testfile_" + UUID.randomUUID());
+ Assume.assumeFalse("Not valid for APPEND BLOB", isAppendBlobEnabled());
+ AzureBlobFileSystem abfs = getFileSystem();
+ NativeAzureFileSystem wasb = getWasbFileSystem();
- // Write
- wasb.create(path, true);
- try (FSDataOutputStream abfsOutputStream = abfs.append(path)) {
- abfsOutputStream.write(TEST_CONTEXT.getBytes());
- abfsOutputStream.flush();
- abfsOutputStream.hsync();
- }
+ Path testFile = path("/testReadFile");
+ Path path = new Path(
+ testFile + "/~12/!008/testfile_" + UUID.randomUUID());
- try (FSDataOutputStream nativeFsStream = wasb.append(path)) {
- nativeFsStream.write(TEST_CONTEXT1.getBytes());
- nativeFsStream.flush();
- nativeFsStream.hsync();
- }
+ // Write
+ wasb.create(path, true);
+ try (FSDataOutputStream abfsOutputStream = abfs.append(path)) {
+ abfsOutputStream.write(TEST_CONTEXT.getBytes());
+ abfsOutputStream.flush();
+ abfsOutputStream.hsync();
+ }
- // Check file status
- ContractTestUtils.assertIsFile(abfs, path);
- // Remove file
- assertDeleted(abfs, path, true);
- }
+ try (FSDataOutputStream nativeFsStream = wasb.append(path)) {
+ nativeFsStream.write(TEST_CONTEXT1.getBytes());
+ nativeFsStream.flush();
+ nativeFsStream.hsync();
}
+
+ // Check file status
+ ContractTestUtils.assertIsFile(abfs, path);
+ // Remove file
+ assertDeleted(abfs, path, true);
}
/**
@@ -514,40 +467,36 @@ public void testScenario4() throws Exception {
*/
@Test
public void testScenario5() throws Exception {
+ Assume.assumeFalse("Not valid for APPEND BLOB", isAppendBlobEnabled());
Configuration conf = getRawConfiguration();
conf.setBoolean(FS_AZURE_ABFS_ENABLE_CHECKSUM_VALIDATION, false);
FileSystem fileSystem = FileSystem.newInstance(conf);
try (AzureBlobFileSystem abfs = (AzureBlobFileSystem) fileSystem) {
- Assume.assumeFalse("Namespace enabled account does not support this test",
- getIsNamespaceEnabled(abfs));
- assumeBlobServiceType();
- Assume.assumeFalse("Not valid for APPEND BLOB", isAppendBlobEnabled());
- try (NativeAzureFileSystem wasb = getWasbFileSystem()) {
-
- Path testFile = path("/testReadFile");
- Path path = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
+ NativeAzureFileSystem wasb = getWasbFileSystem();
- // Write
- abfs.create(path, true);
- try (FSDataOutputStream nativeFsStream = wasb.append(path)) {
- nativeFsStream.write(TEST_CONTEXT.getBytes());
- nativeFsStream.flush();
- nativeFsStream.hsync();
- }
+ Path testFile = path("/testReadFile");
+ Path path = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
- // Check file status
- ContractTestUtils.assertIsFile(abfs, path);
+ // Write
+ abfs.create(path, true);
+ try (FSDataOutputStream nativeFsStream = wasb.append(path)) {
+ nativeFsStream.write(TEST_CONTEXT.getBytes());
+ nativeFsStream.flush();
+ nativeFsStream.hsync();
+ }
- try (BufferedReader br = new BufferedReader(
- new InputStreamReader(abfs.open(path)))) {
- String line = br.readLine();
- assertEquals("Wrong text from " + abfs,
- TEST_CONTEXT, line);
- }
+ // Check file status
+ ContractTestUtils.assertIsFile(abfs, path);
- // Remove file
- assertDeleted(abfs, path, true);
+ try (BufferedReader br = new BufferedReader(
+ new InputStreamReader(abfs.open(path)))) {
+ String line = br.readLine();
+ assertEquals("Wrong text from " + abfs,
+ TEST_CONTEXT, line);
}
+
+ // Remove file
+ assertDeleted(abfs, path, true);
}
}
@@ -557,40 +506,37 @@ public void testScenario5() throws Exception {
*/
@Test
public void testScenario6() throws Exception {
+ Assume.assumeFalse("Not valid for APPEND BLOB", isAppendBlobEnabled());
Configuration conf = getRawConfiguration();
conf.setBoolean(FS_AZURE_ABFS_ENABLE_CHECKSUM_VALIDATION, true);
FileSystem fileSystem = FileSystem.newInstance(conf);
try (AzureBlobFileSystem abfs = (AzureBlobFileSystem) fileSystem) {
- Assume.assumeFalse("Namespace enabled account does not support this test",
- getIsNamespaceEnabled(abfs));
- Assume.assumeFalse("Not valid for APPEND BLOB", isAppendBlobEnabled());
- assumeBlobServiceType();
- try (NativeAzureFileSystem wasb = getWasbFileSystem()) {
-
- Path testFile = path("/testReadFile");
- Path path = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
+ NativeAzureFileSystem wasb = getWasbFileSystem();
- // Write
- abfs.create(path, true);
- try (FSDataOutputStream nativeFsStream = wasb.append(path)) {
- nativeFsStream.write(TEST_CONTEXT.getBytes());
- nativeFsStream.flush();
- nativeFsStream.hsync();
- }
+ Path testFile = path("/testReadFile");
+ Path path = new Path(
+ testFile + "/~12/!008/testfile_" + UUID.randomUUID());
- // Check file status
- ContractTestUtils.assertIsFile(abfs, path);
+ // Write
+ abfs.create(path, true);
+ try (FSDataOutputStream nativeFsStream = wasb.append(path)) {
+ nativeFsStream.write(TEST_CONTEXT.getBytes());
+ nativeFsStream.flush();
+ nativeFsStream.hsync();
+ }
- try (BufferedReader br = new BufferedReader(
- new InputStreamReader(abfs.open(path)))) {
- String line = br.readLine();
- assertEquals("Wrong text from " + abfs,
- TEST_CONTEXT, line);
- }
+ // Check file status
+ ContractTestUtils.assertIsFile(abfs, path);
- // Remove file
- assertDeleted(abfs, path, true);
+ try (BufferedReader br = new BufferedReader(
+ new InputStreamReader(abfs.open(path)))) {
+ String line = br.readLine();
+ assertEquals("Wrong text from " + abfs,
+ TEST_CONTEXT, line);
}
+
+ // Remove file
+ assertDeleted(abfs, path, true);
}
}
@@ -600,42 +546,38 @@ public void testScenario6() throws Exception {
*/
@Test
public void testScenario7() throws Exception {
+ Assume.assumeFalse("Not valid for APPEND BLOB", isAppendBlobEnabled());
Configuration conf = getRawConfiguration();
conf.setBoolean(FS_AZURE_ABFS_ENABLE_CHECKSUM_VALIDATION, true);
FileSystem fileSystem = FileSystem.newInstance(conf);
try (AzureBlobFileSystem abfs = (AzureBlobFileSystem) fileSystem) {
- Assume.assumeFalse("Namespace enabled account does not support this test",
- getIsNamespaceEnabled(abfs));
- Assume.assumeFalse("Not valid for APPEND BLOB", isAppendBlobEnabled());
- try (NativeAzureFileSystem wasb = getWasbFileSystem()) {
+ NativeAzureFileSystem wasb = getWasbFileSystem();
+ Path testFile = path("/testReadFile");
+ Path path = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
- Path testFile = path("/testReadFile");
- Path path = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
+ try (FSDataOutputStream nativeFsStream = wasb.create(path, true)) {
+ nativeFsStream.write(TEST_CONTEXT.getBytes());
+ nativeFsStream.flush();
+ nativeFsStream.hsync();
+ }
- try (FSDataOutputStream nativeFsStream = wasb.create(path, true)) {
- nativeFsStream.write(TEST_CONTEXT.getBytes());
- nativeFsStream.flush();
- nativeFsStream.hsync();
- }
+ // Check file status
+ ContractTestUtils.assertIsFile(abfs, path);
- // Check file status
- ContractTestUtils.assertIsFile(abfs, path);
+ try (BufferedReader br = new BufferedReader(
+ new InputStreamReader(abfs.open(path)))) {
+ String line = br.readLine();
+ assertEquals("Wrong text from " + abfs,
+ TEST_CONTEXT, line);
+ }
+ abfs.create(path, true);
+ FileStatus fileStatus = abfs.getFileStatus(path);
+ Assertions.assertThat(fileStatus.getLen())
+ .as("Expected file length to be 0 after overwrite")
+ .isEqualTo(0L);
- try (BufferedReader br = new BufferedReader(
- new InputStreamReader(abfs.open(path)))) {
- String line = br.readLine();
- assertEquals("Wrong text from " + abfs,
- TEST_CONTEXT, line);
- }
- abfs.create(path, true);
- FileStatus fileStatus = abfs.getFileStatus(path);
- Assertions.assertThat(fileStatus.getLen())
- .as("Expected file length to be 0 after overwrite")
- .isEqualTo(0L);
-
- // Remove file
- assertDeleted(abfs, path, true);
- }
+ // Remove file
+ assertDeleted(abfs, path, true);
}
}
@@ -645,50 +587,47 @@ public void testScenario7() throws Exception {
*/
@Test
public void testScenario8() throws Exception {
+ Assume.assumeFalse("Not valid for APPEND BLOB", isAppendBlobEnabled());
Configuration conf = getRawConfiguration();
conf.setBoolean(FS_AZURE_ABFS_ENABLE_CHECKSUM_VALIDATION, true);
FileSystem fileSystem = FileSystem.newInstance(conf);
try (AzureBlobFileSystem abfs = (AzureBlobFileSystem) fileSystem) {
- Assume.assumeFalse("Namespace enabled account does not support this test",
- getIsNamespaceEnabled(abfs));
- Assume.assumeFalse("Not valid for APPEND BLOB", isAppendBlobEnabled());
- try (NativeAzureFileSystem wasb = getWasbFileSystem()) {
-
- Path testFile = path("/testReadFile");
- Path path = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
+ NativeAzureFileSystem wasb = getWasbFileSystem();
- try (FSDataOutputStream nativeFsStream = wasb.create(path, true)) {
- nativeFsStream.write(TEST_CONTEXT.getBytes());
- nativeFsStream.flush();
- nativeFsStream.hsync();
- }
+ Path testFile = path("/testReadFile");
+ Path path = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
- // Check file status
- ContractTestUtils.assertIsFile(abfs, path);
+ try (FSDataOutputStream nativeFsStream = wasb.create(path, true)) {
+ nativeFsStream.write(TEST_CONTEXT.getBytes());
+ nativeFsStream.flush();
+ nativeFsStream.hsync();
+ }
- try (BufferedReader br = new BufferedReader(
- new InputStreamReader(abfs.open(path)))) {
- String line = br.readLine();
- assertEquals("Wrong text from " + abfs,
- TEST_CONTEXT, line);
- }
- try {
- abfs.create(path, false);
- } catch (IOException e) {
- AbfsRestOperationException restEx = (AbfsRestOperationException) e.getCause();
- if (restEx != null) {
- Assertions.assertThat(restEx.getStatusCode())
- .as("Expected HTTP status code 409 (Conflict) when file already exists")
- .isEqualTo(HTTP_CONFLICT);
- }
- Assertions.assertThat(e.getMessage())
- .as("Expected error message to contain 'AlreadyExists'")
- .contains("AlreadyExists");
- }
+ // Check file status
+ ContractTestUtils.assertIsFile(abfs, path);
- // Remove file
- assertDeleted(abfs, path, true);
+ try (BufferedReader br = new BufferedReader(
+ new InputStreamReader(abfs.open(path)))) {
+ String line = br.readLine();
+ assertEquals("Wrong text from " + abfs,
+ TEST_CONTEXT, line);
+ }
+ try {
+ abfs.create(path, false);
+ } catch (IOException e) {
+ AbfsRestOperationException restEx = (AbfsRestOperationException) e.getCause();
+ if (restEx != null) {
+ Assertions.assertThat(restEx.getStatusCode())
+ .as("Expected HTTP status code 409 (Conflict) when file already exists")
+ .isEqualTo(HTTP_CONFLICT);
+ }
+ Assertions.assertThat(e.getMessage())
+ .as("Expected error message to contain 'AlreadyExists'")
+ .contains("AlreadyExists");
}
+
+ // Remove file
+ assertDeleted(abfs, path, true);
}
}
@@ -698,50 +637,46 @@ public void testScenario8() throws Exception {
*/
@Test
public void testScenario9() throws Exception {
+ Assume.assumeFalse("Not valid for APPEND BLOB", isAppendBlobEnabled());
Configuration conf = getRawConfiguration();
conf.setBoolean(FS_AZURE_ABFS_ENABLE_CHECKSUM_VALIDATION, true);
FileSystem fileSystem = FileSystem.newInstance(conf);
try (AzureBlobFileSystem abfs = (AzureBlobFileSystem) fileSystem) {
- Assume.assumeFalse("Namespace enabled account does not support this test",
- getIsNamespaceEnabled(abfs));
- assumeBlobServiceType();
- Assume.assumeFalse("Not valid for APPEND BLOB", isAppendBlobEnabled());
- try (NativeAzureFileSystem wasb = getWasbFileSystem()) {
-
- Path testFile = path("/testReadFile");
- Path path = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
-
- try (FSDataOutputStream abfsOutputStream = abfs.create(path, true)) {
- abfsOutputStream.write(TEST_CONTEXT.getBytes());
- abfsOutputStream.flush();
- abfsOutputStream.hsync();
- }
+ NativeAzureFileSystem wasb = getWasbFileSystem();
- // Check file status
- ContractTestUtils.assertIsFile(abfs, path);
+ Path testFile = path("/testReadFile");
+ Path path = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
- try (BufferedReader br = new BufferedReader(
- new InputStreamReader(abfs.open(path)))) {
- String line = br.readLine();
- assertEquals("Wrong text from " + abfs,
- TEST_CONTEXT, line);
- }
- wasb.create(path, true);
- FileStatus fileStatus = abfs.getFileStatus(path);
- Assertions.assertThat(fileStatus.getLen())
- .as("Expected file length to be 0 after overwrite")
- .isEqualTo(0L);
+ try (FSDataOutputStream abfsOutputStream = abfs.create(path, true)) {
+ abfsOutputStream.write(TEST_CONTEXT.getBytes());
+ abfsOutputStream.flush();
+ abfsOutputStream.hsync();
+ }
- // Remove file
- assertDeleted(abfs, path, true);
+ // Check file status
+ ContractTestUtils.assertIsFile(abfs, path);
+
+ try (BufferedReader br = new BufferedReader(
+ new InputStreamReader(abfs.open(path)))) {
+ String line = br.readLine();
+ assertEquals("Wrong text from " + abfs,
+ TEST_CONTEXT, line);
}
+ wasb.create(path, true);
+ FileStatus fileStatus = abfs.getFileStatus(path);
+ Assertions.assertThat(fileStatus.getLen())
+ .as("Expected file length to be 0 after overwrite")
+ .isEqualTo(0L);
+
+ // Remove file
+ assertDeleted(abfs, path, true);
}
}
/**
* Scenario 10: Create a file using ABFS and then attempt to create the same file using WASB with overwrite=false.
* Expected Outcome: WASB should fail to create the file as it already exists. The exception should indicate
- * an "AlreadyExists" error with HTTP status code 409 (Conflict).
+ * an "AlreadyExists" error with HTTP status code 409 (Conflict).
*/
@Test
public void testScenario10() throws Exception {
@@ -749,45 +684,41 @@ public void testScenario10() throws Exception {
conf.setBoolean(FS_AZURE_ABFS_ENABLE_CHECKSUM_VALIDATION, true);
FileSystem fileSystem = FileSystem.newInstance(conf);
try (AzureBlobFileSystem abfs = (AzureBlobFileSystem) fileSystem) {
- Assume.assumeFalse("Namespace enabled account does not support this test",
- getIsNamespaceEnabled(abfs));
- try (NativeAzureFileSystem wasb = getWasbFileSystem()) {
-
- Path testFile = path("/testReadFile");
- Path path = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
+ NativeAzureFileSystem wasb = getWasbFileSystem();
+ Path testFile = path("/testReadFile");
+ Path path = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
- try (FSDataOutputStream abfsOutputStream = abfs.create(path, true)) {
- abfsOutputStream.write(TEST_CONTEXT.getBytes());
- abfsOutputStream.flush();
- abfsOutputStream.hsync();
- }
+ try (FSDataOutputStream abfsOutputStream = abfs.create(path, true)) {
+ abfsOutputStream.write(TEST_CONTEXT.getBytes());
+ abfsOutputStream.flush();
+ abfsOutputStream.hsync();
+ }
- // Check file status
- ContractTestUtils.assertIsFile(abfs, path);
+ // Check file status
+ ContractTestUtils.assertIsFile(abfs, path);
- try (BufferedReader br = new BufferedReader(
- new InputStreamReader(abfs.open(path)))) {
- String line = br.readLine();
- assertEquals("Wrong text from " + abfs,
- TEST_CONTEXT, line);
- }
- try {
- wasb.create(path, false);
- } catch (IOException e) {
- AbfsRestOperationException restEx
- = (AbfsRestOperationException) e.getCause();
- if (restEx != null) {
- Assertions.assertThat(restEx.getStatusCode())
- .as("Expected HTTP status code 409 (Conflict) when file already exists")
- .isEqualTo(HTTP_CONFLICT);
- }
- Assertions.assertThat(e.getMessage())
- .as("Expected error message to contain 'exists'")
- .contains("exists");
- }
- // Remove file
- assertDeleted(abfs, path, true);
+ try (BufferedReader br = new BufferedReader(
+ new InputStreamReader(abfs.open(path)))) {
+ String line = br.readLine();
+ assertEquals("Wrong text from " + abfs,
+ TEST_CONTEXT, line);
+ }
+ try {
+ wasb.create(path, false);
+ } catch (IOException e) {
+ AbfsRestOperationException restEx
+ = (AbfsRestOperationException) e.getCause();
+ if (restEx != null) {
+ Assertions.assertThat(restEx.getStatusCode())
+ .as("Expected HTTP status code 409 (Conflict) when file already exists")
+ .isEqualTo(HTTP_CONFLICT);
+ }
+ Assertions.assertThat(e.getMessage())
+ .as("Expected error message to contain 'exists'")
+ .contains("exists");
}
+ // Remove file
+ assertDeleted(abfs, path, true);
}
}
@@ -798,38 +729,34 @@ public void testScenario10() throws Exception {
*/
@Test
public void testScenario11() throws Exception {
+ Assume.assumeFalse("Not valid for APPEND BLOB", isAppendBlobEnabled());
Configuration conf = getRawConfiguration();
conf.setBoolean(FS_AZURE_ABFS_ENABLE_CHECKSUM_VALIDATION, true);
FileSystem fileSystem = FileSystem.newInstance(conf);
try (AzureBlobFileSystem abfs = (AzureBlobFileSystem) fileSystem) {
- Assume.assumeFalse("Namespace enabled account does not support this test",
- getIsNamespaceEnabled(abfs));
- assumeBlobServiceType();
- Assume.assumeFalse("Not valid for APPEND BLOB", isAppendBlobEnabled());
- try (NativeAzureFileSystem wasb = getWasbFileSystem()) {
+ NativeAzureFileSystem wasb = getWasbFileSystem();
- Path testFile = path("/testReadFile");
- Path path = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
+ Path testFile = path("/testReadFile");
+ Path path = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
- // Write
- abfs.create(path, true);
- try (FSDataOutputStream nativeFsStream = wasb.append(path)) {
- nativeFsStream.write(TEST_CONTEXT.getBytes());
- nativeFsStream.flush();
- nativeFsStream.hsync();
- }
+ // Write
+ abfs.create(path, true);
+ try (FSDataOutputStream nativeFsStream = wasb.append(path)) {
+ nativeFsStream.write(TEST_CONTEXT.getBytes());
+ nativeFsStream.flush();
+ nativeFsStream.hsync();
+ }
- // Check file status
- ContractTestUtils.assertIsFile(abfs, path);
+ // Check file status
+ ContractTestUtils.assertIsFile(abfs, path);
- try (BufferedReader br = new BufferedReader(
- new InputStreamReader(abfs.open(path)))) {
- String line = br.readLine();
- assertEquals("Wrong text from " + abfs,
- TEST_CONTEXT, line);
- }
- abfs.delete(path, true);
+ try (BufferedReader br = new BufferedReader(
+ new InputStreamReader(abfs.open(path)))) {
+ String line = br.readLine();
+ assertEquals("Wrong text from " + abfs,
+ TEST_CONTEXT, line);
}
+ abfs.delete(path, true);
}
}
@@ -844,31 +771,28 @@ public void testScenario12() throws Exception {
conf.setBoolean(FS_AZURE_ABFS_ENABLE_CHECKSUM_VALIDATION, true);
FileSystem fileSystem = FileSystem.newInstance(conf);
try (AzureBlobFileSystem abfs = (AzureBlobFileSystem) fileSystem) {
- Assume.assumeFalse("Namespace enabled account does not support this test",
- getIsNamespaceEnabled(abfs));
- try (NativeAzureFileSystem wasb = getWasbFileSystem()) {
+ NativeAzureFileSystem wasb = getWasbFileSystem();
- Path testFile = path("/testReadFile");
- Path path = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
+ Path testFile = path("/testReadFile");
+ Path path = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
- // Write
- try (FSDataOutputStream abfsOutputStream = abfs.create(path, true)) {
- abfsOutputStream.write(TEST_CONTEXT.getBytes());
- abfsOutputStream.flush();
- abfsOutputStream.hsync();
- }
+ // Write
+ try (FSDataOutputStream abfsOutputStream = abfs.create(path, true)) {
+ abfsOutputStream.write(TEST_CONTEXT.getBytes());
+ abfsOutputStream.flush();
+ abfsOutputStream.hsync();
+ }
- // Check file status
- ContractTestUtils.assertIsFile(abfs, path);
+ // Check file status
+ ContractTestUtils.assertIsFile(abfs, path);
- try (BufferedReader br = new BufferedReader(
- new InputStreamReader(abfs.open(path)))) {
- String line = br.readLine();
- assertEquals("Wrong text from " + abfs,
- TEST_CONTEXT, line);
- }
- wasb.delete(path, true);
+ try (BufferedReader br = new BufferedReader(
+ new InputStreamReader(abfs.open(path)))) {
+ String line = br.readLine();
+ assertEquals("Wrong text from " + abfs,
+ TEST_CONTEXT, line);
}
+ wasb.delete(path, true);
}
}
@@ -878,38 +802,34 @@ public void testScenario12() throws Exception {
*/
@Test
public void testScenario13() throws Exception {
+ Assume.assumeFalse("Not valid for APPEND BLOB", isAppendBlobEnabled());
Configuration conf = getRawConfiguration();
conf.setBoolean(FS_AZURE_ABFS_ENABLE_CHECKSUM_VALIDATION, true);
FileSystem fileSystem = FileSystem.newInstance(conf);
try (AzureBlobFileSystem abfs = (AzureBlobFileSystem) fileSystem) {
- Assume.assumeFalse("Namespace enabled account does not support this test",
- getIsNamespaceEnabled(abfs));
- assumeBlobServiceType();
- Assume.assumeFalse("Not valid for APPEND BLOB", isAppendBlobEnabled());
- try (NativeAzureFileSystem wasb = getWasbFileSystem()) {
+ NativeAzureFileSystem wasb = getWasbFileSystem();
- Path testFile = path("/testReadFile");
- Path path = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
+ Path testFile = path("/testReadFile");
+ Path path = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
- // Write
- abfs.create(path, true);
- try (FSDataOutputStream nativeFsStream = wasb.append(path)) {
- nativeFsStream.write(TEST_CONTEXT.getBytes());
- nativeFsStream.flush();
- nativeFsStream.hsync();
- }
+ // Write
+ abfs.create(path, true);
+ try (FSDataOutputStream nativeFsStream = wasb.append(path)) {
+ nativeFsStream.write(TEST_CONTEXT.getBytes());
+ nativeFsStream.flush();
+ nativeFsStream.hsync();
+ }
- // Check file status
- ContractTestUtils.assertIsFile(abfs, path);
+ // Check file status
+ ContractTestUtils.assertIsFile(abfs, path);
- try (BufferedReader br = new BufferedReader(
- new InputStreamReader(wasb.open(path)))) {
- String line = br.readLine();
- assertEquals("Wrong text from " + wasb,
- TEST_CONTEXT, line);
- }
- abfs.delete(path, true);
+ try (BufferedReader br = new BufferedReader(
+ new InputStreamReader(wasb.open(path)))) {
+ String line = br.readLine();
+ assertEquals("Wrong text from " + wasb,
+ TEST_CONTEXT, line);
}
+ abfs.delete(path, true);
}
}
@@ -919,38 +839,34 @@ public void testScenario13() throws Exception {
*/
@Test
public void testScenario14() throws Exception {
+ Assume.assumeFalse("Not valid for APPEND BLOB", isAppendBlobEnabled());
Configuration conf = getRawConfiguration();
conf.setBoolean(FS_AZURE_ABFS_ENABLE_CHECKSUM_VALIDATION, true);
FileSystem fileSystem = FileSystem.newInstance(conf);
try (AzureBlobFileSystem abfs = (AzureBlobFileSystem) fileSystem) {
- Assume.assumeFalse("Namespace enabled account does not support this test",
- getIsNamespaceEnabled(abfs));
- assumeBlobServiceType();
- Assume.assumeFalse("Not valid for APPEND BLOB", isAppendBlobEnabled());
- try (NativeAzureFileSystem wasb = getWasbFileSystem()) {
+ NativeAzureFileSystem wasb = getWasbFileSystem();
- Path testFile = path("/testReadFile");
- Path path = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
+ Path testFile = path("/testReadFile");
+ Path path = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
- // Write
- abfs.create(path, true);
- try (FSDataOutputStream nativeFsStream = wasb.append(path)) {
- nativeFsStream.write(TEST_CONTEXT.getBytes());
- nativeFsStream.flush();
- nativeFsStream.hsync();
- }
+ // Write
+ abfs.create(path, true);
+ try (FSDataOutputStream nativeFsStream = wasb.append(path)) {
+ nativeFsStream.write(TEST_CONTEXT.getBytes());
+ nativeFsStream.flush();
+ nativeFsStream.hsync();
+ }
- // Check file status
- ContractTestUtils.assertIsFile(abfs, path);
+ // Check file status
+ ContractTestUtils.assertIsFile(abfs, path);
- try (BufferedReader br = new BufferedReader(
- new InputStreamReader(wasb.open(path)))) {
- String line = br.readLine();
- assertEquals("Wrong text from " + wasb,
- TEST_CONTEXT, line);
- }
- wasb.delete(path, true);
+ try (BufferedReader br = new BufferedReader(
+ new InputStreamReader(wasb.open(path)))) {
+ String line = br.readLine();
+ assertEquals("Wrong text from " + wasb,
+ TEST_CONTEXT, line);
}
+ wasb.delete(path, true);
}
}
@@ -964,31 +880,28 @@ public void testScenario15() throws Exception {
conf.setBoolean(FS_AZURE_ABFS_ENABLE_CHECKSUM_VALIDATION, true);
FileSystem fileSystem = FileSystem.newInstance(conf);
try (AzureBlobFileSystem abfs = (AzureBlobFileSystem) fileSystem) {
- Assume.assumeFalse("Namespace enabled account does not support this test",
- getIsNamespaceEnabled(abfs));
- try (NativeAzureFileSystem wasb = getWasbFileSystem()) {
+ NativeAzureFileSystem wasb = getWasbFileSystem();
- Path testFile = path("/testReadFile");
- Path path = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
+ Path testFile = path("/testReadFile");
+ Path path = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
- // Write
- try (FSDataOutputStream nativeFsStream = wasb.create(path, true)) {
- nativeFsStream.write(TEST_CONTEXT.getBytes());
- nativeFsStream.flush();
- nativeFsStream.hsync();
- }
+ // Write
+ try (FSDataOutputStream nativeFsStream = wasb.create(path, true)) {
+ nativeFsStream.write(TEST_CONTEXT.getBytes());
+ nativeFsStream.flush();
+ nativeFsStream.hsync();
+ }
- // Check file status
- ContractTestUtils.assertIsFile(abfs, path);
+ // Check file status
+ ContractTestUtils.assertIsFile(abfs, path);
- try (BufferedReader br = new BufferedReader(
- new InputStreamReader(wasb.open(path)))) {
- String line = br.readLine();
- assertEquals("Wrong text from " + wasb,
- TEST_CONTEXT, line);
- }
- abfs.delete(path, true);
+ try (BufferedReader br = new BufferedReader(
+ new InputStreamReader(wasb.open(path)))) {
+ String line = br.readLine();
+ assertEquals("Wrong text from " + wasb,
+ TEST_CONTEXT, line);
}
+ abfs.delete(path, true);
}
}
@@ -998,38 +911,34 @@ public void testScenario15() throws Exception {
*/
@Test
public void testScenario16() throws Exception {
+ Assume.assumeFalse("Not valid for APPEND BLOB", isAppendBlobEnabled());
Configuration conf = getRawConfiguration();
conf.setBoolean(FS_AZURE_ABFS_ENABLE_CHECKSUM_VALIDATION, true);
FileSystem fileSystem = FileSystem.newInstance(conf);
try (AzureBlobFileSystem abfs = (AzureBlobFileSystem) fileSystem) {
- Assume.assumeFalse("Namespace enabled account does not support this test",
- getIsNamespaceEnabled(abfs));
- assumeBlobServiceType();
- Assume.assumeFalse("Not valid for APPEND BLOB", isAppendBlobEnabled());
- try (NativeAzureFileSystem wasb = getWasbFileSystem()) {
+ NativeAzureFileSystem wasb = getWasbFileSystem();
- Path testFile = path("/testReadFile");
- Path path = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
+ Path testFile = path("/testReadFile");
+ Path path = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
- // Write
- wasb.create(path, true);
- try (FSDataOutputStream abfsOutputStream = abfs.append(path)) {
- abfsOutputStream.write(TEST_CONTEXT.getBytes());
- abfsOutputStream.flush();
- abfsOutputStream.hsync();
- }
+ // Write
+ wasb.create(path, true);
+ try (FSDataOutputStream abfsOutputStream = abfs.append(path)) {
+ abfsOutputStream.write(TEST_CONTEXT.getBytes());
+ abfsOutputStream.flush();
+ abfsOutputStream.hsync();
+ }
- // Check file status
- ContractTestUtils.assertIsFile(abfs, path);
+ // Check file status
+ ContractTestUtils.assertIsFile(abfs, path);
- try (BufferedReader br = new BufferedReader(
- new InputStreamReader(abfs.open(path)))) {
- String line = br.readLine();
- assertEquals("Wrong text from " + abfs,
- TEST_CONTEXT, line);
- }
- wasb.delete(path, true);
+ try (BufferedReader br = new BufferedReader(
+ new InputStreamReader(abfs.open(path)))) {
+ String line = br.readLine();
+ assertEquals("Wrong text from " + abfs,
+ TEST_CONTEXT, line);
}
+ wasb.delete(path, true);
}
}
@@ -1043,8 +952,6 @@ public void testScenario17() throws Exception {
conf.setBoolean(FS_AZURE_ABFS_ENABLE_CHECKSUM_VALIDATION, true);
FileSystem fileSystem = FileSystem.newInstance(conf);
try (AzureBlobFileSystem abfs = (AzureBlobFileSystem) fileSystem) {
- Assume.assumeFalse("Namespace enabled account does not support this test",
- getIsNamespaceEnabled(abfs));
Path testFile = path("/testReadFile");
Path path = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
@@ -1086,39 +993,36 @@ public void testScenario18() throws Exception {
conf.setBoolean(FS_AZURE_ABFS_ENABLE_CHECKSUM_VALIDATION, true);
FileSystem fileSystem = FileSystem.newInstance(conf);
try (AzureBlobFileSystem abfs = (AzureBlobFileSystem) fileSystem) {
- Assume.assumeFalse("Namespace enabled account does not support this test",
- getIsNamespaceEnabled(abfs));
- try (NativeAzureFileSystem wasb = getWasbFileSystem()) {
-
- Path testFile = path("/testReadFile");
- Path path = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
+ NativeAzureFileSystem wasb = getWasbFileSystem();
- // Write
- try (FSDataOutputStream nativeFsStream = wasb.create(path, true)) {
- nativeFsStream.write(TEST_CONTEXT.getBytes());
- nativeFsStream.flush();
- nativeFsStream.hsync();
- }
- // --- VALIDATE FILE ---
- FileStatus status = wasb.getFileStatus(path);
- assertIsFile(path, status);
+ Path testFile = path("/testReadFile");
+ Path path = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
- // --- SET XATTR #1 ---
- wasb.setXAttr(path, ATTRIBUTE_NAME_1, ATTRIBUTE_VALUE_1);
- byte[] readValue = wasb.getXAttr(path, ATTRIBUTE_NAME_1);
- ITestAzureBlobFileSystemAttributes.assertAttributeEqual(abfs, readValue, ATTRIBUTE_VALUE_1, "one");
+ // Write
+ try (FSDataOutputStream nativeFsStream = wasb.create(path, true)) {
+ nativeFsStream.write(TEST_CONTEXT.getBytes());
+ nativeFsStream.flush();
+ nativeFsStream.hsync();
+ }
+ // --- VALIDATE FILE ---
+ FileStatus status = wasb.getFileStatus(path);
+ assertIsFile(path, status);
- // --- SET XATTR #2 ---
- wasb.setXAttr(path, ATTRIBUTE_NAME_2, ATTRIBUTE_VALUE_2);
- readValue = wasb.getXAttr(path, ATTRIBUTE_NAME_2);
- ITestAzureBlobFileSystemAttributes.assertAttributeEqual(abfs, readValue, ATTRIBUTE_VALUE_2, "two");
+ // --- SET XATTR #1 ---
+ wasb.setXAttr(path, ATTRIBUTE_NAME_1, ATTRIBUTE_VALUE_1);
+ byte[] readValue = wasb.getXAttr(path, ATTRIBUTE_NAME_1);
+ ITestAzureBlobFileSystemAttributes.assertAttributeEqual(abfs, readValue, ATTRIBUTE_VALUE_1, "one");
- // --- VERIFY XATTR #1 AGAIN ---
- readValue = wasb.getXAttr(path, ATTRIBUTE_NAME_1);
- ITestAzureBlobFileSystemAttributes.assertAttributeEqual(abfs, readValue, ATTRIBUTE_VALUE_1, "one");
+ // --- SET XATTR #2 ---
+ wasb.setXAttr(path, ATTRIBUTE_NAME_2, ATTRIBUTE_VALUE_2);
+ readValue = wasb.getXAttr(path, ATTRIBUTE_NAME_2);
+ ITestAzureBlobFileSystemAttributes.assertAttributeEqual(abfs, readValue, ATTRIBUTE_VALUE_2, "two");
- wasb.delete(path, true);
- }
+ // --- VERIFY XATTR #1 AGAIN ---
+ readValue = wasb.getXAttr(path, ATTRIBUTE_NAME_1);
+ ITestAzureBlobFileSystemAttributes.assertAttributeEqual(abfs, readValue, ATTRIBUTE_VALUE_1, "one");
+
+ wasb.delete(path, true);
}
}
@@ -1132,39 +1036,36 @@ public void testScenario19() throws Exception {
conf.setBoolean(FS_AZURE_ABFS_ENABLE_CHECKSUM_VALIDATION, true);
FileSystem fileSystem = FileSystem.newInstance(conf);
try (AzureBlobFileSystem abfs = (AzureBlobFileSystem) fileSystem) {
- Assume.assumeFalse("Namespace enabled account does not support this test",
- getIsNamespaceEnabled(abfs));
- try (NativeAzureFileSystem wasb = getWasbFileSystem()) {
+ NativeAzureFileSystem wasb = getWasbFileSystem();
- Path testFile = path("/testReadFile");
- Path path = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
+ Path testFile = path("/testReadFile");
+ Path path = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
- // Write
- try (FSDataOutputStream nativeFsStream = wasb.create(path, true)) {
- nativeFsStream.write(TEST_CONTEXT.getBytes());
- nativeFsStream.flush();
- nativeFsStream.hsync();
- }
- // --- VALIDATE FILE ---
- FileStatus status = wasb.getFileStatus(path);
- assertIsFile(path, status);
+ // Write
+ try (FSDataOutputStream nativeFsStream = wasb.create(path, true)) {
+ nativeFsStream.write(TEST_CONTEXT.getBytes());
+ nativeFsStream.flush();
+ nativeFsStream.hsync();
+ }
+ // --- VALIDATE FILE ---
+ FileStatus status = wasb.getFileStatus(path);
+ assertIsFile(path, status);
- // --- SET XATTR #1 ---
- wasb.setXAttr(path, ATTRIBUTE_NAME_1, ATTRIBUTE_VALUE_1);
- byte[] readValue = abfs.getXAttr(path, ATTRIBUTE_NAME_1);
- ITestAzureBlobFileSystemAttributes.assertAttributeEqual(abfs, readValue, ATTRIBUTE_VALUE_1, "one");
+ // --- SET XATTR #1 ---
+ wasb.setXAttr(path, ATTRIBUTE_NAME_1, ATTRIBUTE_VALUE_1);
+ byte[] readValue = abfs.getXAttr(path, ATTRIBUTE_NAME_1);
+ ITestAzureBlobFileSystemAttributes.assertAttributeEqual(abfs, readValue, ATTRIBUTE_VALUE_1, "one");
- // --- SET XATTR #2 ---
- wasb.setXAttr(path, ATTRIBUTE_NAME_2, ATTRIBUTE_VALUE_2);
- readValue = abfs.getXAttr(path, ATTRIBUTE_NAME_2);
- ITestAzureBlobFileSystemAttributes.assertAttributeEqual(abfs, readValue, ATTRIBUTE_VALUE_2, "two");
+ // --- SET XATTR #2 ---
+ wasb.setXAttr(path, ATTRIBUTE_NAME_2, ATTRIBUTE_VALUE_2);
+ readValue = abfs.getXAttr(path, ATTRIBUTE_NAME_2);
+ ITestAzureBlobFileSystemAttributes.assertAttributeEqual(abfs, readValue, ATTRIBUTE_VALUE_2, "two");
- // --- VERIFY XATTR #1 AGAIN ---
- readValue = abfs.getXAttr(path, ATTRIBUTE_NAME_1);
- ITestAzureBlobFileSystemAttributes.assertAttributeEqual(abfs, readValue, ATTRIBUTE_VALUE_1, "one");
+ // --- VERIFY XATTR #1 AGAIN ---
+ readValue = abfs.getXAttr(path, ATTRIBUTE_NAME_1);
+ ITestAzureBlobFileSystemAttributes.assertAttributeEqual(abfs, readValue, ATTRIBUTE_VALUE_1, "one");
- wasb.delete(path, true);
- }
+ wasb.delete(path, true);
}
}
@@ -1176,49 +1077,46 @@ public void testScenario19() throws Exception {
*/
@Test
public void testScenario20() throws Exception {
+ Assume.assumeFalse("Not valid for APPEND BLOB", isAppendBlobEnabled());
Configuration conf = getRawConfiguration();
conf.setBoolean(FS_AZURE_ABFS_ENABLE_CHECKSUM_VALIDATION, true);
FileSystem fileSystem = FileSystem.newInstance(conf);
try (AzureBlobFileSystem abfs = (AzureBlobFileSystem) fileSystem) {
- Assume.assumeFalse("Namespace enabled account does not support this test",
- getIsNamespaceEnabled(abfs));
- Assume.assumeFalse("Not valid for APPEND BLOB", isAppendBlobEnabled());
- try (NativeAzureFileSystem wasb = getWasbFileSystem()) {
+ NativeAzureFileSystem wasb = getWasbFileSystem();
- Path testFile = path("/testReadFile");
- Path path = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
+ Path testFile = path("/testReadFile");
+ Path path = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
- // Write
- try (FSDataOutputStream nativeFsStream = wasb.create(path, true)) {
- nativeFsStream.write(TEST_CONTEXT.getBytes());
- nativeFsStream.flush();
- nativeFsStream.hsync();
- }
- // --- VALIDATE FILE ---
- FileStatus status = wasb.getFileStatus(path);
- assertIsFile(path, status);
-
- // --- SET XATTR #1 ---
- wasb.setXAttr(path, ATTRIBUTE_NAME_1, ATTRIBUTE_VALUE_1);
- byte[] readValue = abfs.getXAttr(path, ATTRIBUTE_NAME_1);
- ITestAzureBlobFileSystemAttributes.assertAttributeEqual(abfs, readValue, ATTRIBUTE_VALUE_1, "one");
-
- // --- SET XATTR #2 ---
- wasb.setXAttr(path, ATTRIBUTE_NAME_2, ATTRIBUTE_VALUE_2);
- readValue = abfs.getXAttr(path, ATTRIBUTE_NAME_2);
- ITestAzureBlobFileSystemAttributes.assertAttributeEqual(abfs, readValue, ATTRIBUTE_VALUE_2, "two");
-
- // --- VERIFY XATTR #1 AGAIN ---
- readValue = abfs.getXAttr(path, ATTRIBUTE_NAME_1);
- ITestAzureBlobFileSystemAttributes.assertAttributeEqual(abfs, readValue, ATTRIBUTE_VALUE_1, "one");
-
- abfs.create(path, true);
- FileStatus fileStatus = abfs.getFileStatus(path);
- Assertions.assertThat(fileStatus.getLen())
- .as("Expected file length to be 0 after overwrite")
- .isEqualTo(0L);
- wasb.delete(path, true);
+ // Write
+ try (FSDataOutputStream nativeFsStream = wasb.create(path, true)) {
+ nativeFsStream.write(TEST_CONTEXT.getBytes());
+ nativeFsStream.flush();
+ nativeFsStream.hsync();
}
+ // --- VALIDATE FILE ---
+ FileStatus status = wasb.getFileStatus(path);
+ assertIsFile(path, status);
+
+ // --- SET XATTR #1 ---
+ wasb.setXAttr(path, ATTRIBUTE_NAME_1, ATTRIBUTE_VALUE_1);
+ byte[] readValue = abfs.getXAttr(path, ATTRIBUTE_NAME_1);
+ ITestAzureBlobFileSystemAttributes.assertAttributeEqual(abfs, readValue, ATTRIBUTE_VALUE_1, "one");
+
+ // --- SET XATTR #2 ---
+ wasb.setXAttr(path, ATTRIBUTE_NAME_2, ATTRIBUTE_VALUE_2);
+ readValue = abfs.getXAttr(path, ATTRIBUTE_NAME_2);
+ ITestAzureBlobFileSystemAttributes.assertAttributeEqual(abfs, readValue, ATTRIBUTE_VALUE_2, "two");
+
+ // --- VERIFY XATTR #1 AGAIN ---
+ readValue = abfs.getXAttr(path, ATTRIBUTE_NAME_1);
+ ITestAzureBlobFileSystemAttributes.assertAttributeEqual(abfs, readValue, ATTRIBUTE_VALUE_1, "one");
+
+ abfs.create(path, true);
+ FileStatus fileStatus = abfs.getFileStatus(path);
+ Assertions.assertThat(fileStatus.getLen())
+ .as("Expected file length to be 0 after overwrite")
+ .isEqualTo(0L);
+ wasb.delete(path, true);
}
}
@@ -1230,50 +1128,46 @@ public void testScenario20() throws Exception {
*/
@Test
public void testScenario21() throws Exception {
+ Assume.assumeFalse("Not valid for APPEND BLOB", isAppendBlobEnabled());
Configuration conf = getRawConfiguration();
conf.setBoolean(FS_AZURE_ABFS_ENABLE_CHECKSUM_VALIDATION, true);
FileSystem fileSystem = FileSystem.newInstance(conf);
try (AzureBlobFileSystem abfs = (AzureBlobFileSystem) fileSystem) {
- Assume.assumeFalse("Namespace enabled account does not support this test",
- getIsNamespaceEnabled(abfs));
- assumeBlobServiceType();
- Assume.assumeFalse("Not valid for APPEND BLOB", isAppendBlobEnabled());
- try (NativeAzureFileSystem wasb = getWasbFileSystem()) {
+ NativeAzureFileSystem wasb = getWasbFileSystem();
- Path testFile = path("/testReadFile");
- Path path = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
+ Path testFile = path("/testReadFile");
+ Path path = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
- // Write
- try (FSDataOutputStream abfsOutputStream = abfs.create(path, true)) {
- abfsOutputStream.write(TEST_CONTEXT.getBytes());
- abfsOutputStream.flush();
- abfsOutputStream.hsync();
- }
- // --- VALIDATE FILE ---
- FileStatus status = wasb.getFileStatus(path);
- assertIsFile(path, status);
-
- // --- SET XATTR #1 ---
- abfs.setXAttr(path, ATTRIBUTE_NAME_1, ATTRIBUTE_VALUE_1);
- byte[] readValue = wasb.getXAttr(path, ATTRIBUTE_NAME_1);
- ITestAzureBlobFileSystemAttributes.assertAttributeEqual(abfs, readValue, ATTRIBUTE_VALUE_1, "one");
-
- // --- SET XATTR #2 ---
- abfs.setXAttr(path, ATTRIBUTE_NAME_2, ATTRIBUTE_VALUE_2);
- readValue = wasb.getXAttr(path, ATTRIBUTE_NAME_2);
- ITestAzureBlobFileSystemAttributes.assertAttributeEqual(abfs, readValue, ATTRIBUTE_VALUE_2, "two");
-
- // --- VERIFY XATTR #1 AGAIN ---
- readValue = wasb.getXAttr(path, ATTRIBUTE_NAME_1);
- ITestAzureBlobFileSystemAttributes.assertAttributeEqual(abfs, readValue, ATTRIBUTE_VALUE_1, "one");
-
- wasb.create(path, true);
- FileStatus fileStatus = abfs.getFileStatus(path);
- Assertions.assertThat(fileStatus.getLen())
- .as("Expected file length to be 0 after overwrite")
- .isEqualTo(0L);
- wasb.delete(path, true);
+ // Write
+ try (FSDataOutputStream abfsOutputStream = abfs.create(path, true)) {
+ abfsOutputStream.write(TEST_CONTEXT.getBytes());
+ abfsOutputStream.flush();
+ abfsOutputStream.hsync();
}
+ // --- VALIDATE FILE ---
+ FileStatus status = wasb.getFileStatus(path);
+ assertIsFile(path, status);
+
+ // --- SET XATTR #1 ---
+ abfs.setXAttr(path, ATTRIBUTE_NAME_1, ATTRIBUTE_VALUE_1);
+ byte[] readValue = wasb.getXAttr(path, ATTRIBUTE_NAME_1);
+ ITestAzureBlobFileSystemAttributes.assertAttributeEqual(abfs, readValue, ATTRIBUTE_VALUE_1, "one");
+
+ // --- SET XATTR #2 ---
+ abfs.setXAttr(path, ATTRIBUTE_NAME_2, ATTRIBUTE_VALUE_2);
+ readValue = wasb.getXAttr(path, ATTRIBUTE_NAME_2);
+ ITestAzureBlobFileSystemAttributes.assertAttributeEqual(abfs, readValue, ATTRIBUTE_VALUE_2, "two");
+
+ // --- VERIFY XATTR #1 AGAIN ---
+ readValue = wasb.getXAttr(path, ATTRIBUTE_NAME_1);
+ ITestAzureBlobFileSystemAttributes.assertAttributeEqual(abfs, readValue, ATTRIBUTE_VALUE_1, "one");
+
+ wasb.create(path, true);
+ FileStatus fileStatus = abfs.getFileStatus(path);
+ Assertions.assertThat(fileStatus.getLen())
+ .as("Expected file length to be 0 after overwrite")
+ .isEqualTo(0L);
+ wasb.delete(path, true);
}
}
@@ -1289,44 +1183,41 @@ public void testScenario22() throws Exception {
conf.setBoolean(FS_AZURE_ABFS_ENABLE_CHECKSUM_VALIDATION, true);
FileSystem fileSystem = FileSystem.newInstance(conf);
try (AzureBlobFileSystem abfs = (AzureBlobFileSystem) fileSystem) {
- Assume.assumeFalse("Namespace enabled account does not support this test",
- getIsNamespaceEnabled(abfs));
- try (NativeAzureFileSystem wasb = getWasbFileSystem()) {
+ NativeAzureFileSystem wasb = getWasbFileSystem();
- Path testFile = path("/testReadFile");
- Path path = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
+ Path testFile = path("/testReadFile");
+ Path path = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
- // Write
- try (FSDataOutputStream nativeFsStream = wasb.create(path, true)) {
- nativeFsStream.write(TEST_CONTEXT.getBytes());
- nativeFsStream.flush();
- nativeFsStream.hsync();
- }
- // --- VALIDATE FILE ---
- FileStatus status = wasb.getFileStatus(path);
- assertIsFile(path, status);
-
- // --- SET XATTR #1 ---
- abfs.setXAttr(path, ATTRIBUTE_NAME_1, ATTRIBUTE_VALUE_1);
- byte[] readValue = wasb.getXAttr(path, ATTRIBUTE_NAME_1);
- ITestAzureBlobFileSystemAttributes.assertAttributeEqual(abfs, readValue, ATTRIBUTE_VALUE_1, "one");
-
- // --- SET XATTR #2 ---
- abfs.setXAttr(path, ATTRIBUTE_NAME_2, ATTRIBUTE_VALUE_2);
- readValue = wasb.getXAttr(path, ATTRIBUTE_NAME_2);
- ITestAzureBlobFileSystemAttributes.assertAttributeEqual(abfs, readValue, ATTRIBUTE_VALUE_2, "two");
-
- // --- VERIFY XATTR #1 AGAIN ---
- readValue = wasb.getXAttr(path, ATTRIBUTE_NAME_1);
- ITestAzureBlobFileSystemAttributes.assertAttributeEqual(abfs, readValue, ATTRIBUTE_VALUE_1, "one");
-
- wasb.create(path, true);
- FileStatus fileStatus = abfs.getFileStatus(path);
- Assertions.assertThat(fileStatus.getLen())
- .as("Expected file length to be 0 after overwrite")
- .isEqualTo(0L);
- wasb.delete(path, true);
+ // Write
+ try (FSDataOutputStream nativeFsStream = wasb.create(path, true)) {
+ nativeFsStream.write(TEST_CONTEXT.getBytes());
+ nativeFsStream.flush();
+ nativeFsStream.hsync();
}
+ // --- VALIDATE FILE ---
+ FileStatus status = wasb.getFileStatus(path);
+ assertIsFile(path, status);
+
+ // --- SET XATTR #1 ---
+ abfs.setXAttr(path, ATTRIBUTE_NAME_1, ATTRIBUTE_VALUE_1);
+ byte[] readValue = wasb.getXAttr(path, ATTRIBUTE_NAME_1);
+ ITestAzureBlobFileSystemAttributes.assertAttributeEqual(abfs, readValue, ATTRIBUTE_VALUE_1, "one");
+
+ // --- SET XATTR #2 ---
+ abfs.setXAttr(path, ATTRIBUTE_NAME_2, ATTRIBUTE_VALUE_2);
+ readValue = wasb.getXAttr(path, ATTRIBUTE_NAME_2);
+ ITestAzureBlobFileSystemAttributes.assertAttributeEqual(abfs, readValue, ATTRIBUTE_VALUE_2, "two");
+
+ // --- VERIFY XATTR #1 AGAIN ---
+ readValue = wasb.getXAttr(path, ATTRIBUTE_NAME_1);
+ ITestAzureBlobFileSystemAttributes.assertAttributeEqual(abfs, readValue, ATTRIBUTE_VALUE_1, "one");
+
+ wasb.create(path, true);
+ FileStatus fileStatus = abfs.getFileStatus(path);
+ Assertions.assertThat(fileStatus.getLen())
+ .as("Expected file length to be 0 after overwrite")
+ .isEqualTo(0L);
+ wasb.delete(path, true);
}
}
@@ -1342,38 +1233,35 @@ public void testScenario23() throws Exception {
conf.setBoolean(FS_AZURE_ABFS_ENABLE_CHECKSUM_VALIDATION, true);
FileSystem fileSystem = FileSystem.newInstance(conf);
try (AzureBlobFileSystem abfs = (AzureBlobFileSystem) fileSystem) {
- Assume.assumeFalse("Namespace enabled account does not support this test",
- getIsNamespaceEnabled(abfs));
- try (NativeAzureFileSystem wasb = getWasbFileSystem()) {
+ NativeAzureFileSystem wasb = getWasbFileSystem();
- Path testFile = path("/testReadFile");
- Path path = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
- // Write
- try (FSDataOutputStream nativeFsStream = wasb.create(path, true)) {
- nativeFsStream.write(TEST_CONTEXT.getBytes());
- nativeFsStream.flush();
- nativeFsStream.hsync();
- }
- // --- VALIDATE FILE ---
- FileStatus status = wasb.getFileStatus(path);
- assertIsFile(path, status);
+ Path testFile = path("/testReadFile");
+ Path path = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
+ // Write
+ try (FSDataOutputStream nativeFsStream = wasb.create(path, true)) {
+ nativeFsStream.write(TEST_CONTEXT.getBytes());
+ nativeFsStream.flush();
+ nativeFsStream.hsync();
+ }
+ // --- VALIDATE FILE ---
+ FileStatus status = wasb.getFileStatus(path);
+ assertIsFile(path, status);
- // --- SET XATTR #1 ---
- abfs.setXAttr(path, ATTRIBUTE_NAME_1, ATTRIBUTE_VALUE_1);
- byte[] readValue = abfs.getXAttr(path, ATTRIBUTE_NAME_1);
- ITestAzureBlobFileSystemAttributes.assertAttributeEqual(abfs, readValue, ATTRIBUTE_VALUE_1, "one");
+ // --- SET XATTR #1 ---
+ abfs.setXAttr(path, ATTRIBUTE_NAME_1, ATTRIBUTE_VALUE_1);
+ byte[] readValue = abfs.getXAttr(path, ATTRIBUTE_NAME_1);
+ ITestAzureBlobFileSystemAttributes.assertAttributeEqual(abfs, readValue, ATTRIBUTE_VALUE_1, "one");
- // --- SET XATTR #2 ---
- wasb.setXAttr(path, ATTRIBUTE_NAME_2, ATTRIBUTE_VALUE_2, CREATE_FLAG);
- readValue = abfs.getXAttr(path, ATTRIBUTE_NAME_2);
- ITestAzureBlobFileSystemAttributes.assertAttributeEqual(abfs, readValue, ATTRIBUTE_VALUE_2, "two");
+ // --- SET XATTR #2 ---
+ wasb.setXAttr(path, ATTRIBUTE_NAME_2, ATTRIBUTE_VALUE_2, CREATE_FLAG);
+ readValue = abfs.getXAttr(path, ATTRIBUTE_NAME_2);
+ ITestAzureBlobFileSystemAttributes.assertAttributeEqual(abfs, readValue, ATTRIBUTE_VALUE_2, "two");
- // --- VERIFY XATTR #1 AGAIN ---
- readValue = abfs.getXAttr(path, ATTRIBUTE_NAME_1);
- ITestAzureBlobFileSystemAttributes.assertAttributeEqual(abfs, readValue, ATTRIBUTE_VALUE_1, "one");
+ // --- VERIFY XATTR #1 AGAIN ---
+ readValue = abfs.getXAttr(path, ATTRIBUTE_NAME_1);
+ ITestAzureBlobFileSystemAttributes.assertAttributeEqual(abfs, readValue, ATTRIBUTE_VALUE_1, "one");
- wasb.delete(path, true);
- }
+ wasb.delete(path, true);
}
}
@@ -1385,44 +1273,41 @@ public void testScenario23() throws Exception {
*/
@Test
public void testScenario24() throws Exception {
+ Assume.assumeFalse("Not valid for APPEND BLOB", isAppendBlobEnabled());
Configuration conf = getRawConfiguration();
conf.setBoolean(FS_AZURE_ABFS_ENABLE_CHECKSUM_VALIDATION, true);
FileSystem fileSystem = FileSystem.newInstance(conf);
try (AzureBlobFileSystem abfs = (AzureBlobFileSystem) fileSystem) {
- Assume.assumeFalse("Namespace enabled account does not support this test",
- getIsNamespaceEnabled(abfs));
- Assume.assumeFalse("Not valid for APPEND BLOB", isAppendBlobEnabled());
- try (NativeAzureFileSystem wasb = getWasbFileSystem()) {
+ NativeAzureFileSystem wasb = getWasbFileSystem();
- Path testFile = path("/testReadFile");
- Path path = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
+ Path testFile = path("/testReadFile");
+ Path path = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
- // Write
- try (FSDataOutputStream abfsOutputStream = abfs.create(path, true)) {
- abfsOutputStream.write(TEST_CONTEXT.getBytes());
- abfsOutputStream.flush();
- abfsOutputStream.hsync();
- }
- // --- VALIDATE FILE ---
- FileStatus status = wasb.getFileStatus(path);
- assertIsFile(path, status);
+ // Write
+ try (FSDataOutputStream abfsOutputStream = abfs.create(path, true)) {
+ abfsOutputStream.write(TEST_CONTEXT.getBytes());
+ abfsOutputStream.flush();
+ abfsOutputStream.hsync();
+ }
+ // --- VALIDATE FILE ---
+ FileStatus status = wasb.getFileStatus(path);
+ assertIsFile(path, status);
- // --- SET XATTR #1 ---
- wasb.setXAttr(path, ATTRIBUTE_NAME_1, ATTRIBUTE_VALUE_1);
- byte[] readValue = abfs.getXAttr(path, ATTRIBUTE_NAME_1);
- ITestAzureBlobFileSystemAttributes.assertAttributeEqual(abfs, readValue, ATTRIBUTE_VALUE_1, "one");
+ // --- SET XATTR #1 ---
+ wasb.setXAttr(path, ATTRIBUTE_NAME_1, ATTRIBUTE_VALUE_1);
+ byte[] readValue = abfs.getXAttr(path, ATTRIBUTE_NAME_1);
+ ITestAzureBlobFileSystemAttributes.assertAttributeEqual(abfs, readValue, ATTRIBUTE_VALUE_1, "one");
- // --- SET XATTR #2 ---
- wasb.setXAttr(path, ATTRIBUTE_NAME_2, ATTRIBUTE_VALUE_2, CREATE_FLAG);
- readValue = abfs.getXAttr(path, ATTRIBUTE_NAME_2);
- ITestAzureBlobFileSystemAttributes.assertAttributeEqual(abfs, readValue, ATTRIBUTE_VALUE_2, "two");
+ // --- SET XATTR #2 ---
+ wasb.setXAttr(path, ATTRIBUTE_NAME_2, ATTRIBUTE_VALUE_2, CREATE_FLAG);
+ readValue = abfs.getXAttr(path, ATTRIBUTE_NAME_2);
+ ITestAzureBlobFileSystemAttributes.assertAttributeEqual(abfs, readValue, ATTRIBUTE_VALUE_2, "two");
- // --- VERIFY XATTR #1 AGAIN ---
- readValue = abfs.getXAttr(path, ATTRIBUTE_NAME_1);
- ITestAzureBlobFileSystemAttributes.assertAttributeEqual(abfs, readValue, ATTRIBUTE_VALUE_1, "one");
+ // --- VERIFY XATTR #1 AGAIN ---
+ readValue = abfs.getXAttr(path, ATTRIBUTE_NAME_1);
+ ITestAzureBlobFileSystemAttributes.assertAttributeEqual(abfs, readValue, ATTRIBUTE_VALUE_1, "one");
- wasb.delete(path, true);
- }
+ wasb.delete(path, true);
}
}
@@ -1438,39 +1323,36 @@ public void testScenario25() throws Exception {
conf.setBoolean(FS_AZURE_ABFS_ENABLE_CHECKSUM_VALIDATION, true);
FileSystem fileSystem = FileSystem.newInstance(conf);
try (AzureBlobFileSystem abfs = (AzureBlobFileSystem) fileSystem) {
- Assume.assumeFalse("Namespace enabled account does not support this test",
- getIsNamespaceEnabled(abfs));
- try (NativeAzureFileSystem wasb = getWasbFileSystem()) {
+ NativeAzureFileSystem wasb = getWasbFileSystem();
- Path testFile = path("/testReadFile");
- Path path = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
+ Path testFile = path("/testReadFile");
+ Path path = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
- // Write
- try (FSDataOutputStream nativeFsStream = wasb.create(path, true)) {
- nativeFsStream.write(TEST_CONTEXT.getBytes());
- nativeFsStream.flush();
- nativeFsStream.hsync();
- }
- // --- VALIDATE FILE ---
- FileStatus status = wasb.getFileStatus(path);
- assertIsFile(path, status);
+ // Write
+ try (FSDataOutputStream nativeFsStream = wasb.create(path, true)) {
+ nativeFsStream.write(TEST_CONTEXT.getBytes());
+ nativeFsStream.flush();
+ nativeFsStream.hsync();
+ }
+ // --- VALIDATE FILE ---
+ FileStatus status = wasb.getFileStatus(path);
+ assertIsFile(path, status);
- // --- SET XATTR #1 ---
- abfs.setXAttr(path, ATTRIBUTE_NAME_1, ATTRIBUTE_VALUE_1);
- byte[] readValue = abfs.getXAttr(path, ATTRIBUTE_NAME_1);
- ITestAzureBlobFileSystemAttributes.assertAttributeEqual(abfs, readValue, ATTRIBUTE_VALUE_1, "one");
+ // --- SET XATTR #1 ---
+ abfs.setXAttr(path, ATTRIBUTE_NAME_1, ATTRIBUTE_VALUE_1);
+ byte[] readValue = abfs.getXAttr(path, ATTRIBUTE_NAME_1);
+ ITestAzureBlobFileSystemAttributes.assertAttributeEqual(abfs, readValue, ATTRIBUTE_VALUE_1, "one");
- // --- SET XATTR #2 ---
- abfs.setXAttr(path, ATTRIBUTE_NAME_2, ATTRIBUTE_VALUE_2, CREATE_FLAG);
- readValue = abfs.getXAttr(path, ATTRIBUTE_NAME_2);
- ITestAzureBlobFileSystemAttributes.assertAttributeEqual(abfs, readValue, ATTRIBUTE_VALUE_2, "two");
+ // --- SET XATTR #2 ---
+ abfs.setXAttr(path, ATTRIBUTE_NAME_2, ATTRIBUTE_VALUE_2, CREATE_FLAG);
+ readValue = abfs.getXAttr(path, ATTRIBUTE_NAME_2);
+ ITestAzureBlobFileSystemAttributes.assertAttributeEqual(abfs, readValue, ATTRIBUTE_VALUE_2, "two");
- // --- VERIFY XATTR #1 AGAIN ---
- readValue = abfs.getXAttr(path, ATTRIBUTE_NAME_1);
- ITestAzureBlobFileSystemAttributes.assertAttributeEqual(abfs, readValue, ATTRIBUTE_VALUE_1, "one");
+ // --- VERIFY XATTR #1 AGAIN ---
+ readValue = abfs.getXAttr(path, ATTRIBUTE_NAME_1);
+ ITestAzureBlobFileSystemAttributes.assertAttributeEqual(abfs, readValue, ATTRIBUTE_VALUE_1, "one");
- wasb.delete(path, true);
- }
+ wasb.delete(path, true);
}
}
@@ -1482,44 +1364,41 @@ public void testScenario25() throws Exception {
*/
@Test
public void testScenario26() throws Exception {
+ Assume.assumeFalse("Not valid for APPEND BLOB", isAppendBlobEnabled());
Configuration conf = getRawConfiguration();
conf.setBoolean(FS_AZURE_ABFS_ENABLE_CHECKSUM_VALIDATION, true);
FileSystem fileSystem = FileSystem.newInstance(conf);
try (AzureBlobFileSystem abfs = (AzureBlobFileSystem) fileSystem) {
- Assume.assumeFalse("Namespace enabled account does not support this test",
- getIsNamespaceEnabled(abfs));
- Assume.assumeFalse("Not valid for APPEND BLOB", isAppendBlobEnabled());
- try (NativeAzureFileSystem wasb = getWasbFileSystem()) {
+ NativeAzureFileSystem wasb = getWasbFileSystem();
- Path testFile = path("/testReadFile");
- Path path = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
+ Path testFile = path("/testReadFile");
+ Path path = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
- // Write
- try (FSDataOutputStream abfsOutputStream = abfs.create(path, true)) {
- abfsOutputStream.write(TEST_CONTEXT.getBytes());
- abfsOutputStream.flush();
- abfsOutputStream.hsync();
- }
- // --- VALIDATE FILE ---
- FileStatus status = abfs.getFileStatus(path);
- assertIsFile(path, status);
+ // Write
+ try (FSDataOutputStream abfsOutputStream = abfs.create(path, true)) {
+ abfsOutputStream.write(TEST_CONTEXT.getBytes());
+ abfsOutputStream.flush();
+ abfsOutputStream.hsync();
+ }
+ // --- VALIDATE FILE ---
+ FileStatus status = abfs.getFileStatus(path);
+ assertIsFile(path, status);
- // --- SET XATTR #1 ---
- wasb.setXAttr(path, ATTRIBUTE_NAME_1, ATTRIBUTE_VALUE_1);
- byte[] readValue = wasb.getXAttr(path, ATTRIBUTE_NAME_1);
- ITestAzureBlobFileSystemAttributes.assertAttributeEqual(abfs, readValue, ATTRIBUTE_VALUE_1, "one");
+ // --- SET XATTR #1 ---
+ wasb.setXAttr(path, ATTRIBUTE_NAME_1, ATTRIBUTE_VALUE_1);
+ byte[] readValue = wasb.getXAttr(path, ATTRIBUTE_NAME_1);
+ ITestAzureBlobFileSystemAttributes.assertAttributeEqual(abfs, readValue, ATTRIBUTE_VALUE_1, "one");
- // --- SET XATTR #2 ---
- wasb.setXAttr(path, ATTRIBUTE_NAME_2, ATTRIBUTE_VALUE_2, CREATE_FLAG);
- readValue = wasb.getXAttr(path, ATTRIBUTE_NAME_2);
- ITestAzureBlobFileSystemAttributes.assertAttributeEqual(abfs, readValue, ATTRIBUTE_VALUE_2, "two");
+ // --- SET XATTR #2 ---
+ wasb.setXAttr(path, ATTRIBUTE_NAME_2, ATTRIBUTE_VALUE_2, CREATE_FLAG);
+ readValue = wasb.getXAttr(path, ATTRIBUTE_NAME_2);
+ ITestAzureBlobFileSystemAttributes.assertAttributeEqual(abfs, readValue, ATTRIBUTE_VALUE_2, "two");
- // --- VERIFY XATTR #1 AGAIN ---
- readValue = wasb.getXAttr(path, ATTRIBUTE_NAME_1);
- ITestAzureBlobFileSystemAttributes.assertAttributeEqual(abfs, readValue, ATTRIBUTE_VALUE_1, "one");
+ // --- VERIFY XATTR #1 AGAIN ---
+ readValue = wasb.getXAttr(path, ATTRIBUTE_NAME_1);
+ ITestAzureBlobFileSystemAttributes.assertAttributeEqual(abfs, readValue, ATTRIBUTE_VALUE_1, "one");
- wasb.delete(path, true);
- }
+ wasb.delete(path, true);
}
}
@@ -1529,49 +1408,46 @@ public void testScenario26() throws Exception {
*/
@Test
public void testScenario27() throws Exception {
+ Assume.assumeFalse("Not valid for APPEND BLOB", isAppendBlobEnabled());
Configuration conf = getRawConfiguration();
conf.setBoolean(FS_AZURE_ABFS_ENABLE_CHECKSUM_VALIDATION, true);
FileSystem fileSystem = FileSystem.newInstance(conf);
try (AzureBlobFileSystem abfs = (AzureBlobFileSystem) fileSystem) {
- Assume.assumeFalse("Namespace enabled account does not support this test",
- getIsNamespaceEnabled(abfs));
- Assume.assumeFalse("Not valid for APPEND BLOB", isAppendBlobEnabled());
- try (NativeAzureFileSystem wasb = getWasbFileSystem()) {
+ NativeAzureFileSystem wasb = getWasbFileSystem();
- Path testFile = path("/testReadFile");
- Path testPath1 = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
- Path testPath2 = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
+ Path testFile = path("/testReadFile");
+ Path testPath1 = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
+ Path testPath2 = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
- // Write
- try (FSDataOutputStream abfsOutputStream = abfs.create(testPath1, true)) {
- abfsOutputStream.write(TEST_CONTEXT.getBytes());
- abfsOutputStream.flush();
- abfsOutputStream.hsync();
- }
+ // Write
+ try (FSDataOutputStream abfsOutputStream = abfs.create(testPath1, true)) {
+ abfsOutputStream.write(TEST_CONTEXT.getBytes());
+ abfsOutputStream.flush();
+ abfsOutputStream.hsync();
+ }
- // Check file status
- ContractTestUtils.assertIsFile(abfs, testPath1);
+ // Check file status
+ ContractTestUtils.assertIsFile(abfs, testPath1);
- try (BufferedReader br = new BufferedReader(
- new InputStreamReader(abfs.open(testPath1)))) {
- String line = br.readLine();
- assertEquals("Wrong text from " + abfs,
- TEST_CONTEXT, line);
- }
- // --- RENAME FILE ---
- boolean renamed = wasb.rename(testPath1, testPath2);
- Assertions.assertThat(renamed)
- .as("Rename failed")
- .isTrue();
-
- // --- LIST FILES IN DIRECTORY ---
- Path parentDir = new Path(testFile + "/~12/!008");
- int noOfFiles = listAllFilesAndDirs(wasb, parentDir);
- Assertions.assertThat(noOfFiles)
- .as("Expected only 1 file or directory under path: %s", parentDir)
- .isEqualTo(1);
- wasb.delete(testPath2, true);
+ try (BufferedReader br = new BufferedReader(
+ new InputStreamReader(abfs.open(testPath1)))) {
+ String line = br.readLine();
+ assertEquals("Wrong text from " + abfs,
+ TEST_CONTEXT, line);
}
+ // --- RENAME FILE ---
+ boolean renamed = wasb.rename(testPath1, testPath2);
+ Assertions.assertThat(renamed)
+ .as("Rename failed")
+ .isTrue();
+
+ // --- LIST FILES IN DIRECTORY ---
+ Path parentDir = new Path(testFile + "/~12/!008");
+ int noOfFiles = listAllFilesAndDirs(wasb, parentDir);
+ Assertions.assertThat(noOfFiles)
+ .as("Expected only 1 file or directory under path: %s", parentDir)
+ .isEqualTo(1);
+ wasb.delete(testPath2, true);
}
}
@@ -1585,44 +1461,43 @@ public void testScenario28() throws Exception {
conf.setBoolean(FS_AZURE_ABFS_ENABLE_CHECKSUM_VALIDATION, true);
FileSystem fileSystem = FileSystem.newInstance(conf);
try (AzureBlobFileSystem abfs = (AzureBlobFileSystem) fileSystem) {
- Assume.assumeFalse("Namespace enabled account does not support this test",
- getIsNamespaceEnabled(abfs));
- try (NativeAzureFileSystem wasb = getWasbFileSystem()) {
+ NativeAzureFileSystem wasb = getWasbFileSystem();
- Path testFile = path("/testReadFile");
- Path testPath1 = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
- Path testPath2 = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
+ Path testFile = path("/testReadFile");
+ Path testPath1 = new Path(
+ testFile + "/~12/!008/testfile_" + UUID.randomUUID());
+ Path testPath2 = new Path(
+ testFile + "/~12/!008/testfile_" + UUID.randomUUID());
- // Write
- try (FSDataOutputStream nativeFsStream = wasb.create(testPath1, true)) {
- nativeFsStream.write(TEST_CONTEXT.getBytes());
- nativeFsStream.flush();
- nativeFsStream.hsync();
- }
+ // Write
+ try (FSDataOutputStream nativeFsStream = wasb.create(testPath1, true)) {
+ nativeFsStream.write(TEST_CONTEXT.getBytes());
+ nativeFsStream.flush();
+ nativeFsStream.hsync();
+ }
- // Check file status
- ContractTestUtils.assertIsFile(abfs, testPath1);
+ // Check file status
+ ContractTestUtils.assertIsFile(abfs, testPath1);
- try (BufferedReader br = new BufferedReader(
- new InputStreamReader(abfs.open(testPath1)))) {
- String line = br.readLine();
- assertEquals("Wrong text from " + abfs,
- TEST_CONTEXT, line);
- }
- // --- RENAME FILE ---
- boolean renamed = abfs.rename(testPath1, testPath2);
- Assertions.assertThat(renamed)
- .as("Rename failed")
- .isTrue();
-
- // --- LIST FILES IN DIRECTORY ---
- Path parentDir = new Path(testFile + "/~12/!008");
- int noOfFiles = listAllFilesAndDirs(abfs, parentDir);
- Assertions.assertThat(noOfFiles)
- .as("Expected only 1 file or directory under path: %s", parentDir)
- .isEqualTo(1);
- wasb.delete(testPath2, true);
+ try (BufferedReader br = new BufferedReader(
+ new InputStreamReader(abfs.open(testPath1)))) {
+ String line = br.readLine();
+ assertEquals("Wrong text from " + abfs,
+ TEST_CONTEXT, line);
}
+ // --- RENAME FILE ---
+ boolean renamed = abfs.rename(testPath1, testPath2);
+ Assertions.assertThat(renamed)
+ .as("Rename failed")
+ .isTrue();
+
+ // --- LIST FILES IN DIRECTORY ---
+ Path parentDir = new Path(testFile + "/~12/!008");
+ int noOfFiles = listAllFilesAndDirs(abfs, parentDir);
+ Assertions.assertThat(noOfFiles)
+ .as("Expected only 1 file or directory under path: %s", parentDir)
+ .isEqualTo(1);
+ wasb.delete(testPath2, true);
}
}
@@ -1632,51 +1507,47 @@ public void testScenario28() throws Exception {
*/
@Test
public void testScenario29() throws Exception {
+ Assume.assumeFalse("Not valid for APPEND BLOB", isAppendBlobEnabled());
Configuration conf = getRawConfiguration();
conf.setBoolean(FS_AZURE_ABFS_ENABLE_CHECKSUM_VALIDATION, true);
FileSystem fileSystem = FileSystem.newInstance(conf);
try (AzureBlobFileSystem abfs = (AzureBlobFileSystem) fileSystem) {
- Assume.assumeFalse("Namespace enabled account does not support this test",
- getIsNamespaceEnabled(abfs));
- assumeBlobServiceType();
- Assume.assumeFalse("Not valid for APPEND BLOB", isAppendBlobEnabled());
- try (NativeAzureFileSystem wasb = getWasbFileSystem()) {
+ NativeAzureFileSystem wasb = getWasbFileSystem();
- Path testFile = path("/testReadFile");
- Path testPath1 = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
- Path testPath2 = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
+ Path testFile = path("/testReadFile");
+ Path testPath1 = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
+ Path testPath2 = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
- // Write
- wasb.create(testPath1, true);
- try (FSDataOutputStream abfsOutputStream = abfs.append(testPath1)) {
- abfsOutputStream.write(TEST_CONTEXT.getBytes());
- abfsOutputStream.flush();
- abfsOutputStream.hsync();
- }
+ // Write
+ wasb.create(testPath1, true);
+ try (FSDataOutputStream abfsOutputStream = abfs.append(testPath1)) {
+ abfsOutputStream.write(TEST_CONTEXT.getBytes());
+ abfsOutputStream.flush();
+ abfsOutputStream.hsync();
+ }
- // Check file status
- ContractTestUtils.assertIsFile(abfs, testPath1);
+ // Check file status
+ ContractTestUtils.assertIsFile(abfs, testPath1);
- try (BufferedReader br = new BufferedReader(
- new InputStreamReader(abfs.open(testPath1)))) {
- String line = br.readLine();
- assertEquals("Wrong text from " + abfs,
- TEST_CONTEXT, line);
- }
- // --- RENAME FILE ---
- boolean renamed = abfs.rename(testPath1, testPath2);
- Assertions.assertThat(renamed)
- .as("Rename failed")
- .isTrue();
-
- // --- LIST FILES IN DIRECTORY ---
- Path parentDir = new Path(testFile + "/~12/!008");
- int noOfFiles = listAllFilesAndDirs(abfs, parentDir);
- Assertions.assertThat(noOfFiles)
- .as("Expected only 1 file or directory under path: %s", parentDir)
- .isEqualTo(1);
- wasb.delete(testPath2, true);
+ try (BufferedReader br = new BufferedReader(
+ new InputStreamReader(abfs.open(testPath1)))) {
+ String line = br.readLine();
+ assertEquals("Wrong text from " + abfs,
+ TEST_CONTEXT, line);
}
+ // --- RENAME FILE ---
+ boolean renamed = abfs.rename(testPath1, testPath2);
+ Assertions.assertThat(renamed)
+ .as("Rename failed")
+ .isTrue();
+
+ // --- LIST FILES IN DIRECTORY ---
+ Path parentDir = new Path(testFile + "/~12/!008");
+ int noOfFiles = listAllFilesAndDirs(abfs, parentDir);
+ Assertions.assertThat(noOfFiles)
+ .as("Expected only 1 file or directory under path: %s", parentDir)
+ .isEqualTo(1);
+ wasb.delete(testPath2, true);
}
}
@@ -1690,9 +1561,7 @@ public void testScenario30() throws Exception {
conf.setBoolean(FS_AZURE_ABFS_ENABLE_CHECKSUM_VALIDATION, true);
FileSystem fileSystem = FileSystem.newInstance(conf);
try (AzureBlobFileSystem abfs = (AzureBlobFileSystem) fileSystem) {
- Assume.assumeFalse("Namespace enabled account does not support this test",
- getIsNamespaceEnabled(abfs));
- try (NativeAzureFileSystem wasb = getWasbFileSystem()) {
+ NativeAzureFileSystem wasb = getWasbFileSystem();
Path testFile = path("/testReadFile");
Path testPath1 = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
@@ -1734,7 +1603,6 @@ public void testScenario30() throws Exception {
.as("Expected only 1 file or directory under path: %s", parentDir)
.isEqualTo(1);
wasb.delete(testPath3, true);
- }
}
}
@@ -1748,9 +1616,7 @@ public void testScenario31() throws Exception {
conf.setBoolean(FS_AZURE_ABFS_ENABLE_CHECKSUM_VALIDATION, true);
FileSystem fileSystem = FileSystem.newInstance(conf);
try (AzureBlobFileSystem abfs = (AzureBlobFileSystem) fileSystem) {
- Assume.assumeFalse("Namespace enabled account does not support this test",
- getIsNamespaceEnabled(abfs));
- try (NativeAzureFileSystem wasb = getWasbFileSystem()) {
+ NativeAzureFileSystem wasb = getWasbFileSystem();
Path testFile = path("/testReadFile");
Path testPath1 = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
@@ -1779,7 +1645,6 @@ public void testScenario31() throws Exception {
Assertions.assertThat(renamed)
.as("Rename operation should have failed but returned true")
.isFalse();
- }
}
}
@@ -1793,46 +1658,43 @@ public void testScenario32() throws Exception {
conf.setBoolean(FS_AZURE_ABFS_ENABLE_CHECKSUM_VALIDATION, true);
FileSystem fileSystem = FileSystem.newInstance(conf);
try (AzureBlobFileSystem abfs = (AzureBlobFileSystem) fileSystem) {
- Assume.assumeFalse("Namespace enabled account does not support this test",
- getIsNamespaceEnabled(abfs));
- try (NativeAzureFileSystem wasb = getWasbFileSystem()) {
+ NativeAzureFileSystem wasb = getWasbFileSystem();
- Path testFile = path("/testReadFile");
- Path testFile1 = path("/testReadFile1");
- Path testPath1 = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
- Path testPath2 = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
- Path testPath3 = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
+ Path testFile = path("/testReadFile");
+ Path testFile1 = path("/testReadFile1");
+ Path testPath1 = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
+ Path testPath2 = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
+ Path testPath3 = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
- // Write
- wasb.mkdirs(testFile);
- try (FSDataOutputStream nativeFsStream = wasb.create(testPath1, true)) {
- nativeFsStream.write(TEST_CONTEXT.getBytes());
- nativeFsStream.flush();
- nativeFsStream.hsync();
- }
- wasb.create(testPath2, true);
- wasb.create(testPath3, true);
+ // Write
+ wasb.mkdirs(testFile);
+ try (FSDataOutputStream nativeFsStream = wasb.create(testPath1, true)) {
+ nativeFsStream.write(TEST_CONTEXT.getBytes());
+ nativeFsStream.flush();
+ nativeFsStream.hsync();
+ }
+ wasb.create(testPath2, true);
+ wasb.create(testPath3, true);
- // Check file status
- ContractTestUtils.assertIsFile(abfs, testPath1);
+ // Check file status
+ ContractTestUtils.assertIsFile(abfs, testPath1);
- try (BufferedReader br = new BufferedReader(
- new InputStreamReader(abfs.open(testPath1)))) {
- String line = br.readLine();
- assertEquals("Wrong text from " + abfs,
- TEST_CONTEXT, line);
- }
- // --- RENAME DIR ---
- boolean renamed = abfs.rename(testFile, testFile1);
- Assertions.assertThat(renamed)
- .as("Rename failed")
- .isTrue();
- // --- LIST FILES IN DIRECTORY ---
- int listResult = listAllFilesAndDirs(abfs, testFile1);
- Assertions.assertThat(listResult)
- .as("Expected only 5 entries under path: %s", testFile1)
- .isEqualTo(5);
+ try (BufferedReader br = new BufferedReader(
+ new InputStreamReader(abfs.open(testPath1)))) {
+ String line = br.readLine();
+ assertEquals("Wrong text from " + abfs,
+ TEST_CONTEXT, line);
}
+ // --- RENAME DIR ---
+ boolean renamed = abfs.rename(testFile, testFile1);
+ Assertions.assertThat(renamed)
+ .as("Rename failed")
+ .isTrue();
+ // --- LIST FILES IN DIRECTORY ---
+ int listResult = listAllFilesAndDirs(abfs, testFile1);
+ Assertions.assertThat(listResult)
+ .as("Expected only 5 entries under path: %s", testFile1)
+ .isEqualTo(5);
}
}
@@ -1842,51 +1704,48 @@ public void testScenario32() throws Exception {
*/
@Test
public void testScenario33() throws Exception {
+ Assume.assumeFalse("Not valid for APPEND BLOB", isAppendBlobEnabled());
Configuration conf = getRawConfiguration();
conf.setBoolean(FS_AZURE_ABFS_ENABLE_CHECKSUM_VALIDATION, true);
FileSystem fileSystem = FileSystem.newInstance(conf);
try (AzureBlobFileSystem abfs = (AzureBlobFileSystem) fileSystem) {
- Assume.assumeFalse("Namespace enabled account does not support this test",
- getIsNamespaceEnabled(abfs));
- Assume.assumeFalse("Not valid for APPEND BLOB", isAppendBlobEnabled());
- try (NativeAzureFileSystem wasb = getWasbFileSystem()) {
+ NativeAzureFileSystem wasb = getWasbFileSystem();
- Path testFile = path("/testReadFile");
- Path testFile1 = path("/testReadFile1");
- Path testPath1 = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
- Path testPath2 = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
- Path testPath3 = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
+ Path testFile = path("/testReadFile");
+ Path testFile1 = path("/testReadFile1");
+ Path testPath1 = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
+ Path testPath2 = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
+ Path testPath3 = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
- // Write
- abfs.mkdirs(testFile);
- try (FSDataOutputStream abfsOutputStream = abfs.create(testPath1, true)) {
- abfsOutputStream.write(TEST_CONTEXT.getBytes());
- abfsOutputStream.flush();
- abfsOutputStream.hsync();
- }
- abfs.create(testPath2, true);
- abfs.create(testPath3, true);
+ // Write
+ abfs.mkdirs(testFile);
+ try (FSDataOutputStream abfsOutputStream = abfs.create(testPath1, true)) {
+ abfsOutputStream.write(TEST_CONTEXT.getBytes());
+ abfsOutputStream.flush();
+ abfsOutputStream.hsync();
+ }
+ abfs.create(testPath2, true);
+ abfs.create(testPath3, true);
- // Check file status
- ContractTestUtils.assertIsFile(abfs, testPath1);
+ // Check file status
+ ContractTestUtils.assertIsFile(abfs, testPath1);
- try (BufferedReader br = new BufferedReader(
- new InputStreamReader(abfs.open(testPath1)))) {
- String line = br.readLine();
- assertEquals("Wrong text from " + abfs,
- TEST_CONTEXT, line);
- }
- // --- RENAME DIR ---
- boolean renamed = wasb.rename(testFile, testFile1);
- Assertions.assertThat(renamed)
- .as("Rename failed")
- .isTrue();
- // --- LIST FILES IN DIRECTORY ---
- int listResult = listAllFilesAndDirs(wasb, testFile1);
- Assertions.assertThat(listResult)
- .as("Expected only 5 entries under path: %s", testFile1)
- .isEqualTo(5);
+ try (BufferedReader br = new BufferedReader(
+ new InputStreamReader(abfs.open(testPath1)))) {
+ String line = br.readLine();
+ assertEquals("Wrong text from " + abfs,
+ TEST_CONTEXT, line);
}
+ // --- RENAME DIR ---
+ boolean renamed = wasb.rename(testFile, testFile1);
+ Assertions.assertThat(renamed)
+ .as("Rename failed")
+ .isTrue();
+ // --- LIST FILES IN DIRECTORY ---
+ int listResult = listAllFilesAndDirs(wasb, testFile1);
+ Assertions.assertThat(listResult)
+ .as("Expected only 5 entries under path: %s", testFile1)
+ .isEqualTo(5);
}
}
@@ -1900,44 +1759,41 @@ public void testScenario34() throws Exception {
conf.setBoolean(FS_AZURE_ABFS_ENABLE_CHECKSUM_VALIDATION, true);
FileSystem fileSystem = FileSystem.newInstance(conf);
try (AzureBlobFileSystem abfs = (AzureBlobFileSystem) fileSystem) {
- Assume.assumeFalse("Namespace enabled account does not support this test",
- getIsNamespaceEnabled(abfs));
- try (NativeAzureFileSystem wasb = getWasbFileSystem()) {
+ NativeAzureFileSystem wasb = getWasbFileSystem();
- Path testFile = path("/testReadFile");
- Path testPath1 = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
- Path testPath2 = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
- Path testPath3 = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
+ Path testFile = path("/testReadFile");
+ Path testPath1 = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
+ Path testPath2 = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
+ Path testPath3 = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
- // Write
- abfs.mkdirs(testFile);
- try (FSDataOutputStream abfsOutputStream = abfs.create(testPath1, true)) {
- abfsOutputStream.write(TEST_CONTEXT.getBytes());
- abfsOutputStream.flush();
- abfsOutputStream.hsync();
- }
- abfs.create(testPath3, true);
+ // Write
+ abfs.mkdirs(testFile);
+ try (FSDataOutputStream abfsOutputStream = abfs.create(testPath1, true)) {
+ abfsOutputStream.write(TEST_CONTEXT.getBytes());
+ abfsOutputStream.flush();
+ abfsOutputStream.hsync();
+ }
+ abfs.create(testPath3, true);
- // Check file status
- ContractTestUtils.assertIsFile(abfs, testPath1);
+ // Check file status
+ ContractTestUtils.assertIsFile(abfs, testPath1);
- try (BufferedReader br = new BufferedReader(
- new InputStreamReader(abfs.open(testPath1)))) {
- String line = br.readLine();
- assertEquals("Wrong text from " + abfs,
- TEST_CONTEXT, line);
- }
- // --- RENAME DIR ---
- boolean renamed = wasb.rename(testPath1, testPath2);
- Assertions.assertThat(renamed)
- .as("Rename failed")
- .isTrue();
- // --- LIST FILES IN DIRECTORY ---
- int listResult = listAllFilesAndDirs(abfs, testFile);
- Assertions.assertThat(listResult)
- .as("Expected only 4 entries under path: %s", testFile)
- .isEqualTo(4);
+ try (BufferedReader br = new BufferedReader(
+ new InputStreamReader(abfs.open(testPath1)))) {
+ String line = br.readLine();
+ assertEquals("Wrong text from " + abfs,
+ TEST_CONTEXT, line);
}
+ // --- RENAME DIR ---
+ boolean renamed = wasb.rename(testPath1, testPath2);
+ Assertions.assertThat(renamed)
+ .as("Rename failed")
+ .isTrue();
+ // --- LIST FILES IN DIRECTORY ---
+ int listResult = listAllFilesAndDirs(abfs, testFile);
+ Assertions.assertThat(listResult)
+ .as("Expected only 4 entries under path: %s", testFile)
+ .isEqualTo(4);
}
}
@@ -1951,44 +1807,41 @@ public void testScenario35() throws Exception {
conf.setBoolean(FS_AZURE_ABFS_ENABLE_CHECKSUM_VALIDATION, true);
FileSystem fileSystem = FileSystem.newInstance(conf);
try (AzureBlobFileSystem abfs = (AzureBlobFileSystem) fileSystem) {
- Assume.assumeFalse("Namespace enabled account does not support this test",
- getIsNamespaceEnabled(abfs));
- try (NativeAzureFileSystem wasb = getWasbFileSystem()) {
+ NativeAzureFileSystem wasb = getWasbFileSystem();
- Path testFile = path("/testReadFile");
- Path testPath1 = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
- Path testPath2 = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
- Path testPath3 = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
+ Path testFile = path("/testReadFile");
+ Path testPath1 = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
+ Path testPath2 = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
+ Path testPath3 = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
- // Write
- wasb.mkdirs(testFile);
- try (FSDataOutputStream nativeFsStream = wasb.create(testPath1, true)) {
- nativeFsStream.write(TEST_CONTEXT.getBytes());
- nativeFsStream.flush();
- nativeFsStream.hsync();
- }
- wasb.create(testPath3, true);
+ // Write
+ wasb.mkdirs(testFile);
+ try (FSDataOutputStream nativeFsStream = wasb.create(testPath1, true)) {
+ nativeFsStream.write(TEST_CONTEXT.getBytes());
+ nativeFsStream.flush();
+ nativeFsStream.hsync();
+ }
+ wasb.create(testPath3, true);
- // Check file status
- ContractTestUtils.assertIsFile(abfs, testPath1);
+ // Check file status
+ ContractTestUtils.assertIsFile(abfs, testPath1);
- try (BufferedReader br = new BufferedReader(
- new InputStreamReader(abfs.open(testPath1)))) {
- String line = br.readLine();
- assertEquals("Wrong text from " + abfs,
- TEST_CONTEXT, line);
- }
- // --- RENAME DIR ---
- boolean renamed = abfs.rename(testPath1, testPath2);
- Assertions.assertThat(renamed)
- .as("Rename failed")
- .isTrue();
- // --- LIST FILES IN DIRECTORY ---
- int listResult = listAllFilesAndDirs(wasb, testFile);
- Assertions.assertThat(listResult)
- .as("Expected only 4 entries under path: %s", testFile)
- .isEqualTo(4);
+ try (BufferedReader br = new BufferedReader(
+ new InputStreamReader(abfs.open(testPath1)))) {
+ String line = br.readLine();
+ assertEquals("Wrong text from " + abfs,
+ TEST_CONTEXT, line);
}
+ // --- RENAME DIR ---
+ boolean renamed = abfs.rename(testPath1, testPath2);
+ Assertions.assertThat(renamed)
+ .as("Rename failed")
+ .isTrue();
+ // --- LIST FILES IN DIRECTORY ---
+ int listResult = listAllFilesAndDirs(wasb, testFile);
+ Assertions.assertThat(listResult)
+ .as("Expected only 4 entries under path: %s", testFile)
+ .isEqualTo(4);
}
}
@@ -2003,38 +1856,35 @@ public void testScenario36() throws Exception {
conf.setBoolean(FS_AZURE_ABFS_ENABLE_CHECKSUM_VALIDATION, true);
FileSystem fileSystem = FileSystem.newInstance(conf);
try (AzureBlobFileSystem abfs = (AzureBlobFileSystem) fileSystem) {
- Assume.assumeFalse("Namespace enabled account does not support this test",
- getIsNamespaceEnabled(abfs));
- try (NativeAzureFileSystem wasb = getWasbFileSystem()) {
+ NativeAzureFileSystem wasb = getWasbFileSystem();
- Path testFile = path("/testReadFile");
- Path testPath1 = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
- Path testPath3 = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
+ Path testFile = path("/testReadFile");
+ Path testPath1 = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
+ Path testPath3 = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
- // Write
- wasb.mkdirs(testFile);
- try (FSDataOutputStream nativeFsStream = wasb.create(testPath1, true)) {
- nativeFsStream.write(TEST_CONTEXT.getBytes());
- nativeFsStream.flush();
- nativeFsStream.hsync();
- }
- wasb.create(testPath3, true);
+ // Write
+ wasb.mkdirs(testFile);
+ try (FSDataOutputStream nativeFsStream = wasb.create(testPath1, true)) {
+ nativeFsStream.write(TEST_CONTEXT.getBytes());
+ nativeFsStream.flush();
+ nativeFsStream.hsync();
+ }
+ wasb.create(testPath3, true);
- // Check file status
- ContractTestUtils.assertIsFile(abfs, testPath1);
+ // Check file status
+ ContractTestUtils.assertIsFile(abfs, testPath1);
- try (BufferedReader br = new BufferedReader(
- new InputStreamReader(abfs.open(testPath1)))) {
- String line = br.readLine();
- assertEquals("Wrong text from " + abfs,
- TEST_CONTEXT, line);
- }
- // --- RENAME DIR ---
- boolean renamed = abfs.rename(testFile, testFile);
- Assertions.assertThat(renamed)
- .as("Rename operation should have failed but returned true")
- .isFalse();
+ try (BufferedReader br = new BufferedReader(
+ new InputStreamReader(abfs.open(testPath1)))) {
+ String line = br.readLine();
+ assertEquals("Wrong text from " + abfs,
+ TEST_CONTEXT, line);
}
+ // --- RENAME DIR ---
+ boolean renamed = abfs.rename(testFile, testFile);
+ Assertions.assertThat(renamed)
+ .as("Rename operation should have failed but returned true")
+ .isFalse();
}
}
@@ -2048,39 +1898,36 @@ public void testScenario37() throws Exception {
conf.setBoolean(FS_AZURE_ABFS_ENABLE_CHECKSUM_VALIDATION, true);
FileSystem fileSystem = FileSystem.newInstance(conf);
try (AzureBlobFileSystem abfs = (AzureBlobFileSystem) fileSystem) {
- Assume.assumeFalse("Namespace enabled account does not support this test",
- getIsNamespaceEnabled(abfs));
- try (NativeAzureFileSystem wasb = getWasbFileSystem()) {
+ NativeAzureFileSystem wasb = getWasbFileSystem();
- Path testFile = path("/testReadFile");
- Path testPath1 = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
- Path testPath2 = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
- Path testPath3 = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
+ Path testFile = path("/testReadFile");
+ Path testPath1 = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
+ Path testPath2 = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
+ Path testPath3 = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
- // Write
- abfs.mkdirs(testFile);
- try (FSDataOutputStream abfsOutputStream = abfs.create(testPath1, true)) {
- abfsOutputStream.write(TEST_CONTEXT.getBytes());
- abfsOutputStream.flush();
- abfsOutputStream.hsync();
- }
- abfs.create(testPath3, true);
+ // Write
+ abfs.mkdirs(testFile);
+ try (FSDataOutputStream abfsOutputStream = abfs.create(testPath1, true)) {
+ abfsOutputStream.write(TEST_CONTEXT.getBytes());
+ abfsOutputStream.flush();
+ abfsOutputStream.hsync();
+ }
+ abfs.create(testPath3, true);
- // Check file status
- ContractTestUtils.assertIsFile(abfs, testPath1);
+ // Check file status
+ ContractTestUtils.assertIsFile(abfs, testPath1);
- try (BufferedReader br = new BufferedReader(
- new InputStreamReader(abfs.open(testPath1)))) {
- String line = br.readLine();
- assertEquals("Wrong text from " + abfs,
- TEST_CONTEXT, line);
- }
- // --- RENAME NON EXISTENT FILE ---
- boolean renamed = wasb.rename(testPath2, testPath3);
- Assertions.assertThat(renamed)
- .as("Rename operation should have failed but returned true")
- .isFalse();
+ try (BufferedReader br = new BufferedReader(
+ new InputStreamReader(abfs.open(testPath1)))) {
+ String line = br.readLine();
+ assertEquals("Wrong text from " + abfs,
+ TEST_CONTEXT, line);
}
+ // --- RENAME NON EXISTENT FILE ---
+ boolean renamed = wasb.rename(testPath2, testPath3);
+ Assertions.assertThat(renamed)
+ .as("Rename operation should have failed but returned true")
+ .isFalse();
}
}
@@ -2094,44 +1941,41 @@ public void testScenario38() throws Exception {
conf.setBoolean(FS_AZURE_ABFS_ENABLE_CHECKSUM_VALIDATION, true);
FileSystem fileSystem = FileSystem.newInstance(conf);
try (AzureBlobFileSystem abfs = (AzureBlobFileSystem) fileSystem) {
- Assume.assumeFalse("Namespace enabled account does not support this test",
- getIsNamespaceEnabled(abfs));
- try (NativeAzureFileSystem wasb = getWasbFileSystem()) {
+ NativeAzureFileSystem wasb = getWasbFileSystem();
- Path testFile = path("/testReadFile");
- Path path = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
+ Path testFile = path("/testReadFile");
+ Path path = new Path(testFile + "/~12/!008/testfile_" + UUID.randomUUID());
- // Write
- try (FSDataOutputStream nativeFsStream = wasb.create(path, true)) {
- nativeFsStream.write(TEST_CONTEXT.getBytes());
- nativeFsStream.flush();
- nativeFsStream.hsync();
- }
- // --- VALIDATE FILE ---
- FileStatus status = wasb.getFileStatus(path);
- assertIsFile(path, status);
-
- // --- SET XATTR #1 ---
- wasb.setXAttr(path, ATTRIBUTE_NAME_1, ATTRIBUTE_VALUE_1);
- byte[] readValue = wasb.getXAttr(path, ATTRIBUTE_NAME_1);
- ITestAzureBlobFileSystemAttributes.assertAttributeEqual(abfs, readValue, ATTRIBUTE_VALUE_1, "one");
-
- // --- SET XATTR #2 ---
- wasb.setXAttr(path, ATTRIBUTE_NAME_2, ATTRIBUTE_VALUE_2);
- readValue = wasb.getXAttr(path, ATTRIBUTE_NAME_2);
- ITestAzureBlobFileSystemAttributes.assertAttributeEqual(abfs, readValue, ATTRIBUTE_VALUE_2, "two");
-
- // --- VERIFY XATTR #1 AGAIN ---
- readValue = wasb.getXAttr(path, ATTRIBUTE_NAME_1);
- ITestAzureBlobFileSystemAttributes.assertAttributeEqual(abfs, readValue, ATTRIBUTE_VALUE_1, "one");
-
- wasb.create(path, true);
- FileStatus fileStatus = abfs.getFileStatus(path);
- Assertions.assertThat(fileStatus.getLen())
- .as("Expected file length to be 0 after overwrite")
- .isEqualTo(0L);
- wasb.delete(path, true);
+ // Write
+ try (FSDataOutputStream nativeFsStream = wasb.create(path, true)) {
+ nativeFsStream.write(TEST_CONTEXT.getBytes());
+ nativeFsStream.flush();
+ nativeFsStream.hsync();
}
+ // --- VALIDATE FILE ---
+ FileStatus status = wasb.getFileStatus(path);
+ assertIsFile(path, status);
+
+ // --- SET XATTR #1 ---
+ wasb.setXAttr(path, ATTRIBUTE_NAME_1, ATTRIBUTE_VALUE_1);
+ byte[] readValue = wasb.getXAttr(path, ATTRIBUTE_NAME_1);
+ ITestAzureBlobFileSystemAttributes.assertAttributeEqual(abfs, readValue, ATTRIBUTE_VALUE_1, "one");
+
+ // --- SET XATTR #2 ---
+ wasb.setXAttr(path, ATTRIBUTE_NAME_2, ATTRIBUTE_VALUE_2);
+ readValue = wasb.getXAttr(path, ATTRIBUTE_NAME_2);
+ ITestAzureBlobFileSystemAttributes.assertAttributeEqual(abfs, readValue, ATTRIBUTE_VALUE_2, "two");
+
+ // --- VERIFY XATTR #1 AGAIN ---
+ readValue = wasb.getXAttr(path, ATTRIBUTE_NAME_1);
+ ITestAzureBlobFileSystemAttributes.assertAttributeEqual(abfs, readValue, ATTRIBUTE_VALUE_1, "one");
+
+ wasb.create(path, true);
+ FileStatus fileStatus = abfs.getFileStatus(path);
+ Assertions.assertThat(fileStatus.getLen())
+ .as("Expected file length to be 0 after overwrite")
+ .isEqualTo(0L);
+ wasb.delete(path, true);
}
}
@@ -2145,9 +1989,7 @@ public void testScenario39() throws Exception {
conf.setBoolean(FS_AZURE_ABFS_ENABLE_CHECKSUM_VALIDATION, true);
FileSystem fileSystem = FileSystem.newInstance(conf);
try (AzureBlobFileSystem abfs = (AzureBlobFileSystem) fileSystem) {
- Assume.assumeFalse("Namespace enabled account does not support this test",
- getIsNamespaceEnabled(abfs));
- try (NativeAzureFileSystem wasb = getWasbFileSystem()) {
+ NativeAzureFileSystem wasb = getWasbFileSystem();
String testRunId = UUID.randomUUID().toString();
Path baseDir = path("/testScenario39_" + testRunId);
@@ -2184,7 +2026,6 @@ public void testScenario39() throws Exception {
Assertions.assertThat(listResult)
.as("Expected only 4 entries under path: %s", testFile)
.isEqualTo(4);
- }
}
}
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsOutputStream.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsOutputStream.java
index 27d071c4c57f7..3392540947ea8 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsOutputStream.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsOutputStream.java
@@ -441,11 +441,10 @@ public void testNoNetworkCallsForSecondFlush() throws Exception {
*/
@Test
public void testResetCalledOnExceptionInRemoteFlush() throws Exception {
- AzureBlobFileSystem fs = Mockito.spy(getFileSystem());
- assumeThat(getIsNamespaceEnabled(fs)).isFalse();
- AzureBlobFileSystemStore store = Mockito.spy(fs.getAbfsStore());
+ assumeHnsDisabled();
assumeBlobServiceType();
assumeThat(isAppendBlobEnabled()).as("Not valid for APPEND BLOB").isFalse();
+ AzureBlobFileSystem fs = Mockito.spy(getFileSystem());
// Create a file and spy on AbfsOutputStream
Path path = new Path("/testFile");
@@ -502,43 +501,48 @@ public void testResetCalledOnExceptionInRemoteFlush() throws Exception {
*/
@Test
public void testNoChecksumComputedWhenConfigFalse() throws Exception {
+ assumeThat(isAppendBlobEnabled()).as("Not valid for APPEND BLOB").isFalse();
+ assumeBlobServiceType();
+ assumeHnsDisabled();
Configuration conf = getRawConfiguration();
conf.setBoolean(FS_AZURE_ABFS_ENABLE_CHECKSUM_VALIDATION, false);
FileSystem fileSystem = FileSystem.newInstance(conf);
- AzureBlobFileSystem fs = (AzureBlobFileSystem) fileSystem;
- assumeThat(getIsNamespaceEnabled(fs)).isFalse();
- AzureBlobFileSystemStore store = Mockito.spy(fs.getAbfsStore());
- assumeBlobServiceType();
- assumeThat(isAppendBlobEnabled()).as("Not valid for APPEND BLOB").isFalse();
-
- // Create spies for the client handler and blob client
- AbfsClientHandler clientHandler = Mockito.spy(store.getClientHandler());
- AbfsBlobClient blobClient = Mockito.spy(clientHandler.getBlobClient());
-
- // Set up the spies to return the mocked objects
- Mockito.doReturn(clientHandler).when(store).getClientHandler();
- Mockito.doReturn(blobClient).when(clientHandler).getBlobClient();
- Mockito.doReturn(blobClient).when(clientHandler).getIngressClient();
- AbfsOutputStream abfsOutputStream = Mockito.spy(
- (AbfsOutputStream) fs.create(new Path("/test/file")).getWrappedStream());
- AzureIngressHandler ingressHandler = Mockito.spy(
- abfsOutputStream.getIngressHandler());
- Mockito.doReturn(ingressHandler).when(abfsOutputStream).getIngressHandler();
- Mockito.doReturn(blobClient).when(ingressHandler).getClient();
- FSDataOutputStream os = Mockito.spy(
- new FSDataOutputStream(abfsOutputStream, null));
- AbfsOutputStream out = (AbfsOutputStream) os.getWrappedStream();
- byte[] bytes = new byte[1024 * 1024 * 4];
- new Random().nextBytes(bytes);
- // Write some bytes and attempt to flush, which should retry
- out.write(bytes);
- out.hsync();
- Assertions.assertThat(Mockito.mockingDetails(blobClient).getInvocations()
- .stream()
- .filter(i -> i.getMethod().getName().equals("addCheckSumHeaderForWrite"))
- .count())
- .as("Expected addCheckSumHeaderForWrite() to be called exactly 0 times")
- .isZero();
+ try (AzureBlobFileSystem fs = (AzureBlobFileSystem) fileSystem) {
+ AzureBlobFileSystemStore store = Mockito.spy(fs.getAbfsStore());
+
+ // Create spies for the client handler and blob client
+ AbfsClientHandler clientHandler = Mockito.spy(store.getClientHandler());
+ AbfsBlobClient blobClient = Mockito.spy(clientHandler.getBlobClient());
+
+ // Set up the spies to return the mocked objects
+ Mockito.doReturn(clientHandler).when(store).getClientHandler();
+ Mockito.doReturn(blobClient).when(clientHandler).getBlobClient();
+ Mockito.doReturn(blobClient).when(clientHandler).getIngressClient();
+ AbfsOutputStream abfsOutputStream = Mockito.spy(
+ (AbfsOutputStream) fs.create(new Path("/test/file"))
+ .getWrappedStream());
+ AzureIngressHandler ingressHandler = Mockito.spy(
+ abfsOutputStream.getIngressHandler());
+ Mockito.doReturn(ingressHandler)
+ .when(abfsOutputStream)
+ .getIngressHandler();
+ Mockito.doReturn(blobClient).when(ingressHandler).getClient();
+ FSDataOutputStream os = Mockito.spy(
+ new FSDataOutputStream(abfsOutputStream, null));
+ AbfsOutputStream out = (AbfsOutputStream) os.getWrappedStream();
+ byte[] bytes = new byte[1024 * 1024 * 4];
+ new Random().nextBytes(bytes);
+ // Write some bytes and attempt to flush, which should retry
+ out.write(bytes);
+ out.hsync();
+ Assertions.assertThat(Mockito.mockingDetails(blobClient).getInvocations()
+ .stream()
+ .filter(
+ i -> i.getMethod().getName().equals("addCheckSumHeaderForWrite"))
+ .count())
+ .as("Expected addCheckSumHeaderForWrite() to be called exactly 0 times")
+ .isZero();
+ }
}
/**
@@ -547,42 +551,47 @@ public void testNoChecksumComputedWhenConfigFalse() throws Exception {
*/
@Test
public void testChecksumComputedWhenConfigTrue() throws Exception {
+ assumeHnsDisabled();
+ assumeBlobServiceType();
+ assumeThat(isAppendBlobEnabled()).as("Not valid for APPEND BLOB")
+ .isFalse();
Configuration conf = getRawConfiguration();
conf.setBoolean(FS_AZURE_ABFS_ENABLE_CHECKSUM_VALIDATION, true);
FileSystem fileSystem = FileSystem.newInstance(conf);
- AzureBlobFileSystem fs = (AzureBlobFileSystem) fileSystem;
- assumeThat(getIsNamespaceEnabled(fs)).isFalse();
- AzureBlobFileSystemStore store = Mockito.spy(fs.getAbfsStore());
- assumeBlobServiceType();
- assumeThat(isAppendBlobEnabled()).as("Not valid for APPEND BLOB").isFalse();
-
- // Create spies for the client handler and blob client
- AbfsClientHandler clientHandler = Mockito.spy(store.getClientHandler());
- AbfsBlobClient blobClient = Mockito.spy(clientHandler.getBlobClient());
-
- // Set up the spies to return the mocked objects
- Mockito.doReturn(clientHandler).when(store).getClientHandler();
- Mockito.doReturn(blobClient).when(clientHandler).getBlobClient();
- Mockito.doReturn(blobClient).when(clientHandler).getIngressClient();
- AbfsOutputStream abfsOutputStream = Mockito.spy(
- (AbfsOutputStream) fs.create(new Path("/test/file")).getWrappedStream());
- AzureIngressHandler ingressHandler = Mockito.spy(
- abfsOutputStream.getIngressHandler());
- Mockito.doReturn(ingressHandler).when(abfsOutputStream).getIngressHandler();
- Mockito.doReturn(blobClient).when(ingressHandler).getClient();
- FSDataOutputStream os = Mockito.spy(
- new FSDataOutputStream(abfsOutputStream, null));
- AbfsOutputStream out = (AbfsOutputStream) os.getWrappedStream();
- byte[] bytes = new byte[1024 * 1024 * 4];
- new Random().nextBytes(bytes);
- // Write some bytes and attempt to flush, which should retry
- out.write(bytes);
- out.hsync();
- Assertions.assertThat(Mockito.mockingDetails(blobClient).getInvocations()
- .stream()
- .filter(i -> i.getMethod().getName().equals("addCheckSumHeaderForWrite"))
- .count())
- .as("Expected addCheckSumHeaderForWrite() to be called exactly once")
- .isEqualTo(1);
+ try (AzureBlobFileSystem fs = (AzureBlobFileSystem) fileSystem) {
+ AzureBlobFileSystemStore store = Mockito.spy(fs.getAbfsStore());
+ // Create spies for the client handler and blob client
+ AbfsClientHandler clientHandler = Mockito.spy(store.getClientHandler());
+ AbfsBlobClient blobClient = Mockito.spy(clientHandler.getBlobClient());
+
+ // Set up the spies to return the mocked objects
+ Mockito.doReturn(clientHandler).when(store).getClientHandler();
+ Mockito.doReturn(blobClient).when(clientHandler).getBlobClient();
+ Mockito.doReturn(blobClient).when(clientHandler).getIngressClient();
+ AbfsOutputStream abfsOutputStream = Mockito.spy(
+ (AbfsOutputStream) fs.create(new Path("/test/file"))
+ .getWrappedStream());
+ AzureIngressHandler ingressHandler = Mockito.spy(
+ abfsOutputStream.getIngressHandler());
+ Mockito.doReturn(ingressHandler)
+ .when(abfsOutputStream)
+ .getIngressHandler();
+ Mockito.doReturn(blobClient).when(ingressHandler).getClient();
+ FSDataOutputStream os = Mockito.spy(
+ new FSDataOutputStream(abfsOutputStream, null));
+ AbfsOutputStream out = (AbfsOutputStream) os.getWrappedStream();
+ byte[] bytes = new byte[1024 * 1024 * 4];
+ new Random().nextBytes(bytes);
+ // Write some bytes and attempt to flush, which should retry
+ out.write(bytes);
+ out.hsync();
+ Assertions.assertThat(Mockito.mockingDetails(blobClient).getInvocations()
+ .stream()
+ .filter(
+ i -> i.getMethod().getName().equals("addCheckSumHeaderForWrite"))
+ .count())
+ .as("Expected addCheckSumHeaderForWrite() to be called exactly once")
+ .isEqualTo(1);
+ }
}
}
From f9a3529b2a13215a8607f50a59fb6cd95be7eef8 Mon Sep 17 00:00:00 2001
From: Anmol Asrani
Date: Wed, 20 Aug 2025 07:06:24 -0700
Subject: [PATCH 25/25] remove unused import
---
.../hadoop/fs/azurebfs/services/ITestAbfsOutputStream.java | 1 -
1 file changed, 1 deletion(-)
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsOutputStream.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsOutputStream.java
index 3392540947ea8..dc68fd7e4d07d 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsOutputStream.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsOutputStream.java
@@ -31,7 +31,6 @@
import java.util.concurrent.TimeUnit;
import org.assertj.core.api.Assertions;
-import org.junit.Assume;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.Timeout;
import org.junit.jupiter.params.ParameterizedClass;