Skip to content

feat(storage:s3): multi-part upload: upload parts concurrently #272

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

Open
wants to merge 1 commit into
base: main
Choose a base branch
from
Open
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
1 change: 1 addition & 0 deletions checkstyle/suppressions.xml
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@
<suppress checks="ClassDataAbstractionCoupling" files=".*Test\.java"/>
<suppress checks="ClassFanOutComplexity" files=".*Test\.java"/>
<suppress checks="ClassFanOutComplexity" files="RemoteStorageManager.java"/>
<suppress checks="ClassDataAbstractionCoupling" files="S3MultiPartOutputStream.java"/>
<suppress checks="ClassDataAbstractionCoupling" files="S3StorageConfig.java"/>
<suppress checks="ClassDataAbstractionCoupling" files="RemoteStorageManager.java"/>
<suppress checks="ClassDataAbstractionCoupling" files="Metrics.java"/>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,11 +18,14 @@

import java.io.ByteArrayInputStream;
import java.io.IOException;
import java.io.InputStream;
import java.io.OutputStream;
import java.nio.ByteBuffer;
import java.util.ArrayList;
import java.util.List;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.ConcurrentLinkedQueue;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.atomic.AtomicInteger;

import com.amazonaws.services.s3.AmazonS3;
import com.amazonaws.services.s3.model.AbortMultipartUploadRequest;
Expand All @@ -31,14 +34,17 @@
import com.amazonaws.services.s3.model.InitiateMultipartUploadResult;
import com.amazonaws.services.s3.model.PartETag;
import com.amazonaws.services.s3.model.UploadPartRequest;
import com.amazonaws.services.s3.model.UploadPartResult;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

