[jira] [Commented] (HADOOP-16193) add extra S3A MPU test to see what happens if a file is created during the MPU

2019-08-29 Thread Thomas Demoor (Jira)


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

Thomas Demoor commented on HADOOP-16193:


[~ehiggs] is correct.

On AWS, the multipart put, the overwriting regular put and the delete might hit 
different servers, causing temporary inconsistencies. This is documented here: 
[https://docs.aws.amazon.com/AmazonS3/latest/dev/Introduction.html#ConsistencyModel
 
|https://docs.aws.amazon.com/AmazonS3/latest/dev/Introduction.html#ConsistencyModel]Evidently,
 strong consistency solves this problem.

What's the purpose of this test? Are we concerned about racing uploads on the 
same name?

 

> add extra S3A MPU test to see what happens if a file is created during the MPU
> --
>
> Key: HADOOP-16193
> URL: https://issues.apache.org/jira/browse/HADOOP-16193
> Project: Hadoop Common
>  Issue Type: Sub-task
>  Components: fs/s3
>Reporter: Steve Loughran
>Assignee: Steve Loughran
>Priority: Minor
> Fix For: 3.1.3
>
>
> Proposed extra test for the S3A MPU: if you create and then delete a file 
> while an MPU is in progress, when you finally complete the MPU the new data 
> is present.
> This verifies that the other FS operations don't somehow cancel the 
> in-progress upload, and that eventual consistency brings the latest value out.



--
This message was sent by Atlassian Jira
(v8.3.2#803003)

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



[jira] [Updated] (HADOOP-11183) Memory-based S3AOutputstream

2015-02-18 Thread Thomas Demoor (JIRA)

 [ 
https://issues.apache.org/jira/browse/HADOOP-11183?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Thomas Demoor updated HADOOP-11183:
---
Attachment: HADOOP-11183-004.patch

004.patch: a rebase of 003.patch vs trunk. Thanks, [~ste...@apache.org].

> Memory-based S3AOutputstream
> 
>
> Key: HADOOP-11183
> URL: https://issues.apache.org/jira/browse/HADOOP-11183
> Project: Hadoop Common
>  Issue Type: Sub-task
>  Components: fs/s3
>Affects Versions: 2.6.0
>Reporter: Thomas Demoor
>Assignee: Thomas Demoor
> Attachments: HADOOP-11183-004.patch, HADOOP-11183.001.patch, 
> HADOOP-11183.002.patch, HADOOP-11183.003.patch, design-comments.pdf
>
>
> Currently s3a buffers files on disk(s) before uploading. This JIRA 
> investigates adding a memory-based upload implementation.
> The motivation is evidently performance: this would be beneficial for users 
> with high network bandwidth to S3 (EC2?) or users that run Hadoop directly on 
> an S3-compatible object store (FYI: my contributions are made in name of 
> Amplidata). 



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (HADOOP-11183) Memory-based S3AOutputstream

2015-02-18 Thread Thomas Demoor (JIRA)

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

Thomas Demoor commented on HADOOP-11183:


I have some open questions: 

*Exceptions*
Server side exceptions are thrown as AmazonServiceException (which extends 
AmazonClientException) returns quite some info on toString():
{code}
public String getMessage() {
return "Status Code: " + getStatusCode() + ", "
+ "AWS Service: " + getServiceName() + ", "
+ "AWS Request ID: " + getRequestId() + ", "
+ "AWS Error Code: " + getErrorCode() + ", "
+ "AWS Error Message: " + super.getMessage();
}
{code}
The error codes are detailed here: 
http://docs.aws.amazon.com/AmazonS3/latest/API/ErrorResponses.html 

A more general AmazonClientException is thrown if the server could not be 
reached (or there is another client side problem). Do you want me to wrap the 
entire list in the link above in standard Java exceptions? I agree that typed 
exceptions are good as they provide more info, but considering the detailed 
response codes, this might not be the top priority. 

*Statistics*

Unlike hdfs, s3a passes null to the wrapping FSDataOutputStream and does the 
statistics counting itself. It counts the bytes transferred to the server 
(double counting retries, etc.) by adding listeners (from AWS lib) to the 
uploads. It also calls statistics.incrementWriteOps(1) on every part of a 
multipart upload. It thus gives an S3-centric view of the filesystem stats, not 
a Hadoop one.

The introduced S3AFastOutputStream leaves bytecounting to FSDataOutputStream 
(cfr. hdfs) and only counts a writeOp per successful create(). It thus has 
different behavior. Should I revert to the S3AOutputStream way or will we 
change that to be HDFS like (in a separate jira)? 

*Failures*

Currently, failure of a MultiPartUpload is only checked upon closing the file. 
So f.i. if the server is unreachable each part waits for the connection setup 
timeout to fail, which takes a while. Once one part has failed, we should abort 
asap. I think adding a callback to each partUpload(ListenableFuture) that sets 
an AtomicBoolean failed = true if it has failed and checking this before 
starting a partUpload allows us to throw the Exception at the start of the next 
partUpload.

> Memory-based S3AOutputstream
> 
>
> Key: HADOOP-11183
> URL: https://issues.apache.org/jira/browse/HADOOP-11183
> Project: Hadoop Common
>  Issue Type: Sub-task
>  Components: fs/s3
>Affects Versions: 2.6.0
>Reporter: Thomas Demoor
>Assignee: Thomas Demoor
> Attachments: HADOOP-11183-004.patch, HADOOP-11183.001.patch, 
> HADOOP-11183.002.patch, HADOOP-11183.003.patch, design-comments.pdf
>
>
> Currently s3a buffers files on disk(s) before uploading. This JIRA 
> investigates adding a memory-based upload implementation.
> The motivation is evidently performance: this would be beneficial for users 
> with high network bandwidth to S3 (EC2?) or users that run Hadoop directly on 
> an S3-compatible object store (FYI: my contributions are made in name of 
> Amplidata). 



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (HADOOP-11262) Enable YARN to use S3A

2015-02-18 Thread Thomas Demoor (JIRA)

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

Thomas Demoor commented on HADOOP-11262:


 Bummer, I simply copied this from the Azure codebase and didn't see any tests 
there so I assumed this was addressed elsewhere.

> Enable YARN to use S3A 
> ---
>
> Key: HADOOP-11262
> URL: https://issues.apache.org/jira/browse/HADOOP-11262
> Project: Hadoop Common
>  Issue Type: Sub-task
>  Components: fs/s3
>Reporter: Thomas Demoor
>Assignee: Thomas Demoor
>  Labels: amazon, s3
> Attachments: HADOOP-11262-2.patch, HADOOP-11262-3.patch, 
> HADOOP-11262-4.patch, HADOOP-11262-5.patch, HADOOP-11262.patch
>
>
> Uses DelegateToFileSystem to expose S3A as an AbstractFileSystem.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (HADOOP-11584) s3a file block size set to 0 in getFileStatus

2015-02-18 Thread Thomas Demoor (JIRA)

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

Thomas Demoor commented on HADOOP-11584:


tests pass vs AWS EU. LGTM

> s3a file block size set to 0 in getFileStatus
> -
>
> Key: HADOOP-11584
> URL: https://issues.apache.org/jira/browse/HADOOP-11584
> Project: Hadoop Common
>  Issue Type: Sub-task
>  Components: fs/s3
>Affects Versions: 2.6.0
>Reporter: Dan Hecht
>Assignee: Brahma Reddy Battula
>Priority: Blocker
> Attachments: HADOOP-10584-003.patch, HADOOP-111584.patch, 
> HADOOP-11584-002.patch
>
>
> The consequence is that mapreduce probably is not splitting s3a files in the 
> expected way. This is similar to HADOOP-5861 (which was for s3n, though s3n 
> was passing 5G rather than 0 for block size).
> FileInputFormat.getSplits() relies on the FileStatus block size being set:
> {code}
> if (isSplitable(job, path)) {
>   long blockSize = file.getBlockSize();
>   long splitSize = computeSplitSize(blockSize, minSize, maxSize);
> {code}
> However, S3AFileSystem does not set the FileStatus block size field. From 
> S3AFileStatus.java:
> {code}
>   // Files
>   public S3AFileStatus(long length, long modification_time, Path path) {
> super(length, false, 1, 0, modification_time, path);
> isEmptyDirectory = false;
>   }
> {code}
> I think it should use S3AFileSystem.getDefaultBlockSize() for each file's 
> block size (where it's currently passing 0).



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Updated] (HADOOP-11183) Memory-based S3AOutputstream

2015-02-20 Thread Thomas Demoor (JIRA)

 [ 
https://issues.apache.org/jira/browse/HADOOP-11183?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Thomas Demoor updated HADOOP-11183:
---
Attachment: HADOOP-11183-005.patch

Marked as unstable. 

The underlying httpclient retries retriable errors and you can control it 
though fs.s3a.attempts.maximum

Did some more investigation and the dominant time it takes for failure is the 
dns failing to resolve. After that the subsequent parts fail fast in line with 
what is set in fs.s3a.establish.timeout. So the fail-fast I had in mind (and 
have implemented) seems premature optimization. Have been testing the current 
code for some time so I think we shouldn't take the risk to put fail-fast in so 
close to 2.7, I'll open up a separate jira for fail-fast.

Added site and core-default documentation. While passing by I corrected  the 
description of the connection timeouts: they are defined in milliseconds, not 
seconds.

> Memory-based S3AOutputstream
> 
>
> Key: HADOOP-11183
> URL: https://issues.apache.org/jira/browse/HADOOP-11183
> Project: Hadoop Common
>  Issue Type: Sub-task
>  Components: fs/s3
>Affects Versions: 2.6.0
>Reporter: Thomas Demoor
>Assignee: Thomas Demoor
> Attachments: HADOOP-11183-004.patch, HADOOP-11183-005.patch, 
> HADOOP-11183.001.patch, HADOOP-11183.002.patch, HADOOP-11183.003.patch, 
> design-comments.pdf
>
>
> Currently s3a buffers files on disk(s) before uploading. This JIRA 
> investigates adding a memory-based upload implementation.
> The motivation is evidently performance: this would be beneficial for users 
> with high network bandwidth to S3 (EC2?) or users that run Hadoop directly on 
> an S3-compatible object store (FYI: my contributions are made in name of 
> Amplidata). 



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Updated] (HADOOP-11183) Memory-based S3AOutputstream

2015-02-20 Thread Thomas Demoor (JIRA)

 [ 
https://issues.apache.org/jira/browse/HADOOP-11183?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Thomas Demoor updated HADOOP-11183:
---
Status: Patch Available  (was: Open)

> Memory-based S3AOutputstream
> 
>
> Key: HADOOP-11183
> URL: https://issues.apache.org/jira/browse/HADOOP-11183
> Project: Hadoop Common
>  Issue Type: Sub-task
>  Components: fs/s3
>Affects Versions: 2.6.0
>Reporter: Thomas Demoor
>Assignee: Thomas Demoor
> Attachments: HADOOP-11183-004.patch, HADOOP-11183-005.patch, 
> HADOOP-11183.001.patch, HADOOP-11183.002.patch, HADOOP-11183.003.patch, 
> design-comments.pdf
>
>
> Currently s3a buffers files on disk(s) before uploading. This JIRA 
> investigates adding a memory-based upload implementation.
> The motivation is evidently performance: this would be beneficial for users 
> with high network bandwidth to S3 (EC2?) or users that run Hadoop directly on 
> an S3-compatible object store (FYI: my contributions are made in name of 
> Amplidata). 



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (HADOOP-11183) Memory-based S3AOutputstream

2015-02-22 Thread Thomas Demoor (JIRA)

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

Thomas Demoor commented on HADOOP-11183:


I ran with the mark as unstable suggestion and implemented your suggestions.

{quote}
write()
2. If I created a 1-byte buffer, would that tail recursion trigger a stack 
overflow? If so, some checks in ctor may be good, such as just hard coded 
minimum & upping low values to it.
{quote}
No, the exector's linkedblockingqueue blocks writes once max.total.tasks are 
waiting.

{quote}
Can we have flush() do a partial write? Would that gain anything in durability 
terms? At the very least, it may make timing of writes more consistent with 
other filesystems.
{quote}
AWS has a minimum upload size of 5MB, so if the buffer is currently smaller 
 I can think of a workaround but it's messy.

{quote}
I can think of some more tests here. If S3AFastOutputStream added a counter of 
#of uploads, a test could grab that stream and verify that writes triggered it, 
flushes triggered it, 0-byte-writes didn't, close cleaned up, etc. Also be 
interesting to time & compare classic vs fast operations & print that in the 
test results.
{quote}
I agree we should add these over time. Counting async operations, is 
non-trivial however. 


> Memory-based S3AOutputstream
> 
>
> Key: HADOOP-11183
> URL: https://issues.apache.org/jira/browse/HADOOP-11183
> Project: Hadoop Common
>  Issue Type: Sub-task
>  Components: fs/s3
>Affects Versions: 2.6.0
>Reporter: Thomas Demoor
>Assignee: Thomas Demoor
> Attachments: HADOOP-11183-004.patch, HADOOP-11183-005.patch, 
> HADOOP-11183-006.patch, HADOOP-11183.001.patch, HADOOP-11183.002.patch, 
> HADOOP-11183.003.patch, design-comments.pdf
>
>
> Currently s3a buffers files on disk(s) before uploading. This JIRA 
> investigates adding a memory-based upload implementation.
> The motivation is evidently performance: this would be beneficial for users 
> with high network bandwidth to S3 (EC2?) or users that run Hadoop directly on 
> an S3-compatible object store (FYI: my contributions are made in name of 
> Amplidata). 



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Updated] (HADOOP-11183) Memory-based S3AOutputstream

2015-02-22 Thread Thomas Demoor (JIRA)

 [ 
https://issues.apache.org/jira/browse/HADOOP-11183?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Thomas Demoor updated HADOOP-11183:
---
Attachment: HADOOP-11183-006.patch

> Memory-based S3AOutputstream
> 
>
> Key: HADOOP-11183
> URL: https://issues.apache.org/jira/browse/HADOOP-11183
> Project: Hadoop Common
>  Issue Type: Sub-task
>  Components: fs/s3
>Affects Versions: 2.6.0
>Reporter: Thomas Demoor
>Assignee: Thomas Demoor
> Attachments: HADOOP-11183-004.patch, HADOOP-11183-005.patch, 
> HADOOP-11183-006.patch, HADOOP-11183.001.patch, HADOOP-11183.002.patch, 
> HADOOP-11183.003.patch, design-comments.pdf
>
>
> Currently s3a buffers files on disk(s) before uploading. This JIRA 
> investigates adding a memory-based upload implementation.
> The motivation is evidently performance: this would be beneficial for users 
> with high network bandwidth to S3 (EC2?) or users that run Hadoop directly on 
> an S3-compatible object store (FYI: my contributions are made in name of 
> Amplidata). 



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (HADOOP-11183) Memory-based S3AOutputstream

2015-02-23 Thread Thomas Demoor (JIRA)

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

Thomas Demoor commented on HADOOP-11183:


Addendum to my previous comment: to be more clear on the flush(): minimum size 
of 5MB is for a partUpload (of a on going multi-part upload). Evidently, one 
can write smaller objects through a single put. 

The reason I think it's safe to put this in 2.7 as unstable is that the 
codepath is never touched when the config flag is set to false (default). It's 
a drop in replacement for S3AOutputStream, but is only used if the user opts in.

> Memory-based S3AOutputstream
> 
>
> Key: HADOOP-11183
> URL: https://issues.apache.org/jira/browse/HADOOP-11183
> Project: Hadoop Common
>  Issue Type: Sub-task
>  Components: fs/s3
>Affects Versions: 2.6.0
>Reporter: Thomas Demoor
>Assignee: Thomas Demoor
> Attachments: HADOOP-11183-004.patch, HADOOP-11183-005.patch, 
> HADOOP-11183-006.patch, HADOOP-11183.001.patch, HADOOP-11183.002.patch, 
> HADOOP-11183.003.patch, design-comments.pdf
>
>
> Currently s3a buffers files on disk(s) before uploading. This JIRA 
> investigates adding a memory-based upload implementation.
> The motivation is evidently performance: this would be beneficial for users 
> with high network bandwidth to S3 (EC2?) or users that run Hadoop directly on 
> an S3-compatible object store (FYI: my contributions are made in name of 
> Amplidata). 



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (HADOOP-11183) Memory-based S3AOutputstream

2015-02-26 Thread Thomas Demoor (JIRA)

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

Thomas Demoor commented on HADOOP-11183:


Thanks Steve.

I agree on the 6MB multipart. Fast upload is not used by default. You still 
have to enable it in core-site.xml FastOutputStream does both single uploads 
and multipart. I added this test just to have a small multipartupload (rather 
than that commented out large multipart test somewhere) to test both 
outpustream implementations. Would you like me to force fast.upload to true?



> Memory-based S3AOutputstream
> 
>
> Key: HADOOP-11183
> URL: https://issues.apache.org/jira/browse/HADOOP-11183
> Project: Hadoop Common
>  Issue Type: Sub-task
>  Components: fs/s3
>Affects Versions: 2.6.0
>Reporter: Thomas Demoor
>Assignee: Thomas Demoor
> Attachments: HADOOP-11183-004.patch, HADOOP-11183-005.patch, 
> HADOOP-11183-006.patch, HADOOP-11183.001.patch, HADOOP-11183.002.patch, 
> HADOOP-11183.003.patch, design-comments.pdf
>
>
> Currently s3a buffers files on disk(s) before uploading. This JIRA 
> investigates adding a memory-based upload implementation.
> The motivation is evidently performance: this would be beneficial for users 
> with high network bandwidth to S3 (EC2?) or users that run Hadoop directly on 
> an S3-compatible object store (FYI: my contributions are made in name of 
> Amplidata). 



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Updated] (HADOOP-11262) Enable YARN to use S3A

2015-03-02 Thread Thomas Demoor (JIRA)

 [ 
https://issues.apache.org/jira/browse/HADOOP-11262?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Thomas Demoor updated HADOOP-11262:
---
Status: Open  (was: Patch Available)

> Enable YARN to use S3A 
> ---
>
> Key: HADOOP-11262
> URL: https://issues.apache.org/jira/browse/HADOOP-11262
> Project: Hadoop Common
>  Issue Type: Sub-task
>  Components: fs/s3
>Reporter: Thomas Demoor
>Assignee: Thomas Demoor
>  Labels: amazon, s3
> Attachments: HADOOP-11262-2.patch, HADOOP-11262-3.patch, 
> HADOOP-11262-4.patch, HADOOP-11262-5.patch, HADOOP-11262.patch
>
>
> Uses DelegateToFileSystem to expose S3A as an AbstractFileSystem.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Updated] (HADOOP-11183) Memory-based S3AOutputstream

2015-03-02 Thread Thomas Demoor (JIRA)

 [ 
https://issues.apache.org/jira/browse/HADOOP-11183?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Thomas Demoor updated HADOOP-11183:
---
Attachment: HADOOP-11183-007.patch

TestClass now forces fast upload to be true. Tests a regular 1MB upload and a 
5+1MB multi-part upload.

> Memory-based S3AOutputstream
> 
>
> Key: HADOOP-11183
> URL: https://issues.apache.org/jira/browse/HADOOP-11183
> Project: Hadoop Common
>  Issue Type: Sub-task
>  Components: fs/s3
>Affects Versions: 2.6.0
>Reporter: Thomas Demoor
>Assignee: Thomas Demoor
> Attachments: HADOOP-11183-004.patch, HADOOP-11183-005.patch, 
> HADOOP-11183-006.patch, HADOOP-11183-007.patch, HADOOP-11183.001.patch, 
> HADOOP-11183.002.patch, HADOOP-11183.003.patch, design-comments.pdf
>
>
> Currently s3a buffers files on disk(s) before uploading. This JIRA 
> investigates adding a memory-based upload implementation.
> The motivation is evidently performance: this would be beneficial for users 
> with high network bandwidth to S3 (EC2?) or users that run Hadoop directly on 
> an S3-compatible object store (FYI: my contributions are made in name of 
> Amplidata). 



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Updated] (HADOOP-11183) Memory-based S3AOutputstream

2015-03-03 Thread Thomas Demoor (JIRA)

 [ 
https://issues.apache.org/jira/browse/HADOOP-11183?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Thomas Demoor updated HADOOP-11183:
---
Attachment: HADOOP-11183-009.patch

Spurious indeed. Clarified threshold and partsize settings.

> Memory-based S3AOutputstream
> 
>
> Key: HADOOP-11183
> URL: https://issues.apache.org/jira/browse/HADOOP-11183
> Project: Hadoop Common
>  Issue Type: Sub-task
>  Components: fs/s3
>Affects Versions: 2.6.0
>Reporter: Thomas Demoor
>Assignee: Thomas Demoor
> Attachments: HADOOP-11183-004.patch, HADOOP-11183-005.patch, 
> HADOOP-11183-006.patch, HADOOP-11183-007.patch, HADOOP-11183-008.patch, 
> HADOOP-11183-009.patch, HADOOP-11183.001.patch, HADOOP-11183.002.patch, 
> HADOOP-11183.003.patch, design-comments.pdf
>
>
> Currently s3a buffers files on disk(s) before uploading. This JIRA 
> investigates adding a memory-based upload implementation.
> The motivation is evidently performance: this would be beneficial for users 
> with high network bandwidth to S3 (EC2?) or users that run Hadoop directly on 
> an S3-compatible object store (FYI: my contributions are made in name of 
> Amplidata). 



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Updated] (HADOOP-11183) Memory-based S3AOutputstream

2015-03-03 Thread Thomas Demoor (JIRA)

 [ 
https://issues.apache.org/jira/browse/HADOOP-11183?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Thomas Demoor updated HADOOP-11183:
---
Attachment: HADOOP-11183-010.patch

Rebased vs trunk

> Memory-based S3AOutputstream
> 
>
> Key: HADOOP-11183
> URL: https://issues.apache.org/jira/browse/HADOOP-11183
> Project: Hadoop Common
>  Issue Type: Sub-task
>  Components: fs/s3
>Affects Versions: 2.6.0
>Reporter: Thomas Demoor
>Assignee: Thomas Demoor
> Attachments: HADOOP-11183-004.patch, HADOOP-11183-005.patch, 
> HADOOP-11183-006.patch, HADOOP-11183-007.patch, HADOOP-11183-008.patch, 
> HADOOP-11183-009.patch, HADOOP-11183-010.patch, HADOOP-11183.001.patch, 
> HADOOP-11183.002.patch, HADOOP-11183.003.patch, design-comments.pdf
>
>
> Currently s3a buffers files on disk(s) before uploading. This JIRA 
> investigates adding a memory-based upload implementation.
> The motivation is evidently performance: this would be beneficial for users 
> with high network bandwidth to S3 (EC2?) or users that run Hadoop directly on 
> an S3-compatible object store (FYI: my contributions are made in name of 
> Amplidata). 



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (HADOOP-11183) Memory-based S3AOutputstream

2015-03-04 Thread Thomas Demoor (JIRA)

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

Thomas Demoor commented on HADOOP-11183:


Thanks, Steve.

> Memory-based S3AOutputstream
> 
>
> Key: HADOOP-11183
> URL: https://issues.apache.org/jira/browse/HADOOP-11183
> Project: Hadoop Common
>  Issue Type: Sub-task
>  Components: fs/s3
>Affects Versions: 2.6.0
>Reporter: Thomas Demoor
>Assignee: Thomas Demoor
> Fix For: 2.7.0
>
> Attachments: HADOOP-11183-004.patch, HADOOP-11183-005.patch, 
> HADOOP-11183-006.patch, HADOOP-11183-007.patch, HADOOP-11183-008.patch, 
> HADOOP-11183-009.patch, HADOOP-11183-010.patch, HADOOP-11183.001.patch, 
> HADOOP-11183.002.patch, HADOOP-11183.003.patch, design-comments.pdf
>
>
> Currently s3a buffers files on disk(s) before uploading. This JIRA 
> investigates adding a memory-based upload implementation.
> The motivation is evidently performance: this would be beneficial for users 
> with high network bandwidth to S3 (EC2?) or users that run Hadoop directly on 
> an S3-compatible object store (FYI: my contributions are made in name of 
> Amplidata). 



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Updated] (HADOOP-11670) Fix IAM instance profile auth for s3a

2015-03-05 Thread Thomas Demoor (JIRA)

 [ 
https://issues.apache.org/jira/browse/HADOOP-11670?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Thomas Demoor updated HADOOP-11670:
---
Attachment: HADOOP-11670-001.patch

Really quick fix. Tested that adding credentials to core-site still works. DID 
NOT TEST IAM.

> Fix IAM instance profile auth for s3a
> -
>
> Key: HADOOP-11670
> URL: https://issues.apache.org/jira/browse/HADOOP-11670
> Project: Hadoop Common
>  Issue Type: Sub-task
>  Components: fs/s3
>Affects Versions: 2.7.0
>Reporter: Adam Budde
> Fix For: 2.7.0
>
> Attachments: HADOOP-11670-001.patch
>
>
> One big advantage provided by the s3a filesystem is the ability to use an IAM 
> instance profile in order to authenticate when attempting to access an S3 
> bucket from an EC2 instance. This eliminates the need to deploy AWS account 
> credentials to the instance or to provide them to Hadoop via the 
> fs.s3a.awsAccessKeyId and fs.s3a.awsSecretAccessKey params.
> The patch submitted to resolve HADOOP-10714 breaks this behavior by using the 
> S3Credentials class to read the value of these two params. The change in 
> question is presented below:
> S3AFileSystem.java, lines 161-170:
> {code}
> // Try to get our credentials or just connect anonymously
> S3Credentials s3Credentials = new S3Credentials();
> s3Credentials.initialize(name, conf);
> AWSCredentialsProviderChain credentials = new AWSCredentialsProviderChain(
> new BasicAWSCredentialsProvider(s3Credentials.getAccessKey(),
> s3Credentials.getSecretAccessKey()),
> new InstanceProfileCredentialsProvider(),
> new AnonymousAWSCredentialsProvider()
> );
> {code}
> As you can see, the getAccessKey() and getSecretAccessKey() methods from the 
> S3Credentials class are now used to provide constructor arguments to 
> BasicAWSCredentialsProvider. These methods will raise an exception if the 
> fs.s3a.awsAccessKeyId or fs.s3a.awsSecretAccessKey params are missing, 
> respectively. If a user is relying on an IAM instance profile to authenticate 
> to an S3 bucket and therefore doesn't supply values for these params, they 
> will receive an exception and won't be able to access the bucket.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Created] (HADOOP-11684) Use thread pool that blocks clients

2015-03-06 Thread Thomas Demoor (JIRA)
Thomas Demoor created HADOOP-11684:
--

 Summary: Use thread pool that blocks clients
 Key: HADOOP-11684
 URL: https://issues.apache.org/jira/browse/HADOOP-11684
 Project: Hadoop Common
  Issue Type: Sub-task
Reporter: Thomas Demoor
Assignee: Thomas Demoor


Currently, if fs.s3a.max.total.tasks are queued and another (part)upload wants 
to start, a RejectedExecutionException is thrown. 

We should use a threadpool that blocks clients, nicely throtthling them, rather 
than throwing an exception. F.i. something similar to 
https://github.com/apache/incubator-s4/blob/master/subprojects/s4-comm/src/main/java/org/apache/s4/comm/staging/BlockingThreadPoolExecutorService.java





--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (HADOOP-9565) Add a Blobstore interface to add to blobstore FileSystems

2015-03-16 Thread Thomas Demoor (JIRA)

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

Thomas Demoor commented on HADOOP-9565:
---

Just got back from holiday, I'll finish up my patch and post it this week (once 
I process the dreaded holiday backlog). 



> Add a Blobstore interface to add to blobstore FileSystems
> -
>
> Key: HADOOP-9565
> URL: https://issues.apache.org/jira/browse/HADOOP-9565
> Project: Hadoop Common
>  Issue Type: Improvement
>  Components: fs, fs/s3, fs/swift
>Affects Versions: 2.6.0
>Reporter: Steve Loughran
>Assignee: Steve Loughran
> Attachments: HADOOP-9565-001.patch, HADOOP-9565-002.patch, 
> HADOOP-9565-003.patch
>
>
> We can make the fact that some {{FileSystem}} implementations are really 
> blobstores, with different atomicity and consistency guarantees, by adding a 
> {{Blobstore}} interface to add to them. 
> This could also be a place to add a {{Copy(Path,Path)}} method, assuming that 
> all blobstores implement at server-side copy operation as a substitute for 
> rename.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (HADOOP-9565) Add a Blobstore interface to add to blobstore FileSystems

2015-03-26 Thread Thomas Demoor (JIRA)

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

Thomas Demoor commented on HADOOP-9565:
---

Any ideas on how we can test the changes in other packages 
(CommandWithDestination, FileOutputCommiiter) vs an AWS bucket with maven? 
We're currently moving offices so I can't access to my test cluster so no 
extensive tests for now. Guess I'll have to wait it out.

Moving to FileContext is probably fine but I'm slightly concerned about 
external projects/ still using the old API. How many are there? 

FYI THe FileOutputCommitter has seen some patches recently (MAPREDUCE-4815, 
MAPREDUCE-6275).


> Add a Blobstore interface to add to blobstore FileSystems
> -
>
> Key: HADOOP-9565
> URL: https://issues.apache.org/jira/browse/HADOOP-9565
> Project: Hadoop Common
>  Issue Type: Improvement
>  Components: fs, fs/s3, fs/swift
>Affects Versions: 2.6.0
>Reporter: Steve Loughran
>Assignee: Steve Loughran
> Attachments: HADOOP-9565-001.patch, HADOOP-9565-002.patch, 
> HADOOP-9565-003.patch
>
>
> We can make the fact that some {{FileSystem}} implementations are really 
> blobstores, with different atomicity and consistency guarantees, by adding a 
> {{Blobstore}} interface to add to them. 
> This could also be a place to add a {{Copy(Path,Path)}} method, assuming that 
> all blobstores implement at server-side copy operation as a substitute for 
> rename.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (HADOOP-11753) TestS3AContractOpen#testOpenReadZeroByteFile fails due to negative range header

2015-03-27 Thread Thomas Demoor (JIRA)

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

Thomas Demoor commented on HADOOP-11753:


Probably against Cloudian backend.

Please see the HTTP [spec| https://tools.ietf.org/html/rfc7233#section-2.1]
{quote} An origin server MUST ignore a Range header field that contains a range 
unit it does not understand. {quote}

If you still use the [old spec | 
https://tools.ietf.org/html/rfc2616#section-14.35.1]
{quote}The recipient of a byte-range-set that includes one or more 
syntactically invalid byte-range-spec values MUST ignore the header field that 
includes that byte-range-set.{quote}

Investigated vs AWS: correct implementation, the request is served as if it 
would be a non-ranged GET. (f.i.: (0,-1) on a 0-byte object returns 0 bytes,  
(0,-1000) on a 4 byte object returns 4 bytes, ...).



> TestS3AContractOpen#testOpenReadZeroByteFile fails due to negative range 
> header
> ---
>
> Key: HADOOP-11753
> URL: https://issues.apache.org/jira/browse/HADOOP-11753
> Project: Hadoop Common
>  Issue Type: Bug
>  Components: fs/s3
>Affects Versions: 3.0.0, 2.7.0
>Reporter: Takenori Sato
>Assignee: Takenori Sato
> Attachments: HADOOP-11753-branch-2.7.001.patch
>
>
> _TestS3AContractOpen#testOpenReadZeroByteFile_ fails as follows.
> {code}
> testOpenReadZeroByteFile(org.apache.hadoop.fs.contract.s3a.TestS3AContractOpen)
>   Time elapsed: 3.312 sec  <<< ERROR!
> com.amazonaws.services.s3.model.AmazonS3Exception: Status Code: 416, AWS 
> Service: Amazon S3, AWS Request ID: A58A95E0D36811E4, AWS Error Code: 
> InvalidRange, AWS Error Message: The requested range cannot be satisfied.
>   at 
> com.amazonaws.http.AmazonHttpClient.handleErrorResponse(AmazonHttpClient.java:798)
>   at 
> com.amazonaws.http.AmazonHttpClient.executeHelper(AmazonHttpClient.java:421)
>   at 
> com.amazonaws.http.AmazonHttpClient.execute(AmazonHttpClient.java:232)
>   at 
> com.amazonaws.services.s3.AmazonS3Client.invoke(AmazonS3Client.java:3528)
>   at 
> com.amazonaws.services.s3.AmazonS3Client.getObject(AmazonS3Client.java:)
>   at 
> org.apache.hadoop.fs.s3a.S3AInputStream.reopen(S3AInputStream.java:91)
>   at 
> org.apache.hadoop.fs.s3a.S3AInputStream.openIfNeeded(S3AInputStream.java:62)
>   at org.apache.hadoop.fs.s3a.S3AInputStream.read(S3AInputStream.java:127)
>   at java.io.FilterInputStream.read(FilterInputStream.java:83)
>   at 
> org.apache.hadoop.fs.contract.AbstractContractOpenTest.testOpenReadZeroByteFile(AbstractContractOpenTest.java:66)
>   at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
>   at 
> sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)
>   at 
> sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
>   at java.lang.reflect.Method.invoke(Method.java:606)
>   at 
> org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:47)
>   at 
> org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12)
>   at 
> org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:44)
>   at 
> org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17)
>   at 
> org.junit.internal.runners.statements.RunBefores.evaluate(RunBefores.java:26)
>   at 
> org.junit.internal.runners.statements.RunAfters.evaluate(RunAfters.java:27)
>   at 
> org.junit.internal.runners.statements.FailOnTimeout$StatementThread.run(FailOnTimeout.java:74)
> {code}
> This is because the header is wrong when calling _S3AInputStream#read_ after 
> _S3AInputStream#open_.
> {code}
> Range: bytes=0--1
> * from 0 to -1
> {code}
> Tested on the latest branch-2.7.
> {quote}
> $ git log
> commit d286673c602524af08935ea132c8afd181b6e2e4
> Author: Jitendra Pandey 
> Date:   Tue Mar 24 16:17:06 2015 -0700
> {quote}



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (HADOOP-11742) mkdir by file system shell fails on an empty bucket

2015-03-27 Thread Thomas Demoor (JIRA)

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

Thomas Demoor commented on HADOOP-11742:


{quote} Then, without this fix, TestS3AContractRootDir failed as follows.{quote}
At my end, without the fix, the test passes vs AWS. 

With the fix the test passes as well so what are you fixing? Can you elaborate? 
Will have a closer look then.

> mkdir by file system shell fails on an empty bucket
> ---
>
> Key: HADOOP-11742
> URL: https://issues.apache.org/jira/browse/HADOOP-11742
> Project: Hadoop Common
>  Issue Type: Bug
>  Components: fs/s3
> Environment: CentOS 7
>Reporter: Takenori Sato
> Attachments: HADOOP-11742-branch-2.7.001.patch, 
> HADOOP-11742-branch-2.7.002.patch
>
>
> I have built the latest 2.7, and tried S3AFileSystem.
> Then found that _mkdir_ fails on an empty bucket, named *s3a* here, as 
> follows:
> {code}
> # hadoop-2.7.0-SNAPSHOT/bin/hdfs dfs -mkdir s3a://s3a/foo
> 15/03/24 03:49:35 DEBUG s3a.S3AFileSystem: Getting path status for 
> s3a://s3a/foo (foo)
> 15/03/24 03:49:36 DEBUG s3a.S3AFileSystem: Not Found: s3a://s3a/foo
> 15/03/24 03:49:36 DEBUG s3a.S3AFileSystem: Getting path status for s3a://s3a/ 
> ()
> 15/03/24 03:49:36 DEBUG s3a.S3AFileSystem: Not Found: s3a://s3a/
> mkdir: `s3a://s3a/foo': No such file or directory
> {code}
> So does _ls_.
> {code}
> # hadoop-2.7.0-SNAPSHOT/bin/hdfs dfs -ls s3a://s3a/
> 15/03/24 03:47:48 DEBUG s3a.S3AFileSystem: Getting path status for s3a://s3a/ 
> ()
> 15/03/24 03:47:48 DEBUG s3a.S3AFileSystem: Not Found: s3a://s3a/
> ls: `s3a://s3a/': No such file or directory
> {code}
> This is how it works via s3n.
> {code}
> # hadoop-2.7.0-SNAPSHOT/bin/hdfs dfs -ls s3n://s3n/
> # hadoop-2.7.0-SNAPSHOT/bin/hdfs dfs -mkdir s3n://s3n/foo
> # hadoop-2.7.0-SNAPSHOT/bin/hdfs dfs -ls s3n://s3n/
> Found 1 items
> drwxrwxrwx   -  0 1970-01-01 00:00 s3n://s3n/foo
> {code}
> The snapshot is the following:
> {quote}
> \# git branch
> \* branch-2.7
>   trunk
> \# git log
> commit 929b04ce3a4fe419dece49ed68d4f6228be214c1
> Author: Harsh J 
> Date:   Sun Mar 22 10:18:32 2015 +0530
> {quote}



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (HADOOP-11753) TestS3AContractOpen#testOpenReadZeroByteFile fails due to negative range header

2015-03-30 Thread Thomas Demoor (JIRA)

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

Thomas Demoor commented on HADOOP-11753:


Not sure we should close this. The change you propose seems harmless (the 
result is the same, the entire object is returned) and it makes the code more 
readable (one no longer needs to know the HTTP spec) so I'm OK with it (+ it 
makes your life easier). [~ste...@apache.org] what do you think?

