You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@iceberg.apache.org by GitBox <gi...@apache.org> on 2020/12/10 06:11:18 UTC

[GitHub] [iceberg] jackye1995 opened a new pull request #1900: AWS: add more S3FileIO tests, cleanup related codebase

jackye1995 opened a new pull request #1900:
URL: https://github.com/apache/iceberg/pull/1900


   @danielcweeks made the following updates to `S3FileIO` related code:
   1. added integration tests that verifies upload behaviors against S3
   2. updated variable names and documentations in `AwsProperties` to be consistent with others, added corresponding tests
   3. fixed invalid reference to private variable `S3URI#VALID_SCHEMES` in doc of `S3FileIO`
   4. muted errorprone warnings:
   
   ```
   /iceberg/aws/src/main/java/org/apache/iceberg/aws/s3/S3RequestUtil.java:35: warning: [UnnecessaryLambda] Returning a lambda from a helper method or saving it in a constant is unnecessary; prefer to implement the functional interface method directly and use a method reference instead.
     private static final Function<ServerSideEncryption, S3Request.Builder> NULL_SSE_SETTER = sse -> null;
                                                                            ^
       (see https://errorprone.info/bugpattern/UnnecessaryLambda)
     Did you mean 'private static  S3Request.Builder nullSseSetter(ServerSideEncryption sse){return null;}'?
   /iceberg/aws/src/main/java/org/apache/iceberg/aws/s3/S3RequestUtil.java:36: warning: [UnnecessaryLambda] Returning a lambda from a helper method or saving it in a constant is unnecessary; prefer to implement the functional interface method directly and use a method reference instead.
     private static final Function<String, S3Request.Builder> NULL_STRING_SETTER = s -> null;
                                                              ^
       (see https://errorprone.info/bugpattern/UnnecessaryLambda)
     Did you mean 'private static  S3Request.Builder nullStringSetter(String s){return null;}'?
   /iceberg/aws/src/main/java/org/apache/iceberg/aws/s3/S3OutputStream.java:92: warning: [StaticAssignmentInConstructor] This assignment is to a static field. Mutating static state from a constructor is highly error-prone.
             executorService = MoreExecutors.getExitingExecutorService(
                             ^
       (see https://errorprone.info/bugpattern/StaticAssignmentInConstructor)
   /iceberg/aws/src/main/java/org/apache/iceberg/aws/s3/S3OutputStream.java:92: warning: [StaticGuardedByInstance] Write to static variable should not be guarded by instance lock 'this'
             executorService = MoreExecutors.getExitingExecutorService(
             ^
       (see https://errorprone.info/bugpattern/StaticGuardedByInstance)
   4 warnings
   ```


----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] danielcweeks commented on a change in pull request #1900: AWS: add more S3FileIO tests, cleanup related codebase

Posted by GitBox <gi...@apache.org>.
danielcweeks commented on a change in pull request #1900:
URL: https://github.com/apache/iceberg/pull/1900#discussion_r540588604



