danielcweeks commented on a change in pull request #1767:
URL: https://github.com/apache/iceberg/pull/1767#discussion_r524458122
##########
File path: aws/src/main/java/org/apache/iceberg/aws/s3/S3OutputStream.java
##########
@@ -69,14 +132,54 @@ public void flush() throws IOException {
@Override
public void write(int b) throws IOException {
+ if (stream.getCount() >= multiPartSize) {
+ newStream();
+ uploadParts();
+ }
+
stream.write(b);
pos += 1;
}
@Override
public void write(byte[] b, int off, int len) throws IOException {
- stream.write(b, off, len);
+ int remaining = len;
+ int relativeOffset = off;
+
+ // Write the remainder of the part size to the staging file
+ // and continue to write new staging files if the write is
+ // larger than the part size.
+ while (stream.getCount() + remaining > multiPartSize) {
+ int writeSize = multiPartSize - (int) stream.getCount();
+
+ stream.write(b, relativeOffset, writeSize);
+ remaining -= writeSize;
+ relativeOffset += writeSize;
+
+ newStream();
+ uploadParts();
+ }
+
+ stream.write(b, relativeOffset, remaining);
pos += len;
+
+ // switch to multipart upload
+ if (multipartUploadId == null && pos >= multiPartSize *
multiPartThresholdFactor) {
+ initializeMultiPartUpload();
+ uploadParts();
+ }
+ }
+
+ private void newStream() throws IOException {
+ if (stream != null) {
+ stream.close();
+ }
+
+ currentStagingFile = File.createTempFile("s3fileio-", ".tmp");
Review comment:
Yeah, this makes sense and should be easy to add once we have properties
pushed all the way down from the catalog/tables (you'll see that mentioned
about in https://github.com/apache/iceberg/pull/1767#discussion_r523154834).
You might get some savings by using st1 volumes for this specifically due to it
being entirely sequential read and write, so that might be interesting to
experiment with.
----------------------------------------------------------------
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]