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 2021/09/03 10:25:58 UTC

[GitHub] [iceberg] xingbowu opened a new pull request #3067: Mock aliyun OSS(Object Storage Service)

xingbowu opened a new pull request #3067:
URL: https://github.com/apache/iceberg/pull/3067


   Aliyun OSS is popular in public cloud service, especially the users from china.  Many use cases of integrating iceberg make aliyun oss as backend storage.  So it benefits community to integrate iceberg with aliyun oss. I would like to contribute several PRs and complete this work. Here is 1st step :  Mock aliyun OSS in UT
   
   Aliyun OSS SDK doesn't support mock local  environment,  and no any plan to develop this feature recently.
   To make unit test of iceberg integration with oss efficiently, this PR mocks a local lightweight aliyun oss behavior for UT similar with s3mock


-- 
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: issues-unsubscribe@iceberg.apache.org

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] openinx commented on a change in pull request #3067: Aliyun: Mock aliyun OSS(Object Storage Service)

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



##########
File path: aliyun/src/test/java/org/apache/iceberg/aliyun/oss/AliyunOSSTestRule.java
##########
@@ -0,0 +1,84 @@
+/*
+ * 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.aliyun.oss;
+
+import com.aliyun.oss.OSS;
+import java.util.UUID;
+import org.junit.rules.TestRule;
+import org.junit.runner.Description;
+import org.junit.runners.model.Statement;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+

Review comment:
       I think we need to add javadoc describing why we need to introduce the `AliyunOSSTestRule` ?




-- 
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: issues-unsubscribe@iceberg.apache.org

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] openinx commented on a change in pull request #3067: Aliyun: Mock aliyun OSS(Object Storage Service)

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



##########
File path: aliyun/src/test/java/org/apache/iceberg/aliyun/oss/mock/AliyunOSSMockRule.java
##########
@@ -0,0 +1,143 @@
+/*
+ * 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.aliyun.oss.mock;
+
+import com.aliyun.oss.OSS;
+import com.aliyun.oss.OSSClientBuilder;
+import java.io.File;
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.nio.file.Files;
+import java.util.Map;
+import org.apache.commons.io.FileUtils;
+import org.apache.iceberg.aliyun.oss.AliyunOSSTestRule;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class AliyunOSSMockRule implements AliyunOSSTestRule {
+  private static final Logger LOG = LoggerFactory.getLogger(AliyunOSSMockRule.class);
+
+  private final Map<String, Object> properties;
+
+  private AliyunOSSMockApp ossMockApp;
+
+  private AliyunOSSMockRule(Map<String, Object> properties) {
+    this.properties = properties;
+  }
+
+  public static Builder builder() {
+    return new Builder();
+  }
+
+  @Override
+  public void start() {
+    ossMockApp = AliyunOSSMockApp.start(properties);
+  }
+
+  @Override
+  public void stop() {
+    ossMockApp.stop();
+  }
+
+  @Override
+  public OSS createOSSClient() {
+    String endpoint = String.format("http://localhost:%s", properties.getOrDefault(
+        AliyunOSSMockApp.PROP_HTTP_PORT,
+        AliyunOSSMockApp.PORT_HTTP_PORT_DEFAULT));
+    return new OSSClientBuilder().build(endpoint, "foo", "bar");
+  }
+
+  @Override
+  public String keyPrefix() {
+    return "mock-objects/";
+  }
+
+  private File rootDir() {
+    Object rootDir = properties.get(AliyunOSSMockApp.PROP_ROOT_DIR);
+    Preconditions.checkNotNull(rootDir, "Root directory cannot be null");
+    return new File(rootDir.toString());
+  }
+
+  @Override
+  public void setUpBucket(String bucket) {
+    createOSSClient().createBucket(bucket);
+  }
+
+  @Override
+  public void tearDownBucket(String bucket) {
+    try {
+      Files.walk(rootDir().toPath())
+          .filter(p -> p.toFile().isFile())
+          .forEach(p -> {
+            try {
+              Files.delete(p);
+            } catch (IOException e) {
+              // delete this files quietly.

Review comment:
       Nit: `files` - > `file`




-- 
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: issues-unsubscribe@iceberg.apache.org

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] openinx commented on a change in pull request #3067: Aliyun: Mock aliyun OSS(Object Storage Service)

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



##########
File path: aliyun/src/test/java/org/apache/iceberg/aliyun/oss/mock/TestLocalAliyunOSS.java
##########
@@ -0,0 +1,184 @@
+/*
+ * 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.aliyun.oss.mock;
+
+import com.aliyun.oss.OSS;
+import com.aliyun.oss.OSSErrorCode;
+import com.aliyun.oss.OSSException;
+import com.aliyun.oss.model.GetObjectRequest;
+import com.aliyun.oss.model.PutObjectResult;
+import java.io.ByteArrayInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.Objects;
+import java.util.Random;
+import java.util.UUID;
+import org.apache.commons.io.IOUtils;
+import org.apache.iceberg.aliyun.oss.AliyunOSSTestRule;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.ClassRule;
+import org.junit.Test;
+
+public class TestLocalAliyunOSS {
+
+  @ClassRule
+  public static final AliyunOSSTestRule OSS_TEST_RULE = AliyunOSSMockRule.builder().silent().build();
+
+  private final OSS oss = OSS_TEST_RULE.createOSSClient();
+  private final String bucketName = OSS_TEST_RULE.testBucketName();
+  private final Random random = new Random(1);
+
+  private static void assertThrows(Runnable runnable, String expectedErrorCode) {
+    try {
+      runnable.run();
+      Assert.fail("No exception was thrown, expected errorCode: " + expectedErrorCode);
+    } catch (OSSException e) {
+      Assert.assertEquals(expectedErrorCode, e.getErrorCode());
+    }
+  }
+
+  @Before
+  public void before() {
+    OSS_TEST_RULE.setUpBucket(bucketName);
+  }
+
+  @After
+  public void after() {
+    OSS_TEST_RULE.tearDownBucket(bucketName);
+  }
+
+  @Test
+  public void testBuckets() {
+    Assert.assertTrue(doesBucketExist(bucketName));
+    assertThrows(() -> oss.createBucket(bucketName), OSSErrorCode.BUCKET_ALREADY_EXISTS);
+
+    oss.deleteBucket(bucketName);
+    Assert.assertFalse(doesBucketExist(bucketName));
+
+    oss.createBucket(bucketName);
+    Assert.assertTrue(doesBucketExist(bucketName));
+  }
+
+  @Test
+  public void testDeleteBucket() {
+    String bucketNotExist = String.format("bucket-not-existing-%s", UUID.randomUUID());
+    assertThrows(() -> oss.deleteBucket(bucketNotExist), OSSErrorCode.NO_SUCH_BUCKET);
+
+    byte[] bytes = new byte[2000];
+    random.nextBytes(bytes);
+
+    oss.putObject(bucketName, "object1", wrap(bytes));
+
+    oss.putObject(bucketName, "object2", wrap(bytes));
+
+    assertThrows(() -> oss.deleteBucket(bucketName), OSSErrorCode.BUCKET_NOT_EMPTY);
+
+    oss.deleteObject(bucketName, "object1");
+    assertThrows(() -> oss.deleteBucket(bucketName), OSSErrorCode.BUCKET_NOT_EMPTY);
+
+    oss.deleteObject(bucketName, "object2");
+    oss.deleteBucket(bucketName);
+    Assert.assertFalse(doesBucketExist(bucketName));
+
+    oss.createBucket(bucketName);
+  }
+
+  @Test
+  public void testPutObject() throws IOException {
+    byte[] bytes = new byte[4 * 1024];
+    random.nextBytes(bytes);
+
+    String bucketNotExist = String.format("bucket-not-existing-%s", UUID.randomUUID());
+    assertThrows(() -> oss.putObject(bucketNotExist, "object", wrap(bytes)), OSSErrorCode.NO_SUCH_BUCKET);
+
+    PutObjectResult result = oss.putObject(bucketName, "object", wrap(bytes));
+    Assert.assertEquals(AliyunOSSMockLocalStore.md5sum(wrap(bytes)), result.getETag());
+  }
+
+  @Test
+  public void testDoesObjectExist() {
+    Assert.assertFalse(oss.doesObjectExist(bucketName, "key"));
+
+    Assert.assertFalse(oss.doesObjectExist(bucketName, "key"));
+
+    byte[] bytes = new byte[4 * 1024];
+    random.nextBytes(bytes);
+    oss.putObject(bucketName, "key", wrap(bytes));
+
+    Assert.assertTrue(oss.doesObjectExist(bucketName, "key"));
+    oss.deleteObject(bucketName, "key");
+  }
+
+  @Test
+  public void testGetObject() throws IOException {
+    String bucketNotExist = String.format("bucket-not-existing-%s", UUID.randomUUID());
+    assertThrows(() -> oss.getObject(bucketNotExist, "key"), OSSErrorCode.NO_SUCH_BUCKET);
+
+    assertThrows(() -> oss.getObject(bucketName, "key"), OSSErrorCode.NO_SUCH_KEY);
+
+    byte[] bytes = new byte[2000];
+    random.nextBytes(bytes);
+
+    oss.putObject(bucketName, "key", new ByteArrayInputStream(bytes));
+
+    byte[] actual = new byte[2000];
+    IOUtils.readFully(oss.getObject(bucketName, "key").getObjectContent(), actual);
+
+    Assert.assertArrayEquals(bytes, actual);
+    oss.deleteObject(bucketName, "key");
+  }
+
+  @Test
+  public void testGetObjectWithRange() throws IOException {
+
+    byte[] bytes = new byte[2000];
+    random.nextBytes(bytes);
+    oss.putObject(bucketName, "key", new ByteArrayInputStream(bytes));
+
+    byte[] actual = new byte[2000];
+    int start = 0;
+    int end = 1999;
+    GetObjectRequest getObjectRequest = new GetObjectRequest(bucketName, "key");
+    getObjectRequest.setRange(start, end);
+    IOUtils.readFully(oss.getObject(getObjectRequest).getObjectContent(), actual);
+
+    Assert.assertArrayEquals(bytes, actual);
+    oss.deleteObject(bucketName, "key");
+  }
+
+  private InputStream wrap(byte[] data) {
+    return new ByteArrayInputStream(data);
+  }
+
+  private boolean doesBucketExist(String bucket) {
+    try {
+      oss.createBucket(bucket);

Review comment:
       Em.. seems we don't provide any bucket existence checking API in the local OSS.  I forgot why we don't implement it ,  seems it's fussy to mock all the object metadata..




-- 
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: issues-unsubscribe@iceberg.apache.org

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] xingbowu commented on a change in pull request #3067: Aliyun: Mock aliyun OSS(Object Storage Service)

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



##########
File path: aliyun/src/test/java/org/apache/iceberg/aliyun/oss/AliyunOSSTestRule.java
##########
@@ -0,0 +1,84 @@
+/*
+ * 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.aliyun.oss;
+
+import com.aliyun.oss.OSS;
+import java.util.UUID;
+import org.junit.rules.TestRule;
+import org.junit.runner.Description;
+import org.junit.runners.model.Statement;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public interface AliyunOSSTestRule extends TestRule {
+  Logger LOG = LoggerFactory.getLogger(AliyunOSSTestRule.class);
+  UUID RANDOM_UUID = java.util.UUID.randomUUID();
+  /**

Review comment:
       I have checked the idea style file "intellij-java-palantir-style.xml",  the default style between two method in the interface is empty as below 
   ![image](https://user-images.githubusercontent.com/12458911/133087831-216e217b-46a2-4915-b7c5-66e3468d6905.png)
   let me know if the style file is incorrect 




-- 
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: issues-unsubscribe@iceberg.apache.org

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] openinx commented on a change in pull request #3067: Aliyun: Mock aliyun OSS(Object Storage Service)

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



##########
File path: aliyun/src/test/java/org/apache/iceberg/aliyun/oss/mock/AliyunOSSMockRule.java
##########
@@ -0,0 +1,143 @@
+/*
+ * 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.aliyun.oss.mock;
+
+import com.aliyun.oss.OSS;
+import com.aliyun.oss.OSSClientBuilder;
+import java.io.File;
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.nio.file.Files;
+import java.util.Map;
+import org.apache.commons.io.FileUtils;
+import org.apache.iceberg.aliyun.oss.AliyunOSSTestRule;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class AliyunOSSMockRule implements AliyunOSSTestRule {
+  private static final Logger LOG = LoggerFactory.getLogger(AliyunOSSMockRule.class);
+
+  private final Map<String, Object> properties;
+
+  private AliyunOSSMockApp ossMockApp;
+
+  private AliyunOSSMockRule(Map<String, Object> properties) {
+    this.properties = properties;
+  }
+
+  public static Builder builder() {
+    return new Builder();
+  }
+
+  @Override
+  public void start() {
+    ossMockApp = AliyunOSSMockApp.start(properties);
+  }
+
+  @Override
+  public void stop() {
+    ossMockApp.stop();
+  }
+
+  @Override
+  public OSS createOSSClient() {
+    String endpoint = String.format("http://localhost:%s", properties.getOrDefault(
+        AliyunOSSMockApp.PROP_HTTP_PORT,
+        AliyunOSSMockApp.PORT_HTTP_PORT_DEFAULT));
+    return new OSSClientBuilder().build(endpoint, "foo", "bar");
+  }
+
+  @Override
+  public String keyPrefix() {
+    return "mock-objects/";
+  }
+
+  private File rootDir() {
+    Object rootDir = properties.get(AliyunOSSMockApp.PROP_ROOT_DIR);
+    Preconditions.checkNotNull(rootDir, "Root directory cannot be null");
+    return new File(rootDir.toString());
+  }
+
+  @Override
+  public void setUpBucket(String bucket) {
+    createOSSClient().createBucket(bucket);
+  }
+
+  @Override
+  public void tearDownBucket(String bucket) {
+    try {
+      Files.walk(rootDir().toPath())
+          .filter(p -> p.toFile().isFile())
+          .forEach(p -> {
+            try {
+              Files.delete(p);
+            } catch (IOException e) {
+              // delete this files quietly.
+            }
+          });
+
+      createOSSClient().deleteBucket(bucket);
+    } catch (IOException e) {
+      throw new UncheckedIOException(e);
+    }
+  }
+
+  public static class Builder {
+    private Map<String, Object> props = Maps.newHashMap();
+
+    public Builder withRootDir(String rootDir) {
+      props.put(AliyunOSSMockApp.PROP_ROOT_DIR, rootDir);
+      return this;
+    }
+
+    public Builder silent() {
+      props.put(AliyunOSSMockApp.PROP_SILENT, true);
+      return this;
+    }
+
+    public AliyunOSSMockRule build() {
+      if (props.get(AliyunOSSMockApp.PROP_ROOT_DIR) == null) {

Review comment:
       Maybe we will still need to set the rootDir back to the properties Map<String, Object> in the above code.  I'm trying not to access the PROP_ROOT_DIR again and again in the current 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: issues-unsubscribe@iceberg.apache.org

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] xingbowu commented on a change in pull request #3067: Aliyun: Mock aliyun OSS(Object Storage Service)

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



##########
File path: aliyun/src/test/java/org/apache/iceberg/aliyun/oss/mock/AliyunOSSMockRule.java
##########
@@ -0,0 +1,143 @@
+/*
+ * 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.aliyun.oss.mock;
+
+import com.aliyun.oss.OSS;
+import com.aliyun.oss.OSSClientBuilder;
+import java.io.File;
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.nio.file.Files;
+import java.util.Map;
+import org.apache.commons.io.FileUtils;
+import org.apache.iceberg.aliyun.oss.AliyunOSSTestRule;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class AliyunOSSMockRule implements AliyunOSSTestRule {
+  private static final Logger LOG = LoggerFactory.getLogger(AliyunOSSMockRule.class);
+
+  private final Map<String, Object> properties;
+
+  private AliyunOSSMockApp ossMockApp;
+
+  private AliyunOSSMockRule(Map<String, Object> properties) {
+    this.properties = properties;
+  }
+
+  public static Builder builder() {
+    return new Builder();
+  }
+
+  @Override
+  public void start() {
+    ossMockApp = AliyunOSSMockApp.start(properties);
+  }
+
+  @Override
+  public void stop() {
+    ossMockApp.stop();
+  }
+
+  @Override
+  public OSS createOSSClient() {
+    String endpoint = String.format("http://localhost:%s", properties.getOrDefault(
+        AliyunOSSMockApp.PROP_HTTP_PORT,
+        AliyunOSSMockApp.PORT_HTTP_PORT_DEFAULT));
+    return new OSSClientBuilder().build(endpoint, "foo", "bar");
+  }
+
+  @Override
+  public String keyPrefix() {
+    return "mock-objects/";
+  }
+
+  private File rootDir() {
+    Object rootDir = properties.get(AliyunOSSMockApp.PROP_ROOT_DIR);
+    Preconditions.checkNotNull(rootDir, "Root directory cannot be null");
+    return new File(rootDir.toString());
+  }
+
+  @Override
+  public void setUpBucket(String bucket) {
+    createOSSClient().createBucket(bucket);
+  }
+
+  @Override
+  public void tearDownBucket(String bucket) {
+    try {
+      Files.walk(rootDir().toPath())
+          .filter(p -> p.toFile().isFile())
+          .forEach(p -> {
+            try {
+              Files.delete(p);
+            } catch (IOException e) {
+              // delete this files quietly.
+            }
+          });
+
+      createOSSClient().deleteBucket(bucket);
+    } catch (IOException e) {
+      throw new UncheckedIOException(e);
+    }
+  }
+
+  public static class Builder {
+    private Map<String, Object> props = Maps.newHashMap();
+
+    public Builder withRootDir(String rootDir) {
+      props.put(AliyunOSSMockApp.PROP_ROOT_DIR, rootDir);
+      return this;
+    }
+
+    public Builder silent() {
+      props.put(AliyunOSSMockApp.PROP_SILENT, true);
+      return this;
+    }
+
+    public AliyunOSSMockRule build() {
+      if (props.get(AliyunOSSMockApp.PROP_ROOT_DIR) == null) {

Review comment:
       yes, we need set the rootDir only if no such element in the properties Map.




-- 
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: issues-unsubscribe@iceberg.apache.org

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] openinx commented on a change in pull request #3067: Aliyun: Mock aliyun OSS(Object Storage Service)

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



##########
File path: build.gradle
##########
@@ -262,6 +262,36 @@ project(':iceberg-data') {
   }
 }
 
+project(':iceberg-aliyun') {
+  dependencies {
+    compile project(':iceberg-api')
+    compile project(':iceberg-common')
+
+    compileOnly 'com.aliyun.oss:aliyun-sdk-oss'
+    compileOnly 'javax.xml.bind:jaxb-api'
+    compileOnly 'javax.activation:activation'
+    compileOnly 'org.glassfish.jaxb:jaxb-runtime'

Review comment:
       Okay, then I will recommend to package all the dependencies jar into a single bundled iceberg-aliyun jar. Because in flink SQL client,  we will need to add [each jar](https://iceberg.apache.org/flink/#preparation-when-using-flink-sql-client) in the shell as the following: 
   
   ```shell
   ./bin/sql-client.sh embedded \
   -j <iceberg-aliyu>.jar \
   -j <aliyun-oss>.jar \
   -j <javax.xml.bind:jaxb-api>.jar \
   -j <org.glassfish.jaxb:jaxb-runtime>.jar 
   shell
   ```
   
   It will be quite tedious for people to add jars one by one to make the iceberg job works ( aws-sdk don't need to package all of them into a bundled jar because aws sdk has provided [it](https://mvnrepository.com/artifact/software.amazon.awssdk/bundle) ( see [document](https://iceberg.apache.org/aws/#enabling-aws-integration)).




-- 
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: issues-unsubscribe@iceberg.apache.org

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] openinx commented on a change in pull request #3067: Aliyun: Mock aliyun OSS(Object Storage Service)

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



##########
File path: aliyun/src/test/java/org/apache/iceberg/aliyun/oss/mock/TestLocalAliyunOSS.java
##########
@@ -0,0 +1,184 @@
+/*
+ * 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.aliyun.oss.mock;
+
+import com.aliyun.oss.OSS;
+import com.aliyun.oss.OSSErrorCode;
+import com.aliyun.oss.OSSException;
+import com.aliyun.oss.model.GetObjectRequest;
+import com.aliyun.oss.model.PutObjectResult;
+import java.io.ByteArrayInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.Objects;
+import java.util.Random;
+import java.util.UUID;
+import org.apache.commons.io.IOUtils;
+import org.apache.iceberg.aliyun.oss.AliyunOSSTestRule;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.ClassRule;
+import org.junit.Test;
+
+public class TestLocalAliyunOSS {
+
+  @ClassRule
+  public static final AliyunOSSTestRule OSS_TEST_RULE = AliyunOSSMockRule.builder().silent().build();
+
+  private final OSS oss = OSS_TEST_RULE.createOSSClient();
+  private final String bucketName = OSS_TEST_RULE.testBucketName();
+  private final Random random = new Random(1);
+
+  private static void assertThrows(Runnable runnable, String expectedErrorCode) {
+    try {
+      runnable.run();
+      Assert.fail("No exception was thrown, expected errorCode: " + expectedErrorCode);
+    } catch (OSSException e) {
+      Assert.assertEquals(expectedErrorCode, e.getErrorCode());
+    }
+  }
+
+  @Before
+  public void before() {
+    OSS_TEST_RULE.setUpBucket(bucketName);
+  }
+
+  @After
+  public void after() {
+    OSS_TEST_RULE.tearDownBucket(bucketName);
+  }
+
+  @Test
+  public void testBuckets() {
+    Assert.assertTrue(doesBucketExist(bucketName));
+    assertThrows(() -> oss.createBucket(bucketName), OSSErrorCode.BUCKET_ALREADY_EXISTS);
+
+    oss.deleteBucket(bucketName);
+    Assert.assertFalse(doesBucketExist(bucketName));
+
+    oss.createBucket(bucketName);
+    Assert.assertTrue(doesBucketExist(bucketName));
+  }
+
+  @Test
+  public void testDeleteBucket() {
+    String bucketNotExist = String.format("bucket-not-existing-%s", UUID.randomUUID());
+    assertThrows(() -> oss.deleteBucket(bucketNotExist), OSSErrorCode.NO_SUCH_BUCKET);
+
+    byte[] bytes = new byte[2000];
+    random.nextBytes(bytes);
+
+    oss.putObject(bucketName, "object1", wrap(bytes));
+
+    oss.putObject(bucketName, "object2", wrap(bytes));
+
+    assertThrows(() -> oss.deleteBucket(bucketName), OSSErrorCode.BUCKET_NOT_EMPTY);
+
+    oss.deleteObject(bucketName, "object1");
+    assertThrows(() -> oss.deleteBucket(bucketName), OSSErrorCode.BUCKET_NOT_EMPTY);
+
+    oss.deleteObject(bucketName, "object2");
+    oss.deleteBucket(bucketName);
+    Assert.assertFalse(doesBucketExist(bucketName));
+
+    oss.createBucket(bucketName);
+  }
+
+  @Test
+  public void testPutObject() throws IOException {
+    byte[] bytes = new byte[4 * 1024];
+    random.nextBytes(bytes);
+
+    String bucketNotExist = String.format("bucket-not-existing-%s", UUID.randomUUID());
+    assertThrows(() -> oss.putObject(bucketNotExist, "object", wrap(bytes)), OSSErrorCode.NO_SUCH_BUCKET);
+
+    PutObjectResult result = oss.putObject(bucketName, "object", wrap(bytes));
+    Assert.assertEquals(AliyunOSSMockLocalStore.md5sum(wrap(bytes)), result.getETag());
+  }
+
+  @Test
+  public void testDoesObjectExist() {
+    Assert.assertFalse(oss.doesObjectExist(bucketName, "key"));
+
+    Assert.assertFalse(oss.doesObjectExist(bucketName, "key"));
+
+    byte[] bytes = new byte[4 * 1024];
+    random.nextBytes(bytes);
+    oss.putObject(bucketName, "key", wrap(bytes));
+
+    Assert.assertTrue(oss.doesObjectExist(bucketName, "key"));
+    oss.deleteObject(bucketName, "key");
+  }
+
+  @Test
+  public void testGetObject() throws IOException {
+    String bucketNotExist = String.format("bucket-not-existing-%s", UUID.randomUUID());
+    assertThrows(() -> oss.getObject(bucketNotExist, "key"), OSSErrorCode.NO_SUCH_BUCKET);
+
+    assertThrows(() -> oss.getObject(bucketName, "key"), OSSErrorCode.NO_SUCH_KEY);
+
+    byte[] bytes = new byte[2000];
+    random.nextBytes(bytes);
+
+    oss.putObject(bucketName, "key", new ByteArrayInputStream(bytes));
+
+    byte[] actual = new byte[2000];
+    IOUtils.readFully(oss.getObject(bucketName, "key").getObjectContent(), actual);
+
+    Assert.assertArrayEquals(bytes, actual);
+    oss.deleteObject(bucketName, "key");
+  }
+
+  @Test
+  public void testGetObjectWithRange() throws IOException {
+

Review comment:
       Nit: I think this line should be removed because it seems not match the whole iceberg project code style , even if we don't put it into the check style rule set.




-- 
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: issues-unsubscribe@iceberg.apache.org

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] xingbowu commented on a change in pull request #3067: Aliyun: Mock aliyun OSS(Object Storage Service)

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



##########
File path: build.gradle
##########
@@ -262,6 +262,36 @@ project(':iceberg-data') {
   }
 }
 
+project(':iceberg-aliyun') {
+  dependencies {
+    compile project(':iceberg-api')
+    compile project(':iceberg-common')
+
+    compileOnly 'com.aliyun.oss:aliyun-sdk-oss'
+    compileOnly 'javax.xml.bind:jaxb-api'
+    compileOnly 'javax.activation:activation'
+    compileOnly 'org.glassfish.jaxb:jaxb-runtime'

Review comment:
       per aliyun oss sdk documentation, https://help.aliyun.com/document_detail/32009.html
   jaxb related dependencies need to be added under java 9 and plus version environment. 




-- 
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: issues-unsubscribe@iceberg.apache.org

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] openinx commented on a change in pull request #3067: Aliyun: Mock aliyun OSS(Object Storage Service)

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



##########
File path: aliyun/src/test/java/org/apache/iceberg/aliyun/oss/mock/TestLocalAliyunOSS.java
##########
@@ -0,0 +1,184 @@
+/*
+ * 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.aliyun.oss.mock;
+
+import com.aliyun.oss.OSS;
+import com.aliyun.oss.OSSErrorCode;
+import com.aliyun.oss.OSSException;
+import com.aliyun.oss.model.GetObjectRequest;
+import com.aliyun.oss.model.PutObjectResult;
+import java.io.ByteArrayInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.Objects;
+import java.util.Random;
+import java.util.UUID;
+import org.apache.commons.io.IOUtils;
+import org.apache.iceberg.aliyun.oss.AliyunOSSTestRule;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.ClassRule;
+import org.junit.Test;
+
+public class TestLocalAliyunOSS {
+
+  @ClassRule
+  public static final AliyunOSSTestRule OSS_TEST_RULE = AliyunOSSMockRule.builder().silent().build();
+
+  private final OSS oss = OSS_TEST_RULE.createOSSClient();
+  private final String bucketName = OSS_TEST_RULE.testBucketName();
+  private final Random random = new Random(1);
+
+  private static void assertThrows(Runnable runnable, String expectedErrorCode) {
+    try {
+      runnable.run();
+      Assert.fail("No exception was thrown, expected errorCode: " + expectedErrorCode);
+    } catch (OSSException e) {
+      Assert.assertEquals(expectedErrorCode, e.getErrorCode());
+    }
+  }
+
+  @Before
+  public void before() {
+    OSS_TEST_RULE.setUpBucket(bucketName);
+  }
+
+  @After
+  public void after() {
+    OSS_TEST_RULE.tearDownBucket(bucketName);
+  }
+
+  @Test
+  public void testBuckets() {
+    Assert.assertTrue(doesBucketExist(bucketName));
+    assertThrows(() -> oss.createBucket(bucketName), OSSErrorCode.BUCKET_ALREADY_EXISTS);
+
+    oss.deleteBucket(bucketName);
+    Assert.assertFalse(doesBucketExist(bucketName));
+
+    oss.createBucket(bucketName);
+    Assert.assertTrue(doesBucketExist(bucketName));
+  }
+
+  @Test
+  public void testDeleteBucket() {
+    String bucketNotExist = String.format("bucket-not-existing-%s", UUID.randomUUID());
+    assertThrows(() -> oss.deleteBucket(bucketNotExist), OSSErrorCode.NO_SUCH_BUCKET);
+
+    byte[] bytes = new byte[2000];
+    random.nextBytes(bytes);
+
+    oss.putObject(bucketName, "object1", wrap(bytes));
+
+    oss.putObject(bucketName, "object2", wrap(bytes));
+
+    assertThrows(() -> oss.deleteBucket(bucketName), OSSErrorCode.BUCKET_NOT_EMPTY);
+
+    oss.deleteObject(bucketName, "object1");
+    assertThrows(() -> oss.deleteBucket(bucketName), OSSErrorCode.BUCKET_NOT_EMPTY);
+
+    oss.deleteObject(bucketName, "object2");
+    oss.deleteBucket(bucketName);
+    Assert.assertFalse(doesBucketExist(bucketName));
+
+    oss.createBucket(bucketName);
+  }
+
+  @Test
+  public void testPutObject() throws IOException {
+    byte[] bytes = new byte[4 * 1024];
+    random.nextBytes(bytes);
+
+    String bucketNotExist = String.format("bucket-not-existing-%s", UUID.randomUUID());
+    assertThrows(() -> oss.putObject(bucketNotExist, "object", wrap(bytes)), OSSErrorCode.NO_SUCH_BUCKET);
+
+    PutObjectResult result = oss.putObject(bucketName, "object", wrap(bytes));
+    Assert.assertEquals(AliyunOSSMockLocalStore.md5sum(wrap(bytes)), result.getETag());
+  }
+
+  @Test
+  public void testDoesObjectExist() {
+    Assert.assertFalse(oss.doesObjectExist(bucketName, "key"));
+
+    Assert.assertFalse(oss.doesObjectExist(bucketName, "key"));
+
+    byte[] bytes = new byte[4 * 1024];
+    random.nextBytes(bytes);
+    oss.putObject(bucketName, "key", wrap(bytes));
+
+    Assert.assertTrue(oss.doesObjectExist(bucketName, "key"));
+    oss.deleteObject(bucketName, "key");
+  }
+
+  @Test
+  public void testGetObject() throws IOException {
+    String bucketNotExist = String.format("bucket-not-existing-%s", UUID.randomUUID());
+    assertThrows(() -> oss.getObject(bucketNotExist, "key"), OSSErrorCode.NO_SUCH_BUCKET);
+
+    assertThrows(() -> oss.getObject(bucketName, "key"), OSSErrorCode.NO_SUCH_KEY);
+
+    byte[] bytes = new byte[2000];
+    random.nextBytes(bytes);
+
+    oss.putObject(bucketName, "key", new ByteArrayInputStream(bytes));
+
+    byte[] actual = new byte[2000];
+    IOUtils.readFully(oss.getObject(bucketName, "key").getObjectContent(), actual);
+
+    Assert.assertArrayEquals(bytes, actual);
+    oss.deleteObject(bucketName, "key");
+  }
+
+  @Test
+  public void testGetObjectWithRange() throws IOException {
+
+    byte[] bytes = new byte[2000];
+    random.nextBytes(bytes);
+    oss.putObject(bucketName, "key", new ByteArrayInputStream(bytes));
+
+    byte[] actual = new byte[2000];
+    int start = 0;
+    int end = 1999;
+    GetObjectRequest getObjectRequest = new GetObjectRequest(bucketName, "key");
+    getObjectRequest.setRange(start, end);
+    IOUtils.readFully(oss.getObject(getObjectRequest).getObjectContent(), actual);
+
+    Assert.assertArrayEquals(bytes, actual);

Review comment:
       It's incorrect to test the object range query by asserting the whole 2000 bytes to the bytes which is read from range [0, 1999], because it did not cover the case that `range` does not affect anything.  If we want to design case to cover the range read, then we must cover the following cases: 
   
   * Range = [0,0]
   * Range = [0,1]
   * Range = [1, 100]
   * Range = [0, 1999]
   * Range = [-1, 100]
   * Range = [100, -1]
   * Range = [-1, -1]
   
   etc.




-- 
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: issues-unsubscribe@iceberg.apache.org

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] openinx commented on a change in pull request #3067: Aliyun: Mock aliyun OSS(Object Storage Service)

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



##########
File path: aliyun/src/test/java/org/apache/iceberg/aliyun/oss/mock/AliyunOSSMockRule.java
##########
@@ -0,0 +1,143 @@
+/*
+ * 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.aliyun.oss.mock;
+
+import com.aliyun.oss.OSS;
+import com.aliyun.oss.OSSClientBuilder;
+import java.io.File;
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.nio.file.Files;
+import java.util.Map;
+import org.apache.commons.io.FileUtils;
+import org.apache.iceberg.aliyun.oss.AliyunOSSTestRule;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class AliyunOSSMockRule implements AliyunOSSTestRule {
+  private static final Logger LOG = LoggerFactory.getLogger(AliyunOSSMockRule.class);
+
+  private final Map<String, Object> properties;
+
+  private AliyunOSSMockApp ossMockApp;
+
+  private AliyunOSSMockRule(Map<String, Object> properties) {
+    this.properties = properties;
+  }
+
+  public static Builder builder() {
+    return new Builder();
+  }
+
+  @Override
+  public void start() {
+    ossMockApp = AliyunOSSMockApp.start(properties);
+  }
+
+  @Override
+  public void stop() {
+    ossMockApp.stop();
+  }
+
+  @Override
+  public OSS createOSSClient() {
+    String endpoint = String.format("http://localhost:%s", properties.getOrDefault(
+        AliyunOSSMockApp.PROP_HTTP_PORT,
+        AliyunOSSMockApp.PORT_HTTP_PORT_DEFAULT));
+    return new OSSClientBuilder().build(endpoint, "foo", "bar");
+  }
+
+  @Override
+  public String keyPrefix() {
+    return "mock-objects/";
+  }
+
+  private File rootDir() {
+    Object rootDir = properties.get(AliyunOSSMockApp.PROP_ROOT_DIR);
+    Preconditions.checkNotNull(rootDir, "Root directory cannot be null");
+    return new File(rootDir.toString());
+  }
+
+  @Override
+  public void setUpBucket(String bucket) {
+    createOSSClient().createBucket(bucket);
+  }
+
+  @Override
+  public void tearDownBucket(String bucket) {
+    try {
+      Files.walk(rootDir().toPath())
+          .filter(p -> p.toFile().isFile())

Review comment:
       In aliyun OSS,  the `<root-dir>/test-bucket/path/to` won't be a specific directory in the real production, but in our local OSS storage app,  it will be a real directory (though it's not a real object that people could see by using aliyun OSS SDK).   




-- 
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: issues-unsubscribe@iceberg.apache.org

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] openinx commented on a change in pull request #3067: Aliyun: Mock aliyun OSS(Object Storage Service)

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



##########
File path: aliyun/src/test/java/org/apache/iceberg/aliyun/oss/AliyunOSSTestRule.java
##########
@@ -0,0 +1,82 @@
+/*
+ * 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.aliyun.oss;
+
+import com.aliyun.oss.OSS;
+import java.util.UUID;
+import org.junit.rules.TestRule;
+import org.junit.runner.Description;
+import org.junit.runners.model.Statement;
+
+/**
+ * API for test Aliyun Object Storage Service (OSS) either local mock or remote server

Review comment:
       > API for test Aliyun Object Storage Service (OSS) either local mock or remote server
   
   Change it to `API for test Aliyun Object Storage Service (OSS)  which is either local mock http server or remote aliyun oss server`




-- 
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: issues-unsubscribe@iceberg.apache.org

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] openinx commented on a change in pull request #3067: Aliyun: Mock aliyun OSS(Object Storage Service)

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



##########
File path: aliyun/src/test/java/org/apache/iceberg/aliyun/oss/mock/TestLocalAliyunOSS.java
##########
@@ -0,0 +1,184 @@
+/*
+ * 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.aliyun.oss.mock;
+
+import com.aliyun.oss.OSS;
+import com.aliyun.oss.OSSErrorCode;
+import com.aliyun.oss.OSSException;
+import com.aliyun.oss.model.GetObjectRequest;
+import com.aliyun.oss.model.PutObjectResult;
+import java.io.ByteArrayInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.Objects;
+import java.util.Random;
+import java.util.UUID;
+import org.apache.commons.io.IOUtils;
+import org.apache.iceberg.aliyun.oss.AliyunOSSTestRule;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.ClassRule;
+import org.junit.Test;
+
+public class TestLocalAliyunOSS {
+
+  @ClassRule
+  public static final AliyunOSSTestRule OSS_TEST_RULE = AliyunOSSMockRule.builder().silent().build();

Review comment:
       @xingbowu How is feeling about this comment ?




-- 
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: issues-unsubscribe@iceberg.apache.org

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] xingbowu closed pull request #3067: Aliyun: Mock aliyun OSS(Object Storage Service)

Posted by GitBox <gi...@apache.org>.
xingbowu closed pull request #3067:
URL: https://github.com/apache/iceberg/pull/3067


   


-- 
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: issues-unsubscribe@iceberg.apache.org

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] openinx commented on a change in pull request #3067: Aliyun: Mock aliyun OSS(Object Storage Service)

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



##########
File path: aliyun/src/test/java/org/apache/iceberg/aliyun/oss/mock/AliyunOSSMockApp.java
##########
@@ -0,0 +1,156 @@
+/*
+ * 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.aliyun.oss.mock;
+
+import java.util.List;
+import java.util.Map;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.springframework.beans.factory.annotation.Autowired;
+import org.springframework.boot.Banner;
+import org.springframework.boot.SpringApplication;
+import org.springframework.boot.autoconfigure.EnableAutoConfiguration;
+import org.springframework.boot.autoconfigure.security.servlet.SecurityAutoConfiguration;
+import org.springframework.boot.builder.SpringApplicationBuilder;
+import org.springframework.context.ConfigurableApplicationContext;
+import org.springframework.context.annotation.Bean;
+import org.springframework.context.annotation.ComponentScan;
+import org.springframework.context.annotation.Configuration;
+import org.springframework.core.convert.converter.Converter;
+import org.springframework.http.MediaType;
+import org.springframework.http.converter.xml.MappingJackson2XmlHttpMessageConverter;
+import org.springframework.util.StringUtils;
+import org.springframework.web.servlet.config.annotation.WebMvcConfigurer;
+
+@SuppressWarnings("checkstyle:AnnotationUseStyle")
+@Configuration
+@EnableAutoConfiguration(exclude = {SecurityAutoConfiguration.class}, excludeName = {
+    "org.springframework.boot.actuate.autoconfigure.security.servlet.ManagementWebSecurityAutoConfiguration"
+})
+@ComponentScan
+public class AliyunOSSMockApp {
+
+  static final String PROP_ROOT_DIR = "root-dir";
+
+  static final String PROP_HTTP_PORT = "server.port";
+  static final int PORT_HTTP_PORT_DEFAULT = 9393;
+
+  static final String PROP_SILENT = "silent";
+
+  @Autowired
+  private ConfigurableApplicationContext context;
+
+  public static AliyunOSSMockApp start(Map<String, Object> properties, String... args) {
+    Map<String, Object> defaults = Maps.newHashMap();
+    defaults.put(PROP_HTTP_PORT, PORT_HTTP_PORT_DEFAULT);
+
+    Banner.Mode bannerMode = Banner.Mode.CONSOLE;
+
+    if (Boolean.parseBoolean(String.valueOf(properties.remove("silent")))) {

Review comment:
       Nit:  Here we can replace the `silent` by the `PROP_SILENT`.




-- 
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: issues-unsubscribe@iceberg.apache.org

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] openinx commented on a change in pull request #3067: Mock aliyun OSS(Object Storage Service)

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



##########
File path: aliyun/src/test/java/org/apache/iceberg/aliyun/oss/mock/AliyunOSSMockApp.java
##########
@@ -0,0 +1,175 @@
+/*
+ * 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.aliyun.oss.mock;
+
+import java.util.List;
+import java.util.Map;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.springframework.beans.factory.annotation.Autowired;
+import org.springframework.beans.factory.annotation.Value;
+import org.springframework.boot.Banner;
+import org.springframework.boot.SpringApplication;
+import org.springframework.boot.autoconfigure.EnableAutoConfiguration;
+import org.springframework.boot.autoconfigure.security.servlet.SecurityAutoConfiguration;
+import org.springframework.boot.builder.SpringApplicationBuilder;
+import org.springframework.context.ConfigurableApplicationContext;
+import org.springframework.context.annotation.Bean;
+import org.springframework.context.annotation.ComponentScan;
+import org.springframework.context.annotation.Configuration;
+import org.springframework.core.convert.converter.Converter;
+import org.springframework.http.MediaType;
+import org.springframework.http.converter.xml.MappingJackson2XmlHttpMessageConverter;
+import org.springframework.util.StringUtils;
+import org.springframework.web.servlet.config.annotation.ContentNegotiationConfigurer;
+import org.springframework.web.servlet.config.annotation.WebMvcConfigurer;
+
+@SuppressWarnings("checkstyle:AnnotationUseStyle")
+@Configuration
+@EnableAutoConfiguration(exclude = {SecurityAutoConfiguration.class}, excludeName = {
+    "org.springframework.boot.actuate.autoconfigure.security.servlet.ManagementWebSecurityAutoConfiguration"
+})
+@ComponentScan
+public class AliyunOSSMockApp {
+
+  static final String PROP_ROOT_DIR = "root-dir";
+
+  static final String PROP_HTTP_PORT = "server.port";
+  static final int PORT_HTTP_PORT_DEFAULT = 9393;
+
+  static final String PROP_SILENT = "silent";
+
+  @Autowired
+  private ConfigurableApplicationContext context;
+
+  @Autowired
+  private Config config;
+
+  public static void main(final String[] args) {
+    start(Maps.newHashMap(), args);
+  }
+
+  public static AliyunOSSMockApp start(Map<String, Object> properties, String... args) {
+    Map<String, Object> defaults = Maps.newHashMap();
+    defaults.put(PROP_HTTP_PORT, PORT_HTTP_PORT_DEFAULT);
+
+    Banner.Mode bannerMode = Banner.Mode.CONSOLE;
+
+    if (Boolean.parseBoolean(String.valueOf(properties.remove("silent")))) {
+      defaults.put("logging.level.root", "WARN");
+      bannerMode = Banner.Mode.OFF;
+    }
+
+    final ConfigurableApplicationContext ctx =
+        new SpringApplicationBuilder(AliyunOSSMockApp.class)
+            .properties(defaults)
+            .properties(properties)
+            .bannerMode(bannerMode)
+            .run(args);
+
+    return ctx.getBean(AliyunOSSMockApp.class);
+  }
+
+  public void stop() {
+    SpringApplication.exit(context, () -> 0);
+  }
+
+  @Configuration
+  static class Config implements WebMvcConfigurer {
+
+    @Value("${" + PROP_HTTP_PORT + "}")
+    private int httpPort;
+
+    @Override
+    public void configureContentNegotiation(final ContentNegotiationConfigurer configurer) {
+      configurer.defaultContentType(MediaType.APPLICATION_FORM_URLENCODED, MediaType.APPLICATION_XML);
+      configurer.favorPathExtension(false);
+      configurer.mediaType("xml", MediaType.TEXT_XML);
+    }
+
+    @Bean
+    Converter<String, Range> rangeConverter() {
+      return new RangeConverter();
+    }
+
+    /**
+     * Creates an HttpMessageConverter for XML.
+     *
+     * @return The configured {@link MappingJackson2XmlHttpMessageConverter}.
+     */
+    @Bean
+    public MappingJackson2XmlHttpMessageConverter getMessageConverter() {
+      List<MediaType> mediaTypes = Lists.newArrayList();
+      mediaTypes.add(MediaType.APPLICATION_XML);
+      mediaTypes.add(MediaType.APPLICATION_FORM_URLENCODED);
+      mediaTypes.add(MediaType.APPLICATION_OCTET_STREAM);
+
+      final MappingJackson2XmlHttpMessageConverter xmlConverter = new MappingJackson2XmlHttpMessageConverter();
+      xmlConverter.setSupportedMediaTypes(mediaTypes);
+
+      return xmlConverter;
+    }
+  }
+
+  private static class RangeConverter implements Converter<String, Range> {
+
+    private static final Pattern REQUESTED_RANGE_PATTERN = Pattern.compile("^bytes=((\\d*)\\-(\\d*))((,\\d*-\\d*)*)");

Review comment:
       Nit: Redundant escape slash '\\-' in RegExp ? 

##########
File path: aliyun/src/test/java/org/apache/iceberg/aliyun/oss/mock/AliyunOSSMockApp.java
##########
@@ -0,0 +1,175 @@
+/*
+ * 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.aliyun.oss.mock;
+
+import java.util.List;
+import java.util.Map;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.springframework.beans.factory.annotation.Autowired;
+import org.springframework.beans.factory.annotation.Value;
+import org.springframework.boot.Banner;
+import org.springframework.boot.SpringApplication;
+import org.springframework.boot.autoconfigure.EnableAutoConfiguration;
+import org.springframework.boot.autoconfigure.security.servlet.SecurityAutoConfiguration;
+import org.springframework.boot.builder.SpringApplicationBuilder;
+import org.springframework.context.ConfigurableApplicationContext;
+import org.springframework.context.annotation.Bean;
+import org.springframework.context.annotation.ComponentScan;
+import org.springframework.context.annotation.Configuration;
+import org.springframework.core.convert.converter.Converter;
+import org.springframework.http.MediaType;
+import org.springframework.http.converter.xml.MappingJackson2XmlHttpMessageConverter;
+import org.springframework.util.StringUtils;
+import org.springframework.web.servlet.config.annotation.ContentNegotiationConfigurer;
+import org.springframework.web.servlet.config.annotation.WebMvcConfigurer;
+
+@SuppressWarnings("checkstyle:AnnotationUseStyle")
+@Configuration
+@EnableAutoConfiguration(exclude = {SecurityAutoConfiguration.class}, excludeName = {
+    "org.springframework.boot.actuate.autoconfigure.security.servlet.ManagementWebSecurityAutoConfiguration"
+})
+@ComponentScan
+public class AliyunOSSMockApp {
+
+  static final String PROP_ROOT_DIR = "root-dir";
+
+  static final String PROP_HTTP_PORT = "server.port";
+  static final int PORT_HTTP_PORT_DEFAULT = 9393;
+
+  static final String PROP_SILENT = "silent";
+
+  @Autowired
+  private ConfigurableApplicationContext context;
+
+  @Autowired
+  private Config config;
+
+  public static void main(final String[] args) {
+    start(Maps.newHashMap(), args);
+  }
+
+  public static AliyunOSSMockApp start(Map<String, Object> properties, String... args) {
+    Map<String, Object> defaults = Maps.newHashMap();
+    defaults.put(PROP_HTTP_PORT, PORT_HTTP_PORT_DEFAULT);
+
+    Banner.Mode bannerMode = Banner.Mode.CONSOLE;
+
+    if (Boolean.parseBoolean(String.valueOf(properties.remove("silent")))) {
+      defaults.put("logging.level.root", "WARN");
+      bannerMode = Banner.Mode.OFF;
+    }
+
+    final ConfigurableApplicationContext ctx =
+        new SpringApplicationBuilder(AliyunOSSMockApp.class)
+            .properties(defaults)
+            .properties(properties)
+            .bannerMode(bannerMode)
+            .run(args);
+
+    return ctx.getBean(AliyunOSSMockApp.class);
+  }
+
+  public void stop() {
+    SpringApplication.exit(context, () -> 0);
+  }
+
+  @Configuration
+  static class Config implements WebMvcConfigurer {
+
+    @Value("${" + PROP_HTTP_PORT + "}")
+    private int httpPort;
+
+    @Override
+    public void configureContentNegotiation(final ContentNegotiationConfigurer configurer) {
+      configurer.defaultContentType(MediaType.APPLICATION_FORM_URLENCODED, MediaType.APPLICATION_XML);
+      configurer.favorPathExtension(false);

Review comment:
       Nit: Looks like the method is deprecated now ?




-- 
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: issues-unsubscribe@iceberg.apache.org

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] openinx commented on a change in pull request #3067: Aliyun: Mock aliyun OSS(Object Storage Service)

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



##########
File path: aliyun/src/test/java/org/apache/iceberg/aliyun/oss/mock/TestLocalAliyunOSS.java
##########
@@ -0,0 +1,184 @@
+/*
+ * 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.aliyun.oss.mock;
+
+import com.aliyun.oss.OSS;
+import com.aliyun.oss.OSSErrorCode;
+import com.aliyun.oss.OSSException;
+import com.aliyun.oss.model.GetObjectRequest;
+import com.aliyun.oss.model.PutObjectResult;
+import java.io.ByteArrayInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.Objects;
+import java.util.Random;
+import java.util.UUID;
+import org.apache.commons.io.IOUtils;
+import org.apache.iceberg.aliyun.oss.AliyunOSSTestRule;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.ClassRule;
+import org.junit.Test;
+
+public class TestLocalAliyunOSS {
+
+  @ClassRule
+  public static final AliyunOSSTestRule OSS_TEST_RULE = AliyunOSSMockRule.builder().silent().build();
+
+  private final OSS oss = OSS_TEST_RULE.createOSSClient();
+  private final String bucketName = OSS_TEST_RULE.testBucketName();
+  private final Random random = new Random(1);
+
+  private static void assertThrows(Runnable runnable, String expectedErrorCode) {
+    try {
+      runnable.run();
+      Assert.fail("No exception was thrown, expected errorCode: " + expectedErrorCode);
+    } catch (OSSException e) {
+      Assert.assertEquals(expectedErrorCode, e.getErrorCode());
+    }
+  }
+
+  @Before
+  public void before() {
+    OSS_TEST_RULE.setUpBucket(bucketName);
+  }
+
+  @After
+  public void after() {
+    OSS_TEST_RULE.tearDownBucket(bucketName);
+  }
+
+  @Test
+  public void testBuckets() {
+    Assert.assertTrue(doesBucketExist(bucketName));
+    assertThrows(() -> oss.createBucket(bucketName), OSSErrorCode.BUCKET_ALREADY_EXISTS);
+
+    oss.deleteBucket(bucketName);
+    Assert.assertFalse(doesBucketExist(bucketName));
+
+    oss.createBucket(bucketName);
+    Assert.assertTrue(doesBucketExist(bucketName));
+  }
+
+  @Test
+  public void testDeleteBucket() {
+    String bucketNotExist = String.format("bucket-not-existing-%s", UUID.randomUUID());
+    assertThrows(() -> oss.deleteBucket(bucketNotExist), OSSErrorCode.NO_SUCH_BUCKET);
+
+    byte[] bytes = new byte[2000];
+    random.nextBytes(bytes);
+
+    oss.putObject(bucketName, "object1", wrap(bytes));
+
+    oss.putObject(bucketName, "object2", wrap(bytes));
+
+    assertThrows(() -> oss.deleteBucket(bucketName), OSSErrorCode.BUCKET_NOT_EMPTY);
+
+    oss.deleteObject(bucketName, "object1");
+    assertThrows(() -> oss.deleteBucket(bucketName), OSSErrorCode.BUCKET_NOT_EMPTY);
+
+    oss.deleteObject(bucketName, "object2");
+    oss.deleteBucket(bucketName);
+    Assert.assertFalse(doesBucketExist(bucketName));
+
+    oss.createBucket(bucketName);
+  }
+
+  @Test
+  public void testPutObject() throws IOException {
+    byte[] bytes = new byte[4 * 1024];
+    random.nextBytes(bytes);
+
+    String bucketNotExist = String.format("bucket-not-existing-%s", UUID.randomUUID());
+    assertThrows(() -> oss.putObject(bucketNotExist, "object", wrap(bytes)), OSSErrorCode.NO_SUCH_BUCKET);
+
+    PutObjectResult result = oss.putObject(bucketName, "object", wrap(bytes));
+    Assert.assertEquals(AliyunOSSMockLocalStore.md5sum(wrap(bytes)), result.getETag());
+  }
+
+  @Test
+  public void testDoesObjectExist() {
+    Assert.assertFalse(oss.doesObjectExist(bucketName, "key"));
+

Review comment:
       Why we need two same assert sentences ? 




-- 
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: issues-unsubscribe@iceberg.apache.org

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] xingbowu commented on pull request #3067: Aliyun: Mock aliyun OSS(Object Storage Service)

Posted by GitBox <gi...@apache.org>.
xingbowu commented on pull request #3067:
URL: https://github.com/apache/iceberg/pull/3067#issuecomment-927301454


   > @xingbowu , Could you please file new issues for the above comment ([1], [2], [3]) and put them in this project dashboard , so that we could easily track the whole progress ?
   > 
   > I plan to get this PR merged once we've tracked those issues and fixed the other minor comments, thanks for the work !
   > 
   > [1] [#3067 (comment)](https://github.com/apache/iceberg/pull/3067#discussion_r703249976)
   > [2] [#3067 (comment)](https://github.com/apache/iceberg/pull/3067#discussion_r703261388)
   > [3] [#3067 (comment)](https://github.com/apache/iceberg/pull/3067#discussion_r711561677)
   
   Issue #3180 opened 


-- 
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: issues-unsubscribe@iceberg.apache.org

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] openinx commented on a change in pull request #3067: Aliyun: Mock aliyun OSS(Object Storage Service)

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



##########
File path: aliyun/src/test/java/org/apache/iceberg/aliyun/oss/mock/AliyunOSSMockRule.java
##########
@@ -0,0 +1,143 @@
+/*
+ * 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.aliyun.oss.mock;
+
+import com.aliyun.oss.OSS;
+import com.aliyun.oss.OSSClientBuilder;
+import java.io.File;
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.nio.file.Files;
+import java.util.Map;
+import org.apache.commons.io.FileUtils;
+import org.apache.iceberg.aliyun.oss.AliyunOSSTestRule;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class AliyunOSSMockRule implements AliyunOSSTestRule {
+  private static final Logger LOG = LoggerFactory.getLogger(AliyunOSSMockRule.class);
+
+  private final Map<String, Object> properties;
+
+  private AliyunOSSMockApp ossMockApp;
+
+  private AliyunOSSMockRule(Map<String, Object> properties) {
+    this.properties = properties;
+  }
+
+  public static Builder builder() {
+    return new Builder();
+  }
+
+  @Override
+  public void start() {
+    ossMockApp = AliyunOSSMockApp.start(properties);
+  }
+
+  @Override
+  public void stop() {
+    ossMockApp.stop();
+  }
+
+  @Override
+  public OSS createOSSClient() {
+    String endpoint = String.format("http://localhost:%s", properties.getOrDefault(
+        AliyunOSSMockApp.PROP_HTTP_PORT,
+        AliyunOSSMockApp.PORT_HTTP_PORT_DEFAULT));
+    return new OSSClientBuilder().build(endpoint, "foo", "bar");
+  }
+
+  @Override
+  public String keyPrefix() {
+    return "mock-objects/";
+  }
+
+  private File rootDir() {
+    Object rootDir = properties.get(AliyunOSSMockApp.PROP_ROOT_DIR);
+    Preconditions.checkNotNull(rootDir, "Root directory cannot be null");
+    return new File(rootDir.toString());
+  }
+
+  @Override
+  public void setUpBucket(String bucket) {
+    createOSSClient().createBucket(bucket);
+  }
+
+  @Override
+  public void tearDownBucket(String bucket) {
+    try {
+      Files.walk(rootDir().toPath())
+          .filter(p -> p.toFile().isFile())
+          .forEach(p -> {
+            try {
+              Files.delete(p);
+            } catch (IOException e) {
+              // delete this files quietly.
+            }
+          });
+
+      createOSSClient().deleteBucket(bucket);
+    } catch (IOException e) {
+      throw new UncheckedIOException(e);
+    }
+  }
+
+  public static class Builder {
+    private Map<String, Object> props = Maps.newHashMap();
+
+    public Builder withRootDir(String rootDir) {
+      props.put(AliyunOSSMockApp.PROP_ROOT_DIR, rootDir);
+      return this;
+    }
+
+    public Builder silent() {
+      props.put(AliyunOSSMockApp.PROP_SILENT, true);
+      return this;
+    }
+
+    public AliyunOSSMockRule build() {
+      if (props.get(AliyunOSSMockApp.PROP_ROOT_DIR) == null) {

Review comment:
       So when the PROP_ROOT_DIR is not set or set to empty string,  we will use the default temporary directory with a current timestamp suffix.  Then we will create the directory by using the `createRootDir` method .  I think it's more clear to decide which root dir value we will use firstly, then check whether it's created for testing: 
   
   ```java
       public AliyunOSSMockRule build() {
         String rootDir = (String) props.get(AliyunOSSMockApp.PROP_ROOT_DIR);
         if (StringUtils.isNullOrEmpty(rootDir)) {
           File dir = new File(FileUtils.getTempDirectory(), "oss-mock-file-store" + System.currentTimeMillis());
           rootDir = dir.getAbsolutePath();
         }
         File root = new File(rootDir);
         root.deleteOnExit();
         root.mkdir();
   
         return new AliyunOSSMockRule(props);
       }
   ```




-- 
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: issues-unsubscribe@iceberg.apache.org

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] xingbowu commented on a change in pull request #3067: Aliyun: Mock aliyun OSS(Object Storage Service)

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



##########
File path: aliyun/src/test/java/org/apache/iceberg/aliyun/oss/mock/TestLocalAliyunOSS.java
##########
@@ -0,0 +1,184 @@
+/*
+ * 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.aliyun.oss.mock;
+
+import com.aliyun.oss.OSS;
+import com.aliyun.oss.OSSErrorCode;
+import com.aliyun.oss.OSSException;
+import com.aliyun.oss.model.GetObjectRequest;
+import com.aliyun.oss.model.PutObjectResult;
+import java.io.ByteArrayInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.Objects;
+import java.util.Random;
+import java.util.UUID;
+import org.apache.commons.io.IOUtils;
+import org.apache.iceberg.aliyun.oss.AliyunOSSTestRule;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.ClassRule;
+import org.junit.Test;
+
+public class TestLocalAliyunOSS {
+
+  @ClassRule
+  public static final AliyunOSSTestRule OSS_TEST_RULE = AliyunOSSMockRule.builder().silent().build();

Review comment:
       yes, I will implement it in the next PR, which include both local and remote part




-- 
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: issues-unsubscribe@iceberg.apache.org

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] openinx commented on a change in pull request #3067: Aliyun: Mock aliyun OSS(Object Storage Service)

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



##########
File path: aliyun/src/test/java/org/apache/iceberg/aliyun/oss/mock/AliyunOSSMockApp.java
##########
@@ -0,0 +1,160 @@
+/*
+ * 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.aliyun.oss.mock;
+
+import java.util.List;
+import java.util.Map;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.springframework.beans.factory.annotation.Autowired;
+import org.springframework.boot.Banner;
+import org.springframework.boot.SpringApplication;
+import org.springframework.boot.autoconfigure.EnableAutoConfiguration;
+import org.springframework.boot.autoconfigure.security.servlet.SecurityAutoConfiguration;
+import org.springframework.boot.builder.SpringApplicationBuilder;
+import org.springframework.context.ConfigurableApplicationContext;
+import org.springframework.context.annotation.Bean;
+import org.springframework.context.annotation.ComponentScan;
+import org.springframework.context.annotation.Configuration;
+import org.springframework.core.convert.converter.Converter;
+import org.springframework.http.MediaType;
+import org.springframework.http.converter.xml.MappingJackson2XmlHttpMessageConverter;
+import org.springframework.util.StringUtils;
+import org.springframework.web.servlet.config.annotation.WebMvcConfigurer;
+
+@SuppressWarnings("checkstyle:AnnotationUseStyle")
+@Configuration
+@EnableAutoConfiguration(exclude = {SecurityAutoConfiguration.class}, excludeName = {
+    "org.springframework.boot.actuate.autoconfigure.security.servlet.ManagementWebSecurityAutoConfiguration"
+})
+@ComponentScan
+public class AliyunOSSMockApp {
+
+  static final String PROP_ROOT_DIR = "root-dir";
+
+  static final String PROP_HTTP_PORT = "server.port";
+  static final int PORT_HTTP_PORT_DEFAULT = 9393;
+
+  static final String PROP_SILENT = "silent";
+
+  @Autowired
+  private ConfigurableApplicationContext context;
+
+  public static void main(final String[] args) {
+    start(Maps.newHashMap(), args);
+  }

Review comment:
       Is their any place that we will start a main thread to start the application in iceberg unit tests ?  If don't, then we iceberg usually don't expose any `main` thread.




-- 
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: issues-unsubscribe@iceberg.apache.org

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] openinx commented on a change in pull request #3067: Aliyun: Mock aliyun OSS(Object Storage Service)

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



##########
File path: aliyun/src/test/java/org/apache/iceberg/aliyun/oss/mock/TestLocalAliyunOSS.java
##########
@@ -0,0 +1,184 @@
+/*
+ * 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.aliyun.oss.mock;
+
+import com.aliyun.oss.OSS;
+import com.aliyun.oss.OSSErrorCode;
+import com.aliyun.oss.OSSException;
+import com.aliyun.oss.model.GetObjectRequest;
+import com.aliyun.oss.model.PutObjectResult;
+import java.io.ByteArrayInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.Objects;
+import java.util.Random;
+import java.util.UUID;
+import org.apache.commons.io.IOUtils;
+import org.apache.iceberg.aliyun.oss.AliyunOSSTestRule;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.ClassRule;
+import org.junit.Test;
+
+public class TestLocalAliyunOSS {
+
+  @ClassRule
+  public static final AliyunOSSTestRule OSS_TEST_RULE = AliyunOSSMockRule.builder().silent().build();
+
+  private final OSS oss = OSS_TEST_RULE.createOSSClient();
+  private final String bucketName = OSS_TEST_RULE.testBucketName();
+  private final Random random = new Random(1);
+
+  private static void assertThrows(Runnable runnable, String expectedErrorCode) {
+    try {
+      runnable.run();
+      Assert.fail("No exception was thrown, expected errorCode: " + expectedErrorCode);
+    } catch (OSSException e) {
+      Assert.assertEquals(expectedErrorCode, e.getErrorCode());
+    }
+  }
+
+  @Before
+  public void before() {
+    OSS_TEST_RULE.setUpBucket(bucketName);
+  }
+
+  @After
+  public void after() {
+    OSS_TEST_RULE.tearDownBucket(bucketName);
+  }
+
+  @Test
+  public void testBuckets() {
+    Assert.assertTrue(doesBucketExist(bucketName));
+    assertThrows(() -> oss.createBucket(bucketName), OSSErrorCode.BUCKET_ALREADY_EXISTS);
+
+    oss.deleteBucket(bucketName);
+    Assert.assertFalse(doesBucketExist(bucketName));
+
+    oss.createBucket(bucketName);
+    Assert.assertTrue(doesBucketExist(bucketName));
+  }
+
+  @Test
+  public void testDeleteBucket() {
+    String bucketNotExist = String.format("bucket-not-existing-%s", UUID.randomUUID());
+    assertThrows(() -> oss.deleteBucket(bucketNotExist), OSSErrorCode.NO_SUCH_BUCKET);
+
+    byte[] bytes = new byte[2000];
+    random.nextBytes(bytes);
+
+    oss.putObject(bucketName, "object1", wrap(bytes));
+
+    oss.putObject(bucketName, "object2", wrap(bytes));
+
+    assertThrows(() -> oss.deleteBucket(bucketName), OSSErrorCode.BUCKET_NOT_EMPTY);
+
+    oss.deleteObject(bucketName, "object1");
+    assertThrows(() -> oss.deleteBucket(bucketName), OSSErrorCode.BUCKET_NOT_EMPTY);
+
+    oss.deleteObject(bucketName, "object2");
+    oss.deleteBucket(bucketName);
+    Assert.assertFalse(doesBucketExist(bucketName));
+
+    oss.createBucket(bucketName);
+  }
+
+  @Test
+  public void testPutObject() throws IOException {
+    byte[] bytes = new byte[4 * 1024];
+    random.nextBytes(bytes);
+
+    String bucketNotExist = String.format("bucket-not-existing-%s", UUID.randomUUID());
+    assertThrows(() -> oss.putObject(bucketNotExist, "object", wrap(bytes)), OSSErrorCode.NO_SUCH_BUCKET);
+
+    PutObjectResult result = oss.putObject(bucketName, "object", wrap(bytes));
+    Assert.assertEquals(AliyunOSSMockLocalStore.md5sum(wrap(bytes)), result.getETag());
+  }
+
+  @Test
+  public void testDoesObjectExist() {
+    Assert.assertFalse(oss.doesObjectExist(bucketName, "key"));
+
+    Assert.assertFalse(oss.doesObjectExist(bucketName, "key"));
+
+    byte[] bytes = new byte[4 * 1024];
+    random.nextBytes(bytes);
+    oss.putObject(bucketName, "key", wrap(bytes));
+
+    Assert.assertTrue(oss.doesObjectExist(bucketName, "key"));
+    oss.deleteObject(bucketName, "key");
+  }
+
+  @Test
+  public void testGetObject() throws IOException {
+    String bucketNotExist = String.format("bucket-not-existing-%s", UUID.randomUUID());
+    assertThrows(() -> oss.getObject(bucketNotExist, "key"), OSSErrorCode.NO_SUCH_BUCKET);
+
+    assertThrows(() -> oss.getObject(bucketName, "key"), OSSErrorCode.NO_SUCH_KEY);
+
+    byte[] bytes = new byte[2000];
+    random.nextBytes(bytes);
+
+    oss.putObject(bucketName, "key", new ByteArrayInputStream(bytes));
+
+    byte[] actual = new byte[2000];
+    IOUtils.readFully(oss.getObject(bucketName, "key").getObjectContent(), actual);
+
+    Assert.assertArrayEquals(bytes, actual);
+    oss.deleteObject(bucketName, "key");
+  }
+
+  @Test
+  public void testGetObjectWithRange() throws IOException {
+

Review comment:
       Extra empty line ?




-- 
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: issues-unsubscribe@iceberg.apache.org

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] openinx commented on a change in pull request #3067: Aliyun: Mock aliyun OSS(Object Storage Service)

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



##########
File path: aliyun/src/test/java/org/apache/iceberg/aliyun/oss/mock/AliyunOSSMockLocalStore.java
##########
@@ -0,0 +1,197 @@
+/*
+ * 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.aliyun.oss.mock;
+
+import com.aliyun.oss.OSSErrorCode;
+import com.aliyun.oss.model.Bucket;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.nio.file.DirectoryStream;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.attribute.BasicFileAttributes;
+import java.security.MessageDigest;
+import java.security.NoSuchAlgorithmException;
+import java.util.List;
+import java.util.Map;
+import org.apache.commons.io.FileUtils;
+import org.apache.commons.io.IOUtils;
+import org.apache.directory.api.util.Hex;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.springframework.beans.factory.annotation.Value;
+import org.springframework.http.MediaType;
+import org.springframework.stereotype.Component;
+
+@Component
+public class AliyunOSSMockLocalStore {
+  private static final Logger LOG = LoggerFactory.getLogger(AliyunOSSMockLocalStore.class);
+
+  private static final String DATA_FILE = ".DATA";
+  private static final String META_FILE = ".META";
+
+  private final File root;
+
+  private final ObjectMapper objectMapper = new ObjectMapper();
+
+  public AliyunOSSMockLocalStore(@Value("${" + AliyunOSSMockApp.PROP_ROOT_DIR + ":}") String rootDir) {
+    Preconditions.checkNotNull(rootDir, "Root directory cannot be null");
+    this.root = new File(rootDir);
+
+    root.deleteOnExit();
+    root.mkdirs();
+
+    LOG.info("Root directory of local OSS store is {}", root);
+  }
+
+  void createBucket(String bucketName) throws IOException {
+    File newBucket = new File(root, bucketName);
+    FileUtils.forceMkdir(newBucket);
+  }
+
+  Bucket getBucket(String bucketName) {
+    List<Bucket> buckets = findBucketsByFilter(file ->
+        Files.isDirectory(file) && file.getFileName().endsWith(bucketName));

Review comment:
       > file.getFileName().endsWith(bucketName)
   
   It's incorrect to use the `file.getFileName().endsWith(bucketName)` to check its bucket name, because for a bucket `test-bucket`, it's possible that we will have an object with name `/path/to/test-bucket/a.txt`.  In that case,  we will create a directory `<root-dir>/path/to/test-bucket` though it's no an object being visiable to OSS SDK, but it will fail to check the existence of this given bucket.




-- 
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: issues-unsubscribe@iceberg.apache.org

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] openinx commented on a change in pull request #3067: Aliyun: Mock aliyun OSS(Object Storage Service)

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



##########
File path: aliyun/src/test/java/org/apache/iceberg/aliyun/oss/util/AliyunOSSURI.java
##########
@@ -0,0 +1,86 @@
+/*
+ * 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.aliyun.oss.util;
+
+import java.util.Set;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+
+public class AliyunOSSURI {

Review comment:
       Do we really need to introduce this class in this PR ?  I see no place are referring to this class. 




-- 
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: issues-unsubscribe@iceberg.apache.org

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] xingbowu commented on a change in pull request #3067: Aliyun: Mock aliyun OSS(Object Storage Service)

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



##########
File path: versions.props
##########
@@ -16,6 +16,10 @@ com.github.ben-manes.caffeine:caffeine = 2.8.4
 org.apache.arrow:arrow-vector = 2.0.0
 org.apache.arrow:arrow-memory-netty = 2.0.0
 com.github.stephenc.findbugs:findbugs-annotations = 1.3.9-1
+com.aliyun.oss:aliyun-sdk-oss = 3.10.2
+javax.xml.bind:jaxb-api = 2.3.1
+javax.activation:activation = 1.1.1
+org.glassfish.jaxb:jaxb-runtime = 2.3.3

Review comment:
       same as above comment




-- 
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: issues-unsubscribe@iceberg.apache.org

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] xingbowu commented on a change in pull request #3067: Aliyun: Mock aliyun OSS(Object Storage Service)

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



##########
File path: aliyun/src/test/java/org/apache/iceberg/aliyun/oss/mock/AliyunOSSMockRule.java
##########
@@ -0,0 +1,143 @@
+/*
+ * 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.aliyun.oss.mock;
+
+import com.aliyun.oss.OSS;
+import com.aliyun.oss.OSSClientBuilder;
+import java.io.File;
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.nio.file.Files;
+import java.util.Map;
+import org.apache.commons.io.FileUtils;
+import org.apache.iceberg.aliyun.oss.AliyunOSSTestRule;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class AliyunOSSMockRule implements AliyunOSSTestRule {
+  private static final Logger LOG = LoggerFactory.getLogger(AliyunOSSMockRule.class);
+
+  private final Map<String, Object> properties;
+
+  private AliyunOSSMockApp ossMockApp;
+
+  private AliyunOSSMockRule(Map<String, Object> properties) {
+    this.properties = properties;
+  }
+
+  public static Builder builder() {
+    return new Builder();
+  }
+
+  @Override
+  public void start() {
+    ossMockApp = AliyunOSSMockApp.start(properties);
+  }
+
+  @Override
+  public void stop() {
+    ossMockApp.stop();
+  }
+
+  @Override
+  public OSS createOSSClient() {
+    String endpoint = String.format("http://localhost:%s", properties.getOrDefault(
+        AliyunOSSMockApp.PROP_HTTP_PORT,
+        AliyunOSSMockApp.PORT_HTTP_PORT_DEFAULT));
+    return new OSSClientBuilder().build(endpoint, "foo", "bar");
+  }
+
+  @Override
+  public String keyPrefix() {
+    return "mock-objects/";
+  }
+
+  private File rootDir() {
+    Object rootDir = properties.get(AliyunOSSMockApp.PROP_ROOT_DIR);
+    Preconditions.checkNotNull(rootDir, "Root directory cannot be null");
+    return new File(rootDir.toString());
+  }
+
+  @Override
+  public void setUpBucket(String bucket) {
+    createOSSClient().createBucket(bucket);
+  }
+
+  @Override
+  public void tearDownBucket(String bucket) {
+    try {
+      Files.walk(rootDir().toPath())
+          .filter(p -> p.toFile().isFile())

Review comment:
       Regarding to the scenario you list above,  the directories are deleted in the deleteBucket. However,  we indeed have problem in another scenario. if we create an object with name "aa/bb/cc.txt", and then remove it. after that we create a new object with name "aa/bb". it has problem because of deleting cc.txt in current logic only. I will fix it in separate 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: issues-unsubscribe@iceberg.apache.org

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] xingbowu commented on a change in pull request #3067: Aliyun: Mock aliyun OSS(Object Storage Service)

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



##########
File path: aliyun/src/test/java/org/apache/iceberg/aliyun/oss/mock/TestLocalAliyunOSS.java
##########
@@ -0,0 +1,184 @@
+/*
+ * 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.aliyun.oss.mock;
+
+import com.aliyun.oss.OSS;
+import com.aliyun.oss.OSSErrorCode;
+import com.aliyun.oss.OSSException;
+import com.aliyun.oss.model.GetObjectRequest;
+import com.aliyun.oss.model.PutObjectResult;
+import java.io.ByteArrayInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.Objects;
+import java.util.Random;
+import java.util.UUID;
+import org.apache.commons.io.IOUtils;
+import org.apache.iceberg.aliyun.oss.AliyunOSSTestRule;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.ClassRule;
+import org.junit.Test;
+
+public class TestLocalAliyunOSS {
+
+  @ClassRule
+  public static final AliyunOSSTestRule OSS_TEST_RULE = AliyunOSSMockRule.builder().silent().build();
+
+  private final OSS oss = OSS_TEST_RULE.createOSSClient();
+  private final String bucketName = OSS_TEST_RULE.testBucketName();
+  private final Random random = new Random(1);
+
+  private static void assertThrows(Runnable runnable, String expectedErrorCode) {
+    try {
+      runnable.run();
+      Assert.fail("No exception was thrown, expected errorCode: " + expectedErrorCode);
+    } catch (OSSException e) {
+      Assert.assertEquals(expectedErrorCode, e.getErrorCode());
+    }
+  }
+
+  @Before
+  public void before() {
+    OSS_TEST_RULE.setUpBucket(bucketName);
+  }
+
+  @After
+  public void after() {
+    OSS_TEST_RULE.tearDownBucket(bucketName);
+  }
+
+  @Test
+  public void testBuckets() {
+    Assert.assertTrue(doesBucketExist(bucketName));
+    assertThrows(() -> oss.createBucket(bucketName), OSSErrorCode.BUCKET_ALREADY_EXISTS);
+
+    oss.deleteBucket(bucketName);
+    Assert.assertFalse(doesBucketExist(bucketName));
+
+    oss.createBucket(bucketName);
+    Assert.assertTrue(doesBucketExist(bucketName));
+  }
+
+  @Test
+  public void testDeleteBucket() {
+    String bucketNotExist = String.format("bucket-not-existing-%s", UUID.randomUUID());
+    assertThrows(() -> oss.deleteBucket(bucketNotExist), OSSErrorCode.NO_SUCH_BUCKET);
+
+    byte[] bytes = new byte[2000];
+    random.nextBytes(bytes);
+
+    oss.putObject(bucketName, "object1", wrap(bytes));
+
+    oss.putObject(bucketName, "object2", wrap(bytes));
+
+    assertThrows(() -> oss.deleteBucket(bucketName), OSSErrorCode.BUCKET_NOT_EMPTY);
+
+    oss.deleteObject(bucketName, "object1");
+    assertThrows(() -> oss.deleteBucket(bucketName), OSSErrorCode.BUCKET_NOT_EMPTY);
+
+    oss.deleteObject(bucketName, "object2");
+    oss.deleteBucket(bucketName);
+    Assert.assertFalse(doesBucketExist(bucketName));
+
+    oss.createBucket(bucketName);
+  }
+
+  @Test
+  public void testPutObject() throws IOException {
+    byte[] bytes = new byte[4 * 1024];
+    random.nextBytes(bytes);
+
+    String bucketNotExist = String.format("bucket-not-existing-%s", UUID.randomUUID());
+    assertThrows(() -> oss.putObject(bucketNotExist, "object", wrap(bytes)), OSSErrorCode.NO_SUCH_BUCKET);
+
+    PutObjectResult result = oss.putObject(bucketName, "object", wrap(bytes));
+    Assert.assertEquals(AliyunOSSMockLocalStore.md5sum(wrap(bytes)), result.getETag());
+  }
+
+  @Test
+  public void testDoesObjectExist() {
+    Assert.assertFalse(oss.doesObjectExist(bucketName, "key"));
+
+    Assert.assertFalse(oss.doesObjectExist(bucketName, "key"));
+
+    byte[] bytes = new byte[4 * 1024];
+    random.nextBytes(bytes);
+    oss.putObject(bucketName, "key", wrap(bytes));
+
+    Assert.assertTrue(oss.doesObjectExist(bucketName, "key"));
+    oss.deleteObject(bucketName, "key");
+  }
+
+  @Test
+  public void testGetObject() throws IOException {
+    String bucketNotExist = String.format("bucket-not-existing-%s", UUID.randomUUID());
+    assertThrows(() -> oss.getObject(bucketNotExist, "key"), OSSErrorCode.NO_SUCH_BUCKET);
+
+    assertThrows(() -> oss.getObject(bucketName, "key"), OSSErrorCode.NO_SUCH_KEY);
+
+    byte[] bytes = new byte[2000];
+    random.nextBytes(bytes);
+
+    oss.putObject(bucketName, "key", new ByteArrayInputStream(bytes));
+
+    byte[] actual = new byte[2000];
+    IOUtils.readFully(oss.getObject(bucketName, "key").getObjectContent(), actual);
+
+    Assert.assertArrayEquals(bytes, actual);
+    oss.deleteObject(bucketName, "key");
+  }
+
+  @Test
+  public void testGetObjectWithRange() throws IOException {
+
+    byte[] bytes = new byte[2000];
+    random.nextBytes(bytes);
+    oss.putObject(bucketName, "key", new ByteArrayInputStream(bytes));
+
+    byte[] actual = new byte[2000];
+    int start = 0;
+    int end = 1999;
+    GetObjectRequest getObjectRequest = new GetObjectRequest(bucketName, "key");
+    getObjectRequest.setRange(start, end);
+    IOUtils.readFully(oss.getObject(getObjectRequest).getObjectContent(), actual);
+
+    Assert.assertArrayEquals(bytes, actual);
+    oss.deleteObject(bucketName, "key");
+  }
+
+  private InputStream wrap(byte[] data) {
+    return new ByteArrayInputStream(data);
+  }
+
+  private boolean doesBucketExist(String bucket) {
+    try {
+      oss.createBucket(bucket);

Review comment:
       it is the simplest way to simulate this api behavior.  we can mock further version if necessary later per upper layer need. may keep it as it is here.




-- 
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: issues-unsubscribe@iceberg.apache.org

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] openinx commented on a change in pull request #3067: Aliyun: Mock aliyun OSS(Object Storage Service)

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



##########
File path: aliyun/src/test/java/org/apache/iceberg/aliyun/oss/mock/AliyunOSSMockRule.java
##########
@@ -0,0 +1,143 @@
+/*
+ * 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.aliyun.oss.mock;
+
+import com.aliyun.oss.OSS;
+import com.aliyun.oss.OSSClientBuilder;
+import java.io.File;
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.nio.file.Files;
+import java.util.Map;
+import org.apache.commons.io.FileUtils;
+import org.apache.iceberg.aliyun.oss.AliyunOSSTestRule;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class AliyunOSSMockRule implements AliyunOSSTestRule {
+  private static final Logger LOG = LoggerFactory.getLogger(AliyunOSSMockRule.class);
+
+  private final Map<String, Object> properties;
+
+  private AliyunOSSMockApp ossMockApp;
+
+  private AliyunOSSMockRule(Map<String, Object> properties) {
+    this.properties = properties;
+  }
+
+  public static Builder builder() {
+    return new Builder();
+  }
+
+  @Override
+  public void start() {
+    ossMockApp = AliyunOSSMockApp.start(properties);
+  }
+
+  @Override
+  public void stop() {
+    ossMockApp.stop();
+  }
+
+  @Override
+  public OSS createOSSClient() {
+    String endpoint = String.format("http://localhost:%s", properties.getOrDefault(
+        AliyunOSSMockApp.PROP_HTTP_PORT,
+        AliyunOSSMockApp.PORT_HTTP_PORT_DEFAULT));
+    return new OSSClientBuilder().build(endpoint, "foo", "bar");
+  }
+
+  @Override
+  public String keyPrefix() {
+    return "mock-objects/";
+  }
+
+  private File rootDir() {
+    Object rootDir = properties.get(AliyunOSSMockApp.PROP_ROOT_DIR);
+    Preconditions.checkNotNull(rootDir, "Root directory cannot be null");
+    return new File(rootDir.toString());
+  }
+
+  @Override
+  public void setUpBucket(String bucket) {
+    createOSSClient().createBucket(bucket);
+  }
+
+  @Override
+  public void tearDownBucket(String bucket) {
+    try {
+      Files.walk(rootDir().toPath())
+          .filter(p -> p.toFile().isFile())

Review comment:
       Why if we have a bucket named `test-bucket`, and have a object named `path/to/a.dat`,  Then the full local path will be `<root-dir>/test-bucket/path/to/a.dat`,   will we remove the `<root-dir>/test-bucket/path/to` directory also ? 




-- 
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: issues-unsubscribe@iceberg.apache.org

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] openinx commented on a change in pull request #3067: Aliyun: Mock aliyun OSS(Object Storage Service)

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



##########
File path: aliyun/src/test/java/org/apache/iceberg/aliyun/oss/mock/TestLocalAliyunOSS.java
##########
@@ -0,0 +1,184 @@
+/*
+ * 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.aliyun.oss.mock;
+
+import com.aliyun.oss.OSS;
+import com.aliyun.oss.OSSErrorCode;
+import com.aliyun.oss.OSSException;
+import com.aliyun.oss.model.GetObjectRequest;
+import com.aliyun.oss.model.PutObjectResult;
+import java.io.ByteArrayInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.Objects;
+import java.util.Random;
+import java.util.UUID;
+import org.apache.commons.io.IOUtils;
+import org.apache.iceberg.aliyun.oss.AliyunOSSTestRule;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.ClassRule;
+import org.junit.Test;
+
+public class TestLocalAliyunOSS {
+
+  @ClassRule
+  public static final AliyunOSSTestRule OSS_TEST_RULE = AliyunOSSMockRule.builder().silent().build();

Review comment:
       I think we need to make this `TestRule` to be configurable so that we could verify this unit tests on both local mock oss services and remote aliyun OSS environment, to guarantee that the local oss app has the same semantics as remote aliyun OSS environment.




-- 
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: issues-unsubscribe@iceberg.apache.org

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] openinx commented on a change in pull request #3067: Aliyun: Mock aliyun OSS(Object Storage Service)

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



##########
File path: aliyun/src/test/java/org/apache/iceberg/aliyun/oss/mock/AliyunOSSMockRule.java
##########
@@ -0,0 +1,143 @@
+/*
+ * 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.aliyun.oss.mock;
+
+import com.aliyun.oss.OSS;
+import com.aliyun.oss.OSSClientBuilder;
+import java.io.File;
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.nio.file.Files;
+import java.util.Map;
+import org.apache.commons.io.FileUtils;
+import org.apache.iceberg.aliyun.oss.AliyunOSSTestRule;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class AliyunOSSMockRule implements AliyunOSSTestRule {
+  private static final Logger LOG = LoggerFactory.getLogger(AliyunOSSMockRule.class);
+
+  private final Map<String, Object> properties;
+
+  private AliyunOSSMockApp ossMockApp;
+
+  private AliyunOSSMockRule(Map<String, Object> properties) {
+    this.properties = properties;
+  }
+
+  public static Builder builder() {
+    return new Builder();
+  }
+
+  @Override
+  public void start() {
+    ossMockApp = AliyunOSSMockApp.start(properties);
+  }
+
+  @Override
+  public void stop() {
+    ossMockApp.stop();
+  }
+
+  @Override
+  public OSS createOSSClient() {
+    String endpoint = String.format("http://localhost:%s", properties.getOrDefault(
+        AliyunOSSMockApp.PROP_HTTP_PORT,
+        AliyunOSSMockApp.PORT_HTTP_PORT_DEFAULT));
+    return new OSSClientBuilder().build(endpoint, "foo", "bar");
+  }
+
+  @Override
+  public String keyPrefix() {
+    return "mock-objects/";
+  }
+
+  private File rootDir() {
+    Object rootDir = properties.get(AliyunOSSMockApp.PROP_ROOT_DIR);
+    Preconditions.checkNotNull(rootDir, "Root directory cannot be null");
+    return new File(rootDir.toString());
+  }
+
+  @Override
+  public void setUpBucket(String bucket) {
+    createOSSClient().createBucket(bucket);
+  }
+
+  @Override
+  public void tearDownBucket(String bucket) {
+    try {
+      Files.walk(rootDir().toPath())
+          .filter(p -> p.toFile().isFile())
+          .forEach(p -> {
+            try {
+              Files.delete(p);
+            } catch (IOException e) {
+              // delete this files quietly.
+            }
+          });
+
+      createOSSClient().deleteBucket(bucket);
+    } catch (IOException e) {
+      throw new UncheckedIOException(e);
+    }
+  }
+
+  public static class Builder {
+    private Map<String, Object> props = Maps.newHashMap();
+
+    public Builder withRootDir(String rootDir) {

Review comment:
       I'm thinking we may don't need to expose a public `withRootDir` in this Builder because people often don't set any specific RootDir in their mocking unit tests, they just use the temporary directory directly. 




-- 
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: issues-unsubscribe@iceberg.apache.org

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] xingbowu commented on pull request #3067: Mock aliyun OSS(Object Storage Service)

Posted by GitBox <gi...@apache.org>.
xingbowu commented on pull request #3067:
URL: https://github.com/apache/iceberg/pull/3067#issuecomment-913286170


   @rdblue @openinx Could you help to take a look at this PR? Thanks in advance.


-- 
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: issues-unsubscribe@iceberg.apache.org

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] openinx commented on a change in pull request #3067: Aliyun: Mock aliyun OSS(Object Storage Service)

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



##########
File path: aliyun/src/test/java/org/apache/iceberg/aliyun/oss/mock/AliyunOSSMockLocalStore.java
##########
@@ -0,0 +1,197 @@
+/*
+ * 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.aliyun.oss.mock;
+
+import com.aliyun.oss.OSSErrorCode;
+import com.aliyun.oss.model.Bucket;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.nio.file.DirectoryStream;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.attribute.BasicFileAttributes;
+import java.security.MessageDigest;
+import java.security.NoSuchAlgorithmException;
+import java.util.List;
+import java.util.Map;
+import org.apache.commons.io.FileUtils;
+import org.apache.commons.io.IOUtils;
+import org.apache.directory.api.util.Hex;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.springframework.beans.factory.annotation.Value;
+import org.springframework.http.MediaType;
+import org.springframework.stereotype.Component;
+
+@Component
+public class AliyunOSSMockLocalStore {
+  private static final Logger LOG = LoggerFactory.getLogger(AliyunOSSMockLocalStore.class);
+
+  private static final String DATA_FILE = ".DATA";
+  private static final String META_FILE = ".META";
+
+  private final File root;
+
+  private final ObjectMapper objectMapper = new ObjectMapper();
+
+  public AliyunOSSMockLocalStore(@Value("${" + AliyunOSSMockApp.PROP_ROOT_DIR + ":}") String rootDir) {
+    Preconditions.checkNotNull(rootDir, "Root directory cannot be null");
+    this.root = new File(rootDir);
+
+    root.deleteOnExit();
+    root.mkdirs();
+
+    LOG.info("Root directory of local OSS store is {}", root);
+  }
+
+  void createBucket(String bucketName) throws IOException {
+    File newBucket = new File(root, bucketName);
+    FileUtils.forceMkdir(newBucket);
+  }
+
+  Bucket getBucket(String bucketName) {
+    List<Bucket> buckets = findBucketsByFilter(file ->
+        Files.isDirectory(file) && file.getFileName().endsWith(bucketName));
+
+    return buckets.size() > 0 ? buckets.get(0) : null;
+  }
+
+  void deleteBucket(String bucketName) throws IOException {
+    Bucket bucket = getBucket(bucketName);
+    Preconditions.checkNotNull(bucket, "Bucket %s shouldn't be null.", bucketName);
+
+    File dir = new File(root, bucket.getName());
+    if (Files.walk(dir.toPath()).anyMatch(p -> p.toFile().isFile())) {
+      throw new AliyunOSSMockLocalController.OssException(409, OSSErrorCode.BUCKET_NOT_EMPTY,
+          "The bucket you tried to delete is not empty. ");
+    }
+
+    FileUtils.deleteDirectory(dir);
+  }
+
+  ObjectMetadata putObject(String bucketName,
+      String fileName,
+      InputStream dataStream,
+      String contentType,
+      String contentEncoding,
+      Map<String, String> userMetaData) throws IOException {
+    File bucketDir = new File(root, bucketName);
+    assert bucketDir.exists() || bucketDir.mkdirs();
+
+    File dataFile = new File(bucketDir, fileName + DATA_FILE);
+    File metaFile = new File(bucketDir, fileName + META_FILE);
+    if (!dataFile.exists()) {
+      dataFile.getParentFile().mkdirs();
+      dataFile.createNewFile();
+    }
+
+    inputStreamToFile(dataStream, dataFile);
+
+    ObjectMetadata metadata = new ObjectMetadata();
+    metadata.setContentLength(dataFile.length());
+    metadata.setContentMD5(md5sum(dataFile.getAbsolutePath()));
+    metadata.setContentType(contentType != null ? contentType : MediaType.APPLICATION_OCTET_STREAM_VALUE);
+    metadata.setContentEncoding(contentEncoding);
+    metadata.setDataFile(dataFile.getAbsolutePath());
+    metadata.setMetaFile(metaFile.getAbsolutePath());
+
+    BasicFileAttributes attributes = Files.readAttributes(dataFile.toPath(), BasicFileAttributes.class);
+    metadata.setLastModificationDate(attributes.lastModifiedTime().toMillis());
+
+    metadata.setUserMetaData(userMetaData);
+
+    objectMapper.writeValue(metaFile, metadata);
+
+    return metadata;
+  }
+
+  void deleteObject(String bucketName, String filename) {
+    File bucketDir = new File(root, bucketName);
+    assert bucketDir.exists();
+
+    File dataFile = new File(bucketDir, filename + DATA_FILE);
+    File metaFile = new File(bucketDir, filename + META_FILE);
+    assert !dataFile.exists() || dataFile.delete();
+    assert !metaFile.exists() || metaFile.delete();
+  }
+
+  ObjectMetadata getObjectMetadata(String bucketName, String filename) throws IOException {
+    File bucketDir = new File(root, bucketName);
+    assert bucketDir.exists();
+
+    File dataFile = new File(bucketDir, filename + DATA_FILE);
+    if (!dataFile.exists()) {
+      return null;
+    }
+
+    File metaFile = new File(bucketDir, filename + META_FILE);
+    return objectMapper.readValue(metaFile, ObjectMetadata.class);
+  }
+
+  static String md5sum(String filepath) throws IOException {
+    try (InputStream is = new FileInputStream(filepath)) {
+      return md5sum(is);
+    }
+  }
+
+  static String md5sum(InputStream is) throws IOException {
+    MessageDigest md;
+    try {
+      md = MessageDigest.getInstance("MD5");
+      md.reset();
+    } catch (NoSuchAlgorithmException e) {
+      throw new RuntimeException(e);
+    }
+    byte[] bytes = new byte[1024];
+    int numBytes;
+
+    while ((numBytes = is.read(bytes)) != -1) {
+      md.update(bytes, 0, numBytes);
+    }
+    return new String(Hex.encodeHex(md.digest()));
+  }
+
+  private static void inputStreamToFile(InputStream inputStream, File targetFile) throws IOException {
+    try (OutputStream outputStream = new FileOutputStream(targetFile)) {
+      IOUtils.copy(inputStream, outputStream);
+    }
+  }
+
+  private List<Bucket> findBucketsByFilter(final DirectoryStream.Filter<Path> filter) {
+    List<Bucket> buckets = Lists.newArrayList();
+
+    try (DirectoryStream<Path> stream = Files.newDirectoryStream(root.toPath(), filter)) {
+      for (final Path path : stream) {
+        buckets.add(new Bucket(path.getFileName().toString()));
+      }
+    } catch (final IOException e) {
+      LOG.error("Could not Iterate over Bucket-Folders", e);

Review comment:
       Nit: `Iterate` -> `iterate`




-- 
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: issues-unsubscribe@iceberg.apache.org

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] openinx commented on a change in pull request #3067: Aliyun: Mock aliyun OSS(Object Storage Service)

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



##########
File path: aliyun/src/test/java/org/apache/iceberg/aliyun/oss/AliyunOSSTestRule.java
##########
@@ -0,0 +1,84 @@
+/*
+ * 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.aliyun.oss;
+
+import com.aliyun.oss.OSS;
+import java.util.UUID;
+import org.junit.rules.TestRule;
+import org.junit.runner.Description;
+import org.junit.runners.model.Statement;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public interface AliyunOSSTestRule extends TestRule {
+  Logger LOG = LoggerFactory.getLogger(AliyunOSSTestRule.class);
+  UUID RANDOM_UUID = java.util.UUID.randomUUID();
+  /**

Review comment:
       In apache iceberg project, we usually leave an empty line before the two code blocks between two methods declaration.  I guess you need to check your intellij code style configuration and also apply the iceberg [checkstyles](https://iceberg.apache.org/community/#setting-up-ide-and-code-style) from the document. 
   
   ![image](https://user-images.githubusercontent.com/5028729/132298216-ab4cbdb3-43d1-4bad-bbe9-4e5d148be100.png)
   




-- 
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: issues-unsubscribe@iceberg.apache.org

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] xingbowu commented on pull request #3067: Aliyun: Mock aliyun OSS(Object Storage Service)

Posted by GitBox <gi...@apache.org>.
xingbowu commented on pull request #3067:
URL: https://github.com/apache/iceberg/pull/3067#issuecomment-913654757


   > Thanks @xingbowu to make a smaller PR for the Aliyun [OSS integration](https://github.com/apache/iceberg/pull/2230) work. I skimmed the whole PR, this PR is trying to introduce the aliyun oss mock app server, so that we could build oss test cases on top of it (We don't need to mock all the called OSS API in a relative complex test cases , such as multi-upload test cases). It's a good thing for us to introduce a simple simulator to align the local mock app and aliyun online oss server.
   > 
   > The most important thing for me is: How do we guarantee the local mock app is aligning correctly to the aliyun online oss server ? In the parent PR #2230 , we introduced a class test rule named OSSTestRule and the rule has two different impl:
   > 
   > * OSSMockRule:  The rule will start a local mini aliyun oss server, which could serving the remote OSS client http requests.
   > * OSSIntegrationTestRule: The rule will prepare testing buckets in the remote oss server, so that the test cases  could write real data to.
   > 
   > For the local oss application, we provide tests cases in [TestLocalOSS](https://github.com/apache/iceberg/blob/3755c967f61035097abbebfe419d17473bf5fefd/aliyun/src/test/java/org/apache/iceberg/aliyun/oss/mock/TestLocalOSS.java) , which was designed to run different TestRule according to the intentional environment variables (will run local mock app by default). If the tests could be passes on both local env and remote oss env, then we could definitely ensure the local oss app is implemented correctly.
   
   
   
   > Thanks @xingbowu to make a smaller PR for the Aliyun [OSS integration](https://github.com/apache/iceberg/pull/2230) work. I skimmed the whole PR, this PR is trying to introduce the aliyun oss mock app server, so that we could build oss test cases on top of it (We don't need to mock all the called OSS API in a relative complex test cases , such as multi-upload test cases). It's a good thing for us to introduce a simple simulator to align the local mock app and aliyun online oss server.
   > 
   > The most important thing for me is: How do we guarantee the local mock app is aligning correctly to the aliyun online oss server ? In the parent PR #2230 , we introduced a class test rule named OSSTestRule and the rule has two different impl:
   > 
   > * OSSMockRule:  The rule will start a local mini aliyun oss server, which could serving the remote OSS client http requests.
   > * OSSIntegrationTestRule: The rule will prepare testing buckets in the remote oss server, so that the test cases  could write real data to.
   > 
   > For the local oss application, we provide tests cases in [TestLocalOSS](https://github.com/apache/iceberg/blob/3755c967f61035097abbebfe419d17473bf5fefd/aliyun/src/test/java/org/apache/iceberg/aliyun/oss/mock/TestLocalOSS.java) , which was designed to run different TestRule according to the intentional environment variables (will run local mock app by default). If the tests could be passes on both local env and remote oss env, then we could definitely ensure the local oss app is implemented correctly.
   
   Thanks a lot for pointing out missing part. I have done rework and added local test here to guarantee quality. comparing with 2230,  implemented more test case, such as range get. 
   
   Additionally, followed up your review comments to simulate basic aliyun oss behavior including create/put/delete and excluded multi-parts , feel free to let me know if you have further comments.


-- 
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: issues-unsubscribe@iceberg.apache.org

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] xingbowu commented on pull request #3067: Aliyun: Mock aliyun OSS(Object Storage Service)

Posted by GitBox <gi...@apache.org>.
xingbowu commented on pull request #3067:
URL: https://github.com/apache/iceberg/pull/3067#issuecomment-919770830


   @openinx  Thanks for your effort and comments, I have reworked them with latest code, feel free to take further review


-- 
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: issues-unsubscribe@iceberg.apache.org

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] openinx commented on pull request #3067: Aliyun: Mock aliyun OSS(Object Storage Service)

Posted by GitBox <gi...@apache.org>.
openinx commented on pull request #3067:
URL: https://github.com/apache/iceberg/pull/3067#issuecomment-919085940


   The broken unit test is : 
   
   ```
   org.apache.iceberg.types.TestReadabilityChecks > testStructWriteReordering STANDARD_ERROR
       [nested.field_b is out of order, before field_a]
   
   org.apache.iceberg.io.TestCloseableGroup > suppressExceptionIfSetSuppressIsTrue STANDARD_ERROR
       [Test worker] ERROR org.apache.iceberg.io.CloseableGroup - Exception suppressed when attempting to close resources
       java.io.IOException: exception1
       	at org.apache.iceberg.io.CloseableGroup.close(CloseableGroup.java:80)
       	at org.apache.iceberg.io.TestCloseableGroup.suppressExceptionIfSetSuppressIsTrue(TestCloseableGroup.java:75)
       	at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
       	at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
       	at java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
       	at java.base/java.lang.reflect.Method.invoke(Method.java:566)
       	at org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:59)
       	at org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12)
       	at org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:56)
       	at org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17)
       	at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306)
       	at org.junit.runners.BlockJUnit4ClassRunner$1.evaluate(BlockJUnit4ClassRunner.java:100)
       	at org.junit.runners.ParentRunner.runLeaf(ParentRunner.java:366)
       	at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:103)
       	at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:63)
       	at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331)
       	at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79)
       	at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329)
       	at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66)
       	at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293)
       	at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306)
       	at org.junit.runners.ParentRunner.run(ParentRunner.java:413)
       	at org.gradle.api.internal.tasks.testing.junit.JUnitTestClassExecutor.runTestClass(JUnitTestClassExecutor.java:110)
       	at org.gradle.api.internal.tasks.testing.junit.JUnitTestClassExecutor.execute(JUnitTestClassExecutor.java:58)
       	at org.gradle.api.internal.tasks.testing.junit.JUnitTestClassExecutor.execute(JUnitTestClassExecutor.java:38)
       	at org.gradle.api.internal.tasks.testing.junit.AbstractJUnitTestClassProcessor.processTestClass(AbstractJUnitTestClassProcessor.java:62)
       	at org.gradle.api.internal.tasks.testing.SuiteTestClassProcessor.processTestClass(SuiteTestClassProcessor.java:51)
       	at jdk.internal.reflect.GeneratedMethodAccessor4.invoke(Unknown Source)
       	at java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
       	at java.base/java.lang.reflect.Method.invoke(Method.java:566)
       	at org.gradle.internal.dispatch.ReflectionDispatch.dispatch(ReflectionDispatch.java:35)
       	at org.gradle.internal.dispatch.ReflectionDispatch.dispatch(ReflectionDispatch.java:24)
       	at org.gradle.internal.dispatch.ContextClassLoaderDispatch.dispatch(ContextClassLoaderDispatch.java:32)
       	at org.gradle.internal.dispatch.ProxyDispatchAdapter$DispatchingInvocationHandler.invoke(ProxyDispatchAdapter.java:93)
       	at com.sun.proxy.$Proxy2.processTestClass(Unknown Source)
       	at org.gradle.api.internal.tasks.testing.worker.TestWorker.processTestClass(TestWorker.java:118)
       	at jdk.internal.reflect.GeneratedMethodAccessor3.invoke(Unknown Source)
       	at java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
       	at java.base/java.lang.reflect.Method.invoke(Method.java:566)
       	at org.gradle.internal.dispatch.ReflectionDispatch.dispatch(ReflectionDispatch.java:35)
       	at org.gradle.internal.dispatch.ReflectionDispatch.dispatch(ReflectionDispatch.java:24)
       	at org.gradle.internal.remote.internal.hub.MessageHubBackedObjectConnection$DispatchWrapper.dispatch(MessageHubBackedObjectConnection.java:175)
       	at org.gradle.internal.remote.internal.hub.MessageHubBackedObjectConnection$DispatchWrapper.dispatch(MessageHubBackedObjectConnection.java:157)
       	at org.gradle.internal.remote.internal.hub.MessageHub$Handler.run(MessageHub.java:404)
       	at org.gradle.internal.concurrent.ExecutorPolicy$CatchAndRecordFailures.onExecute(ExecutorPolicy.java:63)
       	at org.gradle.internal.concurrent.ManagedExecutorImpl$1.run(ManagedExecutorImpl.java:46)
       	at java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128)
       	at java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628)
       	at org.gradle.internal.concurrent.ThreadFactoryImpl$ManagedThreadRunnable.run(ThreadFactoryImpl.java:55)
       	at java.base/java.lang.Thread.run(Thread.java:829)
   ```
   
   Looks like it's unrelated to this PR,  let's reopen to trigger the travis CI once again.


-- 
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: issues-unsubscribe@iceberg.apache.org

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] xingbowu closed pull request #3067: Aliyun: Mock aliyun OSS(Object Storage Service)

Posted by GitBox <gi...@apache.org>.
xingbowu closed pull request #3067:
URL: https://github.com/apache/iceberg/pull/3067


   


-- 
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: issues-unsubscribe@iceberg.apache.org

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] xingbowu commented on a change in pull request #3067: Aliyun: Mock aliyun OSS(Object Storage Service)

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



##########
File path: aliyun/src/test/java/org/apache/iceberg/aliyun/oss/mock/AliyunOSSMockApp.java
##########
@@ -0,0 +1,175 @@
+/*
+ * 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.aliyun.oss.mock;
+
+import java.util.List;
+import java.util.Map;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.springframework.beans.factory.annotation.Autowired;
+import org.springframework.beans.factory.annotation.Value;
+import org.springframework.boot.Banner;
+import org.springframework.boot.SpringApplication;
+import org.springframework.boot.autoconfigure.EnableAutoConfiguration;
+import org.springframework.boot.autoconfigure.security.servlet.SecurityAutoConfiguration;
+import org.springframework.boot.builder.SpringApplicationBuilder;
+import org.springframework.context.ConfigurableApplicationContext;
+import org.springframework.context.annotation.Bean;
+import org.springframework.context.annotation.ComponentScan;
+import org.springframework.context.annotation.Configuration;
+import org.springframework.core.convert.converter.Converter;
+import org.springframework.http.MediaType;
+import org.springframework.http.converter.xml.MappingJackson2XmlHttpMessageConverter;
+import org.springframework.util.StringUtils;
+import org.springframework.web.servlet.config.annotation.ContentNegotiationConfigurer;
+import org.springframework.web.servlet.config.annotation.WebMvcConfigurer;
+
+@SuppressWarnings("checkstyle:AnnotationUseStyle")
+@Configuration
+@EnableAutoConfiguration(exclude = {SecurityAutoConfiguration.class}, excludeName = {
+    "org.springframework.boot.actuate.autoconfigure.security.servlet.ManagementWebSecurityAutoConfiguration"
+})
+@ComponentScan
+public class AliyunOSSMockApp {
+
+  static final String PROP_ROOT_DIR = "root-dir";
+
+  static final String PROP_HTTP_PORT = "server.port";
+  static final int PORT_HTTP_PORT_DEFAULT = 9393;
+
+  static final String PROP_SILENT = "silent";
+
+  @Autowired
+  private ConfigurableApplicationContext context;
+
+  @Autowired
+  private Config config;
+
+  public static void main(final String[] args) {
+    start(Maps.newHashMap(), args);
+  }
+
+  public static AliyunOSSMockApp start(Map<String, Object> properties, String... args) {
+    Map<String, Object> defaults = Maps.newHashMap();
+    defaults.put(PROP_HTTP_PORT, PORT_HTTP_PORT_DEFAULT);
+
+    Banner.Mode bannerMode = Banner.Mode.CONSOLE;
+
+    if (Boolean.parseBoolean(String.valueOf(properties.remove("silent")))) {
+      defaults.put("logging.level.root", "WARN");
+      bannerMode = Banner.Mode.OFF;
+    }
+
+    final ConfigurableApplicationContext ctx =
+        new SpringApplicationBuilder(AliyunOSSMockApp.class)
+            .properties(defaults)
+            .properties(properties)
+            .bannerMode(bannerMode)
+            .run(args);
+
+    return ctx.getBean(AliyunOSSMockApp.class);
+  }
+
+  public void stop() {
+    SpringApplication.exit(context, () -> 0);
+  }
+
+  @Configuration
+  static class Config implements WebMvcConfigurer {
+
+    @Value("${" + PROP_HTTP_PORT + "}")
+    private int httpPort;
+
+    @Override
+    public void configureContentNegotiation(final ContentNegotiationConfigurer configurer) {
+      configurer.defaultContentType(MediaType.APPLICATION_FORM_URLENCODED, MediaType.APPLICATION_XML);
+      configurer.favorPathExtension(false);

Review comment:
       done, remove this one

##########
File path: aliyun/src/test/java/org/apache/iceberg/aliyun/oss/mock/AliyunOSSMockApp.java
##########
@@ -0,0 +1,175 @@
+/*
+ * 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.aliyun.oss.mock;
+
+import java.util.List;
+import java.util.Map;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.springframework.beans.factory.annotation.Autowired;
+import org.springframework.beans.factory.annotation.Value;
+import org.springframework.boot.Banner;
+import org.springframework.boot.SpringApplication;
+import org.springframework.boot.autoconfigure.EnableAutoConfiguration;
+import org.springframework.boot.autoconfigure.security.servlet.SecurityAutoConfiguration;
+import org.springframework.boot.builder.SpringApplicationBuilder;
+import org.springframework.context.ConfigurableApplicationContext;
+import org.springframework.context.annotation.Bean;
+import org.springframework.context.annotation.ComponentScan;
+import org.springframework.context.annotation.Configuration;
+import org.springframework.core.convert.converter.Converter;
+import org.springframework.http.MediaType;
+import org.springframework.http.converter.xml.MappingJackson2XmlHttpMessageConverter;
+import org.springframework.util.StringUtils;
+import org.springframework.web.servlet.config.annotation.ContentNegotiationConfigurer;
+import org.springframework.web.servlet.config.annotation.WebMvcConfigurer;
+
+@SuppressWarnings("checkstyle:AnnotationUseStyle")
+@Configuration
+@EnableAutoConfiguration(exclude = {SecurityAutoConfiguration.class}, excludeName = {
+    "org.springframework.boot.actuate.autoconfigure.security.servlet.ManagementWebSecurityAutoConfiguration"
+})
+@ComponentScan
+public class AliyunOSSMockApp {
+
+  static final String PROP_ROOT_DIR = "root-dir";
+
+  static final String PROP_HTTP_PORT = "server.port";
+  static final int PORT_HTTP_PORT_DEFAULT = 9393;
+
+  static final String PROP_SILENT = "silent";
+
+  @Autowired
+  private ConfigurableApplicationContext context;
+
+  @Autowired
+  private Config config;
+
+  public static void main(final String[] args) {
+    start(Maps.newHashMap(), args);
+  }
+
+  public static AliyunOSSMockApp start(Map<String, Object> properties, String... args) {
+    Map<String, Object> defaults = Maps.newHashMap();
+    defaults.put(PROP_HTTP_PORT, PORT_HTTP_PORT_DEFAULT);
+
+    Banner.Mode bannerMode = Banner.Mode.CONSOLE;
+
+    if (Boolean.parseBoolean(String.valueOf(properties.remove("silent")))) {
+      defaults.put("logging.level.root", "WARN");
+      bannerMode = Banner.Mode.OFF;
+    }
+
+    final ConfigurableApplicationContext ctx =
+        new SpringApplicationBuilder(AliyunOSSMockApp.class)
+            .properties(defaults)
+            .properties(properties)
+            .bannerMode(bannerMode)
+            .run(args);
+
+    return ctx.getBean(AliyunOSSMockApp.class);
+  }
+
+  public void stop() {
+    SpringApplication.exit(context, () -> 0);
+  }
+
+  @Configuration
+  static class Config implements WebMvcConfigurer {
+
+    @Value("${" + PROP_HTTP_PORT + "}")
+    private int httpPort;
+
+    @Override
+    public void configureContentNegotiation(final ContentNegotiationConfigurer configurer) {
+      configurer.defaultContentType(MediaType.APPLICATION_FORM_URLENCODED, MediaType.APPLICATION_XML);
+      configurer.favorPathExtension(false);
+      configurer.mediaType("xml", MediaType.TEXT_XML);
+    }
+
+    @Bean
+    Converter<String, Range> rangeConverter() {
+      return new RangeConverter();
+    }
+
+    /**
+     * Creates an HttpMessageConverter for XML.
+     *
+     * @return The configured {@link MappingJackson2XmlHttpMessageConverter}.
+     */
+    @Bean
+    public MappingJackson2XmlHttpMessageConverter getMessageConverter() {
+      List<MediaType> mediaTypes = Lists.newArrayList();
+      mediaTypes.add(MediaType.APPLICATION_XML);
+      mediaTypes.add(MediaType.APPLICATION_FORM_URLENCODED);
+      mediaTypes.add(MediaType.APPLICATION_OCTET_STREAM);
+
+      final MappingJackson2XmlHttpMessageConverter xmlConverter = new MappingJackson2XmlHttpMessageConverter();
+      xmlConverter.setSupportedMediaTypes(mediaTypes);
+
+      return xmlConverter;
+    }
+  }
+
+  private static class RangeConverter implements Converter<String, Range> {
+
+    private static final Pattern REQUESTED_RANGE_PATTERN = Pattern.compile("^bytes=((\\d*)\\-(\\d*))((,\\d*-\\d*)*)");

Review comment:
       done




-- 
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: issues-unsubscribe@iceberg.apache.org

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] openinx commented on a change in pull request #3067: Aliyun: Mock aliyun OSS(Object Storage Service)

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



