-
Notifications
You must be signed in to change notification settings - Fork 13.9k
[FLINK-39533][s3] Use abort() instead of drain on close/seek when remaining bytes exceed threshold in NativeS3InputStream #28012
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
base: master
Are you sure you want to change the base?
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 |
|---|---|---|
|
|
@@ -27,7 +27,10 @@ | |
| import software.amazon.awssdk.services.s3.model.GetObjectRequest; | ||
| import software.amazon.awssdk.services.s3.model.GetObjectResponse; | ||
|
|
||
| import javax.annotation.concurrent.GuardedBy; | ||
|
|
||
| import java.io.BufferedInputStream; | ||
| import java.io.EOFException; | ||
| import java.io.IOException; | ||
| import java.util.concurrent.locks.ReentrantLock; | ||
|
|
||
|
|
@@ -56,9 +59,16 @@ class NativeS3InputStream extends FSDataInputStream { | |
| private final long contentLength; | ||
| private final int readBufferSize; | ||
|
|
||
| @GuardedBy("lock") | ||
| private ResponseInputStream<GetObjectResponse> currentStream; | ||
|
|
||
| @GuardedBy("lock") | ||
| private BufferedInputStream bufferedStream; | ||
|
|
||
| @GuardedBy("lock") | ||
| private long position; | ||
|
|
||
| @GuardedBy("lock") | ||
| private volatile boolean closed; | ||
|
|
||
| public NativeS3InputStream( | ||
|
|
@@ -108,24 +118,7 @@ private void lazyInitialize() throws IOException { | |
| private void openStreamAtCurrentPosition() throws IOException { | ||
| lock.lock(); | ||
| try { | ||
| if (bufferedStream != null) { | ||
| try { | ||
| bufferedStream.close(); | ||
| } catch (IOException e) { | ||
| LOG.warn("Error closing buffered stream for {}/{}", bucketName, key, e); | ||
| } finally { | ||
| bufferedStream = null; | ||
| } | ||
| } | ||
| if (currentStream != null) { | ||
| try { | ||
| currentStream.close(); | ||
| } catch (IOException e) { | ||
| LOG.warn("Error closing S3 response stream for {}/{}", bucketName, key, e); | ||
| } finally { | ||
| currentStream = null; | ||
| } | ||
| } | ||
| releaseStreams(); | ||
|
|
||
| try { | ||
| GetObjectRequest.Builder requestBuilder = | ||
|
|
@@ -143,27 +136,72 @@ private void openStreamAtCurrentPosition() throws IOException { | |
| currentStream = s3Client.getObject(requestBuilder.build()); | ||
| bufferedStream = new BufferedInputStream(currentStream, readBufferSize); | ||
| } catch (Exception e) { | ||
| if (bufferedStream != null) { | ||
| try { | ||
| bufferedStream.close(); | ||
| } catch (IOException ignored) { | ||
| } | ||
| bufferedStream = null; | ||
| } | ||
| if (currentStream != null) { | ||
| try { | ||
| currentStream.close(); | ||
| } catch (IOException ignored) { | ||
| } | ||
| currentStream = null; | ||
| } | ||
| releaseStreams(); | ||
| throw new IOException("Failed to open S3 stream for " + bucketName + "/" + key, e); | ||
| } | ||
| } finally { | ||
| lock.unlock(); | ||
| } | ||
| } | ||
|
|
||
| /** | ||
| * Aborts the in-flight HTTP connection so that subsequent {@code close()} calls on the stream | ||
| * do not drain remaining bytes over the network. | ||
| * | ||
| * @see ResponseInputStream#abort() | ||
| */ | ||
|
gaborgsomogyi marked this conversation as resolved.
|
||
| @GuardedBy("lock") | ||
| private void abortCurrentStream() { | ||
| assert lock.isHeldByCurrentThread() : "abortCurrentStream() requires lock to be held"; | ||
| if (currentStream != 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. If
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. Good catch — @GuardedBy("lock") is only a static-analysis hint and doesn't actually enforce anything at runtime, so your "silent break on a lock move" scenario was real. I've added a runtime precondition at the top of both @GuardedBy helpers (abortCurrentStream() and releaseStreams()) |
||
| try { | ||
| currentStream.abort(); | ||
| } catch (RuntimeException e) { | ||
| LOG.warn("Error aborting S3 response stream for {}/{}", bucketName, key, e); | ||
| } | ||
| } | ||
| } | ||
|
|
||
| /** | ||
| * Aborts and closes both streams, nulling the references. The abort is called first to prevent | ||
| * {@link ResponseInputStream#close()} from draining remaining bytes over the network. | ||
| * | ||
| * @return the first {@link IOException} encountered (with subsequent ones added as suppressed), | ||
| * or {@code null} if cleanup succeeded without errors | ||
| */ | ||
| @GuardedBy("lock") | ||
| private IOException releaseStreams() { | ||
| assert lock.isHeldByCurrentThread() : "releaseStreams() requires lock to be held"; | ||
| abortCurrentStream(); | ||
| IOException exception = null; | ||
|
|
||
| if (bufferedStream != null) { | ||
| try { | ||
| bufferedStream.close(); | ||
| } catch (IOException e) { | ||
| exception = e; | ||
| LOG.warn("Error closing buffered stream for {}/{}", bucketName, key, e); | ||
| } finally { | ||
| bufferedStream = null; | ||
| } | ||
| } | ||
| if (currentStream != null) { | ||
| try { | ||
| currentStream.close(); | ||
| } catch (IOException e) { | ||
| if (exception == null) { | ||
| exception = e; | ||
| } else { | ||
| exception.addSuppressed(e); | ||
| } | ||
| LOG.warn("Error closing S3 response stream for {}/{}", bucketName, key, e); | ||
| } finally { | ||
| currentStream = null; | ||
| } | ||
| } | ||
| return exception; | ||
| } | ||
|
|
||
| @Override | ||
| public void seek(long desired) throws IOException { | ||
| lock(); | ||
|
|
@@ -172,7 +210,14 @@ public void seek(long desired) throws IOException { | |
| throw new IOException("Stream is closed"); | ||
| } | ||
| if (desired < 0) { | ||
| throw new IOException("Cannot seek to negative position: " + desired); | ||
| throw new EOFException("Cannot seek to negative position: " + desired); | ||
| } | ||
| if (desired > contentLength) { | ||
| throw new EOFException( | ||
| "Cannot seek past end of stream: position=" | ||
| + desired | ||
| + ", length=" | ||
| + contentLength); | ||
| } | ||
|
|
||
| if (desired != position) { | ||
|
|
@@ -270,33 +315,8 @@ public void close() throws IOException { | |
| } | ||
|
|
||
| closed = true; | ||
| IOException exception = null; | ||
|
|
||
| if (bufferedStream != null) { | ||
| try { | ||
| bufferedStream.close(); | ||
| } catch (IOException e) { | ||
| exception = e; | ||
| LOG.warn("Error closing buffered stream for {}/{}", bucketName, key, e); | ||
| } finally { | ||
| bufferedStream = null; | ||
| } | ||
| } | ||
|
|
||
| if (currentStream != null) { | ||
| try { | ||
| currentStream.close(); | ||
| } catch (IOException e) { | ||
| if (exception == null) { | ||
| exception = e; | ||
| } else { | ||
| exception.addSuppressed(e); | ||
| } | ||
| LOG.warn("Error closing S3 response stream for {}/{}", bucketName, key, e); | ||
| } finally { | ||
| currentStream = null; | ||
| } | ||
| } | ||
| IOException exception = releaseStreams(); | ||
|
|
||
| LOG.debug( | ||
| "Closed S3 input stream - bucket: {}, key: {}, final position: {}/{}", | ||
|
|
||
Uh oh!
There was an error while loading. Please reload this page.