|
@@ -25,7 +25,10 @@ import java.net.HttpURLConnection;
|
|
import java.net.MalformedURLException;
|
|
import java.net.MalformedURLException;
|
|
import java.net.URL;
|
|
import java.net.URL;
|
|
import java.net.URLEncoder;
|
|
import java.net.URLEncoder;
|
|
|
|
+import java.security.MessageDigest;
|
|
|
|
+import java.security.NoSuchAlgorithmException;
|
|
import java.util.ArrayList;
|
|
import java.util.ArrayList;
|
|
|
|
+import java.util.Base64;
|
|
import java.util.List;
|
|
import java.util.List;
|
|
import java.util.Locale;
|
|
import java.util.Locale;
|
|
import java.util.UUID;
|
|
import java.util.UUID;
|
|
@@ -34,6 +37,9 @@ import java.util.concurrent.ThreadFactory;
|
|
import java.util.concurrent.TimeUnit;
|
|
import java.util.concurrent.TimeUnit;
|
|
|
|
|
|
import org.apache.hadoop.classification.VisibleForTesting;
|
|
import org.apache.hadoop.classification.VisibleForTesting;
|
|
|
|
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsInvalidChecksumException;
|
|
|
|
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsDriverException;
|
|
|
|
+import org.apache.hadoop.fs.azurebfs.contracts.services.AzureServiceErrorCode;
|
|
import org.apache.hadoop.fs.azurebfs.utils.NamespaceUtil;
|
|
import org.apache.hadoop.fs.azurebfs.utils.NamespaceUtil;
|
|
import org.apache.hadoop.fs.store.LogExactlyOnce;
|
|
import org.apache.hadoop.fs.store.LogExactlyOnce;
|
|
import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystemStore.Permissions;
|
|
import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystemStore.Permissions;
|
|
@@ -76,6 +82,7 @@ import static org.apache.hadoop.fs.azurebfs.AbfsStatistic.RENAME_PATH_ATTEMPTS;
|
|
import static org.apache.hadoop.fs.azurebfs.AzureBlobFileSystemStore.extractEtagHeader;
|
|
import static org.apache.hadoop.fs.azurebfs.AzureBlobFileSystemStore.extractEtagHeader;
|
|
import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.*;
|
|
import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.*;
|
|
import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.DEFAULT_DELETE_CONSIDERED_IDEMPOTENT;
|
|
import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.DEFAULT_DELETE_CONSIDERED_IDEMPOTENT;
|
|
|
|
+import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.ONE_MB;
|
|
import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.SERVER_SIDE_ENCRYPTION_ALGORITHM;
|
|
import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.SERVER_SIDE_ENCRYPTION_ALGORITHM;
|
|
import static org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes.HTTPS_SCHEME;
|
|
import static org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes.HTTPS_SCHEME;
|
|
import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.*;
|
|
import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.*;
|
|
@@ -851,6 +858,11 @@ public class AbfsClient implements Closeable {
|
|
requestHeaders.add(new AbfsHttpHeader(USER_AGENT, userAgentRetry));
|
|
requestHeaders.add(new AbfsHttpHeader(USER_AGENT, userAgentRetry));
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ // Add MD5 Hash of request content as request header if feature is enabled
|
|
|
|
+ if (isChecksumValidationEnabled()) {
|
|
|
|
+ addCheckSumHeaderForWrite(requestHeaders, reqParams, buffer);
|
|
|
|
+ }
|
|
|
|
+
|
|
// AbfsInputStream/AbfsOutputStream reuse SAS tokens for better performance
|
|
// AbfsInputStream/AbfsOutputStream reuse SAS tokens for better performance
|
|
String sasTokenForReuse = appendSASTokenToQuery(path, SASTokenProvider.WRITE_OPERATION,
|
|
String sasTokenForReuse = appendSASTokenToQuery(path, SASTokenProvider.WRITE_OPERATION,
|
|
abfsUriQueryBuilder, cachedSasToken);
|
|
abfsUriQueryBuilder, cachedSasToken);
|
|
@@ -867,7 +879,7 @@ public class AbfsClient implements Closeable {
|
|
sasTokenForReuse);
|
|
sasTokenForReuse);
|
|
try {
|
|
try {
|
|
op.execute(tracingContext);
|
|
op.execute(tracingContext);
|
|
- } catch (AzureBlobFileSystemException e) {
|
|
|
|
|
|
+ } catch (AbfsRestOperationException e) {
|
|
/*
|
|
/*
|
|
If the http response code indicates a user error we retry
|
|
If the http response code indicates a user error we retry
|
|
the same append request with expect header being disabled.
|
|
the same append request with expect header being disabled.
|
|
@@ -877,7 +889,7 @@ public class AbfsClient implements Closeable {
|
|
if someone has taken dependency on the exception message,
|
|
if someone has taken dependency on the exception message,
|
|
which is created using the error string present in the response header.
|
|
which is created using the error string present in the response header.
|
|
*/
|
|
*/
|
|
- int responseStatusCode = ((AbfsRestOperationException) e).getStatusCode();
|
|
|
|
|
|
+ int responseStatusCode = e.getStatusCode();
|
|
if (checkUserError(responseStatusCode) && reqParams.isExpectHeaderEnabled()) {
|
|
if (checkUserError(responseStatusCode) && reqParams.isExpectHeaderEnabled()) {
|
|
LOG.debug("User error, retrying without 100 continue enabled for the given path {}", path);
|
|
LOG.debug("User error, retrying without 100 continue enabled for the given path {}", path);
|
|
reqParams.setExpectHeaderEnabled(false);
|
|
reqParams.setExpectHeaderEnabled(false);
|
|
@@ -889,6 +901,11 @@ public class AbfsClient implements Closeable {
|
|
if (!op.hasResult()) {
|
|
if (!op.hasResult()) {
|
|
throw e;
|
|
throw e;
|
|
}
|
|
}
|
|
|
|
+
|
|
|
|
+ if (isMd5ChecksumError(e)) {
|
|
|
|
+ throw new AbfsInvalidChecksumException(e);
|
|
|
|
+ }
|
|
|
|
+
|
|
if (reqParams.isAppendBlob()
|
|
if (reqParams.isAppendBlob()
|
|
&& appendSuccessCheckOp(op, path,
|
|
&& appendSuccessCheckOp(op, path,
|
|
(reqParams.getPosition() + reqParams.getLength()), tracingContext)) {
|
|
(reqParams.getPosition() + reqParams.getLength()), tracingContext)) {
|
|
@@ -907,6 +924,13 @@ public class AbfsClient implements Closeable {
|
|
throw e;
|
|
throw e;
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ catch (AzureBlobFileSystemException e) {
|
|
|
|
+ // Any server side issue will be returned as AbfsRestOperationException and will be handled above.
|
|
|
|
+ LOG.debug("Append request failed with non server issues for path: {}, offset: {}, position: {}",
|
|
|
|
+ path, reqParams.getoffset(), reqParams.getPosition());
|
|
|
|
+ throw e;
|
|
|
|
+ }
|
|
|
|
+
|
|
return op;
|
|
return op;
|
|
}
|
|
}
|
|
|
|
|
|
@@ -920,6 +944,16 @@ public class AbfsClient implements Closeable {
|
|
&& responseStatusCode < HttpURLConnection.HTTP_INTERNAL_ERROR);
|
|
&& responseStatusCode < HttpURLConnection.HTTP_INTERNAL_ERROR);
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ /**
|
|
|
|
+ * To check if the failure exception returned by server is due to MD5 Mismatch
|
|
|
|
+ * @param e Exception returned by AbfsRestOperation
|
|
|
|
+ * @return boolean whether exception is due to MD5Mismatch or not
|
|
|
|
+ */
|
|
|
|
+ private boolean isMd5ChecksumError(final AbfsRestOperationException e) {
|
|
|
|
+ AzureServiceErrorCode storageErrorCode = e.getErrorCode();
|
|
|
|
+ return storageErrorCode == AzureServiceErrorCode.MD5_MISMATCH;
|
|
|
|
+ }
|
|
|
|
+
|
|
// For AppendBlob its possible that the append succeeded in the backend but the request failed.
|
|
// For AppendBlob its possible that the append succeeded in the backend but the request failed.
|
|
// However a retry would fail with an InvalidQueryParameterValue
|
|
// However a retry would fail with an InvalidQueryParameterValue
|
|
// (as the current offset would be unacceptable).
|
|
// (as the current offset would be unacceptable).
|
|
@@ -1049,10 +1083,16 @@ public class AbfsClient implements Closeable {
|
|
final List<AbfsHttpHeader> requestHeaders = createDefaultHeaders();
|
|
final List<AbfsHttpHeader> requestHeaders = createDefaultHeaders();
|
|
addEncryptionKeyRequestHeaders(path, requestHeaders, false,
|
|
addEncryptionKeyRequestHeaders(path, requestHeaders, false,
|
|
contextEncryptionAdapter, tracingContext);
|
|
contextEncryptionAdapter, tracingContext);
|
|
- requestHeaders.add(new AbfsHttpHeader(RANGE,
|
|
|
|
- String.format("bytes=%d-%d", position, position + bufferLength - 1)));
|
|
|
|
|
|
+ AbfsHttpHeader rangeHeader = new AbfsHttpHeader(RANGE,
|
|
|
|
+ String.format("bytes=%d-%d", position, position + bufferLength - 1));
|
|
|
|
+ requestHeaders.add(rangeHeader);
|
|
requestHeaders.add(new AbfsHttpHeader(IF_MATCH, eTag));
|
|
requestHeaders.add(new AbfsHttpHeader(IF_MATCH, eTag));
|
|
|
|
|
|
|
|
+ // Add request header to fetch MD5 Hash of data returned by server.
|
|
|
|
+ if (isChecksumValidationEnabled(requestHeaders, rangeHeader, bufferLength)) {
|
|
|
|
+ requestHeaders.add(new AbfsHttpHeader(X_MS_RANGE_GET_CONTENT_MD5, TRUE));
|
|
|
|
+ }
|
|
|
|
+
|
|
final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder();
|
|
final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder();
|
|
// AbfsInputStream/AbfsOutputStream reuse SAS tokens for better performance
|
|
// AbfsInputStream/AbfsOutputStream reuse SAS tokens for better performance
|
|
String sasTokenForReuse = appendSASTokenToQuery(path, SASTokenProvider.READ_OPERATION,
|
|
String sasTokenForReuse = appendSASTokenToQuery(path, SASTokenProvider.READ_OPERATION,
|
|
@@ -1069,6 +1109,11 @@ public class AbfsClient implements Closeable {
|
|
bufferLength, sasTokenForReuse);
|
|
bufferLength, sasTokenForReuse);
|
|
op.execute(tracingContext);
|
|
op.execute(tracingContext);
|
|
|
|
|
|
|
|
+ // Verify the MD5 hash returned by server holds valid on the data received.
|
|
|
|
+ if (isChecksumValidationEnabled(requestHeaders, rangeHeader, bufferLength)) {
|
|
|
|
+ verifyCheckSumForRead(buffer, op.getResult(), bufferOffset);
|
|
|
|
+ }
|
|
|
|
+
|
|
return op;
|
|
return op;
|
|
}
|
|
}
|
|
|
|
|
|
@@ -1492,6 +1537,100 @@ public class AbfsClient implements Closeable {
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ /**
|
|
|
|
+ * 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
|
|
|
|
+ */
|
|
|
|
+ private void addCheckSumHeaderForWrite(List<AbfsHttpHeader> 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.
|
|
|
|
+ * @param result HTTP Operation Result.
|
|
|
|
+ * @param bufferOffset Position where data returned by server is saved in buffer.
|
|
|
|
+ * @throws AbfsRestOperationException if Md5Mismatch.
|
|
|
|
+ */
|
|
|
|
+ private void verifyCheckSumForRead(final byte[] buffer,
|
|
|
|
+ final AbfsHttpOperation result, final int bufferOffset)
|
|
|
|
+ throws AbfsRestOperationException {
|
|
|
|
+ // Number of bytes returned by server could be less than or equal to what
|
|
|
|
+ // caller requests. In case it is less, extra bytes will be initialized to 0
|
|
|
|
+ // Server returned MD5 Hash will be computed on what server returned.
|
|
|
|
+ // We need to get exact data that server returned and compute its md5 hash
|
|
|
|
+ // Computed hash should be equal to what server returned.
|
|
|
|
+ int numberOfBytesRead = (int) result.getBytesReceived();
|
|
|
|
+ if (numberOfBytesRead == 0) {
|
|
|
|
+ return;
|
|
|
|
+ }
|
|
|
|
+ String md5HashComputed = computeMD5Hash(buffer, bufferOffset,
|
|
|
|
+ numberOfBytesRead);
|
|
|
|
+ String md5HashActual = result.getResponseHeader(CONTENT_MD5);
|
|
|
|
+ if (!md5HashComputed.equals(md5HashActual)) {
|
|
|
|
+ LOG.debug("Md5 Mismatch Error in Read Operation. Server returned Md5: {}, Client computed Md5: {}", md5HashActual, md5HashComputed);
|
|
|
|
+ throw new AbfsInvalidChecksumException(result.getRequestId());
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ /**
|
|
|
|
+ * Conditions check for allowing checksum support for read operation.
|
|
|
|
+ * Sending MD5 Hash in request headers. For more details see
|
|
|
|
+ * @see <a href="https://learn.microsoft.com/en-us/rest/api/storageservices/datalakestoragegen2/path/read">
|
|
|
|
+ * Path - Read Azure Storage Rest API</a>.
|
|
|
|
+ * 1. Range header must be present as one of the request headers.
|
|
|
|
+ * 2. buffer length must be less than or equal to 4 MB.
|
|
|
|
+ * @param requestHeaders to be checked for range header.
|
|
|
|
+ * @param rangeHeader must be present.
|
|
|
|
+ * @param bufferLength must be less than or equal to 4 MB.
|
|
|
|
+ * @return true if all conditions are met.
|
|
|
|
+ */
|
|
|
|
+ private boolean isChecksumValidationEnabled(List<AbfsHttpHeader> requestHeaders,
|
|
|
|
+ final AbfsHttpHeader rangeHeader, final int bufferLength) {
|
|
|
|
+ return getAbfsConfiguration().getIsChecksumValidationEnabled()
|
|
|
|
+ && requestHeaders.contains(rangeHeader) && bufferLength <= 4 * ONE_MB;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ /**
|
|
|
|
+ * 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 see
|
|
|
|
+ * @see <a href="https://learn.microsoft.com/en-us/rest/api/storageservices/datalakestoragegen2/path/update">
|
|
|
|
+ * Path - Update Azure Rest API</a>.
|
|
|
|
+ * @return true if checksum validation enabled.
|
|
|
|
+ */
|
|
|
|
+ private boolean isChecksumValidationEnabled() {
|
|
|
|
+ return getAbfsConfiguration().getIsChecksumValidationEnabled();
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ /**
|
|
|
|
+ * 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.
|
|
|
|
+ * @param off offset in the array from where actual data starts.
|
|
|
|
+ * @param len length of the data to be used to compute MD5Hash.
|
|
|
|
+ * @return MD5 Hash of the data as String.
|
|
|
|
+ * @throws AbfsRestOperationException if computation fails.
|
|
|
|
+ */
|
|
|
|
+ @VisibleForTesting
|
|
|
|
+ public String computeMD5Hash(final byte[] data, final int off, final int len)
|
|
|
|
+ throws AbfsRestOperationException {
|
|
|
|
+ try {
|
|
|
|
+ MessageDigest md5Digest = MessageDigest.getInstance(MD5);
|
|
|
|
+ md5Digest.update(data, off, len);
|
|
|
|
+ byte[] md5Bytes = md5Digest.digest();
|
|
|
|
+ return Base64.getEncoder().encodeToString(md5Bytes);
|
|
|
|
+ } catch (NoSuchAlgorithmException ex) {
|
|
|
|
+ throw new AbfsDriverException(ex);
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+
|
|
@VisibleForTesting
|
|
@VisibleForTesting
|
|
URL getBaseUrl() {
|
|
URL getBaseUrl() {
|
|
return baseUrl;
|
|
return baseUrl;
|