[ 
https://issues.apache.org/jira/browse/HADOOP-17377?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=18032513#comment-18032513
 ] 

ASF GitHub Bot commented on HADOOP-17377:
-----------------------------------------

steveloughran commented on PR #5273:
URL: https://github.com/apache/hadoop/pull/5273#issuecomment-3437685484

   When these uploads fail we do leave incomplete uploads in progress:
   ```
   Listing uploads under path ""
   job-00-fork-0005/test/testCommitOperations 
141OKG11JHhWF1GOnunHUd9ZzBJ8cUG9z0LsW_4wUGgCXCvDMQM3kRi5IOCUV8FdCHtg_w8SlipfubRtzCQoT5yEpOLv.cWOiOwjEaBzUjnuJORppfXuKy1piHpLnu98
   
job-00-fork-0005/test/testIfMatchTwoMultipartUploadsRaceConditionOneClosesFirst 
yBJpm3zh4DjNQIDtyWgEmWVCk5sehVz5Vzn3QGr_tQT2iOonRp5ErXsQy24yIvnzRxBCZqVapy5VepLeu2udZBT5EXLnKRA3bchvzjtKDlipywSzYlL2N_xLUDCT359I
   job-00-fork-0005/test/testIfNoneMatchTwoConcurrentMultipartUploads 
AnspJPHUoPJqg61t28OvLfAogi6G9ocyx1Dm6XY2C.a_H_onklM0Nr0LIXaPiYlQjZIiH0fTsQ1e2KhEjS9pGxvSKOXq_4YibiGZmFC6rBolmfACMqIRpoeaqYDgzYW4
   job-00-fork-0005/test/testMagicWriteRecovery/file.txt 
KpvoTuVh85Wzm9XuU1EuxbATjb6D.Zv8vEj3z2S6AvJBHCBssy4iphxNhTkLDs7ceEwak4IPtdXED1vRf3geXT7MRMJn8d6feafvHVEgzbD31odpzTLmOaPrU_mFQXGV
   job-00-fork-0005/test/testMagicWriteRecovery/file.txt 
CnrbWU3pzgEGvjRuDuaP43Xcv1eBF5aLknqYaZA1vwO3b1QUIu9QJSiZjuLMYKT9GKw1QXwqoKo4iuxTY1a18bARx4XMEiL98kZBv0TPMaAfXE.70Olh8Q2kTyDlUCSh
   job-00-fork-0005/test/testMagicWriteRecovery/file.txt 
dEVGPBRsuOAzL5pGA02ve9qJhAlNK8lb8khF6laKjo9U0j_aG1xLkHEfPLrmcrcsLxC3R755Yv_uKbzY_Vnoc.nXCprvutM1TZmLLN_7LHrQ0tY0IjYSS6hVzDVlHbvC
   job-00-fork-0006/test/restricted/testCommitEmptyFile/empty-commit.txt 
NOCjVJqycZhkalrvU26F5oIaJP51q055et2N6b74.2JVjiKL8KwrhOhdrtumOrZ2tZWNqaK4iKZ_iosqgehJOiPbWJwxvrfvA5V.dAUTLNqjtEf5tfWh0UXu.vahDy_S5SSgNLFXK.VB82i5MZtOcw--
   job-00/test/tests3ascale/ITestS3AHugeMagicCommits/commit/commit.bin 
lsYNpdn_oiWLwEVvvM621hCvIwDVaL4y_bbwVpQouW1OBThA.P9cR8fZtxvBjGdMY41UH0dTjxGHtF3BXEY8WXqmcnO9QHs_Jy.os781pE3MGzqgzFyxmd0yN6LFcTbq
   test/restricted/testCommitEmptyFile/empty-commit.txt 
T3W9V56Bv_FMhKpgcBgJ1H2wOBkPKk23T0JomesBzZyqiIAu3NiROibAgoZUhWSdoTKSJoOgcn3UWYGOvGBbsHteS_N_c1QoTEp0GE7PNlzDfs1GheJ5SOpUgaEY6MaYdNe0mn0gY48FDXpVB2nqiA--
   test/restricted/testCommitEmptyFile/empty-commit.txt 
.cr4b3xkfze4N24Bj3PAm_ACIyIVuTU4DueDktU1abNu2LJWXH2HKnUu1oOjfnnQwnUXp4VmXBVbZ5aq8E8gVCxN.Oyb7hmGVtESmRjpqIXSW80JrB_0_dqXe.uAT.JH7kEWywAlb4NIqJ5Xz99tvA--
   Total 10 uploads found.
   ```
   
   Most interesting here is `testIfNoneMatchTwoConcurrentMultipartUploads`, 
