Skip to content
Closed
Show file tree
Hide file tree
Changes from 4 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 @@ -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;
}
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Instead, I think we should update AbfsOutputStream.close to wrap the exception, short of having the Java implementers fix this. :)

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

thought of that, but also thought it might be good to differentiate exception raised in the later bits of the flush. Keeping it in close() is cleaner as it puts the problem where it belongs

writeCurrentBufferToService();
flushWrittenBytesToService(isClose);
}
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,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;
Expand All @@ -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.
Expand Down Expand Up @@ -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();
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 {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Double close would have already been passing with these changes. To really test the try-with-resources I think you need to modify the underlying blob after fs.create but before flush so that flush fails and sets AbfsOutputStream.lastError. For example, if you write some data to the blob so that the position held by the AbfsOutputStream is invalid, flush may fail. For example, the following test:

out = fs.create(path)
out.write('a');
// externally write data to the same path so that the next call to flush will fail due to position being off.
intercept(IOException.class, () -> out.flush());
intercept(IOException.class, () -> out.close());

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

You are right, I couldn't think how to reliably replicate it. without going near mockito etc.,

or we add a package level setLastError() call?

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think you can simply create a second stream for the same path, or even call delete on it after writing some data with the other stream. Basically, write to the same path with two streams.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

yes, write+delete does the trick

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();
}
}