You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@asterixdb.apache.org by mb...@apache.org on 2021/06/30 00:11:25 UTC

[asterixdb] 05/23: [NO ISSUE] S3 external datasets: add support to anonymous authentication

This is an automated email from the ASF dual-hosted git repository.

mblow pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/asterixdb.git

commit 0a44dbc96b4be8b160e281e5700a9fbf699cecfb
Author: Hussain Towaileb <Hu...@Couchbase.com>
AuthorDate: Mon Jun 7 04:46:17 2021 +0300

    [NO ISSUE] S3 external datasets: add support to anonymous authentication
    
    - user model changes: no
    - storage format changes: no
    - interface changes: no
    
    Details:
    - Added support to creating external datasets with no
      credentials (anonymous).
    
    Change-Id: I610f5b82d008a661946ae05bb4582d74b4e7161d
    Reviewed-on: https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/11804
    Integration-Tests: Jenkins <je...@fulliautomatix.ics.uci.edu>
    Tested-by: Jenkins <je...@fulliautomatix.ics.uci.edu>
    Reviewed-by: Hussain Towaileb <hu...@gmail.com>
    Reviewed-by: Murtadha Hubail <mh...@apache.org>
---
 .../aws/s3/anonymous_no_auth/test.000.ddl.sqlpp    | 36 ++++++++++++++++++++++
 .../aws/s3/anonymous_no_auth/test.001.ddl.sqlpp    | 36 ++++++++++++++++++++++
 .../aws/s3/anonymous_no_auth/test.002.ddl.sqlpp    | 35 +++++++++++++++++++++
 .../aws/s3/anonymous_no_auth/test.099.ddl.sqlpp    | 20 ++++++++++++
 .../runtimets/testsuite_external_dataset.xml       |  7 +++++
 .../asterix/common/exceptions/ErrorCode.java       |  1 +
 .../src/main/resources/asx_errormsg/en.properties  |  1 +
 .../asterix/external/util/ExternalDataUtils.java   | 36 +++++++++++++++++++---
 8 files changed, 168 insertions(+), 4 deletions(-)

diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/anonymous_no_auth/test.000.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/anonymous_no_auth/test.000.ddl.sqlpp
new file mode 100644
index 0000000..13cfe8a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/anonymous_no_auth/test.000.ddl.sqlpp
@@ -0,0 +1,36 @@
+/*
+ * 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.
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+drop type test if exists;
+create type test as open {
+};
+
+drop dataset test if exists;
+CREATE EXTERNAL DATASET test(test) USING S3 (
+("accessKeyId"="dummyAccessKey"),
+("region"="us-west-2"),
+("serviceEndpoint"="http://localhost:8001"),
+("container"="playground"),
+("definition"="json-data/reviews/single-line/json"),
+("format"="json")
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/anonymous_no_auth/test.001.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/anonymous_no_auth/test.001.ddl.sqlpp
new file mode 100644
index 0000000..b8d0945
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/anonymous_no_auth/test.001.ddl.sqlpp
@@ -0,0 +1,36 @@
+/*
+ * 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.
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+drop type test if exists;
+create type test as open {
+};
+
+drop dataset test if exists;
+CREATE EXTERNAL DATASET test(test) USING S3 (
+("secretAccessKey"="dummySecretKey"),
+("region"="us-west-2"),
+("serviceEndpoint"="http://localhost:8001"),
+("container"="playground"),
+("definition"="json-data/reviews/single-line/json"),
+("format"="json")
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/anonymous_no_auth/test.002.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/anonymous_no_auth/test.002.ddl.sqlpp
new file mode 100644
index 0000000..9eda057
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/anonymous_no_auth/test.002.ddl.sqlpp
@@ -0,0 +1,35 @@
+/*
+ * 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.
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+drop type test if exists;
+create type test as open {
+};
+
+drop dataset test if exists;
+CREATE EXTERNAL DATASET test(test) USING S3 (
+("region"="us-west-2"),
+("serviceEndpoint"="http://localhost:8001"),
+("container"="playground"),
+("definition"="json-data"),
+("format"="json")
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/anonymous_no_auth/test.099.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/anonymous_no_auth/test.099.ddl.sqlpp
new file mode 100644
index 0000000..548e632
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/anonymous_no_auth/test.099.ddl.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * 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.
+ */
+
+drop dataverse test if exists;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_external_dataset.xml b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_external_dataset.xml
index 096e511..43408c9 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_external_dataset.xml
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_external_dataset.xml
@@ -256,5 +256,12 @@
         <output-dir compare="Text">aws/s3/include-exclude/include-12</output-dir>
       </compilation-unit>
     </test-case>
