-
Notifications
You must be signed in to change notification settings - Fork 8.9k
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
HADOOP-19221. S3A: Unable to recover from failure of multipart block …
…upload attempt (#6938) (#7044) This is a major change which handles 400 error responses when uploading large files from memory heap/buffer (or staging committer) and the remote S3 store returns a 500 response from a upload of a block in a multipart upload. The SDK's own streaming code seems unable to fully replay the upload; at attempts to but then blocks and the S3 store returns a 400 response "Your socket connection to the server was not read from or written to within the timeout period. Idle connections will be closed. (Service: S3, Status Code: 400...)" There is an option to control whether or not the S3A client itself attempts to retry on a 50x error other than 503 throttling events (which are independently processed as before) Option: fs.s3a.retry.http.5xx.errors Default: true 500 errors are very rare from standard AWS S3, which has a five nines SLA. It may be more common against S3 Express which has lower guarantees. Third party stores have unknown guarantees, and the exception may indicate a bad server configuration. Consider setting fs.s3a.retry.http.5xx.errors to false when working with such stores. Signification Code changes: There is now a custom set of implementations of software.amazon.awssdk.http.ContentStreamProvidercontent in the class org.apache.hadoop.fs.s3a.impl.UploadContentProviders. These: * Restart on failures * Do not copy buffers/byte buffers into new private byte arrays, so avoid exacerbating memory problems.. There new IOStatistics for specific http error codes -these are collected even when all recovery is performed within the SDK. S3ABlockOutputStream has major changes, including handling of Thread.interrupt() on the main thread, which now triggers and briefly awaits cancellation of any ongoing uploads. If the writing thread is interrupted in close(), it is mapped to an InterruptedIOException. Applications like Hive and Spark must catch these after cancelling a worker thread. Contributed by Steve Loughran
- Loading branch information
1 parent
ea4137b
commit 97abdb4
Showing
53 changed files
with
4,250 additions
and
985 deletions.
There are no files selected for viewing
This file contains 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
199 changes: 199 additions & 0 deletions
199
...project/hadoop-common/src/main/java/org/apache/hadoop/fs/store/ByteBufferInputStream.java
This file contains 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,199 @@ | ||
/* | ||
* 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.store; | ||
|
||
import java.io.EOFException; | ||
import java.io.IOException; | ||
import java.io.InputStream; | ||
import java.nio.ByteBuffer; | ||
|
||
import org.slf4j.Logger; | ||
import org.slf4j.LoggerFactory; | ||
|
||
import org.apache.hadoop.fs.FSExceptionMessages; | ||
import org.apache.hadoop.util.Preconditions; | ||
|
||
/** | ||
* Provide an input stream from a byte buffer; supporting | ||
* {@link #mark(int)}. | ||
*/ | ||
public final class ByteBufferInputStream extends InputStream { | ||
private static final Logger LOG = | ||
LoggerFactory.getLogger(ByteBufferInputStream.class); | ||
|
||
/** Size of the buffer. */ | ||
private final int size; | ||
|
||
/** | ||
* Not final so that in close() it will be set to null, which | ||
* may result in faster cleanup of the buffer. | ||
*/ | ||
private ByteBuffer byteBuffer; | ||
|
||
public ByteBufferInputStream(int size, | ||
ByteBuffer byteBuffer) { | ||
LOG.debug("Creating ByteBufferInputStream of size {}", size); | ||
this.size = size; | ||
this.byteBuffer = byteBuffer; | ||
} | ||
|
||
/** | ||
* After the stream is closed, set the local reference to the byte | ||
* buffer to null; this guarantees that future attempts to use | ||
* stream methods will fail. | ||
*/ | ||
@Override | ||
public synchronized void close() { | ||
LOG.debug("ByteBufferInputStream.close()"); | ||
byteBuffer = null; | ||
} | ||
|
||
/** | ||
* Is the stream open? | ||
* @return true if the stream has not been closed. | ||
*/ | ||
public synchronized boolean isOpen() { | ||
return byteBuffer != null; | ||
} | ||
|
||
/** | ||
* Verify that the stream is open. | ||
* @throws IOException if the stream is closed | ||
*/ | ||
private void verifyOpen() throws IOException { | ||
if (byteBuffer == null) { | ||
throw new IOException(FSExceptionMessages.STREAM_IS_CLOSED); | ||
} | ||
} | ||
|
||
/** | ||
* Check the open state. | ||
* @throws IllegalStateException if the stream is closed. | ||
*/ | ||
private void checkOpenState() { | ||
Preconditions.checkState(isOpen(), | ||
FSExceptionMessages.STREAM_IS_CLOSED); | ||
} | ||
|
||
public synchronized int read() throws IOException { | ||
if (available() > 0) { | ||
return byteBuffer.get() & 0xFF; | ||
} else { | ||
return -1; | ||
} | ||
} | ||
|
||
@Override | ||
public synchronized long skip(long offset) throws IOException { | ||
verifyOpen(); | ||
long newPos = position() + offset; | ||
if (newPos < 0) { | ||
throw new EOFException(FSExceptionMessages.NEGATIVE_SEEK); | ||
} | ||
if (newPos > size) { | ||
throw new EOFException(FSExceptionMessages.CANNOT_SEEK_PAST_EOF); | ||
} | ||
byteBuffer.position((int) newPos); | ||
return newPos; | ||
} | ||
|
||
@Override | ||
public synchronized int available() { | ||
checkOpenState(); | ||
return byteBuffer.remaining(); | ||
} | ||
|
||
/** | ||
* Get the current buffer position. | ||
* @return the buffer position | ||
*/ | ||
public synchronized int position() { | ||
checkOpenState(); | ||
return byteBuffer.position(); | ||
} | ||
|
||
/** | ||
* Check if there is data left. | ||
* @return true if there is data remaining in the buffer. | ||
*/ | ||
public synchronized boolean hasRemaining() { | ||
checkOpenState(); | ||
return byteBuffer.hasRemaining(); | ||
} | ||
|
||
@Override | ||
public synchronized void mark(int readlimit) { | ||
LOG.debug("mark at {}", position()); | ||
checkOpenState(); | ||
byteBuffer.mark(); | ||
} | ||
|
||
@Override | ||
public synchronized void reset() throws IOException { | ||
LOG.debug("reset"); | ||
checkOpenState(); | ||
byteBuffer.reset(); | ||
} | ||
|
||
@Override | ||
public boolean markSupported() { | ||
return true; | ||
} | ||
|
||
/** | ||
* Read in data. | ||
* @param b destination buffer. | ||
* @param offset offset within the buffer. | ||
* @param length length of bytes to read. | ||
* @throws EOFException if the position is negative | ||
* @throws IndexOutOfBoundsException if there isn't space for the | ||
* amount of data requested. | ||
* @throws IllegalArgumentException other arguments are invalid. | ||
*/ | ||
@SuppressWarnings("NullableProblems") | ||
public synchronized int read(byte[] b, int offset, int length) | ||
throws IOException { | ||
Preconditions.checkArgument(length >= 0, "length is negative"); | ||
Preconditions.checkArgument(b != null, "Null buffer"); | ||
if (b.length - offset < length) { | ||
throw new IndexOutOfBoundsException( | ||
FSExceptionMessages.TOO_MANY_BYTES_FOR_DEST_BUFFER | ||
+ ": request length =" + length | ||
+ ", with offset =" + offset | ||
+ "; buffer capacity =" + (b.length - offset)); | ||
} | ||
verifyOpen(); | ||
if (!hasRemaining()) { | ||
return -1; | ||
} | ||
|
||
int toRead = Math.min(length, available()); | ||
byteBuffer.get(b, offset, toRead); | ||
return toRead; | ||
} | ||
|
||
@Override | ||
public String toString() { | ||
return "ByteBufferInputStream{" + | ||
"size=" + size + | ||
", byteBuffer=" + byteBuffer + | ||
((byteBuffer != null) ? ", available=" + byteBuffer.remaining() : "") + | ||
"} " + super.toString(); | ||
} | ||
} |
Oops, something went wrong.