rdblue commented on a change in pull request #3813:
URL: https://github.com/apache/iceberg/pull/3813#discussion_r789134901
##########
File path: aws/src/main/java/org/apache/iceberg/aws/s3/S3OutputStream.java
##########
@@ -311,16 +344,39 @@ private void completeUploads() {
.bucket(location.bucket())
.key(location.key());
+ if (isEtagCheckEnabled) {
+
requestBuilder.contentMD5(BinaryUtils.toBase64(completeMessageDigest.digest()));
+ }
+
S3RequestUtil.configureEncryption(awsProperties, requestBuilder);
S3RequestUtil.configurePermission(awsProperties, requestBuilder);
- s3.putObject(requestBuilder.build(),
RequestBody.fromInputStream(contentStream, contentLength));
+ PutObjectRequest putObjectRequest = requestBuilder.build();
+ try {
+ s3.putObject(putObjectRequest,
RequestBody.fromInputStream(contentStream, contentLength));
+ } catch (UncheckedIOException uncheckedIOException) {
+ logS3RequestAndThrow(uncheckedIOException,
putObjectRequest.toString());
+ }
} else {
uploadParts();
completeMultiPartUpload();
}
}
+ private void closeStream() throws IOException {
+ if (isEtagCheckEnabled) {
+ if (stagingFiles.size() > 0) {
+ stagingFiles.get(stagingFiles.size() -
1).setDigest(currentPartMessageDigest.digest());
+ }
+ }
+ stream.close();
+ }
+
+ private static void logS3RequestAndThrow(UncheckedIOException
uncheckedIOException, String requestString) {
+ LOG.error("S3 Request Failure: {}", requestString);
+ throw uncheckedIOException;
Review comment:
I don't think this method is needed. If the exception is going to be
thrown, then it is the responsibility of whatever catches it to log it or
handle it. This code doesn't know whether it can simply be handled, which is
why it re-throws. So it should also not decide that this needs to be logged.
Patterns like this usually lead to misleading errors or red herrings in logs.
--
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]