##########
File path: aliyun/src/test/java/org/apache/iceberg/aliyun/oss/AliyunOSSTestRule.java
##########
@@ -0,0 +1,84 @@
+/*
+ * 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.aliyun.oss;
+
+import com.aliyun.oss.OSS;
+import java.util.UUID;
+import org.junit.rules.TestRule;
+import org.junit.runner.Description;
+import org.junit.runners.model.Statement;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public interface AliyunOSSTestRule extends TestRule {
+  Logger LOG = LoggerFactory.getLogger(AliyunOSSTestRule.class);

Review comment:
       This line can be removed now ? Seems nobody are referring this.




-- 
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: issues-unsubscribe@iceberg.apache.org

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] openinx merged pull request #3067: Aliyun: Mock aliyun OSS(Object Storage Service)

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


   


-- 
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: issues-unsubscribe@iceberg.apache.org

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] xingbowu commented on a change in pull request #3067: Aliyun: Mock aliyun OSS(Object Storage Service)

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



##########
File path: aliyun/src/test/java/org/apache/iceberg/aliyun/oss/mock/AliyunOSSMockLocalStore.java
##########
@@ -0,0 +1,197 @@
+/*
+ * 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.aliyun.oss.mock;
+
+import com.aliyun.oss.OSSErrorCode;
+import com.aliyun.oss.model.Bucket;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.nio.file.DirectoryStream;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.attribute.BasicFileAttributes;
+import java.security.MessageDigest;
+import java.security.NoSuchAlgorithmException;
+import java.util.List;
+import java.util.Map;
+import org.apache.commons.io.FileUtils;
+import org.apache.commons.io.IOUtils;
+import org.apache.directory.api.util.Hex;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.springframework.beans.factory.annotation.Value;
+import org.springframework.http.MediaType;
+import org.springframework.stereotype.Component;
+
+@Component
+public class AliyunOSSMockLocalStore {
+  private static final Logger LOG = LoggerFactory.getLogger(AliyunOSSMockLocalStore.class);
+
+  private static final String DATA_FILE = ".DATA";
+  private static final String META_FILE = ".META";
+
+  private final File root;
+
+  private final ObjectMapper objectMapper = new ObjectMapper();
+
+  public AliyunOSSMockLocalStore(@Value("${" + AliyunOSSMockApp.PROP_ROOT_DIR + ":}") String rootDir) {
+    Preconditions.checkNotNull(rootDir, "Root directory cannot be null");
+    this.root = new File(rootDir);
+
+    root.deleteOnExit();
+    root.mkdirs();
+
+    LOG.info("Root directory of local OSS store is {}", root);
+  }
+
+  void createBucket(String bucketName) throws IOException {
+    File newBucket = new File(root, bucketName);
+    FileUtils.forceMkdir(newBucket);
+  }
+
+  Bucket getBucket(String bucketName) {
+    List<Bucket> buckets = findBucketsByFilter(file ->
+        Files.isDirectory(file) && file.getFileName().endsWith(bucketName));

Review comment:
       newDirectoryStream only traverses the directory&file under root path. the object with name /test-bucket/a.txt, is under <root-dir>/test-bucket(bucket name)/path/to/test-bucket(object prefix name)/a.txt. so it is no problem here.




-- 
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: issues-unsubscribe@iceberg.apache.org

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] openinx closed pull request #3067: Aliyun: Mock aliyun OSS(Object Storage Service)

Posted by GitBox <gi...@apache.org>.
openinx closed pull request #3067:
URL: https://github.com/apache/iceberg/pull/3067


   


-- 
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: issues-unsubscribe@iceberg.apache.org

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] openinx commented on pull request #3067: Aliyun: Mock aliyun OSS(Object Storage Service)

Posted by GitBox <gi...@apache.org>.
openinx commented on pull request #3067:
URL: https://github.com/apache/iceberg/pull/3067#issuecomment-927243226


   @xingbowu ,  Could you please file new issues for the above comment ([1], [2], [3]) and put them in this project dashboard , so that we could easily track the whole progress  ? 
   
   I plan to get this PR merged once we've tracked those issues and fixed the other minor comments, thanks for the work !
   
   [1] https://github.com/apache/iceberg/pull/3067#discussion_r703249976
   [2] https://github.com/apache/iceberg/pull/3067#discussion_r703261388
   [3] https://github.com/apache/iceberg/pull/3067#discussion_r711561677


-- 
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: issues-unsubscribe@iceberg.apache.org

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] openinx commented on a change in pull request #3067: Aliyun: Mock aliyun OSS(Object Storage Service)

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



##########
File path: versions.props
##########
@@ -16,6 +16,10 @@ com.github.ben-manes.caffeine:caffeine = 2.8.4
 org.apache.arrow:arrow-vector = 2.0.0
 org.apache.arrow:arrow-memory-netty = 2.0.0
 com.github.stephenc.findbugs:findbugs-annotations = 1.3.9-1
+com.aliyun.oss:aliyun-sdk-oss = 3.10.2
+javax.xml.bind:jaxb-api = 2.3.1
+javax.activation:activation = 1.1.1
+org.glassfish.jaxb:jaxb-runtime = 2.3.3

Review comment:
       Why we need to three extra jars  ?  I passed the test cases after removing them ..




-- 
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: issues-unsubscribe@iceberg.apache.org

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] openinx commented on pull request #3067: Aliyun: Mock aliyun OSS(Object Storage Service)

Posted by GitBox <gi...@apache.org>.
openinx commented on pull request #3067:
URL: https://github.com/apache/iceberg/pull/3067#issuecomment-922249336


   I left several comments, almost looks good to me.  @rdblue @jackye1995 would you like to have a double check ?


-- 
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: issues-unsubscribe@iceberg.apache.org

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] openinx commented on a change in pull request #3067: Aliyun: Mock aliyun OSS(Object Storage Service)

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



##########
File path: aliyun/src/test/java/org/apache/iceberg/aliyun/oss/AliyunOSSTestRule.java
##########
@@ -0,0 +1,84 @@
+/*
+ * 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.aliyun.oss;
+
+import com.aliyun.oss.OSS;
+import java.util.UUID;
+import org.junit.rules.TestRule;
+import org.junit.runner.Description;
+import org.junit.runners.model.Statement;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public interface AliyunOSSTestRule extends TestRule {
+  Logger LOG = LoggerFactory.getLogger(AliyunOSSTestRule.class);
+  UUID RANDOM_UUID = java.util.UUID.randomUUID();
+  /**

Review comment:
       I think it's worth to publish a PR to correct the option value in the [intellij-java-palantir-style.xml](https://github.com/apache/iceberg/blob/master/.baseline/idea/intellij-java-palantir-style.xml#L454).




-- 
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: issues-unsubscribe@iceberg.apache.org

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] openinx commented on a change in pull request #3067: Aliyun: Mock aliyun OSS(Object Storage Service)

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



##########
File path: build.gradle
##########
@@ -262,6 +262,36 @@ project(':iceberg-data') {
   }
 }
 
+project(':iceberg-aliyun') {
+  dependencies {
+    compile project(':iceberg-api')
+    compile project(':iceberg-common')
+
+    compileOnly 'com.aliyun.oss:aliyun-sdk-oss'
+    compileOnly 'javax.xml.bind:jaxb-api'
+    compileOnly 'javax.activation:activation'
+    compileOnly 'org.glassfish.jaxb:jaxb-runtime'

Review comment:
       Why do we need to introduce those three jars when adding a testing framework under the aliyun/src/test module?




-- 
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: issues-unsubscribe@iceberg.apache.org

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