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

Steve Loughran commented on HADOOP-13786:
-----------------------------------------

Looks like the latest code is failing in {{TestStagingMRJob}} when there's no 
network as it fails to bond to the test object store. Even if there's binding 
to the mock FS, looks like something is also trying to talk to the real one

{code}

"Finalizer" #3 daemon prio=8 os_prio=31 tid=0x00007fca50035000 nid=0x3803 in 
Object.wait() [0x0000700005281000]
   java.lang.Thread.State: WAITING (on object monitor)
        at java.lang.Object.wait(Native Method)
        - waiting on <0x00000007800339c8> (a java.lang.ref.ReferenceQueue$Lock)
        at java.lang.ref.ReferenceQueue.remove(ReferenceQueue.java:143)
        - locked <0x00000007800339c8> (a java.lang.ref.ReferenceQueue$Lock)
        at java.lang.ref.ReferenceQueue.remove(ReferenceQueue.java:164)
        at java.lang.ref.Finalizer$FinalizerThread.run(Finalizer.java:209)

"Reference Handler" #2 daemon prio=10 os_prio=31 tid=0x00007fca4e80f000 
nid=0x3603 in Object.wait() [0x000070000517e000]
   java.lang.Thread.State: WAITING (on object monitor)
        at java.lang.Object.wait(Native Method)
        - waiting on <0x0000000780033b80> (a java.lang.ref.Reference$Lock)
        at java.lang.Object.wait(Object.java:502)
        at java.lang.ref.Reference.tryHandlePending(Reference.java:191)
        - locked <0x0000000780033b80> (a java.lang.ref.Reference$Lock)
        at java.lang.ref.Reference$ReferenceHandler.run(Reference.java:153)

"main" #1 prio=5 os_prio=31 tid=0x00007fca4d002000 nid=0x1e03 waiting on 
condition [0x000070000475e000]
   java.lang.Thread.State: TIMED_WAITING (sleeping)
        at java.lang.Thread.sleep(Native Method)
        at 
com.amazonaws.http.AmazonHttpClient$RequestExecutor.doPauseBeforeRetry(AmazonHttpClient.java:1654)
        at 
com.amazonaws.http.AmazonHttpClient$RequestExecutor.pauseBeforeRetry(AmazonHttpClient.java:1628)
        at 
com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeOneRequest(AmazonHttpClient.java:1139)
        at 
com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeHelper(AmazonHttpClient.java:1035)
        at 
com.amazonaws.http.AmazonHttpClient$RequestExecutor.doExecute(AmazonHttpClient.java:747)
        at 
com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeWithTimer(AmazonHttpClient.java:721)
        at 
com.amazonaws.http.AmazonHttpClient$RequestExecutor.execute(AmazonHttpClient.java:704)
        at 
com.amazonaws.http.AmazonHttpClient$RequestExecutor.access$500(AmazonHttpClient.java:672)
        at 
com.amazonaws.http.AmazonHttpClient$RequestExecutionBuilderImpl.execute(AmazonHttpClient.java:654)
        at 
com.amazonaws.http.AmazonHttpClient.execute(AmazonHttpClient.java:518)
        at 
com.amazonaws.services.s3.AmazonS3Client.invoke(AmazonS3Client.java:4185)
        at 
com.amazonaws.services.s3.AmazonS3Client.getBucketRegionViaHeadRequest(AmazonS3Client.java:4903)
        at 
com.amazonaws.services.s3.AmazonS3Client.fetchRegionFromCache(AmazonS3Client.java:4878)
        at 
com.amazonaws.services.s3.AmazonS3Client.invoke(AmazonS3Client.java:4170)
        at 
com.amazonaws.services.s3.AmazonS3Client.invoke(AmazonS3Client.java:4132)
        at 
com.amazonaws.services.s3.AmazonS3Client.headBucket(AmazonS3Client.java:1302)
        at 
com.amazonaws.services.s3.AmazonS3Client.doesBucketExist(AmazonS3Client.java:1259)
        at 
org.apache.hadoop.fs.s3a.S3AFileSystem.verifyBucketExists(S3AFileSystem.java:317)
        at 
org.apache.hadoop.fs.s3a.S3AFileSystem.initialize(S3AFileSystem.java:255)
        at 
org.apache.hadoop.fs.FileSystem.createFileSystem(FileSystem.java:3257)
        at org.apache.hadoop.fs.FileSystem.access$200(FileSystem.java:123)
        at 
org.apache.hadoop.fs.FileSystem$Cache.getInternal(FileSystem.java:3306)
        at org.apache.hadoop.fs.FileSystem$Cache.get(FileSystem.java:3274)
        at org.apache.hadoop.fs.FileSystem.get(FileSystem.java:476)
        at org.apache.hadoop.fs.Path.getFileSystem(Path.java:361)
        at 
org.apache.hadoop.fs.s3a.commit.staging.MockedStagingCommitter.getDestination(MockedStagingCommitter.java:69)
        at 