because this initiates then completes an MPU, so as to create a zero byte file. 
It doesn't upload any parts. 
   
   The attempt to complete failed.
   ```
   [ERROR]   
ITestS3APutIfMatchAndIfNoneMatch.testIfNoneMatchTwoConcurrentMultipartUploads:380->createFileWithFlags:190
 » AWSBadRequest Completing multipart upload on 
job-00-fork-0005/test/testIfNoneMatchTwoConcurrentMultipartUploads: 
software.amazon.awssdk.services.s3.model.S3Exception: One or more of the 
specified parts could not be found.  The part may not have been uploaded, or 
the specified entity tag may not match the part's entity tag. (Service: S3, 
Status Code: 400, Request ID: 9JCJ6M5QRDGJNYYS, Extended Request ID: 
Z7Q7+LA0o/5B4xoIGhgo+tVppawZ0UBj7X4RNb+0m9RbOAOwD/Apv1o+KmnW0aypjwmfFlarxjo=) 
(SDK Attempt Count: 1):InvalidPart: One or more of the specified parts could 
not be found.  The part may not have been uploaded, or the specified entity tag 
may not match the part's entity tag. (Service: S3, Status Code: 400, Request 
ID: 9JCJ6M5QRDGJNYYS, Extended Request ID: 
Z7Q7+LA0o/5B4xoIGhgo+tVppawZ0UBj7X4RNb+0m9RbOAOwD/Apv1o+KmnW0aypjwmfFlarxjo=) 
(SDK Attempt Count: 1)          
   ```
   
   Yet the uploads list afterwards finds it
   ```
   job-00-fork-0005/test/testIfNoneMatchTwoConcurrentMultipartUploads 
AnspJPHUoPJqg61t28OvLfAogi6G9ocyx1Dm6XY2C.a_H_onklM0Nr0LIXaPiYlQjZIiH0fTsQ1e2KhEjS9pGxvSKOXq_4YibiGZmFC6rBolmfACMqIRpoeaqYDgzYW4
   ```
   
   I have to conclude that the list of pending uploads was briefly 
offline/inconsistent.
   
   This is presumably so, so rare that there's almost no point retrying here. 
With no retries, every active write/job would have failed, even though the 
system had recovered within a minute.
   
   Maybe we should retry here? I remember a long long time ago the v1 sdk 
didn't retry on failures of the final POST to commit an upload, and how that 
sporadically caused problems. Retrying on MPU failures will allow for recovery 
in the presence of a transient failure here, and the cost of "deletion of all 
pending uploads will take longer to fail all active uploads". 
   
   




