rdblue commented on a change in pull request #3813:
URL: https://github.com/apache/iceberg/pull/3813#discussion_r791973390



##########
File path: aws/src/main/java/org/apache/iceberg/aws/s3/S3OutputStream.java
##########
@@ -178,9 +192,27 @@ private void newStream() throws IOException {
     createStagingDirectoryIfNotExists();
     currentStagingFile = File.createTempFile("s3fileio-", ".tmp", 
stagingDirectory);
     currentStagingFile.deleteOnExit();
-    stagingFiles.add(currentStagingFile);
+    try {
+      currentPartMessageDigest = isChecksumEnabled ? 
MessageDigest.getInstance(digestAlgorithm) : null;
+    } catch (NoSuchAlgorithmException e) {
+      throw new RuntimeException("Failed to create message digest needed for 
s3 checksum checks.", e);
+    }
+
+    stagingFiles.add(new FileAndDigest(currentStagingFile, 
currentPartMessageDigest));
+
+    if (isChecksumEnabled) {
+      DigestOutputStream digestOutputStream = new DigestOutputStream(new 
DigestOutputStream(new BufferedOutputStream(
+          new FileOutputStream(currentStagingFile)), 
currentPartMessageDigest), completeMessageDigest);
 
-    stream = new CountingOutputStream(new BufferedOutputStream(new 
FileOutputStream(currentStagingFile)));
+      // if switched over to multipart threshold already, no need to update 
complete message digest
+      if (multipartUploadId != null) {

Review comment:
       Isn't this known before we create the double `DigestOutputStream`? I 
think you could just check this above and create a single-wrapped stream 
instead.




-- 
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]

Reply via email to