+    <test-case FilePath="external-dataset">
+      <compilation-unit name="aws/s3/anonymous_no_auth">
+        <output-dir compare="Text">anonymous_no_auth</output-dir>
+        <expected-error>ASX3119: Parameter 'secretAccessKey' is required if 'accessKeyId' is provided</expected-error>
+        <expected-error>ASX3119: Parameter 'accessKeyId' is required if 'secretAccessKey' is provided</expected-error>
+      </compilation-unit>
+    </test-case>
   </test-group>
 </test-suite>
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ErrorCode.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ErrorCode.java
index a29ed3b..192113e 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ErrorCode.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ErrorCode.java
@@ -313,6 +313,7 @@ public enum ErrorCode implements IError {
     INPUT_DECODE_FAILURE(3116),
     FAILED_TO_PARSE_MALFORMED_LOG_RECORD(3117),
     ACTIVE_ENTITY_NOT_RUNNING(3118),
+    REQUIRED_PARAM_IF_PARAM_IS_PRESENT(3119),
 
     // Lifecycle management errors
     DUPLICATE_PARTITION_ID(4000),
diff --git a/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties b/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties
index 5012c7a..e3acf9c 100644
--- a/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties
+++ b/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties
@@ -316,6 +316,7 @@
 3116 = Failed to decode input
 3117 = Failed to parse record, malformed log record
 3118 = Active Entity %1$s is not running (it is %2$s)
+3119 = Parameter '%1$s' is required if '%2$s' is provided
 
 # Lifecycle management errors
 4000 = Partition id %1$s for node %2$s already in use by node %3$s
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/ExternalDataUtils.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/ExternalDataUtils.java
index bc9a038..363ec74 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/ExternalDataUtils.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/ExternalDataUtils.java
@@ -18,6 +18,9 @@
  */
 package org.apache.asterix.external.util;
 
+import static org.apache.asterix.common.exceptions.ErrorCode.REQUIRED_PARAM_IF_PARAM_IS_PRESENT;
+import static org.apache.asterix.external.util.ExternalDataConstants.AwsS3.ACCESS_KEY_ID_FIELD_NAME;
+import static org.apache.asterix.external.util.ExternalDataConstants.AwsS3.SECRET_ACCESS_KEY_FIELD_NAME;
 import static org.apache.asterix.external.util.ExternalDataConstants.KEY_DELIMITER;
 import static org.apache.asterix.external.util.ExternalDataConstants.KEY_ESCAPE;
 import static org.apache.asterix.external.util.ExternalDataConstants.KEY_EXTERNAL_SCAN_BUFFER_SIZE;
@@ -63,7 +66,9 @@ import org.apache.hyracks.dataflow.common.data.parsers.IntegerParserFactory;
 import org.apache.hyracks.dataflow.common.data.parsers.LongParserFactory;
 import org.apache.hyracks.dataflow.common.data.parsers.UTF8StringParserFactory;
 
+import software.amazon.awssdk.auth.credentials.AnonymousCredentialsProvider;
 import software.amazon.awssdk.auth.credentials.AwsBasicCredentials;
+import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider;
 import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider;
 import software.amazon.awssdk.core.exception.SdkException;
 import software.amazon.awssdk.regions.Region;
@@ -623,16 +628,25 @@ public class ExternalDataUtils {
          */
         public static S3Client buildAwsS3Client(Map<String, String> configuration) throws CompilationException {
             // TODO(Hussain): Need to ensure that all required parameters are present in a previous step
-            String accessKeyId = configuration.get(ExternalDataConstants.AwsS3.ACCESS_KEY_ID_FIELD_NAME);
-            String secretAccessKey = configuration.get(ExternalDataConstants.AwsS3.SECRET_ACCESS_KEY_FIELD_NAME);
+            String accessKeyId = configuration.get(ACCESS_KEY_ID_FIELD_NAME);
+            String secretAccessKey = configuration.get(SECRET_ACCESS_KEY_FIELD_NAME);
             String regionId = configuration.get(ExternalDataConstants.AwsS3.REGION_FIELD_NAME);
             String serviceEndpoint = configuration.get(ExternalDataConstants.AwsS3.SERVICE_END_POINT_FIELD_NAME);
 
             S3ClientBuilder builder = S3Client.builder();
 
             // Credentials
-            AwsBasicCredentials credentials = AwsBasicCredentials.create(accessKeyId, secretAccessKey);
-            builder.credentialsProvider(StaticCredentialsProvider.create(credentials));
+            AwsCredentialsProvider credentialsProvider;
+
+            // No auth required
+            if (accessKeyId == null) {
+                credentialsProvider = AnonymousCredentialsProvider.create();
+            } else {
+                credentialsProvider =
+                        StaticCredentialsProvider.create(AwsBasicCredentials.create(accessKeyId, secretAccessKey));
+            }
+
+            builder.credentialsProvider(credentialsProvider);
             builder.region(Region.of(regionId));
 
             // Validate the service endpoint if present
@@ -667,6 +681,20 @@ public class ExternalDataUtils {
                 throw new CompilationException(ErrorCode.PARAMETERS_REQUIRED, srcLoc, ExternalDataConstants.KEY_FORMAT);
             }
 
+            // Both parameters should be passed, or neither should be passed (for anonymous/no auth)
+            String accessKeyId = configuration.get(ACCESS_KEY_ID_FIELD_NAME);
+            String secretAccessKey = configuration.get(SECRET_ACCESS_KEY_FIELD_NAME);
+            if (accessKeyId == null || secretAccessKey == null) {
+                // If one is passed, the other is required
+                if (accessKeyId != null) {
+                    throw new CompilationException(REQUIRED_PARAM_IF_PARAM_IS_PRESENT, SECRET_ACCESS_KEY_FIELD_NAME,
+                            ACCESS_KEY_ID_FIELD_NAME);
+                } else if (secretAccessKey != null) {
+                    throw new CompilationException(REQUIRED_PARAM_IF_PARAM_IS_PRESENT, ACCESS_KEY_ID_FIELD_NAME,
+                            SECRET_ACCESS_KEY_FIELD_NAME);
+                }
+            }
+
             validateIncludeExclude(configuration);
 
             // Check if the bucket is present