org.apache.hadoop.fs.s3a.commit.AbstractS3GuardCommitter.initOutput(AbstractS3GuardCommitter.java:122)
        at 
org.apache.hadoop.fs.s3a.commit.AbstractS3GuardCommitter.<init>(AbstractS3GuardCommitter.java:85)
        at 
org.apache.hadoop.fs.s3a.commit.AbstractS3GuardCommitter.<init>(AbstractS3GuardCommitter.java:114)
        at 
org.apache.hadoop.fs.s3a.commit.staging.StagingS3GuardCommitter.<init>(StagingS3GuardCommitter.java:153)
        at 
org.apache.hadoop.fs.s3a.commit.staging.MockedStagingCommitter.<init>(MockedStagingCommitter.java:49)
        at 
org.apache.hadoop.fs.s3a.commit.staging.TestStagingMRJob$S3TextOutputFormat.getOutputCommitter(TestStagingMRJob.java:94)
        - locked <0x0000000780082b60> (a 
org.apache.hadoop.fs.s3a.commit.staging.TestStagingMRJob$S3TextOutputFormat)
        at 
org.apache.hadoop.mapreduce.v2.app.MRAppMaster$3.call(MRAppMaster.java:553)
        at 
org.apache.hadoop.mapreduce.v2.app.MRAppMaster$3.call(MRAppMaster.java:535)
        at 
org.apache.hadoop.mapreduce.v2.app.MRAppMaster.callWithJobClassLoader(MRAppMaster.java:1763)
        at 
org.apache.hadoop.mapreduce.v2.app.MRAppMaster.createOutputCommitter(MRAppMaster.java:535)
        at 
org.apache.hadoop.mapreduce.v2.app.MRAppMaster.serviceInit(MRAppMaster.java:312)
        at 
org.apache.hadoop.service.AbstractService.init(AbstractService.java:163)
        - locked <0x0000000780082cb0> (a java.lang.Object)
        at 
org.apache.hadoop.mapreduce.v2.app.MRAppMaster$6.run(MRAppMaster.java:1721)
        at java.security.AccessController.doPrivileged(Native Method)
        at javax.security.auth.Subject.doAs(Subject.java:422)
        at 
org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1965)
        at 
org.apache.hadoop.mapreduce.v2.app.MRAppMaster.initAndStartAppMaster(MRAppMaster.java:1718)
        at 
org.apache.hadoop.mapreduce.v2.app.MRAppMaster.main(MRAppMaster.java:1649)
{code}

> Add S3Guard committer for zero-rename commits to consistent S3 endpoints
> ------------------------------------------------------------------------
>
>                 Key: HADOOP-13786
>                 URL: https://issues.apache.org/jira/browse/HADOOP-13786
>             Project: Hadoop Common
>          Issue Type: New Feature
>          Components: fs/s3
>    Affects Versions: HADOOP-13345
>            Reporter: Steve Loughran
>            Assignee: Steve Loughran
>         Attachments: HADOOP-13786-HADOOP-13345-001.patch, 
> HADOOP-13786-HADOOP-13345-002.patch, HADOOP-13786-HADOOP-13345-003.patch, 
> HADOOP-13786-HADOOP-13345-004.patch, HADOOP-13786-HADOOP-13345-005.patch, 
> HADOOP-13786-HADOOP-13345-006.patch, HADOOP-13786-HADOOP-13345-006.patch, 
> HADOOP-13786-HADOOP-13345-007.patch, HADOOP-13786-HADOOP-13345-009.patch, 
> HADOOP-13786-HADOOP-13345-010.patch, HADOOP-13786-HADOOP-13345-011.patch, 
> HADOOP-13786-HADOOP-13345-012.patch, HADOOP-13786-HADOOP-13345-013.patch, 
> HADOOP-13786-HADOOP-13345-015.patch, HADOOP-13786-HADOOP-13345-016.patch, 
> HADOOP-13786-HADOOP-13345-017.patch, HADOOP-13786-HADOOP-13345-018.patch, 
> HADOOP-13786-HADOOP-13345-019.patch, HADOOP-13786-HADOOP-13345-020.patch, 
> HADOOP-13786-HADOOP-13345-021.patch, HADOOP-13786-HADOOP-13345-022.patch, 
> HADOOP-13786-HADOOP-13345-023.patch, s3committer-master.zip
>
>
> A goal of this code is "support O(1) commits to S3 repositories in the 
> presence of failures". Implement it, including whatever is needed to 
> demonstrate the correctness of the algorithm. (that is, assuming that s3guard 
> provides a consistent view of the presence/absence of blobs, show that we can 
> commit directly).
> I consider ourselves free to expose the blobstore-ness of the s3 output 
> streams (ie. not visible until the close()), if we need to use that to allow 
> us to abort commit operations.



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)

---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: common-issues-h...@hadoop.apache.org

Reply via email to