-
Notifications
You must be signed in to change notification settings - Fork 9.2k
HADOOP-16906. Add Abortable.abort() interface for streams to enable output stream to be terminated #2667
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
HADOOP-16906. Add Abortable.abort() interface for streams to enable output stream to be terminated #2667
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,36 @@ | ||
| /** | ||
| * Licensed to the Apache Software Foundation (ASF) under one | ||
| * or more contributor license agreements. See the NOTICE file | ||
| * distributed with this work for additional information | ||
| * regarding copyright ownership. The ASF licenses this file | ||
| * to you under the Apache License, Version 2.0 (the | ||
| * "License"); you may not use this file except in compliance | ||
| * with the License. You may obtain a copy of the License at | ||
| * | ||
| * http://www.apache.org/licenses/LICENSE-2.0 | ||
| * | ||
| * Unless required by applicable law or agreed to in writing, software | ||
| * distributed under the License is distributed on an "AS IS" BASIS, | ||
| * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
| * See the License for the specific language governing permissions and | ||
| * limitations under the License. | ||
| */ | ||
|
|
||
| package org.apache.hadoop.fs; | ||
|
|
||
| import org.apache.hadoop.classification.InterfaceStability; | ||
|
|
||
| /** | ||
| * Stream that abort the upload. | ||
| */ | ||
| @InterfaceStability.Unstable | ||
| public interface Abortable { | ||
|
|
||
| /** | ||
| * Abort the upload for the stream. | ||
| * | ||
| * This is to provide ability to cancel the write on stream; once stream is | ||
| * aborted, it should behave as the write was never happened. | ||
| */ | ||
| void abort(); | ||
| } |
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -37,6 +37,7 @@ | |
| import com.amazonaws.services.s3.model.PutObjectRequest; | ||
| import com.amazonaws.services.s3.model.PutObjectResult; | ||
| import com.amazonaws.services.s3.model.UploadPartRequest; | ||
| import org.apache.hadoop.fs.Abortable; | ||
| import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions; | ||
| import org.apache.hadoop.thirdparty.com.google.common.util.concurrent.Futures; | ||
| import org.apache.hadoop.thirdparty.com.google.common.util.concurrent.ListenableFuture; | ||
|
|
@@ -74,7 +75,7 @@ | |
| @InterfaceAudience.Private | ||
| @InterfaceStability.Unstable | ||
| class S3ABlockOutputStream extends OutputStream implements | ||
| StreamCapabilities, IOStatisticsSource { | ||
| StreamCapabilities, IOStatisticsSource, Abortable { | ||
|
|
||
| private static final Logger LOG = | ||
| LoggerFactory.getLogger(S3ABlockOutputStream.class); | ||
|
|
@@ -541,6 +542,10 @@ public boolean hasCapability(String capability) { | |
| case StreamCapabilities.IOSTATISTICS: | ||
| return true; | ||
|
|
||
| // S3A supports abort. | ||
| case StreamCapabilities.ABORTABLE: | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. We can merge both case statements |
||
| return true; | ||
|
|
||
| default: | ||
| return false; | ||
| } | ||
|
|
@@ -551,6 +556,24 @@ public IOStatistics getIOStatistics() { | |
| return iostatistics; | ||
| } | ||
|
|
||
| @Override | ||
| public void abort() { | ||
|
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Migrating comment in HeartSaVioR@63c5588#r46507150
|
||
| if (closed.getAndSet(true)) { | ||
| // already closed | ||
| LOG.debug("Ignoring abort() as stream is already closed"); | ||
| return; | ||
| } | ||
|
|
||
| S3ADataBlocks.DataBlock block = getActiveBlock(); | ||
|
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Migrating comment in HeartSaVioR@63c5588#r46506593
|
||
| try { | ||
| if (multiPartUpload != null) { | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Wondering what happens in case of non multipart upload
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Got it,we are closing the stream before only. |
||
| multiPartUpload.abort(); | ||
|
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Migrating comment in HeartSaVioR@63c5588#r46506765
|
||
| } | ||
| } finally { | ||
| cleanupWithLogger(LOG, block, blockFactory); | ||
| } | ||
| } | ||
|
|
||
| /** | ||
| * Multiple partition upload. | ||
| */ | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -20,6 +20,7 @@ | |
|
|
||
| import org.apache.hadoop.conf.Configuration; | ||
| import org.apache.hadoop.fs.FSDataOutputStream; | ||
| import org.apache.hadoop.fs.FileSystem; | ||
| import org.apache.hadoop.fs.Path; | ||
| import org.apache.hadoop.fs.contract.ContractTestUtils; | ||
| import org.apache.hadoop.fs.s3a.statistics.BlockOutputStreamStatistics; | ||
|
|
@@ -155,4 +156,22 @@ public void testMarkReset() throws Throwable { | |
| markAndResetDatablock(createFactory(getFileSystem())); | ||
| } | ||
|
|
||
| @Test | ||
| public void testAbortAfterWrite() throws Throwable { | ||
| Path dest = path("testAbortAfterWrite"); | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. use getMethodName() |
||
| describe(" testAbortAfterWrite"); | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. nit: A bit more explanatory. |
||
| FileSystem fs = getFileSystem(); | ||
| FSDataOutputStream stream = fs.create(dest, true); | ||
| byte[] data = ContractTestUtils.dataset(16, 'a', 26); | ||
| try { | ||
| stream.write(data); | ||
| stream.abort(); | ||
| // the path should not exist | ||
| ContractTestUtils.assertPathsDoNotExist(fs, "aborted file", dest); | ||
| } finally { | ||
| IOUtils.closeStream(stream); | ||
| // check the path doesn't exist "after" closing stream | ||
| ContractTestUtils.assertPathsDoNotExist(fs, "aborted file", dest); | ||
| } | ||
| } | ||
| } | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -82,4 +82,30 @@ public void testWriteOperationHelperPartLimits() throws Throwable { | |
| () -> woh.newUploadPartRequest(key, | ||
| "uploadId", 50000, 1024, inputStream, null, 0L)); | ||
| } | ||
|
|
||
| static class StreamClosedException extends IOException {} | ||
|
|
||
| @Test | ||
|
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Migrating comment in HeartSaVioR@63c5588#r46506646
|
||
| 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()); | ||
|
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Migrating comment in HeartSaVioR@63c5588#r46507220
|
||
|
|
||
| // check that calling write() will call checkOpen() and throws exception | ||
| doThrow(new StreamClosedException()).when(stream).checkOpen(); | ||
|
|
||
| intercept(StreamClosedException.class, | ||
| () -> stream.write(new byte[] {'a', 'b', 'c'})); | ||
| } | ||
|
|
||
| @Test | ||
| public void testCallingCloseAfterCallingAbort() throws Exception { | ||
| stream.abort(); | ||
|
|
||
| // This shouldn't throw IOException like calling close() multiple times. | ||
| // This will ensure abort() can be called with try-with-resource. | ||
| stream.close(); | ||
| } | ||
| } | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -20,6 +20,9 @@ | |
|
|
||
| import java.io.File; | ||
|
|
||
| import org.apache.hadoop.fs.FSDataOutputStream; | ||
| import org.apache.hadoop.fs.FileSystem; | ||
| import org.apache.hadoop.io.IOUtils; | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. reposition in |
||
| import org.assertj.core.api.Assertions; | ||
| import org.junit.Test; | ||
|
|
||
|
|
@@ -118,4 +121,29 @@ public void testCommitLimitFailure() throws Throwable { | |
| describedAs("commit abort count") | ||
| .isEqualTo(initial + 1); | ||
| } | ||
|
|
||
| @Test | ||
| public void testAbortAfterTwoPartUpload() throws Throwable { | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. javadoc or describe() to explain the test. |
||
| Path file = path(getMethodName()); | ||
|
|
||
| byte[] data = dataset(6 * _1MB, 'a', 'z' - 'a'); | ||
|
|
||
| FileSystem fs = getFileSystem(); | ||
| FSDataOutputStream stream = fs.create(file, true); | ||
| try { | ||
| stream.write(data); | ||
|
|
||
| // From testTwoPartUpload() we know closing stream will finalize uploads | ||
| // and materialize the path. Here we call abort() to abort the upload, | ||
| // and ensure the path is NOT available. (uploads are aborted) | ||
|
|
||
| stream.abort(); | ||
| // the path should not exist | ||
| assertPathDoesNotExist("upload must not have completed", file); | ||
| } finally { | ||
| IOUtils.closeStream(stream); | ||
| // check the path doesn't exist "after" closing stream | ||
| assertPathDoesNotExist("upload must not have completed", file); | ||
| } | ||
| } | ||
| } | ||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
reposition in
org.apache.*block below.