-
Notifications
You must be signed in to change notification settings - Fork 3.2k
[core] fix #9997 - Handle s3a file upload interrupt which results in table metadata pointing to files that doesn't exist #9998
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
Merged
Merged
Changes from all commits
Commits
Show all changes
12 commits
Select commit
Hold shift + click to select a range
4394e90
throw io exception on close if hadoopstreams interrupted
91163ae
edit exception message
32b620f
use fully qualified name instead
c1c022a
add unit test
93fbc9a
update exception message
bb0252f
fix imports
0eafe82
review comments refactor
478ac3d
fix import
87dd6e7
address review comments
d57baa8
update test to mock upload interrupt from another thread
7e24aa9
remove runtime assert for test
e601204
retrigger checks
File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
67 changes: 67 additions & 0 deletions
67
core/src/test/java/org/apache/hadoop/fs/s3a/S3ABlockOutputStream.java
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,67 @@ | ||
| /* | ||
| * 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.s3a; | ||
|
|
||
| import java.io.IOException; | ||
| import java.io.OutputStream; | ||
| import java.util.concurrent.CancellationException; | ||
| import java.util.concurrent.ExecutorService; | ||
| import java.util.concurrent.Executors; | ||
| import java.util.concurrent.Future; | ||
|
|
||
| /** mock class for testing hadoop s3a writer */ | ||
| public class S3ABlockOutputStream extends OutputStream { | ||
| public ExecutorService mockCloseService; | ||
| public Future<?> mockUploadOnClose; | ||
|
|
||
| public S3ABlockOutputStream() { | ||
| mockCloseService = Executors.newSingleThreadExecutor(); | ||
| } | ||
|
|
||
| @Override | ||
| public void write(int b) throws IOException { | ||
| throw new IOException("mocked class, do not use"); | ||
| } | ||
|
|
||
| @Override | ||
| public void close() throws IOException { | ||
| try { | ||
| mockUploadOnClose = | ||
| mockCloseService.submit( | ||
|
stevenzwu marked this conversation as resolved.
|
||
| () -> { | ||
| try { | ||
| Thread.sleep(30 * 1000); | ||
|
stevenzwu marked this conversation as resolved.
|
||
| } catch (InterruptedException e) { | ||
| // ignore | ||
| } | ||
| }); | ||
| mockUploadOnClose.get(); | ||
| } catch (CancellationException | InterruptedException e) { | ||
| // mock interrupt in S3ABlockOutputStream#putObject | ||
| Thread.currentThread().interrupt(); | ||
| } catch (Exception e) { | ||
| throw new RuntimeException(e); | ||
| } | ||
| super.close(); | ||
| } | ||
|
|
||
| public void interruptClose() { | ||
| mockUploadOnClose.cancel(true); | ||
| } | ||
| } | ||
53 changes: 53 additions & 0 deletions
53
core/src/test/java/org/apache/iceberg/hadoop/TestHadoopStreams.java
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,53 @@ | ||
| /* | ||
| * 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.iceberg.hadoop; | ||
|
|
||
| import java.io.IOException; | ||
| import java.util.concurrent.Executors; | ||
| import org.apache.hadoop.fs.FSDataOutputStream; | ||
| import org.apache.hadoop.fs.s3a.S3ABlockOutputStream; | ||
| import org.apache.iceberg.io.PositionOutputStream; | ||
| import org.assertj.core.api.Assertions; | ||
| import org.junit.jupiter.api.Test; | ||
|
|
||
| class TestHadoopStreams { | ||
|
|
||
| @Test | ||
| void closeShouldThrowIOExceptionWhenInterrupted() throws Exception { | ||
|
|
||
| S3ABlockOutputStream s3ABlockOutputStream = new S3ABlockOutputStream(); | ||
| FSDataOutputStream fsDataOutputStream = new FSDataOutputStream(s3ABlockOutputStream, null); | ||
| PositionOutputStream wrap = HadoopStreams.wrap(fsDataOutputStream); | ||
| // interrupt mock upload on close after a delay | ||
| Executors.newSingleThreadExecutor() | ||
| .execute( | ||
| () -> { | ||
| try { | ||
| Thread.sleep(1000); | ||
| } catch (InterruptedException e) { | ||
| throw new RuntimeException(e); | ||
| } | ||
| s3ABlockOutputStream.interruptClose(); | ||
| }); | ||
|
|
||
| Assertions.assertThatThrownBy(wrap::close) | ||
| .isInstanceOf(IOException.class) | ||
| .hasMessage("S3ABlockOutputStream failed to upload object after stream was closed"); | ||
| } | ||
| } |
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
Uh oh!
There was an error while loading. Please reload this page.