-
Notifications
You must be signed in to change notification settings - Fork 9.2k
HADOOP-16785: wasb to raise IOE if write() invoked on a closed stream #1791
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
Changes from 4 commits
ca7a879
15e88b0
a675fe2
a2c8541
cc1b580
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 |
|---|---|---|
|
|
@@ -259,7 +259,15 @@ public synchronized void close() throws IOException { | |
| } | ||
|
|
||
| private synchronized void flushInternal(boolean isClose) throws IOException { | ||
| maybeThrowLastError(); | ||
| try { | ||
| maybeThrowLastError(); | ||
| } catch (IOException e) { | ||
| if (isClose) { | ||
| // wrap existing exception so as to avoid breaking try-with-resources | ||
| throw new IOException("Skipping final flush and write due to " + e, e); | ||
| } else | ||
| throw e; | ||
| } | ||
|
||
| writeCurrentBufferToService(); | ||
| flushWrittenBytesToService(isClose); | ||
| } | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -19,6 +19,8 @@ | |
| package org.apache.hadoop.fs.azurebfs; | ||
|
|
||
| import java.io.FileNotFoundException; | ||
| import java.io.FilterOutputStream; | ||
| import java.io.IOException; | ||
| import java.util.EnumSet; | ||
|
|
||
| import org.junit.Test; | ||
|
|
@@ -29,6 +31,7 @@ | |
| import org.apache.hadoop.fs.permission.FsPermission; | ||
|
|
||
| import static org.apache.hadoop.fs.contract.ContractTestUtils.assertIsFile; | ||
| import static org.apache.hadoop.test.LambdaTestUtils.intercept; | ||
|
|
||
| /** | ||
| * Test create operation. | ||
|
|
@@ -104,4 +107,39 @@ public void testCreateNonRecursive2() throws Exception { | |
| .close(); | ||
| assertIsFile(fs, testFile); | ||
| } | ||
|
|
||
| /** | ||
| * Attempts to use to the ABFS stream after it is closed. | ||
| */ | ||
| @Test | ||
| public void testWriteAfterClose() throws Throwable { | ||
| final AzureBlobFileSystem fs = getFileSystem(); | ||
| Path testPath = new Path(TEST_FOLDER_PATH, TEST_CHILD_FILE); | ||
| FSDataOutputStream out = fs.create(testPath); | ||
| out.close(); | ||
| intercept(IOException.class, () -> out.write('a')); | ||
| intercept(IOException.class, () -> out.write(new byte[]{'a'})); | ||
| // hsync is not ignored on a closed stream | ||
| // out.hsync(); | ||
steveloughran marked this conversation as resolved.
Show resolved
Hide resolved
|
||
| out.flush(); | ||
| out.close(); | ||
| } | ||
|
|
||
| /** | ||
| * Attempts to double close an ABFS output stream from within a | ||
| * FilterOutputStream. | ||
| * That class handles a double failure on close badly if the second | ||
| * exception rethrows the first. | ||
| */ | ||
| @Test | ||
| public void testFilteredDoubleClose() throws Throwable { | ||
|
||
| final AzureBlobFileSystem fs = getFileSystem(); | ||
| Path testPath = new Path(TEST_FOLDER_PATH, TEST_CHILD_FILE); | ||
| FilterOutputStream out = new FilterOutputStream(fs.create(testPath)); | ||
| out.close(); | ||
| intercept(IOException.class, () -> out.write('a')); | ||
| intercept(IOException.class, () -> out.write(new byte[]{'a'})); | ||
| out.flush(); | ||
| out.close(); | ||
| } | ||
| } | ||
Uh oh!
There was an error while loading. Please reload this page.