>From Ritik Raj <[email protected]>:

Ritik Raj has uploaded this change for review. ( 
https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/20571?usp=email )


Change subject: [NO ISSUE][CLOUD] Throw error if azure multipart fails
......................................................................

[NO ISSUE][CLOUD] Throw error if azure multipart fails

- user model changes: no
- storage format changes: no
- interface changes: no

Details:
No official source confirms that commitBlockListWithResponse
is guaranteed to be retriable. Therefore, the implementation
has been updated to fail immediately if the blob storage
server returns an error, instead of retrying the operation.

Ext-ref: MB-69374

Change-Id: I8a2e48efd5f121bd65181d5532fa034179cdd873
---
M 
asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/clients/azure/blobstorage/AzBlobStorageBufferedWriter.java
1 file changed, 18 insertions(+), 27 deletions(-)



  git pull ssh://asterix-gerrit.ics.uci.edu:29418/asterixdb 
refs/changes/71/20571/1

diff --git 
a/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/clients/azure/blobstorage/AzBlobStorageBufferedWriter.java
 
b/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/clients/azure/blobstorage/AzBlobStorageBufferedWriter.java
index d2054c6d..1b4ce50 100644
--- 
a/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/clients/azure/blobstorage/AzBlobStorageBufferedWriter.java
+++ 
b/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/clients/azure/blobstorage/AzBlobStorageBufferedWriter.java
@@ -28,12 +28,12 @@
 import java.util.Base64;
 import java.util.List;
 import java.util.UUID;
-import java.util.concurrent.TimeUnit;

 import org.apache.asterix.cloud.clients.ICloudBufferedWriter;
 import org.apache.asterix.cloud.clients.ICloudGuardian;
 import org.apache.asterix.cloud.clients.profiler.IRequestProfilerLimiter;
 import org.apache.commons.io.IOUtils;
+import org.apache.hyracks.api.exceptions.ErrorCode;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
@@ -43,7 +43,6 @@
 import com.azure.storage.blob.BlobContainerClient;
 import com.azure.storage.blob.models.AccessTier;
 import com.azure.storage.blob.models.BlobRequestConditions;
-import com.azure.storage.blob.models.BlobStorageException;
 import com.azure.storage.blob.options.BlobParallelUploadOptions;
 import com.azure.storage.blob.specialized.BlockBlobClient;
 import com.azure.storage.common.implementation.Constants;
@@ -101,6 +100,7 @@
     private void initBlockBlobUploads(String blockID) {
         if (this.uploadID == null) {
             this.uploadID = blockID;
+            log("STARTED");
         }
     }

@@ -137,32 +137,17 @@
         } else if (PUT_UPLOAD_ID.equals(uploadID)) {
             return;
         }
-        int currRetryAttempt = 0;
-        BlockBlobClient blockBlobClient = 
blobContainerClient.getBlobClient(path).getBlockBlobClient();
-        while (true) {
-            try {
-                guardian.checkWriteAccess(bucket, path);
-                profiler.objectMultipartUpload();
-                blockBlobClient.commitBlockListWithResponse(blockIDArrayList, 
null, null, accessTier,
-                        new 
BlobRequestConditions().setIfNoneMatch(Constants.HeaderConstants.ETAG_WILDCARD),
 null,
-                        Context.NONE);
-                break;
-            } catch (BlobStorageException e) {
-                currRetryAttempt++;
-                if (currRetryAttempt == MAX_RETRIES) {
-                    throw HyracksDataException.create(e);
-                }
-                LOGGER.info(() -> "AzBlob storage write retry, encountered: " 
+ e.getMessage());
-
-                // Backoff for 1 sec for the first 2 retries, and 2 seconds 
from there onward
-                try {
-                    Thread.sleep(TimeUnit.SECONDS.toMillis(currRetryAttempt < 
2 ? 1 : 2));
-                } catch (InterruptedException ex) {
-                    Thread.currentThread().interrupt();
-                    throw HyracksDataException.create(ex);
-                }
-            }
+        try {
+            BlockBlobClient blockBlobClient = 
blobContainerClient.getBlobClient(path).getBlockBlobClient();
+            guardian.checkWriteAccess(bucket, path);
+            profiler.objectMultipartUpload();
+            blockBlobClient.commitBlockListWithResponse(blockIDArrayList, 
null, null, accessTier,
+                    new 
BlobRequestConditions().setIfNoneMatch(Constants.HeaderConstants.ETAG_WILDCARD),
 null,
+                    Context.NONE);
+        } catch (RuntimeException e) {
+            throw HyracksDataException.create(ErrorCode.FAILED_IO_OPERATION, 
e);
         }
+        log("FINISHED");
     }

     @Override
@@ -171,4 +156,10 @@
         // https://github.com/Azure/azure-sdk-for-java/issues/31150
         LOGGER.warn("Multipart upload for {} was aborted", path);
     }
+
+    private void log(String op) {
+        if (LOGGER.isDebugEnabled()) {
+            LOGGER.debug("{} multipart upload for {}", op, path);
+        }
+    }
 }

--
To view, visit https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/20571?usp=email
To unsubscribe, or for help writing mail filters, visit 
https://asterix-gerrit.ics.uci.edu/settings?usp=email

Gerrit-MessageType: newchange
Gerrit-Project: asterixdb
Gerrit-Branch: phoenix
Gerrit-Change-Id: I8a2e48efd5f121bd65181d5532fa034179cdd873
Gerrit-Change-Number: 20571
Gerrit-PatchSet: 1
Gerrit-Owner: Ritik Raj <[email protected]>

Reply via email to