/**
* S3 multipart output stream.
* Enable uploads to S3 with unknown size by feeding input bytes to multiple parts and upload them on close.
*
* <p>OutputStream is used to write sequentially, but
* uploading parts happen asynchronously to reduce full upload latency.
* Concurrency happens within the output stream implementation and does not require changes on the callers.
*
* <p>Requires S3 client and starts a multipart transaction when instantiated. Do not reuse.
*
* <p>{@link S3MultiPartOutputStream} is not thread-safe.
Expand All @@ -54,8 +60,11 @@ public class S3MultiPartOutputStream extends OutputStream {
final int partSize;

private final String uploadId;
private final List<PartETag> partETags = new ArrayList<>();
private final AtomicInteger partNumber = new AtomicInteger(0);

// holds async part upload operations building a list of partETags required when committing
private CompletableFuture<ConcurrentLinkedQueue<PartETag>> partUploads =
CompletableFuture.completedFuture(new ConcurrentLinkedQueue<>());
private boolean closed;

public S3MultiPartOutputStream(final String bucketName,
Expand Down Expand Up @@ -87,15 +96,23 @@ public void write(final byte[] b, final int off, final int len) throws IOExcepti
return;
}
try {
final ByteBuffer source = ByteBuffer.wrap(b, off, len);
while (source.hasRemaining()) {
final int transferred = Math.min(partBuffer.remaining(), source.remaining());
final int offset = source.arrayOffset() + source.position();
// TODO: get rid of this array copying
partBuffer.put(source.array(), offset, transferred);
source.position(source.position() + transferred);
final ByteBuffer currentBatch = ByteBuffer.wrap(b, off, len);
while (currentBatch.hasRemaining()) {
// copy batch to part buffer
final int toCopy = Math.min(partBuffer.remaining(), currentBatch.remaining());
final int positionAfterCopying = currentBatch.position() + toCopy;
currentBatch.limit(positionAfterCopying);
partBuffer.put(currentBatch.slice());

// prepare current batch for next part
currentBatch.clear(); // reset limit
currentBatch.position(positionAfterCopying);

if (!partBuffer.hasRemaining()) {
flushBuffer(0, partSize);
partBuffer.position(0);
partBuffer.limit(partSize);
uploadPart(partBuffer.slice(), partSize);
partBuffer.clear();
}
}
} catch (final RuntimeException e) {
Expand All @@ -105,26 +122,48 @@ public void write(final byte[] b, final int off, final int len) throws IOExcepti
}
}

/**
* Completes pending part uploads
*
* @throws IOException if uploads fail and abort transaction
*/
@Override
public void close() throws IOException {
if (!isClosed()) {
public void flush() throws IOException {
try {
if (partBuffer.position() > 0) {
try {
flushBuffer(partBuffer.arrayOffset(), partBuffer.position());
} catch (final RuntimeException e) {
log.error("Failed to upload last part {}, aborting transaction", uploadId, e);
abortUpload();
throw new IOException(e);
}
// flush missing bytes
final int actualPartSize = partBuffer.position();
partBuffer.position(0);
partBuffer.limit(actualPartSize);
uploadPart(partBuffer.slice(), actualPartSize);
partBuffer.clear();
}
if (!partETags.isEmpty()) {

// wait for requests to be processed
partUploads.join();
} catch (final RuntimeException e) {
log.error("Failed to upload parts {}, aborting transaction", uploadId, e);
abortUpload();
throw new IOException("Failed to flush upload part operations", e);
}
}

@Override
public void close() throws IOException {
if (!isClosed()) {
flush();
if (partNumber.get() > 0) {
try {
completeUpload();
// wait for all uploads to complete successfully before committing
final ConcurrentLinkedQueue<PartETag> tagsQueue = partUploads.get(); // TODO: maybe set a timeout?
final ArrayList<PartETag> partETags = new ArrayList<>(tagsQueue);

completeUpload(partETags);
log.debug("Completed multipart upload {}", uploadId);
} catch (final RuntimeException e) {
} catch (final RuntimeException | InterruptedException | ExecutionException e) {
log.error("Failed to complete multipart upload {}, aborting transaction", uploadId, e);
abortUpload();
throw new IOException(e);
throw new IOException("Failed to complete upload transaction", e);
}
} else {
abortUpload();
Expand All @@ -136,7 +175,7 @@ public boolean isClosed() {
return closed;
}

private void completeUpload() {
private void completeUpload(final List<PartETag> partETags) {
final var request = new CompleteMultipartUploadRequest(bucketName, key, uploadId, partETags);
client.completeMultipartUpload(request);
closed = true;
Expand All @@ -148,24 +187,24 @@ private void abortUpload() {
closed = true;
}

private void flushBuffer(final int offset,
final int actualPartSize) {
final ByteArrayInputStream in = new ByteArrayInputStream(partBuffer.array(), offset, actualPartSize);
uploadPart(in, actualPartSize);
partBuffer.clear();
}

private void uploadPart(final InputStream in, final int actualPartSize) {
final int partNumber = partETags.size() + 1;
final UploadPartRequest uploadPartRequest =
new UploadPartRequest()
.withBucketName(bucketName)
.withKey(key)
.withUploadId(uploadId)
.withPartSize(actualPartSize)
.withPartNumber(partNumber)
.withInputStream(in);
final UploadPartResult uploadResult = client.uploadPart(uploadPartRequest);
partETags.add(uploadResult.getPartETag());
private void uploadPart(final ByteBuffer partBuffer, final int actualPartSize) {
final byte[] partContent = new byte[actualPartSize];
partBuffer.get(partContent, 0, actualPartSize);

final var uploadPartRequest = new UploadPartRequest()
.withBucketName(bucketName)
.withKey(key)
.withUploadId(uploadId)
.withPartSize(actualPartSize)
.withPartNumber(partNumber.incrementAndGet())
.withInputStream(new ByteArrayInputStream(partContent));

// Run request async
partUploads = partUploads.thenCombine(
CompletableFuture.supplyAsync(() -> client.uploadPart(uploadPartRequest)),
(partETags, result) -> {
partETags.add(result.getPartETag());
return partETags;
});
}
}
Loading