rdblue commented on a change in pull request #1767:
URL: https://github.com/apache/iceberg/pull/1767#discussion_r523285106
##########
File path: aws/src/main/java/org/apache/iceberg/aws/s3/S3OutputStream.java
##########
@@ -87,17 +190,105 @@ public void close() throws IOException {
super.close();
closed = true;
+ currentStagingFile = null;
try {
stream.close();
+ completeUploads();
+ } finally {
+ stagingFiles.forEach(f -> {
+ if (f.exists() && !f.delete()) {
+ LOG.warn("Could not delete temporary file: {}", f);
+ }
+ });
+ }
+ }
+
+ private void initializeMultiPartUpload() {
+ multipartUploadId =
s3.createMultipartUpload(CreateMultipartUploadRequest.builder()
+ .bucket(location.bucket()).key(location.key()).build()).uploadId();
+ }
+
+ private void uploadParts() {
+ // exit if multipart has not been initiated
+ if (multipartUploadId == null) {
+ return;
+ }
+
+ stagingFiles.stream()
+ // do not upload the file currently being written
+ .filter(f -> currentStagingFile == null ||
!currentStagingFile.equals(f))
+ // do not upload any files that have already been processed
+ .filter(Predicates.not(multiPartMap::containsKey))
+ .forEach(f -> {
+ UploadPartRequest uploadRequest = UploadPartRequest.builder()
+ .bucket(location.bucket())
+ .key(location.key())
+ .uploadId(multipartUploadId)
+ .partNumber(stagingFiles.indexOf(f) + 1)
+ .contentLength(f.length())
+ .build();
+
+ CompletableFuture<CompletedPart> future =
CompletableFuture.supplyAsync(
+ () -> {
+ UploadPartResponse response = s3.uploadPart(uploadRequest,
RequestBody.fromFile(f));
+ return
CompletedPart.builder().eTag(response.eTag()).partNumber(uploadRequest.partNumber()).build();
+ },
+ executorService
+ );
+
+ multiPartMap.put(f, future);
+ });
+ }
+
+ private void completeMultiPartUpload() throws IOException {
+ try {
+ List<CompletedPart> completedParts =
+ multiPartMap.values()
+ .stream()
+ .map(CompletableFuture::join)
+ .sorted(Comparator.comparing(CompletedPart::partNumber))
+ .collect(Collectors.toList());
+
+ s3.completeMultipartUpload(CompleteMultipartUploadRequest.builder()
+ .bucket(location.bucket()).key(location.key())
+ .uploadId(multipartUploadId)
+
.multipartUpload(CompletedMultipartUpload.builder().parts(completedParts).build()).build());
+ } catch (CompletionException e) {
+ abortUpload();
+ throw new IOException("Multipart upload failed for upload id: " +
multipartUploadId, e);
+ }
+ }
+
+ private void abortUpload() {
+ if (multipartUploadId != null) {
+ s3.abortMultipartUpload(AbortMultipartUploadRequest.builder()
+
.bucket(location.bucket()).key(location.key()).uploadId(multipartUploadId).build());
+ }
+ }
+
+ private void completeUploads() throws IOException {
+ if (multipartUploadId == null) {
+ long contentLength = stagingFiles.stream().mapToLong(File::length).sum();
+ InputStream contentStream = new BufferedInputStream(stagingFiles.stream()
+ .map(S3OutputStream::uncheckedInputStream)
+ .reduce(SequenceInputStream::new).orElseGet(() -> new
ByteArrayInputStream(new byte[0])));
Review comment:
Nit: Could you move `orElseGet` to a new line?
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
For queries about this service, please contact Infrastructure at:
[email protected]
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]