##########
File path: aws/src/integration/java/org/apache/iceberg/aws/s3/S3MultipartUploadTest.java
##########
@@ -0,0 +1,192 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.aws.s3;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.Random;
+import java.util.UUID;
+import java.util.stream.IntStream;
+import org.apache.iceberg.aws.AwsClientUtil;
+import org.apache.iceberg.aws.AwsIntegTestUtil;
+import org.apache.iceberg.aws.AwsProperties;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.io.PositionOutputStream;
+import org.apache.iceberg.io.SeekableInputStream;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import software.amazon.awssdk.services.s3.S3Client;
+
+/**
+ * Long-running tests to ensure multipart upload logic is resilient
+ */
+public class S3MultipartUploadTest {
+
+  private final Random random = new Random(1);
+  private static S3Client s3;
+  private static String bucketName;
+  private static String prefix;
+  private String objectUri;
+
+  @BeforeClass
+  public static void beforeClass() {
+    s3 = AwsClientUtil.defaultS3Client();
+    bucketName = AwsIntegTestUtil.testBucketName();
+    prefix = UUID.randomUUID().toString();
+  }
+
+  @AfterClass
+  public static void afterClass() {
+    AwsIntegTestUtil.cleanS3Bucket(s3, bucketName, prefix);
+  }
+
+  @Before
+  public void before() {
+    String objectKey = String.format("%s/%s", prefix, UUID.randomUUID().toString());
+    objectUri = String.format("s3://%s/%s", bucketName, objectKey);
+  }
+
+  @Test
+  public void testManyParts_writeWithInt() throws IOException {
+    AwsProperties properties = new AwsProperties();
+    properties.setS3FileIoMultiPartSize(AwsProperties.S3FILEIO_MULTIPART_SIZE_MIN);
+    S3FileIO io = new S3FileIO(() -> s3, properties);
+    PositionOutputStream outputStream = io.newOutputFile(objectUri).create();
+    for (int i = 0; i < 100; i++) {

Review comment:
       No, I just mixed up this test and the next one and missed the inner loop here.  However, you might be able to combine some of the upload and content validation into a single test, but it looks like you already have some thoughts on it, so I'll wait.
   
   I guess there's two minor questions I have:
   
   1) Is it reasonable to be creating large files in S3 as part of the integration test (I'm not clear on if we run these as part of our actual build or it's left up to users to run in their own accounts).
   2) Are there cases where we think the s3mock wouldn't catch something that these tests would?




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] danielcweeks commented on a change in pull request #1900: AWS: add more S3FileIO tests, cleanup related codebase

Posted by GitBox <gi...@apache.org>.
danielcweeks commented on a change in pull request #1900:
URL: https://github.com/apache/iceberg/pull/1900#discussion_r541096376



