[
https://issues.apache.org/jira/browse/HADOOP-16906?focusedWorklogId=545216&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-545216
]
ASF GitHub Bot logged work on HADOOP-16906:
-------------------------------------------
Author: ASF GitHub Bot
Created on: 01/Feb/21 09:48
Start Date: 01/Feb/21 09:48
Worklog Time Spent: 10m
Work Description: HeartSaVioR commented on a change in pull request #2667:
URL: https://github.com/apache/hadoop/pull/2667#discussion_r567685195
##########
File path:
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ABlockOutputStream.java
##########
@@ -551,6 +556,24 @@ public IOStatistics getIOStatistics() {
return iostatistics;
}
+ @Override
+ public void abort() {
Review comment:
Migrating comment in
https://github.com/HeartSaVioR/hadoop/commit/63c5588d46d422a66ec620ae310f9a860db890b3#r46507150
> the new "cloud ready" API calls always return a CompletableFuture, to
emphasise that the op may take time and to allow the caller to do something
while waiting. Would we want to do this here? I'm not convinced it is
appropriate. Instead we say
>
> 1. call must guarantee that after this is invoked,. close() will not
materialize the file at its final path
> 2. it may communicate with the store to cancel an operation; which may
retry. Errors will be stored.
> 3. there may still/also be async IO to the store after the call returns,
but this must maintain the requirement "not visible"
> 4. And close() may do some IO to cancel
##########
File path:
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ABlockOutputStream.java
##########
@@ -551,6 +556,24 @@ public IOStatistics getIOStatistics() {
return iostatistics;
}
+ @Override
+ public void abort() {
+ if (closed.getAndSet(true)) {
+ // already closed
+ LOG.debug("Ignoring abort() as stream is already closed");
+ return;
+ }
+
+ S3ADataBlocks.DataBlock block = getActiveBlock();
Review comment:
Migrating comment in
https://github.com/HeartSaVioR/hadoop/commit/63c5588d46d422a66ec620ae310f9a860db890b3#r46506593
> I think we are going to have to worry about this a bit more, because we
may have queued >1 block for upload in a separate thread. They'll maybe need
interruption, or at least, when they finish, see if they should immediately
cancel the upload. This won't make any difference in the semantics of abort()
(the final upload has been killed), I just don't want to run up any bills.
##########
File path:
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ABlockOutputStream.java
##########
@@ -551,6 +556,24 @@ public IOStatistics getIOStatistics() {
return iostatistics;
}
+ @Override
+ public void abort() {
+ if (closed.getAndSet(true)) {
+ // already closed
+ LOG.debug("Ignoring abort() as stream is already closed");
+ return;
+ }
+
+ S3ADataBlocks.DataBlock block = getActiveBlock();
+ try {
+ if (multiPartUpload != null) {
+ multiPartUpload.abort();
Review comment:
Migrating comment in
https://github.com/HeartSaVioR/hadoop/commit/63c5588d46d422a66ec620ae310f9a860db890b3#r46506765
> ok, don't worry so much about my prev comment. That cancels all the
outstanding futures.
##########
File path:
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3ABlockOutputStream.java
##########
@@ -82,4 +82,22 @@ public void testWriteOperationHelperPartLimits() throws
Throwable {
() -> woh.newUploadPartRequest(key,
"uploadId", 50000, 1024, inputStream, null, 0L));
}
+
+ @Test
+ public void testStreamClosedAfterAbort() throws Exception {
+ stream.abort();
+
+ // This verification replaces testing various operations after calling
abort:
+ // after calling abort, stream is closed like calling close().
+ intercept(IOException.class, () -> stream.checkOpen());
Review comment:
Migrating comment in
https://github.com/HeartSaVioR/hadoop/commit/63c5588d46d422a66ec620ae310f9a860db890b3#r46507220
> should also verify that stream.write() raises an IOE. We could raise a
subclass of IOE to indicate this was a checkOpen failure for a stricter test
##########
File path:
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3ABlockOutputStream.java
##########
@@ -82,4 +82,22 @@ public void testWriteOperationHelperPartLimits() throws
Throwable {
() -> woh.newUploadPartRequest(key,
"uploadId", 50000, 1024, inputStream, null, 0L));
}
+
+ @Test
Review comment:
Migrating comment in
https://github.com/HeartSaVioR/hadoop/commit/63c5588d46d422a66ec620ae310f9a860db890b3#r46506646
> tests are good. We will need to do an ITest too, which can be done in
ITestS3ABlockOutputArray
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
For queries about this service, please contact Infrastructure at:
[email protected]
Issue Time Tracking
-------------------
Worklog Id: (was: 545216)
Time Spent: 0.5h (was: 20m)
> Add some Abortable.abort() interface for streams etc which can be terminated
> ----------------------------------------------------------------------------
>
> Key: HADOOP-16906
> URL: https://issues.apache.org/jira/browse/HADOOP-16906
> Project: Hadoop Common
> Issue Type: Sub-task
> Components: fs, fs/azure, fs/s3
> Affects Versions: 3.3.0
> Reporter: Steve Loughran
> Priority: Minor
> Labels: pull-request-available
> Time Spent: 0.5h
> Remaining Estimate: 0h
>
> Some IO we want to be able to abort rather than close cleanly, especially if
> the inner stream is an HTTP connection which itself supports some abort()
> method. For example: uploads to an object where we want to cancel the upload
> without close() making an incomplete write visible.
> Proposed: Add a generic interface which things like streams can implement
> {code}
> AbortableIO {
> public void abortIO() throws IOE;
> }
> {code}
> +do for s3a output stream. I wouldn't do this a passthrough on
> FSDataOutputStream because we need to consider what expectations callers have
> of an operation being "aborted"
--
This message was sent by Atlassian Jira
(v8.3.4#803005)
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]