> ABFS: MsiTokenProvider doesn't retry HTTP 429 from the Instance Metadata 
> Service
> --------------------------------------------------------------------------------
>
>                 Key: HADOOP-17377
>                 URL: https://issues.apache.org/jira/browse/HADOOP-17377
>             Project: Hadoop Common
>          Issue Type: Bug
>          Components: fs/azure
>    Affects Versions: 3.2.1
>            Reporter: Brandon
>            Priority: Major
>              Labels: pull-request-available
>
> *Summary*
>  The instance metadata service has its own guidance for error handling and 
> retry which are different from the Blob store. 
> [https://docs.microsoft.com/en-us/azure/active-directory/managed-identities-azure-resources/how-to-use-vm-token#error-handling]
> In particular, it responds with HTTP 429 if request rate is too high. Whereas 
> Blob store will respond with HTTP 503. The retry policy used only accounts 
> for the latter as it will retry any status >=500. This can result in job 
> instability when running multiple processes on the same host.
> *Environment*
>  * Spark talking to an ABFS store
>  * Hadoop 3.2.1
>  * Running on an Azure VM with user-assigned identity, ABFS configured to use 
> MsiTokenProvider
>  * 6 executor processes on each VM
> *Example*
>  Here's an example error message and stack trace. It's always the same stack 
> trace. This appears in logs a few hundred to low thousands of times a day. 
> It's luckily skating by since the download operation is wrapped in 3 retries.
> {noformat}
> AADToken: HTTP connection failed for getting token from AzureAD. Http 
> response: 429 null
> Content-Type: application/json; charset=utf-8 Content-Length: 90 Request ID:  
> Proxies: none
> First 1K of Body: {"error":"invalid_request","error_description":"Temporarily 
> throttled, too many requests"}
>       at 
> org.apache.hadoop.fs.azurebfs.services.AbfsRestOperation.executeHttpOperation(AbfsRestOperation.java:190)
>       at 
> org.apache.hadoop.fs.azurebfs.services.AbfsRestOperation.execute(AbfsRestOperation.java:125)
>       at 
> org.apache.hadoop.fs.azurebfs.services.AbfsClient.getAclStatus(AbfsClient.java:506)
>       at 
> org.apache.hadoop.fs.azurebfs.services.AbfsClient.getAclStatus(AbfsClient.java:489)
>       at 
> org.apache.hadoop.fs.azurebfs.AzureBlobFileSystemStore.getIsNamespaceEnabled(AzureBlobFileSystemStore.java:208)
>       at 
> org.apache.hadoop.fs.azurebfs.AzureBlobFileSystemStore.getFileStatus(AzureBlobFileSystemStore.java:473)
>       at 
> org.apache.hadoop.fs.azurebfs.AzureBlobFileSystem.getFileStatus(AzureBlobFileSystem.java:437)
>       at org.apache.hadoop.fs.FileSystem.isFile(FileSystem.java:1717)
>       at org.apache.spark.util.Utils$.fetchHcfsFile(Utils.scala:747)
>       at org.apache.spark.util.Utils$.doFetchFile(Utils.scala:724)
>       at org.apache.spark.util.Utils$.fetchFile(Utils.scala:496)
>       at 
> org.apache.spark.executor.Executor.$anonfun$updateDependencies$7(Executor.scala:812)
>       at 
> org.apache.spark.executor.Executor.$anonfun$updateDependencies$7$adapted(Executor.scala:803)
>       at 
> scala.collection.TraversableLike$WithFilter.$anonfun$foreach$1(TraversableLike.scala:792)
>       at 
> scala.collection.mutable.HashMap.$anonfun$foreach$1(HashMap.scala:149)
>       at scala.collection.mutable.HashTable.foreachEntry(HashTable.scala:237)
>       at scala.collection.mutable.HashTable.foreachEntry$(HashTable.scala:230)
>       at scala.collection.mutable.HashMap.foreachEntry(HashMap.scala:44)
>       at scala.collection.mutable.HashMap.foreach(HashMap.scala:149)
>       at 
> scala.collection.TraversableLike$WithFilter.foreach(TraversableLike.scala:791)
>       at 
> org.apache.spark.executor.Executor.org$apache$spark$executor$Executor$$updateDependencies(Executor.scala:803)
>       at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:375)
>       at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
>       at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
>       at java.lang.Thread.run(Thread.java:748){noformat}
>  CC [~mackrorysd], [[email protected]]



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to