Your other change (HADOOP-11742) is higher risk, we want to be really sure you 
don't break other backends and AWS is the standard against which we can all run 
tests, so some more justification is required.

> TestS3AContractOpen#testOpenReadZeroByteFile fails due to negative range 
> header
> ---
>
> Key: HADOOP-11753
> URL: https://issues.apache.org/jira/browse/HADOOP-11753
> Project: Hadoop Common
>  Issue Type: Bug
>  Components: fs/s3
>Affects Versions: 3.0.0, 2.7.0
>Reporter: Takenori Sato
>Assignee: Takenori Sato
> Attachments: HADOOP-11753-branch-2.7.001.patch
>
>
> _TestS3AContractOpen#testOpenReadZeroByteFile_ fails as follows.
> {code}
> testOpenReadZeroByteFile(org.apache.hadoop.fs.contract.s3a.TestS3AContractOpen)
>   Time elapsed: 3.312 sec  <<< ERROR!
> com.amazonaws.services.s3.model.AmazonS3Exception: Status Code: 416, AWS 
> Service: Amazon S3, AWS Request ID: A58A95E0D36811E4, AWS Error Code: 
> InvalidRange, AWS Error Message: The requested range cannot be satisfied.
>   at 
> com.amazonaws.http.AmazonHttpClient.handleErrorResponse(AmazonHttpClient.java:798)
>   at 
> com.amazonaws.http.AmazonHttpClient.executeHelper(AmazonHttpClient.java:421)
>   at 
> com.amazonaws.http.AmazonHttpClient.execute(AmazonHttpClient.java:232)
>   at 
> com.amazonaws.services.s3.AmazonS3Client.invoke(AmazonS3Client.java:3528)
>   at 
> com.amazonaws.services.s3.AmazonS3Client.getObject(AmazonS3Client.java:)
>   at 
> org.apache.hadoop.fs.s3a.S3AInputStream.reopen(S3AInputStream.java:91)
>   at 
> org.apache.hadoop.fs.s3a.S3AInputStream.openIfNeeded(S3AInputStream.java:62)
>   at org.apache.hadoop.fs.s3a.S3AInputStream.read(S3AInputStream.java:127)
>   at java.io.FilterInputStream.read(FilterInputStream.java:83)
>   at 
> org.apache.hadoop.fs.contract.AbstractContractOpenTest.testOpenReadZeroByteFile(AbstractContractOpenTest.java:66)
>   at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
>   at 
> sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)
>   at 
> sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
>   at java.lang.reflect.Method.invoke(Method.java:606)
>   at 
> org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:47)
>   at 
> org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12)
>   at 
> org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:44)
>   at 
> org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17)
>   at 
> org.junit.internal.runners.statements.RunBefores.evaluate(RunBefores.java:26)
>   at 
> org.junit.internal.runners.statements.RunAfters.evaluate(RunAfters.java:27)
>   at 
> org.junit.internal.runners.statements.FailOnTimeout$StatementThread.run(FailOnTimeout.java:74)
> {code}
> This is because the header is wrong when calling _S3AInputStream#read_ after 
> _S3AInputStream#open_.
> {code}
> Range: bytes=0--1
> * from 0 to -1
> {code}
> Tested on the latest branch-2.7.
> {quote}
> $ git log
> commit d286673c602524af08935ea132c8afd181b6e2e4
> Author: Jitendra Pandey 
> Date:   Tue Mar 24 16:17:06 2015 -0700
> {quote}



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Updated] (HADOOP-11684) S3a to use thread pool that blocks clients

