Skip to content

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

Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -406,7 +406,7 @@ public static <E extends Throwable> E intercept(
throws Exception {
try {
eval.call();
throw new AssertionError("Expected an exception");
throw new AssertionError("Expected an exception of type " + clazz);
} catch (Throwable e) {
if (clazz.isAssignableFrom(e.getClass())) {
return (E)e;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -1083,6 +1083,7 @@ public synchronized void close() throws IOException {
*/
@Override
public void write(int b) throws IOException {
checkOpen();
try {
out.write(b);
} catch(IOException e) {
Expand All @@ -1106,6 +1107,7 @@ public void write(int b) throws IOException {
*/
@Override
public void write(byte[] b) throws IOException {
checkOpen();
try {
out.write(b);
} catch(IOException e) {
Expand Down Expand Up @@ -1136,6 +1138,7 @@ public void write(byte[] b) throws IOException {
*/
@Override
public void write(byte[] b, int off, int len) throws IOException {
checkOpen();
try {
out.write(b, off, len);
} catch(IOException e) {
Expand Down Expand Up @@ -1198,6 +1201,17 @@ public void setEncodedKey(String anEncodedKey) {
private void restoreKey() throws IOException {
store.rename(getEncodedKey(), getKey());
}

/**
* Check for the stream being open.
* @throws IOException if the stream is closed.
*/
private void checkOpen() throws IOException {
if (out == null) {
throw new IOException(FSExceptionMessages.STREAM_IS_CLOSED);
}
}

}

private URI uri;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -43,6 +43,8 @@
import org.apache.hadoop.fs.StreamCapabilities;
import org.apache.hadoop.fs.Syncable;

import static org.apache.hadoop.io.IOUtils.wrapException;

/**
* The BlobFsOutputStream for Rest AbfsClient.
*/
Expand Down Expand Up @@ -246,6 +248,12 @@ public synchronized void close() throws IOException {
try {
flushInternal(true);
threadExecutor.shutdown();
} catch (IOException e) {
// Problems surface in try-with-resources clauses if
// the exception thrown in a close == the one already thrown
// -so we wrap any exception with a new one.
// See HADOOP-16785
throw wrapException(path, e.getMessage(), e);
} finally {
lastError = new IOException(FSExceptionMessages.STREAM_IS_CLOSED);
buffer = null;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -52,7 +52,7 @@ public abstract class AbstractWasbTestBase extends AbstractWasbTestWithTimeout
@Before
public void setUp() throws Exception {
AzureBlobStorageTestAccount account = createTestAccount();
assumeNotNull(account);
assumeNotNull("test account", account);
bindToTestAccount(account);
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@
package org.apache.hadoop.fs.azure;

import java.io.FileNotFoundException;
import java.io.IOException;

import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FSDataOutputStream;
Expand All @@ -30,7 +31,9 @@
import org.junit.After;
import org.junit.Test;

import static org.apache.hadoop.fs.FSExceptionMessages.STREAM_IS_CLOSED;
import static org.apache.hadoop.fs.azure.ExceptionHandlingTestHelper.*;
import static org.apache.hadoop.test.LambdaTestUtils.intercept;

/**
* Single threaded exception handling.
Expand Down Expand Up @@ -265,6 +268,23 @@ public void testSingleThreadedPageBlobOpenScenario() throws Throwable {
inputStream = fs.open(testPath);
}

/**
* Attempts to write to the azure stream after it is closed will raise
* an IOException.
*/
@Test
public void testWriteAfterClose() throws Throwable {
FSDataOutputStream out = fs.create(testPath);
out.close();
intercept(IOException.class, STREAM_IS_CLOSED,
() -> out.write('a'));
intercept(IOException.class, STREAM_IS_CLOSED,
() -> out.write(new byte[]{'a'}));
out.hsync();
out.flush();
out.close();
}

@After
public void tearDown() throws Exception {
if (inputStream != null) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@
package org.apache.hadoop.fs.azurebfs;

import java.io.FileNotFoundException;
import java.io.IOException;
import java.util.EnumSet;

import org.junit.Test;
Expand All @@ -27,8 +28,10 @@
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.test.GenericTestUtils;

import static org.apache.hadoop.fs.contract.ContractTestUtils.assertIsFile;
import static org.apache.hadoop.test.LambdaTestUtils.intercept;

/**
* Test create operation.
Expand Down Expand Up @@ -104,4 +107,52 @@ 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();
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 testTryWithResources() throws Throwable {
final AzureBlobFileSystem fs = getFileSystem();
Path testPath = new Path(TEST_FOLDER_PATH, TEST_CHILD_FILE);
try (FSDataOutputStream out = fs.create(testPath)) {
out.write('1');
out.hsync();
// this will cause the next write to failAll
fs.delete(testPath, false);
out.write('2');
out.hsync();
fail("Expected a failure");
} catch (FileNotFoundException fnfe) {
// the exception raised in close() must be in the caught exception's
// suppressed list
Throwable[] suppressed = fnfe.getSuppressed();
assertEquals("suppressed count", 1, suppressed.length);
Throwable inner = suppressed[0];
if (!(inner instanceof IOException)) {
throw inner;
}
GenericTestUtils.assertExceptionContains(fnfe.getMessage(), inner);
}
}
}