##########
File path: aws/src/integration/java/org/apache/iceberg/aws/s3/S3MultipartUploadTest.java
##########
@@ -0,0 +1,192 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.aws.s3;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.Random;
+import java.util.UUID;
+import java.util.stream.IntStream;
+import org.apache.iceberg.aws.AwsClientUtil;
+import org.apache.iceberg.aws.AwsIntegTestUtil;
+import org.apache.iceberg.aws.AwsProperties;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.io.PositionOutputStream;
+import org.apache.iceberg.io.SeekableInputStream;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import software.amazon.awssdk.services.s3.S3Client;
+
+/**
+ * Long-running tests to ensure multipart upload logic is resilient
+ */
+public class S3MultipartUploadTest {
+
+  private final Random random = new Random(1);
+  private static S3Client s3;
+  private static String bucketName;
+  private static String prefix;
+  private String objectUri;
+
+  @BeforeClass
+  public static void beforeClass() {
+    s3 = AwsClientUtil.defaultS3Client();
+    bucketName = AwsIntegTestUtil.testBucketName();
+    prefix = UUID.randomUUID().toString();
+  }
+
+  @AfterClass
+  public static void afterClass() {
+    AwsIntegTestUtil.cleanS3Bucket(s3, bucketName, prefix);
+  }
+
+  @Before
+  public void before() {
+    String objectKey = String.format("%s/%s", prefix, UUID.randomUUID().toString());
+    objectUri = String.format("s3://%s/%s", bucketName, objectKey);
+  }
+
+  @Test
+  public void testManyParts_writeWithInt() throws IOException {
+    AwsProperties properties = new AwsProperties();
+    properties.setS3FileIoMultiPartSize(AwsProperties.S3FILEIO_MULTIPART_SIZE_MIN);
+    S3FileIO io = new S3FileIO(() -> s3, properties);
+    PositionOutputStream outputStream = io.newOutputFile(objectUri).create();
+    for (int i = 0; i < 100; i++) {

Review comment:
       Yeah, looks good.  It seems for #2 there are a number of things we won't be able to test against s3mock (like sts) so it makes sense to add these integration tests once we have an account.
   
   Thanks!




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] jackye1995 commented on a change in pull request #1900: AWS: add more S3FileIO tests, cleanup related codebase

Posted by GitBox <gi...@apache.org>.
jackye1995 commented on a change in pull request #1900:
URL: https://github.com/apache/iceberg/pull/1900#discussion_r540582399



##########
File path: aws/src/integration/java/org/apache/iceberg/aws/s3/S3MultipartUploadTest.java
##########
@@ -0,0 +1,192 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.aws.s3;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.Random;
+import java.util.UUID;
+import java.util.stream.IntStream;
+import org.apache.iceberg.aws.AwsClientUtil;
+import org.apache.iceberg.aws.AwsIntegTestUtil;
+import org.apache.iceberg.aws.AwsProperties;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.io.PositionOutputStream;
+import org.apache.iceberg.io.SeekableInputStream;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import software.amazon.awssdk.services.s3.S3Client;
+
+/**
+ * Long-running tests to ensure multipart upload logic is resilient
+ */
+public class S3MultipartUploadTest {
+
+  private final Random random = new Random(1);
+  private static S3Client s3;
+  private static String bucketName;
+  private static String prefix;
+  private String objectUri;
+
+  @BeforeClass
+  public static void beforeClass() {
+    s3 = AwsClientUtil.defaultS3Client();
+    bucketName = AwsIntegTestUtil.testBucketName();
+    prefix = UUID.randomUUID().toString();
+  }
+
+  @AfterClass
+  public static void afterClass() {
+    AwsIntegTestUtil.cleanS3Bucket(s3, bucketName, prefix);
+  }
+
+  @Before
+  public void before() {
+    String objectKey = String.format("%s/%s", prefix, UUID.randomUUID().toString());
+    objectUri = String.format("s3://%s/%s", bucketName, objectKey);
+  }
+
+  @Test
+  public void testManyParts_writeWithInt() throws IOException {
+    AwsProperties properties = new AwsProperties();
+    properties.setS3FileIoMultiPartSize(AwsProperties.S3FILEIO_MULTIPART_SIZE_MIN);
+    S3FileIO io = new S3FileIO(() -> s3, properties);
+    PositionOutputStream outputStream = io.newOutputFile(objectUri).create();
+    for (int i = 0; i < 100; i++) {

Review comment:
       > that would only be writing a single byte, so 100 bytes in this case
   
   There is an internal loop `for (int j = 0; j < AwsProperties.S3FILEIO_MULTIPART_SIZE_MIN; j++)`.
   
   > You might want to look at the S3Outputstream test because you can actually validate the operations performed like this
   
   the tests here are trying to verify against actual result in s3 instead of verifying the number of calls, because I know those are verified in the tests you referenced. But I think I am being very not DRY here, let me refactor the tests a little bit




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] jackye1995 commented on a change in pull request #1900: AWS: add more S3FileIO tests, cleanup related codebase

Posted by GitBox <gi...@apache.org>.
jackye1995 commented on a change in pull request #1900:
URL: https://github.com/apache/iceberg/pull/1900#discussion_r540704169



##########
File path: aws/src/integration/java/org/apache/iceberg/aws/s3/S3MultipartUploadTest.java
##########
@@ -0,0 +1,192 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.aws.s3;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.Random;
+import java.util.UUID;
+import java.util.stream.IntStream;
+import org.apache.iceberg.aws.AwsClientUtil;
+import org.apache.iceberg.aws.AwsIntegTestUtil;
+import org.apache.iceberg.aws.AwsProperties;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.io.PositionOutputStream;
+import org.apache.iceberg.io.SeekableInputStream;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import software.amazon.awssdk.services.s3.S3Client;
+
+/**
+ * Long-running tests to ensure multipart upload logic is resilient
+ */
+public class S3MultipartUploadTest {
+
+  private final Random random = new Random(1);
+  private static S3Client s3;
+  private static String bucketName;
+  private static String prefix;
+  private String objectUri;
+
+  @BeforeClass
+  public static void beforeClass() {
+    s3 = AwsClientUtil.defaultS3Client();
+    bucketName = AwsIntegTestUtil.testBucketName();
+    prefix = UUID.randomUUID().toString();
+  }
+
+  @AfterClass
+  public static void afterClass() {
+    AwsIntegTestUtil.cleanS3Bucket(s3, bucketName, prefix);
+  }
+
+  @Before
+  public void before() {
+    String objectKey = String.format("%s/%s", prefix, UUID.randomUUID().toString());
+    objectUri = String.format("s3://%s/%s", bucketName, objectKey);
+  }
+
+  @Test
+  public void testManyParts_writeWithInt() throws IOException {
+    AwsProperties properties = new AwsProperties();
+    properties.setS3FileIoMultiPartSize(AwsProperties.S3FILEIO_MULTIPART_SIZE_MIN);
+    S3FileIO io = new S3FileIO(() -> s3, properties);
+    PositionOutputStream outputStream = io.newOutputFile(objectUri).create();
+    for (int i = 0; i < 100; i++) {

Review comment:
       @danielcweeks refactored tests, please let me know if it looks good to you.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] danielcweeks commented on a change in pull request #1900: AWS: add more S3FileIO tests, cleanup related codebase

Posted by GitBox <gi...@apache.org>.
danielcweeks commented on a change in pull request #1900:
URL: https://github.com/apache/iceberg/pull/1900#discussion_r540565526



##########
File path: aws/src/integration/java/org/apache/iceberg/aws/s3/S3MultipartUploadTest.java
##########
@@ -0,0 +1,192 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.aws.s3;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.Random;
+import java.util.UUID;
+import java.util.stream.IntStream;
+import org.apache.iceberg.aws.AwsClientUtil;
+import org.apache.iceberg.aws.AwsIntegTestUtil;
+import org.apache.iceberg.aws.AwsProperties;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.io.PositionOutputStream;
+import org.apache.iceberg.io.SeekableInputStream;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import software.amazon.awssdk.services.s3.S3Client;
+
+/**
+ * Long-running tests to ensure multipart upload logic is resilient
+ */
+public class S3MultipartUploadTest {
+
+  private final Random random = new Random(1);
+  private static S3Client s3;
+  private static String bucketName;
+  private static String prefix;
+  private String objectUri;
+
+  @BeforeClass
+  public static void beforeClass() {
+    s3 = AwsClientUtil.defaultS3Client();
+    bucketName = AwsIntegTestUtil.testBucketName();
+    prefix = UUID.randomUUID().toString();
+  }
+
+  @AfterClass
+  public static void afterClass() {
+    AwsIntegTestUtil.cleanS3Bucket(s3, bucketName, prefix);
+  }
+
+  @Before
+  public void before() {
+    String objectKey = String.format("%s/%s", prefix, UUID.randomUUID().toString());
+    objectUri = String.format("s3://%s/%s", bucketName, objectKey);
+  }
+
+  @Test
+  public void testManyParts_writeWithInt() throws IOException {
+    AwsProperties properties = new AwsProperties();
+    properties.setS3FileIoMultiPartSize(AwsProperties.S3FILEIO_MULTIPART_SIZE_MIN);
+    S3FileIO io = new S3FileIO(() -> s3, properties);
+    PositionOutputStream outputStream = io.newOutputFile(objectUri).create();
+    for (int i = 0; i < 100; i++) {

Review comment:
       Maybe I'm missing something, but I don't think these are actually testing the multipart upload.  If we're writing with the OutputStream::write interface, that would only be writing a single byte, so 100 bytes in this case.  That wouldn't be enough to trigger the multipart behavior.
   
   I think that's the case for most of the tests I see here.  You might want to look at the S3Outputstream test because you can actually validate the operations performed like this: https://github.com/apache/iceberg/blob/master/aws/src/test/java/org/apache/iceberg/aws/s3/S3OutputStreamTest.java#L109




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] danielcweeks merged pull request #1900: AWS: add more S3FileIO tests, cleanup related codebase

Posted by GitBox <gi...@apache.org>.
danielcweeks merged pull request #1900:
URL: https://github.com/apache/iceberg/pull/1900


   


----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] jackye1995 commented on a change in pull request #1900: AWS: add more S3FileIO tests, cleanup related codebase

Posted by GitBox <gi...@apache.org>.
jackye1995 commented on a change in pull request #1900:
URL: https://github.com/apache/iceberg/pull/1900#discussion_r540599723



##########
File path: aws/src/integration/java/org/apache/iceberg/aws/s3/S3MultipartUploadTest.java
##########
@@ -0,0 +1,192 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.aws.s3;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.Random;
+import java.util.UUID;
+import java.util.stream.IntStream;
+import org.apache.iceberg.aws.AwsClientUtil;
+import org.apache.iceberg.aws.AwsIntegTestUtil;
+import org.apache.iceberg.aws.AwsProperties;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.io.PositionOutputStream;
+import org.apache.iceberg.io.SeekableInputStream;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import software.amazon.awssdk.services.s3.S3Client;
+
+/**
+ * Long-running tests to ensure multipart upload logic is resilient
+ */
+public class S3MultipartUploadTest {
+
+  private final Random random = new Random(1);
+  private static S3Client s3;
+  private static String bucketName;
+  private static String prefix;
+  private String objectUri;
+
+  @BeforeClass
+  public static void beforeClass() {
+    s3 = AwsClientUtil.defaultS3Client();
+    bucketName = AwsIntegTestUtil.testBucketName();
+    prefix = UUID.randomUUID().toString();
+  }
+
+  @AfterClass
+  public static void afterClass() {
+    AwsIntegTestUtil.cleanS3Bucket(s3, bucketName, prefix);
+  }
+
+  @Before
+  public void before() {
+    String objectKey = String.format("%s/%s", prefix, UUID.randomUUID().toString());
+    objectUri = String.format("s3://%s/%s", bucketName, objectKey);
+  }
+
+  @Test
+  public void testManyParts_writeWithInt() throws IOException {
+    AwsProperties properties = new AwsProperties();
+    properties.setS3FileIoMultiPartSize(AwsProperties.S3FILEIO_MULTIPART_SIZE_MIN);
+    S3FileIO io = new S3FileIO(() -> s3, properties);
+    PositionOutputStream outputStream = io.newOutputFile(objectUri).create();
+    for (int i = 0; i < 100; i++) {

Review comment:
       > 1. Is it reasonable to be creating large files in S3 as part of the integration test (I'm not clear on if we run these as part of our actual build or it's left up to users to run in their own accounts).
   
   I don't expect this to be run for every actual build, and the tests take quite a while to complete, so it's mostly for users to run in their own account. With that being said, I am in progress of potentially getting an account to run these tests for all PRs committing to the aws module with cost covered.
   
   > 2. Are there cases where we think the s3mock wouldn't catch something that these tests would?
   
   It is hard to say how different is the actual S3 compared to S3mock, so this serves as a line of defense to catch potentially different behaviors and potential errors during non-local network calls.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] danielcweeks commented on a change in pull request #1900: AWS: add more S3FileIO tests, cleanup related codebase

Posted by GitBox <gi...@apache.org>.
danielcweeks commented on a change in pull request #1900:
URL: https://github.com/apache/iceberg/pull/1900#discussion_r540588604



##########
File path: aws/src/integration/java/org/apache/iceberg/aws/s3/S3MultipartUploadTest.java
##########
@@ -0,0 +1,192 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.aws.s3;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.Random;
+import java.util.UUID;
+import java.util.stream.IntStream;
+import org.apache.iceberg.aws.AwsClientUtil;
+import org.apache.iceberg.aws.AwsIntegTestUtil;
+import org.apache.iceberg.aws.AwsProperties;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.io.PositionOutputStream;
+import org.apache.iceberg.io.SeekableInputStream;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import software.amazon.awssdk.services.s3.S3Client;
+
+/**
+ * Long-running tests to ensure multipart upload logic is resilient
+ */
+public class S3MultipartUploadTest {
+
+  private final Random random = new Random(1);
+  private static S3Client s3;
+  private static String bucketName;
+  private static String prefix;
+  private String objectUri;
+
+  @BeforeClass
+  public static void beforeClass() {
+    s3 = AwsClientUtil.defaultS3Client();
+    bucketName = AwsIntegTestUtil.testBucketName();
+    prefix = UUID.randomUUID().toString();
+  }
+
+  @AfterClass
+  public static void afterClass() {
+    AwsIntegTestUtil.cleanS3Bucket(s3, bucketName, prefix);
+  }
+
+  @Before
+  public void before() {
+    String objectKey = String.format("%s/%s", prefix, UUID.randomUUID().toString());
+    objectUri = String.format("s3://%s/%s", bucketName, objectKey);
+  }
+
+  @Test
+  public void testManyParts_writeWithInt() throws IOException {
+    AwsProperties properties = new AwsProperties();
+    properties.setS3FileIoMultiPartSize(AwsProperties.S3FILEIO_MULTIPART_SIZE_MIN);
+    S3FileIO io = new S3FileIO(() -> s3, properties);
+    PositionOutputStream outputStream = io.newOutputFile(objectUri).create();
+    for (int i = 0; i < 100; i++) {

Review comment:
       No, I just mixed up this test and the next one and missed the inner loop here.  However, you might be able to combine some of the upload and content validation into a single test, but it looks like you already have some thoughts on it, so I'll wait.
   
   I guess there's two minor questions I have:
   
   1) Is it reasonable to be creating large files in as part of the integration test (I'm not clear on if we run these as part of our actual build or it's left up to users to run in their own accounts).
   2) Are there cases where we think the s3mock wouldn't catch something that these tests would?




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] jackye1995 commented on a change in pull request #1900: AWS: add more S3FileIO tests, cleanup related codebase

Posted by GitBox <gi...@apache.org>.
jackye1995 commented on a change in pull request #1900:
URL: https://github.com/apache/iceberg/pull/1900#discussion_r540582399



##########
File path: aws/src/integration/java/org/apache/iceberg/aws/s3/S3MultipartUploadTest.java
##########
@@ -0,0 +1,192 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.aws.s3;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.Random;
+import java.util.UUID;
+import java.util.stream.IntStream;
+import org.apache.iceberg.aws.AwsClientUtil;
+import org.apache.iceberg.aws.AwsIntegTestUtil;
+import org.apache.iceberg.aws.AwsProperties;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.io.PositionOutputStream;
+import org.apache.iceberg.io.SeekableInputStream;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import software.amazon.awssdk.services.s3.S3Client;
+
+/**
+ * Long-running tests to ensure multipart upload logic is resilient
+ */
+public class S3MultipartUploadTest {
+
+  private final Random random = new Random(1);
+  private static S3Client s3;
+  private static String bucketName;
+  private static String prefix;
+  private String objectUri;
+
+  @BeforeClass
+  public static void beforeClass() {
+    s3 = AwsClientUtil.defaultS3Client();
+    bucketName = AwsIntegTestUtil.testBucketName();
+    prefix = UUID.randomUUID().toString();
+  }
+
+  @AfterClass
+  public static void afterClass() {
+    AwsIntegTestUtil.cleanS3Bucket(s3, bucketName, prefix);
+  }
+
+  @Before
+  public void before() {
+    String objectKey = String.format("%s/%s", prefix, UUID.randomUUID().toString());
+    objectUri = String.format("s3://%s/%s", bucketName, objectKey);
+  }
+
+  @Test
+  public void testManyParts_writeWithInt() throws IOException {
+    AwsProperties properties = new AwsProperties();
+    properties.setS3FileIoMultiPartSize(AwsProperties.S3FILEIO_MULTIPART_SIZE_MIN);
+    S3FileIO io = new S3FileIO(() -> s3, properties);
+    PositionOutputStream outputStream = io.newOutputFile(objectUri).create();
+    for (int i = 0; i < 100; i++) {

Review comment:
       > that would only be writing a single byte, so 100 bytes in this case
   There is an internal loop `for (int j = 0; j < AwsProperties.S3FILEIO_MULTIPART_SIZE_MIN; j++)`.
   
   > You might want to look at the S3Outputstream test because you can actually validate the operations performed like this
   
   the tests here are trying to verify against actual result in s3 instead of verifying the number of calls, because I know those are verified in the tests you referenced. But I think I am being very not DRY here, let me refactor the tests a little bit




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org