2015-04-14 Thread Thomas Demoor (JIRA)

 [ 
https://issues.apache.org/jira/browse/HADOOP-11684?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Thomas Demoor updated HADOOP-11684:
---
Attachment: HADOOP-11684-001.patch

001.patch
The threadpool is a mashup of the code that was in S3AFileSystem and the S4 
code in the link in the description above.

This required bumping the AWS SDK dependency from version 1.7.4 to 1.7.8, where 
the constructor for TransferManager can be passed any ExecutorService instead 
of only a ThreadPool. However, I decided to upgrade the version to a very 
recent version (1.9.27), as from 1.9 onwards, the different components of the 
sdk can be imported individually. We now only include S3, resulting in much 
smaller binaries. 

BEWARE: To prove that the inclduded test fails when the rest of the patch is 
not applied one has to uncomment a line as Constants.MAX_THREADS is removed 
from the codebase by the patch. 

As a side effect, the version upgrade also fixes following bug in 
TransferManager: multiPartThreshold is now a long instead of an int.

 

> S3a to use thread pool that blocks clients
> --
>
> Key: HADOOP-11684
> URL: https://issues.apache.org/jira/browse/HADOOP-11684
> Project: Hadoop Common
>  Issue Type: Sub-task
>  Components: fs/s3
>Affects Versions: 2.7.0
>Reporter: Thomas Demoor
>Assignee: Thomas Demoor
> Attachments: HADOOP-11684-001.patch
>
>
> Currently, if fs.s3a.max.total.tasks are queued and another (part)upload 
> wants to start, a RejectedExecutionException is thrown. 
> We should use a threadpool that blocks clients, nicely throtthling them, 
> rather than throwing an exception. F.i. something similar to 
> https://github.com/apache/incubator-s4/blob/master/subprojects/s4-comm/src/main/java/org/apache/s4/comm/staging/BlockingThreadPoolExecutorService.java



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (HADOOP-11918) Listing an empty s3a root directory throws FileNotFound.

2015-05-08 Thread Thomas Demoor (JIRA)

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

Thomas Demoor commented on HADOOP-11918:


Same bug, same fix conceptually but I prefer the code in HADOOP-11918

> Listing an empty s3a root directory throws FileNotFound.
> 
>
> Key: HADOOP-11918
> URL: https://issues.apache.org/jira/browse/HADOOP-11918
> Project: Hadoop Common
>  Issue Type: Bug
>Affects Versions: 2.7.0
>Reporter: Lei (Eddy) Xu
>Assignee: Lei (Eddy) Xu
>Priority: Minor
>  Labels: BB2015-05-TBR, s3
> Attachments: HADOOP-11918.000.patch, HADOOP-11918.001.patch
>
>
> With an empty s3 bucket and run
> {code}
> $ hadoop fs -D... -ls s3a://hdfs-s3a-test/
> 15/05/04 15:21:34 WARN util.NativeCodeLoader: Unable to load native-hadoop 
> library for your platform... using builtin-java classes where applicable
> ls: `s3a://hdfs-s3a-test/': No such file or directory
> {code}



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (HADOOP-9565) Add a Blobstore interface to add to blobstore FileSystems

2015-05-11 Thread Thomas Demoor (JIRA)

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

Thomas Demoor commented on HADOOP-9565:
---

FYI, currently testing my implementation. 

> Add a Blobstore interface to add to blobstore FileSystems
> -
>
> Key: HADOOP-9565
> URL: https://issues.apache.org/jira/browse/HADOOP-9565
> Project: Hadoop Common
>  Issue Type: Improvement
>  Components: fs, fs/s3, fs/swift
>Affects Versions: 2.6.0
>Reporter: Steve Loughran
>Assignee: Steve Loughran
>  Labels: BB2015-05-TBR
> Attachments: HADOOP-9565-001.patch, HADOOP-9565-002.patch, 
> HADOOP-9565-003.patch
>
>
> We can make the fact that some {{FileSystem}} implementations are really 
> blobstores, with different atomicity and consistency guarantees, by adding a 
> {{Blobstore}} interface to add to them. 
> This could also be a place to add a {{Copy(Path,Path)}} method, assuming that 
> all blobstores implement at server-side copy operation as a substitute for 
> rename.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (HADOOP-11918) Listing an empty s3a root directory throws FileNotFound.

2015-05-11 Thread Thomas Demoor (JIRA)

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

Thomas Demoor commented on HADOOP-11918:


Not yet, focusing on HADOOP-9565 first.

> Listing an empty s3a root directory throws FileNotFound.
> 
>
> Key: HADOOP-11918
> URL: https://issues.apache.org/jira/browse/HADOOP-11918
> Project: Hadoop Common
>  Issue Type: Bug
>Affects Versions: 2.7.0
>Reporter: Lei (Eddy) Xu
>Assignee: Lei (Eddy) Xu
>Priority: Minor
>  Labels: BB2015-05-TBR, s3
> Attachments: HADOOP-11918.000.patch, HADOOP-11918.001.patch
>
>
> With an empty s3 bucket and run
> {code}
> $ hadoop fs -D... -ls s3a://hdfs-s3a-test/
> 15/05/04 15:21:34 WARN util.NativeCodeLoader: Unable to load native-hadoop 
> library for your platform... using builtin-java classes where applicable
> ls: `s3a://hdfs-s3a-test/': No such file or directory
> {code}



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (HADOOP-11262) Enable YARN to use S3A

2015-05-11 Thread Thomas Demoor (JIRA)

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

Thomas Demoor commented on HADOOP-11262:


[~jghoman],
we're indeed still planning to add these tests. You'll see some progress here 
from me or [~PieterReuse], who recently joined my team, soon (I want to wrap up 
some work on HADOOP-9565 first).



> Enable YARN to use S3A 
> ---
>
> Key: HADOOP-11262
> URL: https://issues.apache.org/jira/browse/HADOOP-11262
> Project: Hadoop Common
>  Issue Type: Sub-task
>  Components: fs/s3
>Reporter: Thomas Demoor
>Assignee: Thomas Demoor
>  Labels: amazon, s3
> Attachments: HADOOP-11262-2.patch, HADOOP-11262-3.patch, 
> HADOOP-11262-4.patch, HADOOP-11262-5.patch, HADOOP-11262.patch
>
>
> Uses DelegateToFileSystem to expose S3A as an AbstractFileSystem.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (HADOOP-11918) Listing an empty s3a root directory throws FileNotFound.

2015-05-18 Thread Thomas Demoor (JIRA)

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

Thomas Demoor commented on HADOOP-11918:


Sorry, here you go: [https://issues.apache.org/jira/browse/HADOOP-11742]

> Listing an empty s3a root directory throws FileNotFound.
> 
>
> Key: HADOOP-11918
> URL: https://issues.apache.org/jira/browse/HADOOP-11918
> Project: Hadoop Common
>  Issue Type: Bug
>Affects Versions: 2.7.0
>Reporter: Lei (Eddy) Xu
>Assignee: Lei (Eddy) Xu
>Priority: Minor
>  Labels: BB2015-05-TBR, s3
> Attachments: HADOOP-11918.000.patch, HADOOP-11918.001.patch
>
>
> With an empty s3 bucket and run
> {code}
> $ hadoop fs -D... -ls s3a://hdfs-s3a-test/
> 15/05/04 15:21:34 WARN util.NativeCodeLoader: Unable to load native-hadoop 
> library for your platform... using builtin-java classes where applicable
> ls: `s3a://hdfs-s3a-test/': No such file or directory
> {code}



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (HADOOP-12444) Consider implementing lazy seek in S3AInputStream

2016-02-19 Thread Thomas Demoor (JIRA)

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

Thomas Demoor commented on HADOOP-12444:


Ran tests: get through readFully fails: HTTP 416- InvalidRange. 

Needs further investigation.

> Consider implementing lazy seek in S3AInputStream
> -
>
> Key: HADOOP-12444
> URL: https://issues.apache.org/jira/browse/HADOOP-12444
> Project: Hadoop Common
>  Issue Type: Sub-task
>  Components: fs/s3
>Affects Versions: 2.7.1
>Reporter: Rajesh Balamohan
>Assignee: Rajesh Balamohan
> Attachments: HADOOP-12444.1.patch, HADOOP-12444.2.patch, 
> HADOOP-12444.WIP.patch
>
>
> - Currently, "read(long position, byte[] buffer, int offset, int length)" is 
> not implemented in S3AInputStream (unlike DFSInputStream). So, 
> "readFully(long position, byte[] buffer, int offset, int length)" in 
> S3AInputStream goes through the default implementation of seek(), read(), 
> seek() in FSInputStream. 
> - However, seek() in S3AInputStream involves re-opening of connection to S3 
> everytime 
> (https://github.com/apache/hadoop/blob/trunk/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInputStream.java#L115).
>   
> - It would be good to consider having a lazy seek implementation to reduce 
> connection overheads to S3. (e.g Presto implements lazy seek. 
> https://github.com/facebook/presto/blob/master/presto-hive/src/main/java/com/facebook/presto/hive/PrestoS3FileSystem.java#L623)



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Created] (HADOOP-12878) Impersonate hosts in s3a for better data locality handling

2016-03-03 Thread Thomas Demoor (JIRA)
Thomas Demoor created HADOOP-12878:
--

 Summary: Impersonate hosts in s3a for better data locality handling
 Key: HADOOP-12878
 URL: https://issues.apache.org/jira/browse/HADOOP-12878
 Project: Hadoop Common
  Issue Type: Sub-task
Reporter: Thomas Demoor
Assignee: Thomas Demoor


Currently, {{localhost}} is passed as locality for each block, causing all 
blocks involved in job to initially target the same node (RM), before being 
moved by the scheduler (to a rack-local node). This reduces parallelism for 
jobs (with short-lived mappers). 

We should mimic Azures implementation: a config setting 
{{fs.s3a.block.location.impersonatedhost}} where the user can enter the list of 
hostnames in the cluster to return to {{getFileBlockLocations}}. 

Possible optimization: for larger systems, it might be better to return N (5?) 
random hostnames to prevent passing a huge array (the downstream code assumes 
size = O(3)).



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (HADOOP-11687) Ignore x-emc-* headers when copying an Amazon S3 object

2016-03-04 Thread Thomas Demoor (JIRA)

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

Thomas Demoor commented on HADOOP-11687:


This should also fix another bug. The current clone implementation also copies 
reponse headers (f.i. ETag, Accept-Ranges) from the source object to the new 
copy request. AWS can handle these superfluous headers but other 
implementations might not handle this well. 

Looks OK at fist glance, slight concerns:
* have we got everything? Will review in more detail later.
* we'll need to keep this up to date with every AWS version bump (check if they 
have added / changed something).

> Ignore x-emc-* headers when copying an Amazon S3 object
> ---
>
> Key: HADOOP-11687
> URL: https://issues.apache.org/jira/browse/HADOOP-11687
> Project: Hadoop Common
>  Issue Type: Improvement
>  Components: fs/s3
>Affects Versions: 2.7.0
>Reporter: Denis Jannot
> Attachments: HADOOP-11687.001.patch
>
>
> The EMC ViPR/ECS object storage platform uses proprietary headers starting by 
> x-emc-* (like Amazon does with x-amz-*).
> Headers starting by x-emc-* should be included in the signature computation, 
> but it's not done by the Amazon S3 Java SDK (it's done by the EMC S3 SDK).
> When s3a copy an object it copies all the headers, but when the object 
> includes x-emc-* headers, it generates a signature mismatch.
> Removing the x-emc-* headers from the copy would allow s3a to be compatible 
> with the EMC ViPR/ECS object storage platform.
> Removing the x-* which aren't x-amz-* headers from the copy would allow s3a 
> to be compatible with any object storage platform which is using proprietary 
> headers



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (HADOOP-13131) Add tests to verify that S3A supports SSE-S3 encryption

2016-05-19 Thread Thomas Demoor (JIRA)

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

Thomas Demoor commented on HADOOP-13131:


[~PieterReuse] ran some test on our "non-AWS s3 implementation which supports 
SSE(AES)" and things went as expected. Only the error you expected for the 
invalid algorithm was wrapped in an IOException but I noticed this is being 
addressed in another patch.

You are correct, most implementations ignore headers they do not support (if 
possible) and I assume they would simply store the object unencrypted.

> Add tests to verify that S3A supports SSE-S3 encryption
> ---
>
> Key: HADOOP-13131
> URL: https://issues.apache.org/jira/browse/HADOOP-13131
> Project: Hadoop Common
>  Issue Type: Sub-task
>  Components: fs/s3
>Affects Versions: 2.7.2
>Reporter: Steve Loughran
>Assignee: Steve Loughran
>Priority: Minor
> Attachments: HADOOP-13131-001.patch, HADOOP-13131-002.patch, 
> HADOOP-13131-003.patch, HADOOP-13131-004.patch, HADOOP-13131-005.patch
>
>   Original Estimate: 1h
>  Remaining Estimate: 1h
>
> Although S3A claims to support server-side S3 encryption (and does, if you 
> set the option), we don't have any test to verify this. Of course, as the 
> encryption is transparent, it's hard to test.
> Here's what I propose
> # a test which sets encryption = AES256; expects things to work as normal.
> # a test which sets encyption = DES and expects any operation creating a file 
> or directory to fail with a 400 "bad request" error



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

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



[jira] [Commented] (HADOOP-11684) S3a to use thread pool that blocks clients

2015-10-01 Thread Thomas Demoor (JIRA)

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

Thomas Demoor commented on HADOOP-11684:


S3a has 2 modes for uploading: 
* fs.s3a.fast.upload=false (default): S3AOutputStream.java
** files are buffered to local disk first, on fs.close() the upload to S3 is 
initiated
** similar behaviour to s3n, other 3d party filesystems
** downsides: throughput of local disk, remaining space on local disk, delayed 
start of upload 
* fs.s3a.fast.upload=true: S3AFastOutputStream.java
** Hadoop writes are buffered in memory, if written data > threshold: multipart 
is initiated, uploading multiple parts in parallel in different *threads* (as 
soon as the data is in memory)
** EMR probably does something similar
** in this mode, fs.s3a.multipart.size should be set to something like 64 or 
128MB, similar to hdfs block size.
** downsides: buffers data in memory inside JVM (~ fs.s3a.multipart.size * 
(fs.s3a.threads.max + fs.s3a.max.total.tasks) +1 ), HADOOP-12387 will improve 
memory management

In fast mode, more threads / queued parts improve parallelism but require 
additional memory buffer space. Setting max.total.tasks=1000 certainly runs the 
JVM OOM here, as do applications that write files from separate threads (with 
CallerRuns, not with Blocking Threadpool). In default mode, the threadpool is 
used by the AWS SDK TransferManager.

Indeed, the blocking threadpool is non-trivial (semaphores,...) and thus 
higher-risk. Is there similar code in HDFS we could inspect / reuse?


> S3a to use thread pool that blocks clients
> --
>
> Key: HADOOP-11684
> URL: https://issues.apache.org/jira/browse/HADOOP-11684
> Project: Hadoop Common
>  Issue Type: Sub-task
>  Components: fs/s3
>Affects Versions: 2.7.0
>Reporter: Thomas Demoor
>Assignee: Thomas Demoor
> Attachments: HADOOP-11684-001.patch, HADOOP-11684-002.patch, 
> HADOOP-11684-003.patch
>
>
> Currently, if fs.s3a.max.total.tasks are queued and another (part)upload 
> wants to start, a RejectedExecutionException is thrown. 
> We should use a threadpool that blocks clients, nicely throtthling them, 
> rather than throwing an exception. F.i. something similar to 
> https://github.com/apache/incubator-s4/blob/master/subprojects/s4-comm/src/main/java/org/apache/s4/comm/staging/BlockingThreadPoolExecutorService.java



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (HADOOP-12444) Consider implementing lazy seek in S3AInputStream

2015-10-15 Thread Thomas Demoor (JIRA)

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

Thomas Demoor commented on HADOOP-12444:


Hi [~rajesh.balamohan], that performance improvement number looks nice. It's on 
my review list but I'm currently swamped. Will try to get it done in the coming 
weeks.



> Consider implementing lazy seek in S3AInputStream
> -
>
> Key: HADOOP-12444
> URL: https://issues.apache.org/jira/browse/HADOOP-12444
> Project: Hadoop Common
>  Issue Type: Sub-task
>  Components: fs/s3
>Affects Versions: 2.7.1
>Reporter: Rajesh Balamohan
>Assignee: Rajesh Balamohan
> Attachments: HADOOP-12444.1.patch, HADOOP-12444.WIP.patch
>
>
> - Currently, "read(long position, byte[] buffer, int offset, int length)" is 
> not implemented in S3AInputStream (unlike DFSInputStream). So, 
> "readFully(long position, byte[] buffer, int offset, int length)" in 
> S3AInputStream goes through the default implementation of seek(), read(), 
> seek() in FSInputStream. 
> - However, seek() in S3AInputStream involves re-opening of connection to S3 
> everytime 
> (https://github.com/apache/hadoop/blob/trunk/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInputStream.java#L115).
>   
> - It would be good to consider having a lazy seek implementation to reduce 
> connection overheads to S3. (e.g Presto implements lazy seek. 
> https://github.com/facebook/presto/blob/master/presto-hive/src/main/java/com/facebook/presto/hive/PrestoS3FileSystem.java#L623)



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (HADOOP-12496) AWS SDK version

2015-10-21 Thread Thomas Demoor (JIRA)

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

Thomas Demoor commented on HADOOP-12496:


Hi Yongjia,

HADOOP-12269 upgraded aws-sdk-s3 to 1.10.6

> AWS SDK version
> ---
>
> Key: HADOOP-12496
> URL: https://issues.apache.org/jira/browse/HADOOP-12496
> Project: Hadoop Common
>  Issue Type: Improvement
>  Components: fs/s3
>Reporter: Yongjia Wang
>
> hadoop-aws jar still depends on the very old 1.7.4 version of aws-java-sdk.
> In newer versions of SDK, there is incompatible API changes that leads to the 
> following error when trying to use the S3A class and newer versions of sdk 
> presents.
> This is because S3A is calling the method with "int" as the parameter type 
> while the new SDK is expecting "long". This makes it impossible to use 
> kinesis + s3a in the same process.
> It would be very helpful to upgrade hadoop-awas's aws-sdk version.
> java.lang.NoSuchMethodError: 
> com.amazonaws.services.s3.transfer.TransferManagerConfiguration.setMultipartUploadThreshold(I)V
>   at 
> org.apache.hadoop.fs.s3a.S3AFileSystem.initialize(S3AFileSystem.java:285)
>   at 
> org.apache.hadoop.fs.FileSystem.createFileSystem(FileSystem.java:2653)
>   at org.apache.hadoop.fs.FileSystem.access$200(FileSystem.java:92)
>   at 
> org.apache.hadoop.fs.FileSystem$Cache.getInternal(FileSystem.java:2687)
>   at org.apache.hadoop.fs.FileSystem$Cache.get(FileSystem.java:2669)
>   at org.apache.hadoop.fs.FileSystem.get(FileSystem.java:371)
>   at org.apache.hadoop.fs.Path.getFileSystem(Path.java:295)
>   at 
> org.apache.spark.sql.execution.datasources.ResolvedDataSource$.apply(ResolvedDataSource.scala:130)
>   at org.apache.spark.sql.DataFrameReader.load(DataFrameReader.scala:114)
>   at org.apache.spark.sql.DataFrameReader.load(DataFrameReader.scala:104)
>   at $iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:29)
>   at $iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:34)
>   at $iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:36)
>   at $iwC$$iwC$$iwC$$iwC$$iwC.(:38)
>   at $iwC$$iwC$$iwC$$iwC.(:40)
>   at $iwC$$iwC$$iwC.(:42)
>   at $iwC$$iwC.(:44)
>   at $iwC.(:46)
>   at (:48)
>   at .(:52)
>   at .()
>   at .(:7)
>   at .()
>   at $print()
>   at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
>   at 
> sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)
>   at 
> sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
>   at java.lang.reflect.Method.invoke(Method.java:606)
>   at 
> org.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:1065)
>   at 
> org.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1340)
>   at 
> org.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:840)
>   at org.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:871)
>   at org.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:819)
>   at 
> org.apache.zeppelin.spark.SparkInterpreter.interpretInput(SparkInterpreter.java:655)
>   at 
> org.apache.zeppelin.spark.SparkInterpreter.interpret(SparkInterpreter.java:620)
>   at 
> org.apache.zeppelin.spark.SparkInterpreter.interpret(SparkInterpreter.java:613)
>   at 
> org.apache.zeppelin.interpreter.ClassloaderInterpreter.interpret(ClassloaderInterpreter.java:57)
>   at 
> org.apache.zeppelin.interpreter.LazyOpenInterpreter.interpret(LazyOpenInterpreter.java:93)
>   at 
> org.apache.zeppelin.interpreter.remote.RemoteInterpreterServer$InterpretJob.jobRun(RemoteInterpreterServer.java:276)
>   at org.apache.zeppelin.scheduler.Job.run(Job.java:170)
>   at 
> org.apache.zeppelin.scheduler.FIFOScheduler$1.run(FIFOScheduler.java:118)
>   at 
> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471)
>   at java.util.concurrent.FutureTask.run(FutureTask.java:262)
>   at 
> java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$201(ScheduledThreadPoolExecutor.java:178)
>   at 
> java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:292)
>   at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
>   at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
>   at java.lang.Thread.run(Thread.java:745)



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (HADOOP-10400) Incorporate new S3A FileSystem implementation

2015-10-23 Thread Thomas Demoor (JIRA)

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

Thomas Demoor commented on HADOOP-10400:


Alternatively, the last patch uploaded in HADOOP-9565 also does direct output 
commiting.

> Incorporate new S3A FileSystem implementation
> -
>
> Key: HADOOP-10400
> URL: https://issues.apache.org/jira/browse/HADOOP-10400
> Project: Hadoop Common
>  Issue Type: New Feature
>  Components: fs, fs/s3
>Affects Versions: 2.4.0
>Reporter: Jordan Mendelson
>Assignee: Jordan Mendelson
> Fix For: 2.6.0
>
> Attachments: HADOOP-10400-1.patch, HADOOP-10400-2.patch, 
> HADOOP-10400-3.patch, HADOOP-10400-4.patch, HADOOP-10400-5.patch, 
> HADOOP-10400-6.patch, HADOOP-10400-7.patch, HADOOP-10400-8-branch-2.patch, 
> HADOOP-10400-8.patch, HADOOP-10400-branch-2.patch
>
>
> The s3native filesystem has a number of limitations (some of which were 
> recently fixed by HADOOP-9454). This patch adds an s3a filesystem which uses 
> the aws-sdk instead of the jets3t library. There are a number of improvements 
> over s3native including:
> - Parallel copy (rename) support (dramatically speeds up commits on large 
> files)
> - AWS S3 explorer compatible empty directories files "xyz/" instead of 
> "xyz_$folder$" (reduces littering)
> - Ignores s3native created _$folder$ files created by s3native and other S3 
> browsing utilities
> - Supports multiple output buffer dirs to even out IO when uploading files
> - Supports IAM role-based authentication
> - Allows setting a default canned ACL for uploads (public, private, etc.)
> - Better error recovery handling
> - Should handle input seeks without having to download the whole file (used 
> for splits a lot)
> This code is a copy of https://github.com/Aloisius/hadoop-s3a with patches to 
> various pom files to get it to build against trunk. I've been using 0.0.1 in 
> production with CDH 4 for several months and CDH 5 for a few days. The 
> version here is 0.0.2 which changes around some keys to hopefully bring the 
> key name style more inline with the rest of hadoop 2.x.
> *Tunable parameters:*
> fs.s3a.access.key - Your AWS access key ID (omit for role authentication)
> fs.s3a.secret.key - Your AWS secret key (omit for role authentication)
> fs.s3a.connection.maximum - Controls how many parallel connections 
> HttpClient spawns (default: 15)
> fs.s3a.connection.ssl.enabled - Enables or disables SSL connections to S3 
> (default: true)
> fs.s3a.attempts.maximum - How many times we should retry commands on 
> transient errors (default: 10)
> fs.s3a.connection.timeout - Socket connect timeout (default: 5000)
> fs.s3a.paging.maximum - How many keys to request from S3 when doing 
> directory listings at a time (default: 5000)
> fs.s3a.multipart.size - How big (in bytes) to split a upload or copy 
> operation up into (default: 104857600)
> fs.s3a.multipart.threshold - Until a file is this large (in bytes), use 
> non-parallel upload (default: 2147483647)
> fs.s3a.acl.default - Set a canned ACL on newly created/copied objects 
> (private | public-read | public-read-write | authenticated-read | 
> log-delivery-write | bucket-owner-read | bucket-owner-full-control)
> fs.s3a.multipart.purge - True if you want to purge existing multipart 
> uploads that may not have been completed/aborted correctly (default: false)
> fs.s3a.multipart.purge.age - Minimum age in seconds of multipart uploads 
> to purge (default: 86400)
> fs.s3a.buffer.dir - Comma separated list of directories that will be used 
> to buffer file writes out of (default: uses ${hadoop.tmp.dir}/s3a )
> *Caveats*:
> Hadoop uses a standard output committer which uploads files as 
> filename.COPYING before renaming them. This can cause unnecessary performance 
> issues with S3 because it does not have a rename operation and S3 already 
> verifies uploads against an md5 that the driver sets on the upload request. 
> While this FileSystem should be significantly faster than the built-in 
> s3native driver because of parallel copy support, you may want to consider 
> setting a null output committer on our jobs to further improve performance.
> Because S3 requires the file length and MD5 to be known before a file is 
> uploaded, all output is buffered out to a temporary file first similar to the 
> s3native driver.
> Due to the lack of native rename() for S3, renaming extremely large files or 
> directories make take a while. Unfortunately, there is no way to notify 
> hadoop that progress is still being made for rename operations, so your job 
> may time out unless you increase the task timeout.
> This driver will fully ignore _$folder$ files. This was necessary so that it 
> could int

[jira] [Commented] (HADOOP-12548) read s3 creds from a file

2015-11-04 Thread Thomas Demoor (JIRA)

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

Thomas Demoor commented on HADOOP-12548:


I would like to point you to HADOOP-11670, where an earlier patch to let s3a 
use S3Credentials was reverted as that broke IAM support. 

> read s3 creds from a file
> -
>
> Key: HADOOP-12548
> URL: https://issues.apache.org/jira/browse/HADOOP-12548
> Project: Hadoop Common
>  Issue Type: New Feature
>  Components: fs/s3
>Reporter: Allen Wittenauer
>
> It would be good if we could read s3 creds from a file rather than via a java 
> property.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (HADOOP-11262) Enable YARN to use S3A

2015-11-04 Thread Thomas Demoor (JIRA)

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

Thomas Demoor commented on HADOOP-11262:


Thanks for your time reviewing this. [~cnauroth] is correct. In hindsight, 
probably should have appended "as defaultFS" to the title of this jira. Feel 
free to adjust it.

[~eddyxu], [~PieterReuse] is on holiday this week, we'll address your remarks 
next week.

> Enable YARN to use S3A 
> ---
>
> Key: HADOOP-11262
> URL: https://issues.apache.org/jira/browse/HADOOP-11262
> Project: Hadoop Common
>  Issue Type: Sub-task
>  Components: fs/s3
>Reporter: Thomas Demoor
>Assignee: Pieter Reuse
>  Labels: amazon, s3
> Attachments: HADOOP-11262-2.patch, HADOOP-11262-3.patch, 
> HADOOP-11262-4.patch, HADOOP-11262-5.patch, HADOOP-11262-6.patch, 
> HADOOP-11262-7.patch, HADOOP-11262.patch
>
>
> Uses DelegateToFileSystem to expose S3A as an AbstractFileSystem.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (HADOOP-12548) read s3 creds from a Credential Provider

2015-11-15 Thread Thomas Demoor (JIRA)

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

Thomas Demoor commented on HADOOP-12548:


Relating to HADOOP-11694: s3a phase II improvements and bugfixes. Do we want to 
make this a subtask?

> read s3 creds from a Credential Provider
> 
>
> Key: HADOOP-12548
> URL: https://issues.apache.org/jira/browse/HADOOP-12548
> Project: Hadoop Common
>  Issue Type: New Feature
>  Components: fs/s3
>Reporter: Allen Wittenauer
>Assignee: Larry McCay
> Attachments: CredentialProviderAPIforS3FS-002.pdf, 
> HADOOP-12548-01.patch, HADOOP-12548-02.patch, HADOOP-12548-03.patch, 
> HADOOP-12548-04.patch
>
>
> It would be good if we could read s3 creds from a source other than via a 
> java property/Hadoop configuration option



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (HADOOP-12632) Allow S3A to use Temporary Credentials for Auth

2015-12-09 Thread Thomas Demoor (JIRA)

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

Thomas Demoor commented on HADOOP-12632:


Hi Joe,

at first glance this seems to duplicate HADOOP-12537, which is under active 
development by [~mackrorysd]]. Could you review that and check if your use case 
is covered?

Marking as duplicate, please correct me if I'm mistaken.

> Allow S3A to use Temporary Credentials for Auth
> ---
>
> Key: HADOOP-12632
> URL: https://issues.apache.org/jira/browse/HADOOP-12632
> Project: Hadoop Common
>  Issue Type: Improvement
>  Components: fs/s3
>Affects Versions: 2.7.1
>Reporter: Joe Ellis
>Priority: Critical
> Attachments: HADOOP-12632.patch
>
>
> In order to use temporary credentials, specifically Federation Tokens in my 
> case, you must include the session token along with the access key and secret 
> key when making requests to S3. I'd like another config property that will 
> allow a session token to be specified and then passed along to AWS SDK calls.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Resolved] (HADOOP-12632) Allow S3A to use Temporary Credentials for Auth

2015-12-09 Thread Thomas Demoor (JIRA)

 [ 
https://issues.apache.org/jira/browse/HADOOP-12632?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Thomas Demoor resolved HADOOP-12632.

Resolution: Duplicate

> Allow S3A to use Temporary Credentials for Auth
> ---
>
> Key: HADOOP-12632
> URL: https://issues.apache.org/jira/browse/HADOOP-12632
> Project: Hadoop Common
>  Issue Type: Improvement
>  Components: fs/s3
>Affects Versions: 2.7.1
>Reporter: Joe Ellis
>Priority: Critical
> Attachments: HADOOP-12632.patch
>
>
> In order to use temporary credentials, specifically Federation Tokens in my 
> case, you must include the session token along with the access key and secret 
> key when making requests to S3. I'd like another config property that will 
> allow a session token to be specified and then passed along to AWS SDK calls.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (HADOOP-12671) Inconsistent configuration values and incorrect comments

2015-12-23 Thread Thomas Demoor (JIRA)

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

Thomas Demoor commented on HADOOP-12671:


Good catches, [~tianyin]!

>From inspecting the history 
>(https://github.com/apache/hadoop/commits/trunk/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java)
> most of these seem to be in since the beginning and subsequent editors 
>(including myself :() didn't spot the inconsistencies. Thanks for bringing 
>these up.

Milliseconds indeed (see 
https://github.com/aws/aws-sdk-java/blob/1.10.6/aws-java-sdk-core/src/main/java/com/amazonaws/ClientConfiguration.java#L140)

Some remarks: 
* I feel we should set {{fs.s3a.connection.establish.timeout}} and 
{{fs.s3a.connection.timeout}} to the same value (20k or 50k).
* I would do the change for {{fs.s3a.multipart.purge.age}} the other way 
around:  set to 86400 in the code. This aborts all ongoing multipartuploads 
older than the configured value as a means of "garbage collection". For people 
with slow connections 4h is too short imho. 
* FYI: The s3 documentation 
(https://github.com/apache/hadoop/blob/trunk/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md)
 also holds these values so needs to be kept in sync (state in 3 different 
places -> trouble :P)

> Inconsistent configuration values and incorrect comments
> 
>
> Key: HADOOP-12671
> URL: https://issues.apache.org/jira/browse/HADOOP-12671
> Project: Hadoop Common
>  Issue Type: Bug
>  Components: conf, documentation, fs/s3
>Affects Versions: 2.7.1, 2.6.2
>Reporter: Tianyin Xu
>Assignee: Tianyin Xu
> Attachments: HADOOP-12671.000.patch
>
>
> The two values in [core-default.xml | 
> https://hadoop.apache.org/docs/r2.7.1/hadoop-project-dist/hadoop-common/core-default.xml]
>  are wrong. 
> {{fs.s3a.multipart.purge.age}}
> {{fs.s3a.connection.timeout}}
> {{fs.s3a.connection.establish.timeout}}
> \\
> \\
> *1. {{fs.s3a.multipart.purge.age}}*
> (in both {{2.6.2}} and {{2.7.1}})
> In [core-default.xml | 
> https://hadoop.apache.org/docs/r2.7.1/hadoop-project-dist/hadoop-common/core-default.xml],
>  the value is {{86400}} ({{24}} hours), while in the code it is {{14400}} 
> ({{4}} hours).
> \\
> \\
> *2. {{fs.s3a.connection.timeout}}*
> (only appear in {{2.6.2}})
> In [core-default.xml (2.6.2) | 
> https://hadoop.apache.org/docs/r2.6.2/hadoop-project-dist/hadoop-common/core-default.xml],
>  the value is {{5000}}, while in the code it is {{5}}.
> {code}
>   // seconds until we give up on a connection to s3
>   public static final String SOCKET_TIMEOUT = "fs.s3a.connection.timeout";
>   public static final int DEFAULT_SOCKET_TIMEOUT = 5;
> {code}
> \\
> *3. {{fs.s3a.connection.establish.timeout}}*
> (only appear in {{2.7.1}})
> In [core-default.xml (2.7.1)| 
> https://hadoop.apache.org/docs/r2.7.1/hadoop-project-dist/hadoop-common/core-default.xml],
>  the value is {{5000}}, while in the code it is {{5}}.
> {code}
>   // seconds until we give up trying to establish a connection to s3
>   public static final String ESTABLISH_TIMEOUT = 
> "fs.s3a.connection.establish.timeout";
>   public static final int DEFAULT_ESTABLISH_TIMEOUT = 5;
> {code}
> \\
> btw, the code comments are wrong! The two parameters are in the unit of 
> *milliseconds* instead of *seconds*...
> {code}
> -  // seconds until we give up on a connection to s3
> +  // milliseconds until we give up on a connection to s3
> ...
> -  // seconds until we give up trying to establish a connection to s3
> +  // milliseconds until we give up trying to establish a connection to s3
> {code}



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (HADOOP-11262) Enable YARN to use S3A

2016-01-13 Thread Thomas Demoor (JIRA)

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

Thomas Demoor commented on HADOOP-11262:


Thank you for supporting this feature and for your reviews, advice and help 
[~eddyxu], [~cnauroth], [~ste...@apache.org], [~mackrorysd].

Ping [~jghoman]: you showed initial interest in this, simply notifying you that 
it's been merged in.

> Enable YARN to use S3A 
> ---
>
> Key: HADOOP-11262
> URL: https://issues.apache.org/jira/browse/HADOOP-11262
> Project: Hadoop Common
>  Issue Type: Sub-task
>  Components: fs/s3
>Reporter: Thomas Demoor
>Assignee: Pieter Reuse
>  Labels: amazon, s3
> Fix For: 2.8.0
>
> Attachments: HADOOP-11262-10.patch, HADOOP-11262-2.patch, 
> HADOOP-11262-3.patch, HADOOP-11262-4.patch, HADOOP-11262-5.patch, 
> HADOOP-11262-6.patch, HADOOP-11262-7.patch, HADOOP-11262-8.patch, 
> HADOOP-11262-9.patch, HADOOP-11262.patch
>
>
> Uses DelegateToFileSystem to expose S3A as an AbstractFileSystem.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Resolved] (HADOOP-12763) S3AFileSystem And Hadoop FsShell Operations

2016-02-03 Thread Thomas Demoor (JIRA)

 [ 
https://issues.apache.org/jira/browse/HADOOP-12763?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Thomas Demoor resolved HADOOP-12763.

Resolution: Duplicate

Hi Stephen, I think this has been fixed in HADOOP-11918. Closing as duplicate, 
please reach out if that doesn't resolve your issue.

> S3AFileSystem And Hadoop FsShell Operations
> ---
>
> Key: HADOOP-12763
> URL: https://issues.apache.org/jira/browse/HADOOP-12763
> Project: Hadoop Common
>  Issue Type: Bug
>  Components: fs/s3
>Affects Versions: 2.7.1
>Reporter: Stephen Montgomery
>
> Hi,
> I'm looking at the Hadoop S3A Filesystem and FS Shell commands (specifically 
> -ls and -copyFromLocal/Put).
> 1. Create S3 bucket eg test-s3a-bucket.
> 2. List bucket contents using S3A and get an error: 
> $ hadoop fs -Dfs.s3n.awsAccessKeyId=... -Dfs.s3n.awsSecretAccessKey=... 
> -Dfs.s3a.access.key=... -Dfs.s3a.secret.key=... -ls s3a://test-s3a-bucket/
> 16/02/03 16:31:13 WARN util.NativeCodeLoader: Unable to load native-hadoop 
> library for your platform... using builtin-java classes where applicable
> ls: `s3a://test-s3a-bucket/': No such file or directory
> 3. List bucket contents using S3N and get no results (fair enough):
> $ hadoop fs -Dfs.s3n.awsAccessKeyId=... -Dfs.s3n.awsSecretAccessKey=... 
> -Dfs.s3a.access.key=... -Dfs.s3a.secret.key=... -ls s3n://test-s3a-bucket/
> 16/02/03 16:32:41 WARN util.NativeCodeLoader: Unable to load native-hadoop 
> library for your platform... using builtin-java classes where applicable
> 4. Attempt to copy a file from local fs to S3A and get an error (with or 
> without the trailing slash):
> $ hadoop fs -Dfs.s3n.awsAccessKeyId=... -Dfs.s3n.awsSecretAccessKey=... 
> -Dfs.s3a.access.key=... -Dfs.s3a.secret.key=... -copyFromLocal /tmp/zz 
> s3a://test-s3a-bucket/
> 16/02/03 16:35:02 WARN util.NativeCodeLoader: Unable to load native-hadoop 
> library for your platform... using builtin-java classes where applicable
> copyFromLocal: `s3a://test-s3a-bucket/': No such file or directory
> 5. Attempt to copy a file from local fs to S3N and works:
> $ hadoop fs -Dfs.s3n.awsAccessKeyId=... -Dfs.s3n.awsSecretAccessKey=... 
> -Dfs.s3a.access.key=... -Dfs.s3a.secret.key=... -copyFromLocal /tmp/zz 
> s3n://test-s3a-bucket/
> 16/02/03 16:36:17 WARN util.NativeCodeLoader: Unable to load native-hadoop 
> library for your platform... using builtin-java classes where applicable
> 16/02/03 16:36:18 INFO s3native.NativeS3FileSystem: OutputStream for key 
> 'zz._COPYING_' writing to tempfile 
> '/tmp/hadoop-monty/s3/output-9212095517127973121.tmp'
> 16/02/03 16:36:18 INFO s3native.NativeS3FileSystem: OutputStream for key 
> 'zz._COPYING_' closed. Now beginning upload
> 16/02/03 16:36:18 INFO s3native.NativeS3FileSystem: OutputStream for key 
> 'zz._COPYING_' upload complete
> $ hadoop fs -Dfs.s3n.awsAccessKeyId=... -Dfs.s3n.awsSecretAccessKey=... 
> -Dfs.s3a.access.key=... -Dfs.s3a.secret.key=... -ls s3a://test-s3a-bucket/
> 16/02/03 16:36:44 WARN util.NativeCodeLoader: Unable to load native-hadoop 
> library for your platform... using builtin-java classes where applicable
> Found 1 items
> -rw-rw-rw-   1200 2016-02-03 16:36 s3a://test-s3a-bucket/zz
> It seems that basic filesystem operations can't be performed with an 
> empty/new bucket. I have been able to populate buckets with distcp but I 
> wonder if this is because I was copying directories instead of individual 
> files.
> I know that S3A uses AmazonS3 client and S3N uses jet3t so different 
> underlying implementations/potentially different behaviours but I mainly used 
> s3n for illustration purposes (and it looks like it's working as expected).
> Can someone confirm this behaviour. Is it expected?
> Thanks,
> Stephen



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Resolved] (HADOOP-12319) S3AFastOutputStream has no ability to apply backpressure

2016-02-06 Thread Thomas Demoor (JIRA)

 [ 
https://issues.apache.org/jira/browse/HADOOP-12319?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Thomas Demoor resolved HADOOP-12319.

Resolution: Fixed

Fixed by HADOOP-11684

> S3AFastOutputStream has no ability to apply backpressure
> 
>
> Key: HADOOP-12319
> URL: https://issues.apache.org/jira/browse/HADOOP-12319
> Project: Hadoop Common
>  Issue Type: Improvement
>  Components: fs/s3
>Affects Versions: 2.7.0
>Reporter: Colin Marc
>Priority: Critical
>
> Currently, users of S3AFastOutputStream can control memory usage with a few 
> settings: {{fs.s3a.threads.core,max}}, which control the number of active 
> uploads (specifically as arguments to a {{ThreadPoolExecutor}}), and 
> {{fs.s3a.max.total.tasks}}, which controls the size of the feeding queue for 
> the {{ThreadPoolExecutor}}.
> However, a user can get an almost *guaranteed* crash if the throughput of the 
> writing job is higher than the total S3 throughput, because there is never 
> any backpressure or blocking on calls to {{write}}.
> If {{fs.s3a.max.total.tasks}} is set high (the default is 1000), then 
> {{write}} calls will continue to add data to the queue, which can eventually 
> OOM. But if the user tries to set it lower, then writes will fail when the 
> queue is full; the {{ThreadPoolExecutor}} will reject the part with 
> {{java.util.concurrent.RejectedExecutionException}}.
> Ideally, calls to {{write}} should *block, not fail* when the queue is full, 
> so as to apply backpressure on whatever the writing process is.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (HADOOP-12292) Make use of DeleteObjects optional

2016-02-06 Thread Thomas Demoor (JIRA)

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

Thomas Demoor commented on HADOOP-12292:


Thanks [~ste...@apache.org] for adding the test and committing. Saw it this 
afternoon and launched a test run. By the time I got home, it was committed. 
For the record, the test came out fine vs EU-west1 on my end:

{{ Running org.apache.hadoop.fs.s3a.scale.TestS3ADeleteFilesOneByOne
Tests run: 2, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 666.261 sec - 
in org.apache.hadoop.fs.s3a.scale.TestS3ADeleteFilesOneByOne }}

I think it would make our life easier to backport HADOOP-11684 in branch-2. It 
fixes an obvious bug (see f.i. HADOOP-12319). Can we add a deprecation warning 
or take other actions to make it branch-2 material?

> Make use of DeleteObjects optional
> --
>
> Key: HADOOP-12292
> URL: https://issues.apache.org/jira/browse/HADOOP-12292
> Project: Hadoop Common
>  Issue Type: Sub-task
>  Components: fs/s3
>Reporter: Thomas Demoor
>Assignee: Thomas Demoor
> Fix For: 2.8.0
>
> Attachments: HADOOP-12292-001.patch, HADOOP-12292-002.patch, 
> HADOOP-12292-003.patch, HADOOP-12292-004.patch, HADOOP-12292-005.patch, 
> HADOOP-12292-branch-2-005.patch
>
>
> The {{DeleteObjectsRequest}} was not part of the initial S3 API, but was 
> added later. This patch allows one to configure s3a to replace each 
> multidelete request by consecutive single deletes. Evidently, this setting is 
> disabled by default as this causes slower deletes.
> The main motivation is to enable legacy S3-compatible object stores to make 
> the transition from s3n (which does not use multidelete) to s3a, fully 
> allowing the planned s3n deprecation.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (HADOOP-12548) read s3 creds from a Credential Provider

2016-02-13 Thread Thomas Demoor (JIRA)

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

Thomas Demoor commented on HADOOP-12548:


Hi [~lmccay],

thanks, finally someone cleaned up our initialize() mess ;) Possibly, some 
outstanding patches will no longer merge but that's a minor inconvenience, it 
needed to be done.
# My take:
** Don't think s3  is still used, the 'inode-tricks' it tries to pull-off are 
guaranteed data loss :P 
** s3n is still used but is planned to be deprecated once s3a is considered 
fully stable: [~ste...@apache.org] has been rejecting non-critical patches to 
s3n and pointing those people to s3a for a while now (recent example: 
HADOOP-12353). FYI Most work on stabilizing s3a is grouped in HADOOP-11694. 
HADOOP-9565 is also relevant (2x write performance). 
#  Good idea. Now that your patch has secured the access credentials it makes 
sense to do the same for the proxy password.


> read s3 creds from a Credential Provider
> 
>
> Key: HADOOP-12548
> URL: https://issues.apache.org/jira/browse/HADOOP-12548
> Project: Hadoop Common
>  Issue Type: New Feature
>  Components: fs/s3
>Reporter: Allen Wittenauer
>Assignee: Larry McCay
> Attachments: CredentialProviderAPIforS3FS-002.pdf, 
> HADOOP-12548-01.patch, HADOOP-12548-02.patch, HADOOP-12548-03.patch, 
> HADOOP-12548-04.patch, HADOOP-12548-05.patch, HADOOP-12548-06.patch, 
> HADOOP-12548-07.patch
>
>
> It would be good if we could read s3 creds from a source other than via a 
> java property/Hadoop configuration option



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (HADOOP-12548) read s3 creds from a Credential Provider

2016-02-13 Thread Thomas Demoor (JIRA)

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

Thomas Demoor commented on HADOOP-12548:


Right, I forgot we started using [~aw]'s magic. Thanks!

> read s3 creds from a Credential Provider
> 
>
> Key: HADOOP-12548
> URL: https://issues.apache.org/jira/browse/HADOOP-12548
> Project: Hadoop Common
>  Issue Type: New Feature
>  Components: fs/s3
>Reporter: Allen Wittenauer
>Assignee: Larry McCay
> Attachments: CredentialProviderAPIforS3FS-002.pdf, 
> HADOOP-12548-01.patch, HADOOP-12548-02.patch, HADOOP-12548-03.patch, 
> HADOOP-12548-04.patch, HADOOP-12548-05.patch, HADOOP-12548-06.patch, 
> HADOOP-12548-07.patch
>
>
> It would be good if we could read s3 creds from a source other than via a 
> java property/Hadoop configuration option



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (HADOOP-12353) S3 Native filesystem does not retry all connection failures

2016-02-13 Thread Thomas Demoor (JIRA)

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

Thomas Demoor commented on HADOOP-12353:


[~ste...@apache.org]: s3a does retry as the AWS SDK has a default exponential 
backoff retry policy that retries IO, HTTP 5xx, 
..(http://docs.aws.amazon.com/AWSJavaSDK/latest/javadoc/com/amazonaws/ClientConfiguration.html#DEFAULT_RETRY_POLICY).
 We could make this configurable.

[~raviprak]: You make a valid point but s3n is unmaintained code that's planned 
to be deprecated (people are focusing on s3a). A previous harmless-looking s3n 
patch broke things in an unexpected way hence I understand 
[~ste...@apache.org]'s point as well. He didn't -1 the patch, he simply doesn't 
want to be held responsible if it breaks things, but someone else might.

> S3 Native filesystem does not retry all connection failures
> ---
>
> Key: HADOOP-12353
> URL: https://issues.apache.org/jira/browse/HADOOP-12353
> Project: Hadoop Common
>  Issue Type: Bug
>  Components: fs/s3
>Affects Versions: 2.7.1
>Reporter: Mariusz Strzelecki
>Assignee: Mariusz Strzelecki
>Priority: Minor
> Attachments: HADOOP-12353.001.patch, HADOOP-12353.002.patch
>
>
> Current implementation of NativeS3FileSystem.java uses RetryProxy that 
> retries exceptions that may occur on network communication with S3 API, but 
> these exceptions must be exact instances of IOException:
> https://github.com/apache/hadoop/blob/master/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3native/NativeS3FileSystem.java#L349
> Our tests show that HttpClient throws IOException subclasses which are not 
> handled by Proxy.
> Additionally, not all methods that call S3 API are listed to be handled, i.e. 
> storeEmptyFile and retrieveMetadata are missing.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Assigned] (HADOOP-9565) Add a Blobstore interface to add to blobstore FileSystems

2016-02-13 Thread Thomas Demoor (JIRA)

 [ 
https://issues.apache.org/jira/browse/HADOOP-9565?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Thomas Demoor reassigned HADOOP-9565:
-

Assignee: Thomas Demoor  (was: Steve Loughran)

> Add a Blobstore interface to add to blobstore FileSystems
> -
>
> Key: HADOOP-9565
> URL: https://issues.apache.org/jira/browse/HADOOP-9565
> Project: Hadoop Common
>  Issue Type: Improvement
>  Components: fs, fs/s3, fs/swift
>Affects Versions: 2.6.0
>Reporter: Steve Loughran
>Assignee: Thomas Demoor
>  Labels: BB2015-05-TBR
> Attachments: HADOOP-9565-001.patch, HADOOP-9565-002.patch, 
> HADOOP-9565-003.patch, HADOOP-9565-004.patch
>
>
> We can make the fact that some {{FileSystem}} implementations are really 
> blobstores, with different atomicity and consistency guarantees, by adding a 
> {{Blobstore}} interface to add to them. 
> This could also be a place to add a {{Copy(Path,Path)}} method, assuming that 
> all blobstores implement at server-side copy operation as a substitute for 
> rename.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Updated] (HADOOP-9565) Add a Blobstore interface to add to blobstore FileSystems

2016-02-13 Thread Thomas Demoor (JIRA)

 [ 
https://issues.apache.org/jira/browse/HADOOP-9565?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Thomas Demoor updated HADOOP-9565:
--
Attachment: HADOOP-9565-005.patch

Adjusted distcp: removed tempfile usage for object stores that support direct 
concurrent writes. Evidently, this yields a 2x speedup . 

AFAIK we have most things covered now?


> Add a Blobstore interface to add to blobstore FileSystems
> -
>
> Key: HADOOP-9565
> URL: https://issues.apache.org/jira/browse/HADOOP-9565
> Project: Hadoop Common
>  Issue Type: Improvement
>  Components: fs, fs/s3, fs/swift
>Affects Versions: 2.6.0
>Reporter: Steve Loughran
>Assignee: Thomas Demoor
>  Labels: BB2015-05-TBR
> Attachments: HADOOP-9565-001.patch, HADOOP-9565-002.patch, 
> HADOOP-9565-003.patch, HADOOP-9565-004.patch, HADOOP-9565-005.patch
>
>
> We can make the fact that some {{FileSystem}} implementations are really 
> blobstores, with different atomicity and consistency guarantees, by adding a 
> {{Blobstore}} interface to add to them. 
> This could also be a place to add a {{Copy(Path,Path)}} method, assuming that 
> all blobstores implement at server-side copy operation as a substitute for 
> rename.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (HADOOP-9565) Add a Blobstore interface to add to blobstore FileSystems

2016-02-13 Thread Thomas Demoor (JIRA)

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

Thomas Demoor commented on HADOOP-9565:
---

Also fixed the javadoc issues (thanks [~emres]).

> Add a Blobstore interface to add to blobstore FileSystems
> -
>
> Key: HADOOP-9565
> URL: https://issues.apache.org/jira/browse/HADOOP-9565
> Project: Hadoop Common
>  Issue Type: Improvement
>  Components: fs, fs/s3, fs/swift
>Affects Versions: 2.6.0
>Reporter: Steve Loughran
>Assignee: Thomas Demoor
>  Labels: BB2015-05-TBR
> Attachments: HADOOP-9565-001.patch, HADOOP-9565-002.patch, 
> HADOOP-9565-003.patch, HADOOP-9565-004.patch, HADOOP-9565-005.patch
>
>
> We can make the fact that some {{FileSystem}} implementations are really 
> blobstores, with different atomicity and consistency guarantees, by adding a 
> {{Blobstore}} interface to add to them. 
> This could also be a place to add a {{Copy(Path,Path)}} method, assuming that 
> all blobstores implement at server-side copy operation as a substitute for 
> rename.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Assigned] (HADOOP-9565) Add a Blobstore interface to add to blobstore FileSystems

2017-01-13 Thread Thomas Demoor (JIRA)

 [ 
https://issues.apache.org/jira/browse/HADOOP-9565?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Thomas Demoor reassigned HADOOP-9565:
-

Assignee: Thomas Demoor  (was: Pieter Reuse)

> Add a Blobstore interface to add to blobstore FileSystems
> -
>
> Key: HADOOP-9565
> URL: https://issues.apache.org/jira/browse/HADOOP-9565
> Project: Hadoop Common
>  Issue Type: Improvement
>  Components: fs, fs/s3, fs/swift
>Affects Versions: 2.6.0
>Reporter: Steve Loughran
>Assignee: Thomas Demoor
> Attachments: HADOOP-9565-001.patch, HADOOP-9565-002.patch, 
> HADOOP-9565-003.patch, HADOOP-9565-004.patch, HADOOP-9565-005.patch, 
> HADOOP-9565-006.patch, HADOOP-9565-branch-2-007.patch
>
>
> We can make the fact that some {{FileSystem}} implementations are really 
> blobstores, with different atomicity and consistency guarantees, by adding a 
> {{Blobstore}} interface to add to them. 
> This could also be a place to add a {{Copy(Path,Path)}} method, assuming that 
> all blobstores implement at server-side copy operation as a substitute for 
> rename.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

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



[jira] [Commented] (HADOOP-13974) S3a CLI to support list/purge of pending multipart commits

2017-02-01 Thread Thomas Demoor (JIRA)

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

Thomas Demoor commented on HADOOP-13974:


For purging we use TransferManagers abortMultipartUploads which under the hood 
does exactly that: listMultipartuploads and then iterate that list and abort 1 
by 1.

> S3a CLI to support list/purge of pending multipart commits
> --
>
> Key: HADOOP-13974
> URL: https://issues.apache.org/jira/browse/HADOOP-13974
> Project: Hadoop Common
>  Issue Type: Sub-task
>  Components: fs/s3
>Affects Versions: HADOOP-13345
>Reporter: Steve Loughran
>
> The S3A CLI will need to be able to list and delete pending multipart 
> commits. 
> We can do the cleanup already via fs.s3a properties. The CLI will let scripts 
> stat for outstanding data (have a different exit code) and permit batch jobs 
> to explicitly trigger cleanups.
> This will become critical with the multipart committer, as there's a 
> significantly higher likelihood of commits remaining outstanding.
> We may also want to be able to enumerate/cancel all pending commits in the FS 
> tree



--
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



[jira] [Commented] (HADOOP-14028) S3A block output streams don't delete temporary files in multipart uploads

2017-02-06 Thread Thomas Demoor (JIRA)

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

Thomas Demoor commented on HADOOP-14028:


I'll do some testing tomorrow.

> S3A block output streams don't delete temporary files in multipart uploads
> --
>
> Key: HADOOP-14028
> URL: https://issues.apache.org/jira/browse/HADOOP-14028
> Project: Hadoop Common
>  Issue Type: Bug
>  Components: fs/s3
>Affects Versions: 2.8.0
> Environment: JDK 8 + ORC 1.3.0 + hadoop-aws 3.0.0-alpha2
>Reporter: Seth Fitzsimmons
>Assignee: Steve Loughran
>Priority: Critical
> Attachments: HADOOP-14028-branch-2-001.patch, 
> HADOOP-14028-branch-2.8-002.patch, HADOOP-14028-branch-2.8-003.patch, 
> HADOOP-14028-branch-2.8-004.patch
>
>
> I have `fs.s3a.fast.upload` enabled with 3.0.0-alpha2 (it's exactly what I 
> was looking for after running into the same OOM problems) and don't see it 
> cleaning up the disk-cached blocks.
> I'm generating a ~50GB file on an instance with ~6GB free when the process 
> starts. My expectation is that local copies of the blocks would be deleted 
> after those parts finish uploading, but I'm seeing more than 15 blocks in 
> /tmp (and none of them have been deleted thus far).
> I see that DiskBlock deletes temporary files when closed, but is it closed 
> after individual blocks have finished uploading or when the entire file has 
> been fully written to the FS (full upload completed, including all parts)?
> As a temporary workaround to avoid running out of space, I'm listing files, 
> sorting by atime, and deleting anything older than the first 20: `ls -ut | 
> tail -n +21 | xargs rm`
> Steve Loughran says:
> > They should be deleted as soon as the upload completes; the close() call 
> > that the AWS httpclient makes on the input stream triggers the deletion. 
> > Though there aren't tests for it, as I recall.



--
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



[jira] [Commented] (HADOOP-14071) S3a: Failed to reset the request input stream

2017-02-10 Thread Thomas Demoor (JIRA)

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

Thomas Demoor commented on HADOOP-14071:


For the ByteArrayInputStream & ByteBufferInputStream I don't think we currently 
set  {{request.getRequestClientOptions().setReadLimit}}. My understanding is 
that, based on the above, we should do this. Is that correct?


> S3a: Failed to reset the request input stream
> -
>
> Key: HADOOP-14071
> URL: https://issues.apache.org/jira/browse/HADOOP-14071
> Project: Hadoop Common
>  Issue Type: Bug
>  Components: fs/s3
>Affects Versions: 3.0.0-alpha2
>Reporter: Seth Fitzsimmons
>
> When using the patch from HADOOP-14028, I fairly consistently get {{Failed to 
> reset the request input stream}} exceptions. They're more likely to occur the 
> larger the file that's being written (70GB in the extreme case, but it needs 
> to be one file).
> {code}
> 2017-02-10 04:21:43 WARN S3ABlockOutputStream:692 - Transfer failure of block 
> FileBlock{index=416, 
> destFile=/tmp/hadoop-root/s3a/s3ablock-0416-4228067786955989475.tmp, 
> state=Upload, dataSize=11591473, limit=104857600}
> 2017-02-10 04:21:43 WARN S3AInstrumentation:777 - Closing output stream 
> statistics while data is still marked as pending upload in 
> OutputStreamStatistics{blocksSubmitted=416, blocksInQueue=0, blocksActive=0, 
> blockUploadsCompleted=416, blockUploadsFailed=3, 
> bytesPendingUpload=209747761, bytesUploaded=43317747712, blocksAllocated=416, 
> blocksReleased=416, blocksActivelyAllocated=0, 
> exceptionsInMultipartFinalize=0, transferDuration=1389936 ms, 
> queueDuration=519 ms, averageQueueTime=1 ms, totalUploadDuration=1390455 ms, 
> effectiveBandwidth=3.1153649497466657E7 bytes/s}
> at org.apache.hadoop.fs.s3a.S3AUtils.extractException(S3AUtils.java:200)
> at org.apache.hadoop.fs.s3a.S3AUtils.translateException(S3AUtils.java:128)
> Exception in thread "main" org.apache.hadoop.fs.s3a.AWSClientIOException: 
> Multi-part upload with id 
> 'Xx.ezqT5hWrY1W92GrcodCip88i8rkJiOcom2nuUAqHtb6aQX__26FYh5uYWKlRNX5vY5ktdmQWlOovsbR8CLmxUVmwFkISXxDRHeor8iH9nPhI3OkNbWJJBLrvB3xLUuLX0zvGZWo7bUrAKB6IGxA--'
>  to 2017/planet-170206.orc on 2017/planet-170206.orc: 
> com.amazonaws.ResetException: Failed to reset the request input stream; If 
> the request involves an input stream, the maximum stream buffer size can be 
> configured via request.getRequestClientOptions().setReadLimit(int): Failed to 
> reset the request input stream; If the request involves an input stream, the 
> maximum stream buffer size can be configured via 
> request.getRequestClientOptions().setReadLimit(int)
> at 
> org.apache.hadoop.fs.s3a.S3ABlockOutputStream$MultiPartUpload.waitForAllPartUploads(S3ABlockOutputStream.java:539)
> at 
> org.apache.hadoop.fs.s3a.S3ABlockOutputStream$MultiPartUpload.access$100(S3ABlockOutputStream.java:456)
> at 
> org.apache.hadoop.fs.s3a.S3ABlockOutputStream.close(S3ABlockOutputStream.java:351)
> at 
> org.apache.hadoop.fs.FSDataOutputStream$PositionCache.close(FSDataOutputStream.java:72)
> at org.apache.hadoop.fs.FSDataOutputStream.close(FSDataOutputStream.java:101)
> at org.apache.orc.impl.PhysicalFsWriter.close(PhysicalFsWriter.java:221)
> at org.apache.orc.impl.WriterImpl.close(WriterImpl.java:2827)
> at net.mojodna.osm2orc.standalone.OsmPbf2Orc.convert(OsmPbf2Orc.java:296)
> at net.mojodna.osm2orc.Osm2Orc.main(Osm2Orc.java:47)
> Caused by: com.amazonaws.ResetException: Failed to reset the request input 
> stream; If the request involves an input stream, the maximum stream buffer 
> size can be configured via request.getRequestClientOptions().setReadLimit(int)
> at 
> com.amazonaws.http.AmazonHttpClient$RequestExecutor.resetRequestInputStream(AmazonHttpClient.java:1221)
> at 
> com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeOneRequest(AmazonHttpClient.java:1042)
> at 
> com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeHelper(AmazonHttpClient.java:948)
> at 
> org.apache.hadoop.fs.s3a.SemaphoredDelegatingExecutor$CallableWithPermitRelease.call(SemaphoredDelegatingExecutor.java:222)
> at 
> com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeWithTimer(AmazonHttpClient.java:635)
> at 
> com.amazonaws.http.AmazonHttpClient$RequestExecutor.execute(AmazonHttpClient.java:618)
> at 
> com.amazonaws.http.AmazonHttpClient$RequestExecutor.doExecute(AmazonHttpClient.java:661)
> at 
> com.amazonaws.http.AmazonHttpClient$RequestExecutionBuilderImpl.execute(AmazonHttpClient.java:573)
> at com.amazonaws.http.AmazonHttpClient.execute(AmazonHttpClient.java:445)
> at com.amazonaws.services.s3.AmazonS3Client.invoke(AmazonS3Client.java:4041)
> at 
> com.amazonaws.services.s3.AmazonS3Client.doUploadPart(AmazonS3Client.java:3041)
> at 
> com.amazonaws.services.s3.Amazon

[jira] [Commented] (HADOOP-14028) S3A block output streams don't delete temporary files in multipart uploads

2017-02-15 Thread Thomas Demoor (JIRA)

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

Thomas Demoor commented on HADOOP-14028:


Not sure yet, looking at the aws sdk code but I'm still confused.

This is the comment for readlimit in {{com.amazonaws.RequestClientOptions}}
{code}
 /**
 * Used to enable mark-and-reset for
 * non-mark-and-resettable non-file input stream for up to 128K memory
 * buffering by default. Add 1 to get around an implementation quirk of
 * BufferedInputStream.
 *
 * Retries after reading {@link #DEFAULT_STREAM_BUFFER_SIZE} bytes would
 * fail to reset the underlying input stream as the mark position would
 * have been invalidated.
 *
 */
{code}
The upload / partupload code gets this from a legacy system property and puts 
it in awsreq variable but the awsreq variable doesn't seem to be used anymore.  
Code is here {{com/amazonaws/services/s3/AmazonS3Client.java:1627}} and here 
{{com/amazonaws/services/s3/AmazonS3Client.java:3130}} (sdk 1.11.86)

So don't think it's set for our inputstreams. I guess we currently use the 
default 128K+1byte?



> S3A block output streams don't delete temporary files in multipart uploads
> --
>
> Key: HADOOP-14028
> URL: https://issues.apache.org/jira/browse/HADOOP-14028
> Project: Hadoop Common
>  Issue Type: Bug
>  Components: fs/s3
>Affects Versions: 2.8.0
> Environment: JDK 8 + ORC 1.3.0 + hadoop-aws 3.0.0-alpha2
>Reporter: Seth Fitzsimmons
>Assignee: Steve Loughran
>Priority: Critical
> Attachments: HADOOP-14028-branch-2-001.patch, 
> HADOOP-14028-branch-2.8-002.patch, HADOOP-14028-branch-2.8-003.patch, 
> HADOOP-14028-branch-2.8-004.patch, HADOOP-14028-branch-2.8-005.patch
>
>
> I have `fs.s3a.fast.upload` enabled with 3.0.0-alpha2 (it's exactly what I 
> was looking for after running into the same OOM problems) and don't see it 
> cleaning up the disk-cached blocks.
> I'm generating a ~50GB file on an instance with ~6GB free when the process 
> starts. My expectation is that local copies of the blocks would be deleted 
> after those parts finish uploading, but I'm seeing more than 15 blocks in 
> /tmp (and none of them have been deleted thus far).
> I see that DiskBlock deletes temporary files when closed, but is it closed 
> after individual blocks have finished uploading or when the entire file has 
> been fully written to the FS (full upload completed, including all parts)?
> As a temporary workaround to avoid running out of space, I'm listing files, 
> sorting by atime, and deleting anything older than the first 20: `ls -ut | 
> tail -n +21 | xargs rm`
> Steve Loughran says:
> > They should be deleted as soon as the upload completes; the close() call 
> > that the AWS httpclient makes on the input stream triggers the deletion. 
> > Though there aren't tests for it, as I recall.



--
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



[jira] [Commented] (HADOOP-13282) S3 blob etags to be made visible in status/getFileChecksum() calls

2017-02-16 Thread Thomas Demoor (JIRA)

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

Thomas Demoor commented on HADOOP-13282:


ETag:
* singlepart= md5sum
* multipart = (md5sum of the concatenation of the md5sum of each part) + "-" + 
number of parts


> S3 blob etags to be made visible in status/getFileChecksum() calls
> --
>
> Key: HADOOP-13282
> URL: https://issues.apache.org/jira/browse/HADOOP-13282
> Project: Hadoop Common
>  Issue Type: Sub-task
>  Components: fs/s3
>Affects Versions: 2.9.0
>Reporter: Steve Loughran
>Priority: Minor
>
> If the etags of blobs were exported via {{getFileChecksum()}}, it'd be 
> possible to probe for a blob being in sync with a local file. Distcp could 
> use this to decide whether to skip a file or not.
> Now, there's a problem there: distcp needs source and dest filesystems to 
> implement the same algorithm. It'd only work out the box if you were copying 
> between S3 instances. There are also quirks with encryption and multipart: 
> [s3 
> docs|http://docs.aws.amazon.com/AmazonS3/latest/API/RESTCommonResponseHeaders.html].
>  At the very least, it's something which could be used when indexing the FS, 
> to check for changes later.



--
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



[jira] [Commented] (HADOOP-13946) Document how HDFS updates timestamps in the FS spec; compare with object stores

2017-03-07 Thread Thomas Demoor (JIRA)

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

Thomas Demoor commented on HADOOP-13946:


my previous remarks have been addressed. didn't find anything more in my run 
through. 
+1

> Document how HDFS updates timestamps in the FS spec; compare with object 
> stores
> ---
>
> Key: HADOOP-13946
> URL: https://issues.apache.org/jira/browse/HADOOP-13946
> Project: Hadoop Common
>  Issue Type: Improvement
>  Components: documentation, fs
>Affects Versions: 2.7.3
>Reporter: Steve Loughran
>Assignee: Steve Loughran
>Priority: Minor
> Attachments: HADOOP-13946-001.patch, HADOOP-13946-002.patch, 
> HADOOP-13946-003.patch, HADOOP-13946-004.patch
>
>
> SPARK-17159 shows that the behavior of when HDFS updates timestamps isn't 
> well documented. Document these in the FS spec.
> I'm not going to add tests for this, as it is so very dependent on FS 
> implementations, as in "POSIX filesystems may behave differently from HDFS". 
> If someone knows what happens there, their contribution is welcome.



--
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



[jira] [Commented] (HADOOP-13786) Add S3Guard committer for zero-rename commits to consistent S3 endpoints

2017-03-10 Thread Thomas Demoor (JIRA)

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

Thomas Demoor commented on HADOOP-13786:


Hi Ryan, thanks for contributing to this effort.

Both approaches have the same concept of having data persistence but postponing 
object visibility through holding back on the multipart commit. +1 on sharing 
common code and having 2 committers. 

Thoughts after my first pass, on 009.patch:

StagingCommitter:
* I can see how this is ideal in a pre s3ablockoutputstream world. Using the 
stage to disk allows code which is conceptually similar to the exisiting 
filoutputcommitter 
* +: well tested and lots of failure handling (crash during overwrite 
protection.)
* +: flexible, we could still use s3a as the wrappedCommiter for s3guard / 
consistent s3 implementations and thus not require a hdfs cluster to be around 
* -: s3client in committer itself (another threadpool, will they fight? :P)
* -: staging to local disk not essentail and can cause issues (disk out of 
space, disk throughput limitation = network is faster, large files ...) = one 
of the reasons we made the blockoutputstreams 


MagicCommiter:
* +: less moving parts (no local disk, no second committer for the metadata)
* +: s3 complexity solved by reusing s3ablockoutputstream
* -: less mileage / testing: if things break where will there be garbage?
* Saw quite some unused variables, methods, etc. but I considered this work in 
progress so /ignore
* There are some extra HEAD requests to verify that things really happened 
which I think are superfluous. For instance, 
org/apache/hadoop/fs/s3a/commit/FileCommitActions.java:78 the HEAD after PUT. 
Is this to get around eventual consistency? This is head after put, so should 
be consistent even on aws. Maybe useful for PUT overwrite but then you also 
need to check the ETag. I think this HEAD should be optional behind some config 
flag.

Varia:
* Don't think the default (hdfs) workflow has crash-during-jobcommit protection 
(fileoutputcommitter algo v2 already makes files visible in taskcommit afaik) , 
so not sure it is required for our committers.





> 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, 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



[jira] [Commented] (HADOOP-13786) Add S3Guard committer for zero-rename commits to consistent S3 endpoints

2017-03-22 Thread Thomas Demoor (JIRA)

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

Thomas Demoor commented on HADOOP-13786:


On the staging wrappedCommitter exception on s3a. Could we have a config flag 
to override that? I think that would satisfy test sanity but still enable 
consistent s3 implementations to run without deploying an hdfs cluster just for 
this.

I'd be interested in a tour, in my first review I quickly found out that the 
rabbithole is deep ;)

> 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, 
> 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



[jira] [Commented] (HADOOP-14305) S3A SSE tests won't run in parallel: Bad request in directory GetFileStatus

2017-04-19 Thread Thomas Demoor (JIRA)

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

Thomas Demoor commented on HADOOP-14305:


[~ste...@apache.org] We currently only support SSE-S3, so I fear we won't be 
able to provide additional insight on the SSE-C stuff. Will run the SSE-S3 
stuff through our testing.

> S3A SSE tests won't run in parallel: Bad request in directory GetFileStatus
> ---
>
> Key: HADOOP-14305
> URL: https://issues.apache.org/jira/browse/HADOOP-14305
> Project: Hadoop Common
>  Issue Type: Sub-task
>  Components: fs/s3, test
>Affects Versions: 2.9.0
>Reporter: Steve Loughran
>Assignee: Steve Moist
>Priority: Minor
> Attachments: HADOOP-14305-001.patch
>
>
> The S3a encryption tests all run in parallel (they were interfering with each 
> other, apparently). This adds ~1 min to the test runs.
> They should run in serial. That they fail when this is attempted due to Bad 
> Auth problems must be considered a serious problem, as is indicates issues 
> related to working with SSE encryption from hadoop



--
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



[jira] [Commented] (HADOOP-9565) Add a Blobstore interface to add to blobstore FileSystems

2017-05-04 Thread Thomas Demoor (JIRA)

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

Thomas Demoor commented on HADOOP-9565:
---

This ticket has been worked on by multiple people.

Steve made a foundation to "detect" objectstores and expose their semantics.
We added a "directoutputcommitter", I think this is now made redundant by 
HADOOP-13786 so you should definitely take that out. iirc we also made some 
distcp hacks which might need to go out.

> Add a Blobstore interface to add to blobstore FileSystems
> -
>
> Key: HADOOP-9565
> URL: https://issues.apache.org/jira/browse/HADOOP-9565
> Project: Hadoop Common
>  Issue Type: Improvement
>  Components: fs, fs/s3, fs/swift
>Affects Versions: 2.6.0
>Reporter: Steve Loughran
>Assignee: Thomas Demoor
> Attachments: HADOOP-9565-001.patch, HADOOP-9565-002.patch, 
> HADOOP-9565-003.patch, HADOOP-9565-004.patch, HADOOP-9565-005.patch, 
> HADOOP-9565-006.patch, HADOOP-9565-008.patch, HADOOP-9565-branch-2-007.patch
>
>
> We can make the fact that some {{FileSystem}} implementations are really 
> blobstores, with different atomicity and consistency guarantees, by adding a 
> {{Blobstore}} interface to add to them. 
> This could also be a place to add a {{Copy(Path,Path)}} method, assuming that 
> all blobstores implement at server-side copy operation as a substitute for 
> rename.



--
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



[jira] [Commented] (HADOOP-13695) S3A to use a thread pool for async path operations

2016-10-25 Thread Thomas Demoor (JIRA)

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

Thomas Demoor commented on HADOOP-13695:


We really like the idea of the separate threadpools per operation type.

I see multiple types of operations:
* small payload: HEAD, DELETE
* potentially big payload + O(objectsize) duration: GET, PUT
* small payload + O(objectsize) duration: PUT-COPY
* moderate payload + O(listsize) duration: LIST, MULTIDELETE

[~ste...@apache.org], I agree, finding a way to parallelize all renames 
(PUT-COPY) would achieve some of the goals we had for HADOOP-9565: close to 2x 
speedup on FileOutputCommiter.commitJob() and distCp. Related to that, we've 
also been brainstorming about an "object storage friendly" FileOutputCommitter 
based on S3 versioning. What is your thinking here?

Also, HADOOP-13600 and HADOOP-13407 should be linked tickets imho.

> S3A to use a thread pool for async path operations
> --
>
> Key: HADOOP-13695
> URL: https://issues.apache.org/jira/browse/HADOOP-13695
> Project: Hadoop Common
>  Issue Type: Sub-task
>  Components: fs/s3
>Affects Versions: 2.8.0
>Reporter: Steve Loughran
>
> S3A path operations are often slow due to directory scanning, mock directory 
> create/delete, etc. Many of these can be done asynchronously
> * because deletion is eventually consistent, deleting parent dirs after an 
> operation has returned doesn't alter the behaviour, except in the special 
> case of : operation failure.
> * scanning for paths/parents of a file in the create operation only needs to 
> complete before the close() operation instantiates the object, no need to 
> block create().
> * parallelized COPY calls would permit asynchronous rename.
> We could either use the thread pool used for block writes, or somehow isolate 
> low cost path ops (GET, DELETE) from the more expensive calls (COPY, PUT) so 
> that a thread doing basic IO doesn't block for the duration of the long op. 
> Maybe also use {{Semaphore.tryAcquire()}} and only start async work if there 
> actually is an idle thread, doing it synchronously if not. Maybe it depends 
> on the operation. path query/cleanup before/after a write is something which 
> could be scheduled as just more futures to schedule in the block write.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

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



[jira] [Commented] (HADOOP-13826) S3A Deadlock in multipart copy due to thread pool limits.

2016-11-23 Thread Thomas Demoor (JIRA)

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

Thomas Demoor commented on HADOOP-13826:


Mmmh, sorry,  I caused this when introducing the bounded pool to protect 
against OOM when streaming the upload from a memory buffer. To my defense, at 
time of implementation we were programming against aws-sdk 1.7.4 where the docs 
don't seem to mention this :( 

The bounded threadpool blocks the clients (data producers) so memory usage is 
bounded. The TransferManager has built in throttling so they do not need this 
and advocate using an unbounded pool. However, the TransferManager does not 
offer functionality to do "block-based" uploads. So for S3ABlockOutputStream we 
need a different solution. The first thing that came to my mind was two 
threadpools (1 unbounded for TransferManager, 1 bounded for Block) which have a 
shared limit for number of active (uploading) threads but would like something 
simpler if possible. Have not yet studied [~mackrorysd]'s solution in detail, 
will have a look.

> S3A Deadlock in multipart copy due to thread pool limits.
> -
>
> Key: HADOOP-13826
> URL: https://issues.apache.org/jira/browse/HADOOP-13826
> Project: Hadoop Common
>  Issue Type: Bug
>  Components: fs/s3
>Affects Versions: 2.7.3
>Reporter: Sean Mackrory
> Attachments: HADOOP-13826.001.patch, HADOOP-13826.002.patch
>
>
> In testing HIVE-15093 we have encountered deadlocks in the s3a connector. The 
> TransferManager javadocs 
> (http://docs.aws.amazon.com/AWSJavaSDK/latest/javadoc/com/amazonaws/services/s3/transfer/TransferManager.html)
>  explain how this is possible:
> {quote}It is not recommended to use a single threaded executor or a thread 
> pool with a bounded work queue as control tasks may submit subtasks that 
> can't complete until all sub tasks complete. Using an incorrectly configured 
> thread pool may cause a deadlock (I.E. the work queue is filled with control 
> tasks that can't finish until subtasks complete but subtasks can't execute 
> because the queue is filled).{quote}



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

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



[jira] [Commented] (HADOOP-13826) S3A Deadlock in multipart copy due to thread pool limits.

2016-11-23 Thread Thomas Demoor (JIRA)

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

Thomas Demoor commented on HADOOP-13826:


Yes, that is set on the AmazonS3Client, which we use both for the TM as for our 
block uploads. The threadpools would share the connections, just as would let 
them share the number of active threads (as per my previous comment).

> S3A Deadlock in multipart copy due to thread pool limits.
> -
>
> Key: HADOOP-13826
> URL: https://issues.apache.org/jira/browse/HADOOP-13826
> Project: Hadoop Common
>  Issue Type: Bug
>  Components: fs/s3
>Affects Versions: 2.7.3
>Reporter: Sean Mackrory
> Attachments: HADOOP-13826.001.patch, HADOOP-13826.002.patch
>
>
> In testing HIVE-15093 we have encountered deadlocks in the s3a connector. The 
> TransferManager javadocs 
> (http://docs.aws.amazon.com/AWSJavaSDK/latest/javadoc/com/amazonaws/services/s3/transfer/TransferManager.html)
>  explain how this is possible:
> {quote}It is not recommended to use a single threaded executor or a thread 
> pool with a bounded work queue as control tasks may submit subtasks that 
> can't complete until all sub tasks complete. Using an incorrectly configured 
> thread pool may cause a deadlock (I.E. the work queue is filled with control 
> tasks that can't finish until subtasks complete but subtasks can't execute 
> because the queue is filled).{quote}



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

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



[jira] [Commented] (HADOOP-13421) Switch to v2 of the S3 List Objects API in S3A

2016-11-29 Thread Thomas Demoor (JIRA)

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

Thomas Demoor commented on HADOOP-13421:


Thanks for reaching out [~ste...@apache.org] ([~PieterReuse] discussed with 
me). 

This API call is quite recent (May 4th 2016) so there are a lot of legacy 
systems which will not support this. Our latest release does but I think if we 
don't make this optional the majority of other s3 endpoints will brake. 

Making this optional seems relatively low cost. I quickly checked and we don't 
seem to use the {{marker}} functionality in the V1 implementation which is 
replaced by {{start-after}} in V2.

> Switch to v2 of the S3 List Objects API in S3A
> --
>
> Key: HADOOP-13421
> URL: https://issues.apache.org/jira/browse/HADOOP-13421
> Project: Hadoop Common
>  Issue Type: Sub-task
>  Components: fs/s3
>Affects Versions: 2.8.0
>Reporter: Steven K. Wong
>Priority: Minor
>
> Unlike [version 
> 1|http://docs.aws.amazon.com/AmazonS3/latest/API/RESTBucketGET.html] of the 
> S3 List Objects API, [version 
> 2|http://docs.aws.amazon.com/AmazonS3/latest/API/v2-RESTBucketGET.html] by 
> default does not fetch object owner information, which S3A doesn't need 
> anyway. By switching to v2, there will be less data to transfer/process. 
> Also, it should be more robust when listing a versioned bucket with "a large 
> number of delete markers" ([according to 
> AWS|https://aws.amazon.com/releasenotes/Java/0735652458007581]).
> Methods in S3AFileSystem that use this API include:
> * getFileStatus(Path)
> * innerDelete(Path, boolean)
> * innerListStatus(Path)
> * innerRename(Path, Path)
> Requires AWS SDK 1.10.75 or later.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

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



[jira] [Commented] (HADOOP-13826) S3A Deadlock in multipart copy due to thread pool limits.

2016-11-30 Thread Thomas Demoor (JIRA)

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

Thomas Demoor commented on HADOOP-13826:


I think [~mackrorysd]'s implementation is heading in the right direction.

Some questions / suggestions:
* The {{controlTypes}} do not have a large memory and bandwidth impact as they 
carry little payload. Consequently, I think we can allow a lot of active 
threads here and the waiting room can be unbounded. I hope this would fix the 
issues [~mackrorysd] is still encountering. (In contrast to my earlier thinking 
above, I don't think the number of active threads needs to be shared between 
the two types, it seems unlikely that {{controlTypes}} will use significant 
resources) 
* The {{subTaskTypes}} have the potential to overwhelm memory and bandwidth 
usage and should thus be run from the  bounded threadpool. We need to take care 
that all relevant classes are captured here.
* I am not 100% sure if what I propose here would eliminate all deadlocks. I do 
not understand the deadlock scenario entirely (yet) from the discussion above. 
If you would have more insight please help me out.

> S3A Deadlock in multipart copy due to thread pool limits.
> -
>
> Key: HADOOP-13826
> URL: https://issues.apache.org/jira/browse/HADOOP-13826
> Project: Hadoop Common
>  Issue Type: Bug
>  Components: fs/s3
>Affects Versions: 2.7.3
>Reporter: Sean Mackrory
>Assignee: Sean Mackrory
>Priority: Critical
> Attachments: HADOOP-13826.001.patch, HADOOP-13826.002.patch
>
>
> In testing HIVE-15093 we have encountered deadlocks in the s3a connector. The 
> TransferManager javadocs 
> (http://docs.aws.amazon.com/AWSJavaSDK/latest/javadoc/com/amazonaws/services/s3/transfer/TransferManager.html)
>  explain how this is possible:
> {quote}It is not recommended to use a single threaded executor or a thread 
> pool with a bounded work queue as control tasks may submit subtasks that 
> can't complete until all sub tasks complete. Using an incorrectly configured 
> thread pool may cause a deadlock (I.E. the work queue is filled with control 
> tasks that can't finish until subtasks complete but subtasks can't execute 
> because the queue is filled).{quote}



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

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



[jira] [Commented] (HADOOP-13695) S3A to use a thread pool for async path operations

2016-12-06 Thread Thomas Demoor (JIRA)

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

Thomas Demoor commented on HADOOP-13695:


The netflix solution is "dirty but efficient" if you can live with the 
downsides.

The other solution is I think the best way to eliminate all race conditions, 
but would require to always do multipart and expose the multipart 
complete/abort calls to the committer. For a regular PUT, once sufficient bytes 
are transferred, the server will commit the "transaction" and return 200 OK, 
you cannot cancel it. By only using multipart, we could let the committer keep 
track of all MultipartUploadIds, and then complete with the id of the "winner" 
and abort the "losing speculative copies.



> S3A to use a thread pool for async path operations
> --
>
> Key: HADOOP-13695
> URL: https://issues.apache.org/jira/browse/HADOOP-13695
> Project: Hadoop Common
>  Issue Type: Sub-task
>  Components: fs/s3
>Affects Versions: 2.8.0
>Reporter: Steve Loughran
>
> S3A path operations are often slow due to directory scanning, mock directory 
> create/delete, etc. Many of these can be done asynchronously
> * because deletion is eventually consistent, deleting parent dirs after an 
> operation has returned doesn't alter the behaviour, except in the special 
> case of : operation failure.
> * scanning for paths/parents of a file in the create operation only needs to 
> complete before the close() operation instantiates the object, no need to 
> block create().
> * parallelized COPY calls would permit asynchronous rename.
> We could either use the thread pool used for block writes, or somehow isolate 
> low cost path ops (GET, DELETE) from the more expensive calls (COPY, PUT) so 
> that a thread doing basic IO doesn't block for the duration of the long op. 
> Maybe also use {{Semaphore.tryAcquire()}} and only start async work if there 
> actually is an idle thread, doing it synchronously if not. Maybe it depends 
> on the operation. path query/cleanup before/after a write is something which 
> could be scheduled as just more futures to schedule in the block write.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

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



[jira] [Commented] (HADOOP-13826) S3A Deadlock in multipart copy due to thread pool limits.

2016-12-12 Thread Thomas Demoor (JIRA)

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

Thomas Demoor commented on HADOOP-13826:


[~mackrorysd] your last patch seems simple but I think it might do the trick. I 
like simple solutions.

{{putObject()}} uses the (unbounded) TransferManager, {{putObjectDirect()}}  
and {{uploadPart()}} use the bounded threadpool so I think the (potentially) 
memory intensive parts are nicely isolated and under control.
 
My only slight concern is that now both pools can have {{MAX_THREADS}} active. 
From my reading of the code, both threadpools cannot be doing large object PUTs 
at the same time (an instance of s3a uses the block-based uploads or the 
regular S3AOutputstream, never both at the same time). What is possible, is 
that during a large block-based upload, which is saturating the bounded 
executor, another client thread might {{rename}} a directory, invoking a lot of 
parallel copies, hence saturating the TransferManager. But copies are are not 
data-intensive (see below) so I assume this is manageable.

I like [~ste...@apache.org]'s ideas for further separating out the different 
types of operations, but have one remark: for me COPY is not similar to PUT. 
COPY is completely server-side and is thus generally much less 
resource-intensive and much quicker than PUT (the smaller your bandwidth to S3, 
the bigger the difference becomes).

> S3A Deadlock in multipart copy due to thread pool limits.
> -
>
> Key: HADOOP-13826
> URL: https://issues.apache.org/jira/browse/HADOOP-13826
> Project: Hadoop Common
>  Issue Type: Bug
>  Components: fs/s3
>Affects Versions: 2.7.3
>Reporter: Sean Mackrory
>Assignee: Sean Mackrory
>Priority: Critical
> Attachments: HADOOP-13826.001.patch, HADOOP-13826.002.patch, 
> HADOOP-13826.003.patch
>
>
> In testing HIVE-15093 we have encountered deadlocks in the s3a connector. The 
> TransferManager javadocs 
> (http://docs.aws.amazon.com/AWSJavaSDK/latest/javadoc/com/amazonaws/services/s3/transfer/TransferManager.html)
>  explain how this is possible:
> {quote}It is not recommended to use a single threaded executor or a thread 
> pool with a bounded work queue as control tasks may submit subtasks that 
> can't complete until all sub tasks complete. Using an incorrectly configured 
> thread pool may cause a deadlock (I.E. the work queue is filled with control 
> tasks that can't finish until subtasks complete but subtasks can't execute 
> because the queue is filled).{quote}



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

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



[jira] [Created] (HADOOP-13912) S3a Multipart Committer (avoid rename)

2016-12-16 Thread Thomas Demoor (JIRA)
Thomas Demoor created HADOOP-13912:
--

 Summary: S3a Multipart Committer (avoid rename)
 Key: HADOOP-13912
 URL: https://issues.apache.org/jira/browse/HADOOP-13912
 Project: Hadoop Common
  Issue Type: Sub-task
Reporter: Thomas Demoor
Assignee: Thomas Demoor


Object stores do not have an efficient rename operation, which is used by the 
Hadoop FileOutputCommitter to atomically promote the "winning" attempt out of 
the multiple (speculative) attempts to the final path. These slow job commits 
are one of the main friction points when using object stores in Hadoop.There 
have been quite some attempts at resolving this: HADOOP-9565, Apache Spark 
DirectOutputCommitters, ... but they have proven not to be robust in face of 
adversity (network partitions, ...).

The current ticket proposes to do the atomic commit by using the S3 Multipart 
API, which allows multiple concurrent uploads on the same objectname, each in 
its own "temporary space, identified by the UploadId which is returned as a 
response to InitiateMultipartUpload. Every attempt writes directly to the final 
{{outputPath}}. Data is uploaded using Put Part and as a response an ETag for 
the part is returned and stored. The CompleteMultipartUpload is postponed. 
Instead, we persist the UploadId (using a _temporary subdir or elsewhere) and 
the ETags. When a certain "job" wins {{CompleteMultipartUpload}} is called for 
each of its files using the proper list of Part ETags. 

Completing a MultipartUpload is a metadata only operation (internally in S3) 
and is thus orders of magnitude faster than the rename-based approach which 
moves all the data. 

Required work: 
* Expose the multipart initiate and complete calls in S3AOutputStream to 
S3AFilesystem 
* Use these multipart calls in a custom committer as described above. I propose 
to build on the S3ACommitter [~ste...@apache.org] is doing for HADOOP-13786




--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

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



[jira] [Commented] (HADOOP-13786) add output committer which uses s3guard for consistent commits to S3

2016-12-16 Thread Thomas Demoor (JIRA)

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

Thomas Demoor commented on HADOOP-13786:


Had my write-up ready but you beat me too actually submitting 
[~ste...@apache.org]

Filed HADOOP-13912 so we can track this separately, but I think it should go on 
top on what you have already done in the current ticket.



> add output committer which uses s3guard for consistent commits to S3
> 
>
> Key: HADOOP-13786
> URL: https://issues.apache.org/jira/browse/HADOOP-13786
> Project: Hadoop Common
>  Issue Type: Sub-task
>  Components: fs/s3
>Affects Versions: 3.0.0-alpha2
>Reporter: Steve Loughran
>Assignee: Steve Loughran
>
> 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.4#6332)

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



[jira] [Commented] (HADOOP-13912) S3a Multipart Committer (avoid rename)

2016-12-19 Thread Thomas Demoor (JIRA)

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

Thomas Demoor commented on HADOOP-13912:


Extensive write-up by [~ste...@apache.org] based on the conf call we ahd this 
week where I explained the idea:
[https://github.com/steveloughran/hadoop/blob/s3guard/HADOOP-13786-committer/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/delayed-put-commit.md]

> S3a Multipart Committer (avoid rename)
> --
>
> Key: HADOOP-13912
> URL: https://issues.apache.org/jira/browse/HADOOP-13912
> Project: Hadoop Common
>  Issue Type: Sub-task
>  Components: fs/s3
>Reporter: Thomas Demoor
>Assignee: Thomas Demoor
>
> Object stores do not have an efficient rename operation, which is used by the 
> Hadoop FileOutputCommitter to atomically promote the "winning" attempt out of 
> the multiple (speculative) attempts to the final path. These slow job commits 
> are one of the main friction points when using object stores in Hadoop.There 
> have been quite some attempts at resolving this: HADOOP-9565, Apache Spark 
> DirectOutputCommitters, ... but they have proven not to be robust in face of 
> adversity (network partitions, ...).
> The current ticket proposes to do the atomic commit by using the S3 Multipart 
> API, which allows multiple concurrent uploads on the same objectname, each in 
> its own "temporary space, identified by the UploadId which is returned as a 
> response to InitiateMultipartUpload. Every attempt writes directly to the 
> final {{outputPath}}. Data is uploaded using Put Part and as a response an 
> ETag for the part is returned and stored. The CompleteMultipartUpload is 
> postponed. Instead, we persist the UploadId (using a _temporary subdir or 
> elsewhere) and the ETags. When a certain "job" wins 
> {{CompleteMultipartUpload}} is called for each of its files using the proper 
> list of Part ETags. 
> Completing a MultipartUpload is a metadata only operation (internally in S3) 
> and is thus orders of magnitude faster than the rename-based approach which 
> moves all the data. 
> Required work: 
> * Expose the multipart initiate and complete calls in S3AOutputStream to 
> S3AFilesystem 
> * Use these multipart calls in a custom committer as described above. I 
> propose to build on the S3ACommitter [~ste...@apache.org] is doing for 
> HADOOP-13786



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

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



[jira] [Commented] (HADOOP-13946) Document how HDFS updates timestamps in the FS spec; compare with object stores

2017-01-03 Thread Thomas Demoor (JIRA)

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

Thomas Demoor commented on HADOOP-13946:


Another late review remark:
bq. The file only becomes visible at the end of the write operation; this also 
sets the creation time of the file.

On S3 an object (file) is only visible at the end of the write operation, but 
the creation / modification timestamp (Last-Modified HTTP header) is that of 
the *start* of the write operation.


> Document how HDFS updates timestamps in the FS spec; compare with object 
> stores
> ---
>
> Key: HADOOP-13946
> URL: https://issues.apache.org/jira/browse/HADOOP-13946
> Project: Hadoop Common
>  Issue Type: Improvement
>  Components: documentation, fs
>Affects Versions: 2.7.3
>Reporter: Steve Loughran
>Assignee: Steve Loughran
>Priority: Minor
> Fix For: 2.8.0, 3.0.0-alpha2
>
> Attachments: HADOOP-13946-001.patch
>
>
> SPARK-17159 shows that the behavior of when HDFS updates timestamps isn't 
> well documented. Document these in the FS spec.
> I'm not going to add tests for this, as it is so very dependent on FS 
> implementations, as in "POSIX filesystems may behave differently from HDFS". 
> If someone knows what happens there, their contribution is welcome.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

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



[jira] [Commented] (HADOOP-13946) Document how HDFS updates timestamps in the FS spec; compare with object stores

2017-01-05 Thread Thomas Demoor (JIRA)

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

Thomas Demoor commented on HADOOP-13946:


Yup, same for multipart. The timestamp of the initiate multipart upload request 
is used as creation/modification time. 

> Document how HDFS updates timestamps in the FS spec; compare with object 
> stores
> ---
>
> Key: HADOOP-13946
> URL: https://issues.apache.org/jira/browse/HADOOP-13946
> Project: Hadoop Common
>  Issue Type: Improvement
>  Components: documentation, fs
>Affects Versions: 2.7.3
>Reporter: Steve Loughran
>Assignee: Steve Loughran
>Priority: Minor
> Fix For: 2.8.0, 3.0.0-alpha2
>
> Attachments: HADOOP-13946-001.patch
>
>
> SPARK-17159 shows that the behavior of when HDFS updates timestamps isn't 
> well documented. Document these in the FS spec.
> I'm not going to add tests for this, as it is so very dependent on FS 
> implementations, as in "POSIX filesystems may behave differently from HDFS". 
> If someone knows what happens there, their contribution is welcome.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

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



[jira] [Commented] (HADOOP-13946) Document how HDFS updates timestamps in the FS spec; compare with object stores

2017-01-08 Thread Thomas Demoor (JIRA)

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

Thomas Demoor commented on HADOOP-13946:


A copy is an independent immutable object (which happens to have the same data 
contents as the source). It's creation time is that of the PUT-COPY request, 
not that of the "sibling" object.

> Document how HDFS updates timestamps in the FS spec; compare with object 
> stores
> ---
>
> Key: HADOOP-13946
> URL: https://issues.apache.org/jira/browse/HADOOP-13946
> Project: Hadoop Common
>  Issue Type: Improvement
>  Components: documentation, fs
>Affects Versions: 2.7.3
>Reporter: Steve Loughran
>Assignee: Steve Loughran
>Priority: Minor
> Fix For: 2.8.0, 3.0.0-alpha2
>
> Attachments: HADOOP-13946-001.patch, HADOOP-13946-002.patch
>
>
> SPARK-17159 shows that the behavior of when HDFS updates timestamps isn't 
> well documented. Document these in the FS spec.
> I'm not going to add tests for this, as it is so very dependent on FS 
> implementations, as in "POSIX filesystems may behave differently from HDFS". 
> If someone knows what happens there, their contribution is welcome.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

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



[jira] [Commented] (HADOOP-12878) Impersonate hosts in s3a for better data locality handling

2016-06-10 Thread Thomas Demoor (JIRA)

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

Thomas Demoor commented on HADOOP-12878:


[~cnauroth]: I assumed this would be used as follows: by having  
{{fs.wasb/s3a.block.location.impersonatedhost=host1,host2,...,host50}}  you 
would impersonate the block to be local on all nodes, the scheduler would then 
be offered options. An optimization would be to return 5 random hosts.
[~dweeks]: Thanks for your interest!
* The current s3a implementation causes scheduling delays as it declares 
localhost as the only locality for all blocks. 
* The proposed solution can indeed be suboptimal under load but it's always an 
improvement over the current situation
* I agree that using "*"  woud be easier and more correct but I think 
[~ste...@apache.org] might have a point. Testing... 
* Returning the s3 endpoint so things are always off-rack seems like an 
interesting idea. Testing...

[~PieterReuse] has verified that what I originally proposed speeds things up. 
We are checking if any of Daniel's proposals would work out as well. Expect an 
update next week.



> Impersonate hosts in s3a for better data locality handling
> --
>
> Key: HADOOP-12878
> URL: https://issues.apache.org/jira/browse/HADOOP-12878
> Project: Hadoop Common
>  Issue Type: Sub-task
>  Components: fs/s3
>Affects Versions: 2.8.0
>Reporter: Thomas Demoor
>Assignee: Thomas Demoor
>
> Currently, {{localhost}} is passed as locality for each block, causing all 
> blocks involved in job to initially target the same node (RM), before being 
> moved by the scheduler (to a rack-local node). This reduces parallelism for 
> jobs (with short-lived mappers). 
> We should mimic Azures implementation: a config setting 
> {{fs.s3a.block.location.impersonatedhost}} where the user can enter the list 
> of hostnames in the cluster to return to {{getFileBlockLocations}}. 
> Possible optimization: for larger systems, it might be better to return N 
> (5?) random hostnames to prevent passing a huge array (the downstream code 
> assumes size = O(3)).



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

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



[jira] [Commented] (HADOOP-12878) Impersonate hosts in s3a for better data locality handling

2016-07-08 Thread Thomas Demoor (JIRA)

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

Thomas Demoor commented on HADOOP-12878:


[~cnauroth] pointed out that the issue I originally ran into could also be 
avoided by setting {{yarn.scheduler.capacity.node-locality-delay=0}}. However, 
there are non-YARN usecases and downstream projects that would benefit from 
locality faking, so I think this still makes sense.

> Impersonate hosts in s3a for better data locality handling
> --
>
> Key: HADOOP-12878
> URL: https://issues.apache.org/jira/browse/HADOOP-12878
> Project: Hadoop Common
>  Issue Type: Sub-task
>  Components: fs/s3
>Affects Versions: 2.8.0
>Reporter: Thomas Demoor
>Assignee: Thomas Demoor
>
> Currently, {{localhost}} is passed as locality for each block, causing all 
> blocks involved in job to initially target the same node (RM), before being 
> moved by the scheduler (to a rack-local node). This reduces parallelism for 
> jobs (with short-lived mappers). 
> We should mimic Azures implementation: a config setting 
> {{fs.s3a.block.location.impersonatedhost}} where the user can enter the list 
> of hostnames in the cluster to return to {{getFileBlockLocations}}. 
> Possible optimization: for larger systems, it might be better to return N 
> (5?) random hostnames to prevent passing a huge array (the downstream code 
> assumes size = O(3)).



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

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



[jira] [Commented] (HADOOP-9565) Add a Blobstore interface to add to blobstore FileSystems

2016-08-05 Thread Thomas Demoor (JIRA)

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

Thomas Demoor commented on HADOOP-9565:
---

Steve the "avoid data write" thing you mention is exactly why these direct 
outputcommitters (and what I did for the FileOutputCommitter) work on object 
stores. Multiple writers can write to the same object concurrently. At any 
point, the last-started successfully-completed write is what is visible.

Regular put: 
* Content length (=N) communicated at start of request. 
* Once N bytes hit S3 the object becomes visible
* If hadoop task aborts before writing N bytes the upload will timeout and the 
object version is garbage collected by S3. 
MulitpartUpload:
* Requires explicit API call to complete (or abort)
* Only when complete API call is used the object becomes visible
* If hadoop task fails the upload will remain to be active (s3a has the purge 
functionality to automatically clean these up after a certain period) but the 
object is NOT visible

The interesting thing to think about are network partitions.




> Add a Blobstore interface to add to blobstore FileSystems
> -
>
> Key: HADOOP-9565
> URL: https://issues.apache.org/jira/browse/HADOOP-9565
> Project: Hadoop Common
>  Issue Type: Improvement
>  Components: fs, fs/s3, fs/swift
>Affects Versions: 2.6.0
>Reporter: Steve Loughran
>Assignee: Pieter Reuse
> Attachments: HADOOP-9565-001.patch, HADOOP-9565-002.patch, 
> HADOOP-9565-003.patch, HADOOP-9565-004.patch, HADOOP-9565-005.patch, 
> HADOOP-9565-006.patch, HADOOP-9565-branch-2-007.patch
>
>
> We can make the fact that some {{FileSystem}} implementations are really 
> blobstores, with different atomicity and consistency guarantees, by adding a 
> {{Blobstore}} interface to add to them. 
> This could also be a place to add a {{Copy(Path,Path)}} method, assuming that 
> all blobstores implement at server-side copy operation as a substitute for 
> rename.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

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



[jira] [Comment Edited] (HADOOP-9565) Add a Blobstore interface to add to blobstore FileSystems

2016-08-05 Thread Thomas Demoor (JIRA)

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

Thomas Demoor edited comment on HADOOP-9565 at 8/5/16 2:41 PM:
---

Steve the "avoid data write" thing you mention is exactly why these direct 
outputcommitters (and what I did for the FileOutputCommitter) work on object 
stores. Multiple writers can write to the same object concurrently. At any 
point, the last-started successfully-completed write is what is visible.

Regular put: 
* Content length (=N) communicated at start of request. 
* Once N bytes hit S3 the object becomes visible
* If hadoop task aborts before writing N bytes the upload will timeout and the 
object version is garbage collected by S3. 

MulitpartUpload:
* Requires explicit API call to complete (or abort)
* Only when complete API call is used the object becomes visible
* If hadoop task fails the upload will remain to be active (s3a has the purge 
functionality to automatically clean these up after a certain period) but the 
object is NOT visible

The interesting thing to think about are network partitions.





was (Author: thomas demoor):
Steve the "avoid data write" thing you mention is exactly why these direct 
outputcommitters (and what I did for the FileOutputCommitter) work on object 
stores. Multiple writers can write to the same object concurrently. At any 
point, the last-started successfully-completed write is what is visible.

Regular put: 
* Content length (=N) communicated at start of request. 
* Once N bytes hit S3 the object becomes visible
* If hadoop task aborts before writing N bytes the upload will timeout and the 
object version is garbage collected by S3. 
MulitpartUpload:
* Requires explicit API call to complete (or abort)
* Only when complete API call is used the object becomes visible
* If hadoop task fails the upload will remain to be active (s3a has the purge 
functionality to automatically clean these up after a certain period) but the 
object is NOT visible

The interesting thing to think about are network partitions.




> Add a Blobstore interface to add to blobstore FileSystems
> -
>
> Key: HADOOP-9565
> URL: https://issues.apache.org/jira/browse/HADOOP-9565
> Project: Hadoop Common
>  Issue Type: Improvement
>  Components: fs, fs/s3, fs/swift
>Affects Versions: 2.6.0
>Reporter: Steve Loughran
>Assignee: Pieter Reuse
> Attachments: HADOOP-9565-001.patch, HADOOP-9565-002.patch, 
> HADOOP-9565-003.patch, HADOOP-9565-004.patch, HADOOP-9565-005.patch, 
> HADOOP-9565-006.patch, HADOOP-9565-branch-2-007.patch
>
>
> We can make the fact that some {{FileSystem}} implementations are really 
> blobstores, with different atomicity and consistency guarantees, by adding a 
> {{Blobstore}} interface to add to them. 
> This could also be a place to add a {{Copy(Path,Path)}} method, assuming that 
> all blobstores implement at server-side copy operation as a substitute for 
> rename.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

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



[jira] [Commented] (HADOOP-11684) S3a to use thread pool that blocks clients

2016-09-08 Thread Thomas Demoor (JIRA)

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

Thomas Demoor commented on HADOOP-11684:


[~ste...@apache.org] you should not make the queue (tasks) long. Indeed the 
consumed memory is (threads + tasks + 1) * partitionSize) (the 1 is due to the 
buffer itself )

The point of this ticket was to introduce the BlockingThreadPoolExecutorService 
that, in contrast to the default Executorservice, does not error out when the 
queue is full but blocks ALL clients from making progress (= using additional 
memory), not even in the calling thread (thus different from CallerRunsPolicy). 

Threads = number of parallel busy (part)uploads 
Tasks = waiting uploads (already in memory)

The intended usage is setting the number of threads and tasks to small values 
(threads=3, (waiting)tasks= 1). Additional work coming in from the application 
will be blocked due to the special executorservice.




> S3a to use thread pool that blocks clients
> --
>
> Key: HADOOP-11684
> URL: https://issues.apache.org/jira/browse/HADOOP-11684
> Project: Hadoop Common
>  Issue Type: Sub-task
>  Components: fs/s3
>Affects Versions: 2.7.0
>Reporter: Thomas Demoor
>Assignee: Thomas Demoor
> Fix For: 3.0.0-alpha1
>
> Attachments: HADOOP-11684-001.patch, HADOOP-11684-002.patch, 
> HADOOP-11684-003.patch, HADOOP-11684-004.patch, HADOOP-11684-005.patch, 
> HADOOP-11684-006.patch
>
>
> Currently, if fs.s3a.max.total.tasks are queued and another (part)upload 
> wants to start, a RejectedExecutionException is thrown. 
> We should use a threadpool that blocks clients, nicely throtthling them, 
> rather than throwing an exception. F.i. something similar to 
> https://github.com/apache/incubator-s4/blob/master/subprojects/s4-comm/src/main/java/org/apache/s4/comm/staging/BlockingThreadPoolExecutorService.java



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

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



[jira] [Commented] (HADOOP-13568) S3AFastOutputStream to implement flush()

2016-09-08 Thread Thomas Demoor (JIRA)

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

Thomas Demoor commented on HADOOP-13568:


This might be hard as only the last part of a MPU can be smaller then 5MB. So I 
fear that the subsequent complete operation would fail if flush() is called 
when the buffer contains less than 5MB.

http://docs.aws.amazon.com/AmazonS3/latest/API/mpUploadUploadPart.html ??Each 
part must be at least 5 MB in size, except the last part. There is no size 
limit on the last part of your multipart upload.??


> S3AFastOutputStream to implement flush()
> 
>
> Key: HADOOP-13568
> URL: https://issues.apache.org/jira/browse/HADOOP-13568
> Project: Hadoop Common
>  Issue Type: Sub-task
>  Components: fs/s3
>Reporter: Steve Loughran
>Priority: Minor
>
> {{S3AFastOutputStream}} doesn't implement {{flush()}}, so it's a no-op.
> Really it should trigger a multipart upload of the current buffer.
> Note that simply calling {{uploadBuffer()} isn't enough...do that and things 
> fail.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

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



[jira] [Commented] (HADOOP-13262) set multipart delete timeout to 5 * 60s in S3ATestUtils.createTestFileSystem

2016-09-20 Thread Thomas Demoor (JIRA)

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

Thomas Demoor commented on HADOOP-13262:


+1

> set multipart delete timeout to 5 * 60s in S3ATestUtils.createTestFileSystem
> 
>
> Key: HADOOP-13262
> URL: https://issues.apache.org/jira/browse/HADOOP-13262
> Project: Hadoop Common
>  Issue Type: Sub-task
>  Components: fs/s3, test
>Affects Versions: 2.8.0
> Environment: parallel test runs
>Reporter: Steve Loughran
>Assignee: Steve Loughran
> Attachments: HADOOP-13262-001.patch
>
>
> HADOOP-13139 patch 003 test runs show that the multipart tests are failing on 
> parallel runs. The cause of this is that the FS init logic in 
> {{S3ATestUtils.createTestFileSystem}} sets the expiry to 0: any in-progress 
> multipart uploads will fail. 
> setting a 5 minute expiry will clean up from old runs, but not break anything 
> in progress.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

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



[jira] [Commented] (HADOOP-11262) Enable YARN to use S3A

2015-06-09 Thread Thomas Demoor (JIRA)

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

Thomas Demoor commented on HADOOP-11262:


We (well, [~PieterReuse] actually :p) have implemented all FileContext tests + 
a test that spins up MiniYARNCluster. Some minor code additions / changes to 
make things work. Have been running the patch on our test cluster for a week 
now. So far things look good but want to check some more corner cases before 
uploading the patch.


> Enable YARN to use S3A 
> ---
>
> Key: HADOOP-11262
> URL: https://issues.apache.org/jira/browse/HADOOP-11262
> Project: Hadoop Common
>  Issue Type: Sub-task
>  Components: fs/s3
>Reporter: Thomas Demoor
>Assignee: Thomas Demoor
>  Labels: amazon, s3
> Attachments: HADOOP-11262-2.patch, HADOOP-11262-3.patch, 
> HADOOP-11262-4.patch, HADOOP-11262-5.patch, HADOOP-11262.patch
>
>
> Uses DelegateToFileSystem to expose S3A as an AbstractFileSystem.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (HADOOP-11918) Listing an empty s3a root directory throws FileNotFound.

2015-06-18 Thread Thomas Demoor (JIRA)

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

Thomas Demoor commented on HADOOP-11918:


We preferred the test approach of this patch over that of HADOOP-11742. 
However, as other tests sometimes leave artifacts behind, the test had 
transient problems and we propose to fix this by forcing an empty root in the 
test case for this patch , as [~PieterReuse] pointed out. [~eddyxu], did you 
see the same behaviour?


> Listing an empty s3a root directory throws FileNotFound.
> 
>
> Key: HADOOP-11918
> URL: https://issues.apache.org/jira/browse/HADOOP-11918
> Project: Hadoop Common
>  Issue Type: Bug
>Affects Versions: 2.7.0
>Reporter: Lei (Eddy) Xu
>Assignee: Lei (Eddy) Xu
>Priority: Minor
>  Labels: BB2015-05-TBR, s3
> Attachments: HADOOP-11918-002.patch, HADOOP-11918.000.patch, 
> HADOOP-11918.001.patch
>
>
> With an empty s3 bucket and run
> {code}
> $ hadoop fs -D... -ls s3a://hdfs-s3a-test/
> 15/05/04 15:21:34 WARN util.NativeCodeLoader: Unable to load native-hadoop 
> library for your platform... using builtin-java classes where applicable
> ls: `s3a://hdfs-s3a-test/': No such file or directory
> {code}



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Updated] (HADOOP-12169) ListStatus on empty dir in S3A lists itself instead of returning an empty list

2015-07-01 Thread Thomas Demoor (JIRA)

 [ 
https://issues.apache.org/jira/browse/HADOOP-12169?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Thomas Demoor updated HADOOP-12169:
---
Issue Type: Sub-task  (was: Bug)
Parent: HADOOP-11694

> ListStatus on empty dir in S3A lists itself instead of returning an empty list
> --
>
> Key: HADOOP-12169
> URL: https://issues.apache.org/jira/browse/HADOOP-12169
> Project: Hadoop Common
>  Issue Type: Sub-task
>  Components: fs/s3
>Reporter: Pieter Reuse
>Assignee: Pieter Reuse
> Attachments: HADOOP-12169-001.patch
>
>
> Upon testing the patch for HADOOP-11918, I stumbled upon a weird behaviour 
> this introduces to the S3AFileSystem-class. Calling ListStatus() on an empty 
> bucket returns an empty list, while doing the same on an empty directory, 
> returns an array of length 1 containing only this directory itself.
> The bugfix is quite simple. In the line of code {code}...if 
> (keyPath.equals(f)...{code} (S3AFileSystem:758), keyPath is qualified wrt. 
> the fs and f is not. Therefore, this returns false while it shouldn't. The 
> bugfix to make f qualified in this line of code.
> More formally: accoring to the formal definition of [The Hadoop FileSystem 
> API 
> Definition|https://hadoop.apache.org/docs/r2.6.0/hadoop-project-dist/hadoop-common/filesystem/],
>  more specifically FileSystem.listStatus, only child elements of a directory 
> should be returned upon a listStatus()-call.
> In detail: 
> {code}
> elif isDir(FS, p): result [getFileStatus(c) for c in children(FS, p) where 
> f(c) == True]
> {code}
> and
> {code}
> def children(FS, p) = {q for q in paths(FS) where parent(q) == p}
> {code}
> Which translates to the result of listStatus on an empty directory being an 
> empty list. This is the same behaviour as ls has in Unix, which is what 
> someone would expect from a FileSystem.
> Note: it seemed appropriate to add the test of this patch to the same file as 
> the test for HADOOP-11918, but as a result, one of the two will have to be 
> rebased wrt. the other before being applied to trunk.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Updated] (HADOOP-11262) Enable YARN to use S3A

2015-07-02 Thread Thomas Demoor (JIRA)

 [ 
https://issues.apache.org/jira/browse/HADOOP-11262?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Thomas Demoor updated HADOOP-11262:
---
Assignee: Pieter Reuse  (was: Thomas Demoor)

> Enable YARN to use S3A 
> ---
>
> Key: HADOOP-11262
> URL: https://issues.apache.org/jira/browse/HADOOP-11262
> Project: Hadoop Common
>  Issue Type: Sub-task
>  Components: fs/s3
>Reporter: Thomas Demoor
>Assignee: Pieter Reuse
>  Labels: amazon, s3
> Attachments: HADOOP-11262-2.patch, HADOOP-11262-3.patch, 
> HADOOP-11262-4.patch, HADOOP-11262-5.patch, HADOOP-11262.patch
>
>
> Uses DelegateToFileSystem to expose S3A as an AbstractFileSystem.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Updated] (HADOOP-9565) Add a Blobstore interface to add to blobstore FileSystems

2015-07-02 Thread Thomas Demoor (JIRA)

 [ 
https://issues.apache.org/jira/browse/HADOOP-9565?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Thomas Demoor updated HADOOP-9565:
--
Attachment: HADOOP-9565-004.patch

004 introduces an objectStore-aware FileOutputCommitter: it writes directly to 
the outputPath if in-progress/failed writes are not visible on the object store.

[~jnp], [~anu], [~cnauroth] and [~jghoman]: this is the approach I think 
HDFS-7240 can easily use this to avoid the "rename issue".

The implementation optios were constrained by the fact that FileOutputCommitter 
has a stable public interface:
* implementing the changes by subclassing was not possible
* the public / protected methods could not be changed. Often they are also 
static (for use in MR1), further limiting options.

Some smaller stylistic changes (checkstyle, etc.) to the code in 003.patch

> Add a Blobstore interface to add to blobstore FileSystems
> -
>
> Key: HADOOP-9565
> URL: https://issues.apache.org/jira/browse/HADOOP-9565
> Project: Hadoop Common
>  Issue Type: Improvement
>  Components: fs, fs/s3, fs/swift
>Affects Versions: 2.6.0
>Reporter: Steve Loughran
>Assignee: Steve Loughran
>  Labels: BB2015-05-TBR
> Attachments: HADOOP-9565-001.patch, HADOOP-9565-002.patch, 
> HADOOP-9565-003.patch, HADOOP-9565-004.patch
>
>
> We can make the fact that some {{FileSystem}} implementations are really 
> blobstores, with different atomicity and consistency guarantees, by adding a 
> {{Blobstore}} interface to add to them. 
> This could also be a place to add a {{Copy(Path,Path)}} method, assuming that 
> all blobstores implement at server-side copy operation as a substitute for 
> rename.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (HADOOP-9565) Add a Blobstore interface to add to blobstore FileSystems

2015-07-10 Thread Thomas Demoor (JIRA)

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

Thomas Demoor commented on HADOOP-9565:
---

Thanks for your review [~eddyxu]. Indeed, the patch contains code similar to 
that of [HADOOP-11525] as [~ste...@apache.org] previously merged that into 
003.patch. My additions to his work (diff between 003.patch and 004.patch) are 
mainly to the FileOutputCommitter.

h4.FileOutputCommitter
Most parts of the larger ecosystem use FileOutputCommitter: MapReduce, Spark, 
Tez, ... HBase is a notable counterexample: I don' know if they use rename a 
lot, I do know append is an operation typically not supported on object stores 
(thus I think HBase on an objectstore is not a very good usecase). I think you 
have a better view of the ecosystem than I do: do you know of hadoop filesystem 
"users" that do not use FileOutputCommitter but do "commit by renaming" 
themselves? I am willing to try to optimize them for objectstores, if the 
usecase makes sense. I will have a look at making {{distcp}} object-store aware 
as well (so expect 005.patch :P).

Currently, what we do is basically {{if(canWeUseAtomicWrite())}}, which then 
triggers one of the 2 (relatively) separate code paths. I don't think there's a 
simple one-liner one can change from {{write()}} to {{atomicWrite()}}: as HDFS 
is a single-writer POSIX-style filesystem, {{write()}} is accompanied by a 
whole scheme of other operations that together make up the (higher-level) 
atomic "commit operation". For instance for the FileOutputCommitter: object 
stores want a clean {{atomicWrite()}} but for HDFS it uses a scheme of 
temporary directories for each attempt and at the end it commits the 
"successful" attempt by renaming and deletes all other attempts. I do not see 
how that can be replaced by a simple interface {{write()}}/{{atomicWrite()}} 
while keeping backwards-compatibility (FileOutputCommitter is hardcoded in lots 
of applications), but suggestions would be very welcome.

h4.Slow / Async operations
I think the flags for slow operations are a good idea. I think 
[~ste...@apache.org]'s comment above on adding async operations takes that idea 
one step further. Some renames / deletes can be async, other can't but 
evidently only the client can know if async is possible for their codepath. I 
think these are good ideas to offer more options to users in the future, but it 
will require some PR to get them picked up. The code that is currently in the 
patch aims at things we can improve in a manner that is invisible to end-users. 
I think therefore these are best tracked in separate issues. 

[~ste...@apache.org] and [~ndimiduk]: afaik TTL is a feature of AWS Cloudfront 
(cache): i.e. TTL in the cache, get from S3 after that. This does not affect 
the S3 object. Furthermore, S3 has an "Object Expiration" feature, but its 
policy is defined per bucket so I'm not sure it's directly applicable here.

> Add a Blobstore interface to add to blobstore FileSystems
> -
>
> Key: HADOOP-9565
> URL: https://issues.apache.org/jira/browse/HADOOP-9565
> Project: Hadoop Common
>  Issue Type: Improvement
>  Components: fs, fs/s3, fs/swift
>Affects Versions: 2.6.0
>Reporter: Steve Loughran
>Assignee: Steve Loughran
>  Labels: BB2015-05-TBR
> Attachments: HADOOP-9565-001.patch, HADOOP-9565-002.patch, 
> HADOOP-9565-003.patch, HADOOP-9565-004.patch
>
>
> We can make the fact that some {{FileSystem}} implementations are really 
> blobstores, with different atomicity and consistency guarantees, by adding a 
> {{Blobstore}} interface to add to them. 
> This could also be a place to add a {{Copy(Path,Path)}} method, assuming that 
> all blobstores implement at server-side copy operation as a substitute for 
> rename.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (HADOOP-12267) s3a failure due to integer overflow bug in AWS SDK

2015-07-24 Thread Thomas Demoor (JIRA)

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

Thomas Demoor commented on HADOOP-12267:


Hi Aaron,

in HADOOP-11684 I have bumped to 1.9.x (we have been testing this for a month 
now and all is well). Note that other bugs fixed in the aws-sdk (multi-part 
threshold from int -> long ) require some code changes in s3a. 

You will see in the comments that [~ste...@apache.org] requested to pull out 
the aws-sdk upgrade to a separate patch. I am doing that today, will link to 
the new issue then.

Another main benefit of 1.9+ is that s3 is a separate library. We no longer 
need to pull in the entire sdk.

> s3a failure due to integer overflow bug in AWS SDK
> --
>
> Key: HADOOP-12267
> URL: https://issues.apache.org/jira/browse/HADOOP-12267
> Project: Hadoop Common
>  Issue Type: Bug
>  Components: fs/s3
>Affects Versions: 2.6.0
>Reporter: Aaron Fabbri
>Assignee: Aaron Fabbri
> Attachments: HADOOP-12267.2.6.0.001.patch, 
> HADOOP-12267.2.7.1.001.patch
>
>
> Under high load writing to Amazon AWS S3 storage, a client can be throttled 
> enough to encounter 24 retries in a row.
> The amazon http client code (in aws-java-sdk jar) has a bug in its 
> exponential backoff retry code, that causes integer overflow, and a call to 
> Thread.sleep() with a negative value, which causes client to bail out with an 
> exception (see below).
> Bug has been fixed in aws-java-sdk:
> https://github.com/aws/aws-sdk-java/pull/388
> We need to pick this up for hadoop-tools/hadoop-aws.
> Error: java.io.IOException: File copy failed: hdfs://path-redacted --> 
> s3a://path-redacted
> at 
> org.apache.hadoop.tools.mapred.CopyMapper.copyFileWithRetry(CopyMapper.java:284)
> at org.apache.hadoop.tools.mapred.CopyMapper.map(CopyMapper.java:252) 
> at org.apache.hadoop.tools.mapred.CopyMapper.map(CopyMapper.java:50)  
> at org.apache.hadoop.mapreduce.Mapper.run(Mapper.java:145) 
> at org.apache.hadoop.mapred.MapTask.runNewMapper(MapTask.java:784)
> at org.apache.hadoop.mapred.MapTask.run(MapTask.java:341) 
> at org.apache.hadoop.mapred.YarnChild$2.run(YarnChild.java:168)
> at java.security.AccessController.doPrivileged(Native Method) 
> at javax.security.auth.Subject.doAs(Subject.java:415) 
> at 
> org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1642)
> at org.apache.hadoop.mapred.YarnChild.main(YarnChild.java:163) Caused by: 
> java.io.IOException: Couldn't run retriable-command: Copying 
> hdfs://path-redacted to s3a://path-redacted
> at 
> org.apache.hadoop.tools.util.RetriableCommand.execute(RetriableCommand.java:101)
>  
> at 
> org.apache.hadoop.tools.mapred.CopyMapper.copyFileWithRetry(CopyMapper.java:280)
>  
> ... 10 more 
> Caused by: com.amazonaws.AmazonClientException: Unable to complete transfer: 
> timeout value is negative
> at 
> com.amazonaws.services.s3.transfer.internal.AbstractTransfer.unwrapExecutionException(AbstractTransfer.java:300)
> at 
> com.amazonaws.services.s3.transfer.internal.AbstractTransfer.rethrowExecutionException(AbstractTransfer.java:284)
> at 
> com.amazonaws.services.s3.transfer.internal.CopyImpl.waitForCopyResult(CopyImpl.java:67)
>  
> at org.apache.hadoop.fs.s3a.S3AFileSystem.copyFile(S3AFileSystem.java:943) 
> at org.apache.hadoop.fs.s3a.S3AFileSystem.rename(S3AFileSystem.java:357) 
> at 
> org.apache.hadoop.tools.mapred.RetriableFileCopyCommand.promoteTmpToTarget(RetriableFileCopyCommand.java:220)
> at 
> org.apache.hadoop.tools.mapred.RetriableFileCopyCommand.doCopy(RetriableFileCopyCommand.java:137)
>  
> at 
> org.apache.hadoop.tools.mapred.RetriableFileCopyCommand.doExecute(RetriableFileCopyCommand.java:100)
> at 
> org.apache.hadoop.tools.util.RetriableCommand.execute(RetriableCommand.java:87)
>  
> ... 11 more 
> Caused by: java.lang.IllegalArgumentException: timeout value is negative
> at java.lang.Thread.sleep(Native Method) 
> at 
> com.amazonaws.http.AmazonHttpClient.pauseBeforeNextRetry(AmazonHttpClient.java:864)
> at 
> com.amazonaws.http.AmazonHttpClient.executeHelper(AmazonHttpClient.java:353) 
> at com.amazonaws.http.AmazonHttpClient.execute(AmazonHttpClient.java:232) 
> at com.amazonaws.services.s3.AmazonS3Client.invoke(AmazonS3Client.java:3528)
> at 
> com.amazonaws.services.s3.AmazonS3Client.copyObject(AmazonS3Client.java:1507)
> at 
> com.amazonaws.services.s3.transfer.internal.CopyCallable.copyInOneChunk(CopyCallable.java:143)
> at 
> com.amazonaws.services.s3.transfer.internal.CopyCallable.call(CopyCallable.java:131)
>  
> at 
> com.amazonaws.services.s3.transfer.internal.CopyMonitor.copy(CopyMonitor.java:189)
>  
> at 
> com.amazonaws.services.s3.transfer.internal.CopyMonitor.call(CopyMonitor.java:134)
>  
> at 
> com.amazonaws.services.

[jira] [Created] (HADOOP-12269) Update aws-sdk dependency version

2015-07-24 Thread Thomas Demoor (JIRA)
Thomas Demoor created HADOOP-12269:
--

 Summary: Update aws-sdk dependency version
 Key: HADOOP-12269
 URL: https://issues.apache.org/jira/browse/HADOOP-12269
 Project: Hadoop Common
  Issue Type: Sub-task
  Components: fs/s3
Reporter: Thomas Demoor






--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Updated] (HADOOP-12269) Update aws-sdk dependency version

2015-07-24 Thread Thomas Demoor (JIRA)

 [ 
https://issues.apache.org/jira/browse/HADOOP-12269?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Thomas Demoor updated HADOOP-12269:
---
Description: This was originally part of HADOOP-11684

> Update aws-sdk dependency version
> -
>
> Key: HADOOP-12269
> URL: https://issues.apache.org/jira/browse/HADOOP-12269
> Project: Hadoop Common
>  Issue Type: Sub-task
>  Components: fs/s3
>Reporter: Thomas Demoor
>
> This was originally part of HADOOP-11684



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Updated] (HADOOP-12269) Update aws-sdk dependency version

2015-07-24 Thread Thomas Demoor (JIRA)

 [ 
https://issues.apache.org/jira/browse/HADOOP-12269?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Thomas Demoor updated HADOOP-12269:
---
Description: This was originally part of HADOOP-11684, pulling out to this 
separate subtask as requested by [~ste...@apache.org]  (was: This was 
originally part of HADOOP-11684)

> Update aws-sdk dependency version
> -
>
> Key: HADOOP-12269
> URL: https://issues.apache.org/jira/browse/HADOOP-12269
> Project: Hadoop Common
>  Issue Type: Sub-task
>  Components: fs/s3
>Reporter: Thomas Demoor
>
> This was originally part of HADOOP-11684, pulling out to this separate 
> subtask as requested by [~ste...@apache.org]



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Updated] (HADOOP-12269) Update aws-sdk dependency to 1.10.6

2015-07-24 Thread Thomas Demoor (JIRA)

 [ 
https://issues.apache.org/jira/browse/HADOOP-12269?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Thomas Demoor updated HADOOP-12269:
---
Summary: Update aws-sdk dependency to 1.10.6  (was: Update aws-sdk 
dependency version)

> Update aws-sdk dependency to 1.10.6
> ---
>
> Key: HADOOP-12269
> URL: https://issues.apache.org/jira/browse/HADOOP-12269
> Project: Hadoop Common
>  Issue Type: Sub-task
>  Components: fs/s3
>Reporter: Thomas Demoor
>
> This was originally part of HADOOP-11684, pulling out to this separate 
> subtask as requested by [~ste...@apache.org]



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Updated] (HADOOP-12269) Update aws-sdk dependency to 1.10.6

2015-07-24 Thread Thomas Demoor (JIRA)

 [ 
https://issues.apache.org/jira/browse/HADOOP-12269?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Thomas Demoor updated HADOOP-12269:
---
Attachment: HADOOP-12269-001.patch

Patch bumps from aws-sdk-1.7.4 to aws-sdk-s3-1.10.6. 

* Only S3 library so smaller binary (possible since 1.9). 
* Multipart threshold changed from int to long as the corresponding change 
(bugfix) was made in the aws sdk
* Added config setting to make overriding the signing algorithm possible. This 
to keep object stores that still use the previous signing algorithm functional 
with s3. Set this config setting to {{S3Signer}} to get v2 authentication.

> Update aws-sdk dependency to 1.10.6
> ---
>
> Key: HADOOP-12269
> URL: https://issues.apache.org/jira/browse/HADOOP-12269
> Project: Hadoop Common
>  Issue Type: Sub-task
>  Components: fs/s3
>Reporter: Thomas Demoor
> Attachments: HADOOP-12269-001.patch
>
>
> This was originally part of HADOOP-11684, pulling out to this separate 
> subtask as requested by [~ste...@apache.org]



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (HADOOP-12267) s3a failure due to integer overflow bug in AWS SDK

2015-07-24 Thread Thomas Demoor (JIRA)

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

Thomas Demoor commented on HADOOP-12267:


I have isolated the aws-sdk bump in HADOOP-12269

> s3a failure due to integer overflow bug in AWS SDK
> --
>
> Key: HADOOP-12267
> URL: https://issues.apache.org/jira/browse/HADOOP-12267
> Project: Hadoop Common
>  Issue Type: Bug
>  Components: fs/s3
>Affects Versions: 2.6.0
>Reporter: Aaron Fabbri
>Assignee: Aaron Fabbri
> Attachments: HADOOP-12267.2.6.0.001.patch, 
> HADOOP-12267.2.7.1.001.patch
>
>
> Under high load writing to Amazon AWS S3 storage, a client can be throttled 
> enough to encounter 24 retries in a row.
> The amazon http client code (in aws-java-sdk jar) has a bug in its 
> exponential backoff retry code, that causes integer overflow, and a call to 
> Thread.sleep() with a negative value, which causes client to bail out with an 
> exception (see below).
> Bug has been fixed in aws-java-sdk:
> https://github.com/aws/aws-sdk-java/pull/388
> We need to pick this up for hadoop-tools/hadoop-aws.
> Error: java.io.IOException: File copy failed: hdfs://path-redacted --> 
> s3a://path-redacted
> at 
> org.apache.hadoop.tools.mapred.CopyMapper.copyFileWithRetry(CopyMapper.java:284)
> at org.apache.hadoop.tools.mapred.CopyMapper.map(CopyMapper.java:252) 
> at org.apache.hadoop.tools.mapred.CopyMapper.map(CopyMapper.java:50)  
> at org.apache.hadoop.mapreduce.Mapper.run(Mapper.java:145) 
> at org.apache.hadoop.mapred.MapTask.runNewMapper(MapTask.java:784)
> at org.apache.hadoop.mapred.MapTask.run(MapTask.java:341) 
> at org.apache.hadoop.mapred.YarnChild$2.run(YarnChild.java:168)
> at java.security.AccessController.doPrivileged(Native Method) 
> at javax.security.auth.Subject.doAs(Subject.java:415) 
> at 
> org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1642)
> at org.apache.hadoop.mapred.YarnChild.main(YarnChild.java:163) Caused by: 
> java.io.IOException: Couldn't run retriable-command: Copying 
> hdfs://path-redacted to s3a://path-redacted
> at 
> org.apache.hadoop.tools.util.RetriableCommand.execute(RetriableCommand.java:101)
>  
> at 
> org.apache.hadoop.tools.mapred.CopyMapper.copyFileWithRetry(CopyMapper.java:280)
>  
> ... 10 more 
> Caused by: com.amazonaws.AmazonClientException: Unable to complete transfer: 
> timeout value is negative
> at 
> com.amazonaws.services.s3.transfer.internal.AbstractTransfer.unwrapExecutionException(AbstractTransfer.java:300)
> at 
> com.amazonaws.services.s3.transfer.internal.AbstractTransfer.rethrowExecutionException(AbstractTransfer.java:284)
> at 
> com.amazonaws.services.s3.transfer.internal.CopyImpl.waitForCopyResult(CopyImpl.java:67)
>  
> at org.apache.hadoop.fs.s3a.S3AFileSystem.copyFile(S3AFileSystem.java:943) 
> at org.apache.hadoop.fs.s3a.S3AFileSystem.rename(S3AFileSystem.java:357) 
> at 
> org.apache.hadoop.tools.mapred.RetriableFileCopyCommand.promoteTmpToTarget(RetriableFileCopyCommand.java:220)
> at 
> org.apache.hadoop.tools.mapred.RetriableFileCopyCommand.doCopy(RetriableFileCopyCommand.java:137)
>  
> at 
> org.apache.hadoop.tools.mapred.RetriableFileCopyCommand.doExecute(RetriableFileCopyCommand.java:100)
> at 
> org.apache.hadoop.tools.util.RetriableCommand.execute(RetriableCommand.java:87)
>  
> ... 11 more 
> Caused by: java.lang.IllegalArgumentException: timeout value is negative
> at java.lang.Thread.sleep(Native Method) 
> at 
> com.amazonaws.http.AmazonHttpClient.pauseBeforeNextRetry(AmazonHttpClient.java:864)
> at 
> com.amazonaws.http.AmazonHttpClient.executeHelper(AmazonHttpClient.java:353) 
> at com.amazonaws.http.AmazonHttpClient.execute(AmazonHttpClient.java:232) 
> at com.amazonaws.services.s3.AmazonS3Client.invoke(AmazonS3Client.java:3528)
> at 
> com.amazonaws.services.s3.AmazonS3Client.copyObject(AmazonS3Client.java:1507)
> at 
> com.amazonaws.services.s3.transfer.internal.CopyCallable.copyInOneChunk(CopyCallable.java:143)
> at 
> com.amazonaws.services.s3.transfer.internal.CopyCallable.call(CopyCallable.java:131)
>  
> at 
> com.amazonaws.services.s3.transfer.internal.CopyMonitor.copy(CopyMonitor.java:189)
>  
> at 
> com.amazonaws.services.s3.transfer.internal.CopyMonitor.call(CopyMonitor.java:134)
>  
> at 
> com.amazonaws.services.s3.transfer.internal.CopyMonitor.call(CopyMonitor.java:46)
>   
> at java.util.concurrent.FutureTask.run(FutureTask.java:262) 
> at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
> at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
>  
> at java.lang.Thread.run(Thread.java:745) 



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Updated] (HADOOP-12269) Update aws-sdk dependency to 1.10.6

2015-07-26 Thread Thomas Demoor (JIRA)

 [ 
https://issues.apache.org/jira/browse/HADOOP-12269?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Thomas Demoor updated HADOOP-12269:
---
Target Version/s: 2.8.0

As soon as possible (just like all my other patches :P). For "outsiders" it's 
not really trivial to find out what the next version will be. I see discussions 
on the dev lists on 2.8 vs. 3.0 ... I kinda sense 2.8 is winning the front. 
Targeting 2.8 for now.

Concerning the 2.6-7.x branches: [~fabbri], I think your patch moves past 
1.7.6, which has the int -> long bugfix on multipart threshold in the aws sdk. 
So you need to backport those changes as well or verify that that does not 
cause issues. 

> Update aws-sdk dependency to 1.10.6
> ---
>
> Key: HADOOP-12269
> URL: https://issues.apache.org/jira/browse/HADOOP-12269
> Project: Hadoop Common
>  Issue Type: Sub-task
>  Components: fs/s3
>Reporter: Thomas Demoor
> Attachments: HADOOP-12269-001.patch
>
>
> This was originally part of HADOOP-11684, pulling out to this separate 
> subtask as requested by [~ste...@apache.org]



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Assigned] (HADOOP-12269) Update aws-sdk dependency to 1.10.6

2015-07-26 Thread Thomas Demoor (JIRA)

 [ 
https://issues.apache.org/jira/browse/HADOOP-12269?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Thomas Demoor reassigned HADOOP-12269:
--

Assignee: Thomas Demoor

> Update aws-sdk dependency to 1.10.6
> ---
>
> Key: HADOOP-12269
> URL: https://issues.apache.org/jira/browse/HADOOP-12269
> Project: Hadoop Common
>  Issue Type: Sub-task
>  Components: fs/s3
>Reporter: Thomas Demoor
>Assignee: Thomas Demoor
> Attachments: HADOOP-12269-001.patch
>
>
> This was originally part of HADOOP-11684, pulling out to this separate 
> subtask as requested by [~ste...@apache.org]



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (HADOOP-12269) Update aws-sdk dependency to 1.10.6

2015-07-28 Thread Thomas Demoor (JIRA)

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

Thomas Demoor commented on HADOOP-12269:


Setting defaults is difficult as the "correct value depends greatly on the 
bandwidth between your Hadoop cluster and the S3(-compatible) backend and the 
reason you use multipartupload: limit size of "unit that gets lost" on a bad 
connection (WAN scenario),  starting upload before all data is in 
(s3a.fast.upload=true), ...

I'm setting it back to 2GB  (although that's certainly a lot too high for WAN). 
 Adjusting the defaults merits its own JIRA as the other s3a defaults are also, 
mmh, not fantastic.

> Update aws-sdk dependency to 1.10.6
> ---
>
> Key: HADOOP-12269
> URL: https://issues.apache.org/jira/browse/HADOOP-12269
> Project: Hadoop Common
>  Issue Type: Sub-task
>  Components: fs/s3
>Reporter: Thomas Demoor
>Assignee: Thomas Demoor
> Attachments: HADOOP-12269-001.patch
>
>
> This was originally part of HADOOP-11684, pulling out to this separate 
> subtask as requested by [~ste...@apache.org]



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


  1   2   3   >