jackye1995 commented on a change in pull request #3813:
URL: https://github.com/apache/iceberg/pull/3813#discussion_r781457888
##########
File path: aws/src/main/java/org/apache/iceberg/aws/s3/S3OutputStream.java
##########
@@ -216,43 +247,43 @@ private void uploadParts() {
return;
}
- stagingFiles.stream()
+ stagingFilesWithETags.stream()
// do not upload the file currently being written
- .filter(f -> closed || !f.equals(currentStagingFile))
+ .filter(f -> closed || !f.file().equals(currentStagingFile))
// do not upload any files that have already been processed
- .filter(Predicates.not(multiPartMap::containsKey))
+ .filter(Predicates.not(f -> multiPartMap.containsKey(f.file())))
.forEach(f -> {
UploadPartRequest.Builder requestBuilder =
UploadPartRequest.builder()
.bucket(location.bucket())
.key(location.key())
.uploadId(multipartUploadId)
- .partNumber(stagingFiles.indexOf(f) + 1)
- .contentLength(f.length());
+ .partNumber(stagingFilesWithETags.indexOf(f) + 1)
+ .contentLength(f.file().length());
S3RequestUtil.configureEncryption(awsProperties, requestBuilder);
UploadPartRequest uploadRequest = requestBuilder.build();
CompletableFuture<CompletedPart> future =
CompletableFuture.supplyAsync(
() -> {
- UploadPartResponse response = s3.uploadPart(uploadRequest,
RequestBody.fromFile(f));
+ UploadPartResponse response = s3.uploadPart(uploadRequest,
RequestBody.fromFile(f.file()));
+ checkEtag(f.eTag(), response.eTag());
Review comment:
The integration tests in `/aws/src/integration` are added for such a
purpose, I think we can leverage that.
Unfortunately it is not run as a part of CI yet because it requires
connection to an AWS account, we are working on getting one for Iceberg. If you
do not have any AWS infrastructure that could be used to run it, I can run it
for you after you publish you tests in this PR.
--
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.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]