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 2020/07/19 22:10:43 UTC

[asterixdb] branch master updated (866d4c8 -> d4ca925)

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

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


    from 866d4c8  [NO ISSUE][COMP] Refactor drop dataverse and drop function
     new 26e411e  [ASTERIXDB-2754][COMP] LoadRecordFieldsRule must clone expression
     new 9ac7816  [NO ISSUE][RT] Support External Datasets Rebalance
     new 8a64bf8  [ASTERIXDB-2757] Properly handle include/exclude pattern translation
     new b44f5c3  [NO ISSUE] Redact sensitive data from statement logging
     new ad3ee2e  [ASTERIXDB-2758][TEST] Test for ensuring translation from wildcard pattern to REGEX is done properly
     new c476a98  [NO ISSUE][STO] Use Index Resolved Path
     new d4ca925  Merge branch 'gerrit/mad-hatter'

The 7 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 .../optimizer/rules/LoadRecordFieldsRule.java      |   2 +-
 .../api/http/server/QueryServiceServlet.java       |   7 +-
 .../message/ExecuteStatementRequestMessage.java    |   2 +-
 .../asterix/hyracks/bootstrap/CCApplication.java   |   3 +
 .../asterix/hyracks/bootstrap/NCApplication.java   |   3 +
 .../org/apache/asterix/utils/RebalanceUtil.java    |   8 +-
 .../org/apache/asterix/utils/RedactionUtil.java    |  41 +++----
 .../external_dataset/aws/WildCardToRegexTest.java  |  97 +++++++++++++++++
 .../optimizerts/queries/common-expr-01.sqlpp       |  65 +++++++++++
 .../optimizerts/results/common-expr-01.plan        |  68 ++++++++++++
 .../include-exclude/include-11/test.000.ddl.sqlpp  |  35 +++---
 .../test.001.query.sqlpp                           |   0
 .../include-11}/test.099.ddl.sqlpp                 |   0
 .../include-exclude/include-12/test.000.ddl.sqlpp  |  37 ++++---
 .../test.001.query.sqlpp                           |   0
 .../include-12}/test.099.ddl.sqlpp                 |   0
 .../include-10}/result.001.adm                     |   0
 .../{include-9 => include-11}/result.001.adm       |   0
 .../include-12}/result.001.adm                     |   0
 .../runtimets/testsuite_external_dataset.xml       |  11 +-
 .../record/reader/aws/AwsS3InputStreamFactory.java |   4 +-
 .../asterix/external/util/ExternalDataUtils.java   | 119 ++++++++++++---------
 .../evaluators/functions/StringEvaluatorUtils.java |   4 +-
 .../storage/am/common/build/IndexBuilder.java      |  27 ++---
 .../java/org/apache/hyracks/util/ILogRedactor.java |   8 ++
 .../org/apache/hyracks/util/LogRedactionUtil.java  |   9 ++
 26 files changed, 422 insertions(+), 128 deletions(-)
 copy hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/LogRedactionUtil.java => asterixdb/asterix-app/src/main/java/org/apache/asterix/utils/RedactionUtil.java (53%)
 create mode 100644 asterixdb/asterix-app/src/test/java/org/apache/asterix/test/external_dataset/aws/WildCardToRegexTest.java
 create mode 100644 asterixdb/asterix-app/src/test/resources/optimizerts/queries/common-expr-01.sqlpp
 create mode 100644 asterixdb/asterix-app/src/test/resources/optimizerts/results/common-expr-01.plan
 copy hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/ILogRedactor.java => asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/include-11/test.000.ddl.sqlpp (62%)
 copy asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/{include-all => include-11}/test.001.query.sqlpp (100%)
 copy asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/{no-files-returned/include-no-files => include-exclude/include-11}/test.099.ddl.sqlpp (100%)
 copy hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/ILogRedactor.java => asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/include-12/test.000.ddl.sqlpp (56%)
 copy asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/{include-all => include-12}/test.001.query.sqlpp (100%)
 copy asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/{no-files-returned/include-no-files => include-exclude/include-12}/test.099.ddl.sqlpp (100%)
 copy asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/{no-files-returned/include-no-files => include-exclude/include-10}/result.001.adm (100%)
 copy asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/include-exclude/{include-9 => include-11}/result.001.adm (100%)
 copy asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/{no-files-returned/include-no-files => include-exclude/include-12}/result.001.adm (100%)


[asterixdb] 06/07: [NO ISSUE][STO] Use Index Resolved Path

Posted by mb...@apache.org.
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 c476a98360abcde9cfaca5dc1a84658f3482c886
Author: Murtadha Hubail <mh...@apache.org>
AuthorDate: Tue Jul 14 13:38:25 2020 +0300

    [NO ISSUE][STO] Use Index Resolved Path
    
    - user model changes: no
    - storage format changes: no
    - interface changes: no
    
    Details:
    - When attempting to drop an existing index before recreating
      it, use the index resolved path based on the NC io devices
      resolver.
    
    Change-Id: I201ff17dc6deb02032d5e8603b28b7711ed0efab
    Reviewed-on: https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/7183
    Integration-Tests: Jenkins <je...@fulliautomatix.ics.uci.edu>
    Tested-by: Jenkins <je...@fulliautomatix.ics.uci.edu>
    Reviewed-by: Murtadha Hubail <mh...@apache.org>
    Reviewed-by: Till Westmann <ti...@apache.org>
---
 .../storage/am/common/build/IndexBuilder.java      | 27 +++++++++++-----------
 1 file changed, 14 insertions(+), 13 deletions(-)

diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/build/IndexBuilder.java b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/build/IndexBuilder.java
index f62860a..45bfed1 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/build/IndexBuilder.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/build/IndexBuilder.java
@@ -47,6 +47,7 @@ public class IndexBuilder implements IIndexBuilder {
     protected final IResourceFactory localResourceFactory;
     protected final boolean durable;
     private final IResourceIdFactory resourceIdFactory;
+    private final String resourceRelPath;
 
     /*
      * Ideally, we should not pass resource id factory to the constructor since we can obtain it through
@@ -62,6 +63,7 @@ public class IndexBuilder implements IIndexBuilder {
         this.localResourceFactory = localResourceFactory;
         this.durable = durable;
         this.resourceRef = resourceRef;
+        resourceRelPath = resourceRef.getRelativePath();
     }
 
     @Override
@@ -72,35 +74,34 @@ public class IndexBuilder implements IIndexBuilder {
             // physical artifact that the LocalResourceRepository is managing (e.g. a file containing the resource Id).
             // Once the index has been created, a new resource Id can be generated.
             ILocalResourceRepository localResourceRepository = storageManager.getLocalResourceRepository(ctx);
-            LocalResource lr = localResourceRepository.get(resourceRef.getRelativePath());
+            LocalResource lr = localResourceRepository.get(resourceRelPath);
             long resourceId = lr == null ? -1 : lr.getId();
             if (resourceId != -1) {
-                localResourceRepository.delete(resourceRef.getRelativePath());
+                localResourceRepository.delete(resourceRelPath);
             }
             resourceId = resourceIdFactory.createId();
             IResource resource = localResourceFactory.createResource(resourceRef);
             lr = new LocalResource(resourceId, ITreeIndexFrame.Constants.VERSION, durable, resource);
-            IIndex index = lcManager.get(resourceRef.getRelativePath());
+            IIndex index = lcManager.get(resourceRelPath);
             if (index != null) {
                 //how is this right?????????? <needs to be fixed>
                 //The reason for this is to handle many cases such as:
                 //1. Crash while delete index is running (we don't do global cleanup on restart)
                 //2. Node leaves and then join with old data
-                LOGGER.log(Level.WARN,
-                        "Removing existing index on index create for the index: " + resourceRef.getRelativePath());
-                lcManager.unregister(resourceRef.getRelativePath());
+                LOGGER.log(Level.WARN, "Removing existing index on index create for the index: " + resourceRelPath);
+                lcManager.unregister(resourceRelPath);
                 index.destroy();
             } else {
-                if (resourceRef.getFile().exists()) {
+                final FileReference resolvedResourceRef = ctx.getIoManager().resolve(resourceRelPath);
+                if (resolvedResourceRef.getFile().exists()) {
                     // Index is not registered but the index file exists
                     // This is another big problem that we need to disallow soon
                     // We can only disallow this if we have a global cleanup after crash
                     // on reboot
-                    LOGGER.log(Level.WARN,
-                            "Deleting " + resourceRef.getRelativePath()
-                                    + " on index create. The index is not registered"
-                                    + " but the file exists in the filesystem");
-                    IoUtil.delete(resourceRef);
+                    LOGGER.warn(
+                            "Deleting {} on index create. The index is not registered but the file exists in the filesystem",
+                            resolvedResourceRef);
+                    IoUtil.delete(resolvedResourceRef);
                 }
                 index = resource.createInstance(ctx);
             }
@@ -110,7 +111,7 @@ public class IndexBuilder implements IIndexBuilder {
             } catch (IOException e) {
                 throw HyracksDataException.create(e);
             }
-            lcManager.register(resourceRef.getRelativePath(), index);
+            lcManager.register(resourceRelPath, index);
         }
     }
 }


[asterixdb] 07/07: Merge branch 'gerrit/mad-hatter'

Posted by mb...@apache.org.
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 d4ca9254a95dd18c34cad491186d9961a372f0af
Merge: 866d4c8 c476a98
Author: Michael Blow <mb...@apache.org>
AuthorDate: Sun Jul 19 12:41:41 2020 -0400

    Merge branch 'gerrit/mad-hatter'
    
    Change-Id: Iba32a467109f15093d9e0347b819c8e46066cd5d

 .../optimizer/rules/LoadRecordFieldsRule.java      |   2 +-
 .../api/http/server/QueryServiceServlet.java       |   7 +-
 .../message/ExecuteStatementRequestMessage.java    |   2 +-
 .../asterix/hyracks/bootstrap/CCApplication.java   |   3 +
 .../asterix/hyracks/bootstrap/NCApplication.java   |   3 +
 .../org/apache/asterix/utils/RebalanceUtil.java    |   8 +-
 .../org/apache/asterix/utils/RedactionUtil.java    |  41 +++----
 .../external_dataset/aws/WildCardToRegexTest.java  |  97 +++++++++++++++++
 .../optimizerts/queries/common-expr-01.sqlpp       |  65 +++++++++++
 .../optimizerts/results/common-expr-01.plan        |  68 ++++++++++++
 .../include-exclude/include-11/test.000.ddl.sqlpp  |  35 +++---
 .../include-11/test.001.query.sqlpp                |  21 +---
 .../include-exclude/include-11/test.099.ddl.sqlpp  |  20 +---
 .../include-exclude/include-12/test.000.ddl.sqlpp  |  37 ++++---
 .../include-12/test.001.query.sqlpp                |  21 +---
 .../include-exclude/include-12/test.099.ddl.sqlpp  |  20 +---
 .../s3/include-exclude/include-10/result.001.adm   |   1 +
 .../s3/include-exclude/include-11/result.001.adm   |   1 +
 .../s3/include-exclude/include-12/result.001.adm   |   1 +
 .../runtimets/testsuite_external_dataset.xml       |  11 +-
 .../record/reader/aws/AwsS3InputStreamFactory.java |   4 +-
 .../asterix/external/util/ExternalDataUtils.java   | 119 ++++++++++++---------
 .../evaluators/functions/StringEvaluatorUtils.java |   4 +-
 .../storage/am/common/build/IndexBuilder.java      |  27 ++---
 .../java/org/apache/hyracks/util/ILogRedactor.java |   8 ++
 .../org/apache/hyracks/util/LogRedactionUtil.java  |   9 ++
 26 files changed, 431 insertions(+), 204 deletions(-)

diff --cc asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/CCApplication.java
index 3eeed59,0f0620c..e80995c
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/CCApplication.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/CCApplication.java
@@@ -91,9 -86,7 +91,10 @@@ import org.apache.asterix.runtime.utils
  import org.apache.asterix.translator.IStatementExecutorFactory;
  import org.apache.asterix.translator.Receptionist;
  import org.apache.asterix.util.MetadataBuiltinFunctions;
+ import org.apache.asterix.utils.RedactionUtil;
 +import org.apache.commons.csv.CSVFormat;
 +import org.apache.commons.csv.CSVParser;
 +import org.apache.commons.csv.CSVRecord;
  import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
  import org.apache.hyracks.api.application.IServiceContext;
  import org.apache.hyracks.api.client.IHyracksClientConnection;


[asterixdb] 05/07: [ASTERIXDB-2758][TEST] Test for ensuring translation from wildcard pattern to REGEX is done properly

Posted by mb...@apache.org.
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 ad3ee2eb480306d859add1f20a0adb5904c589a3
Author: Hussain Towaileb <Hu...@Gmail.com>
AuthorDate: Thu Jul 9 22:02:06 2020 +0300

    [ASTERIXDB-2758][TEST] Test for ensuring translation from wildcard pattern to REGEX is done properly
    
    - user model changes: no
    - storage format changes: no
    - interface changes: no
    
    Details:
    - Added a unit test to ensure the translation from wildcard
      patterns to regex pattern is done properly.
    
    Change-Id: Ieb7fb8d86e1b33ac351c3a6f9f14c3b77eac2988
    Reviewed-on: https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/7164
    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: Ali Alsuliman <al...@gmail.com>
---
 .../external_dataset/aws/WildCardToRegexTest.java  | 97 ++++++++++++++++++++++
 1 file changed, 97 insertions(+)

diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/external_dataset/aws/WildCardToRegexTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/external_dataset/aws/WildCardToRegexTest.java
new file mode 100644
index 0000000..f15b657
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/external_dataset/aws/WildCardToRegexTest.java
@@ -0,0 +1,97 @@
+/*
+ * 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.asterix.test.external_dataset.aws;
+
+import static org.apache.asterix.external.util.ExternalDataUtils.patternToRegex;
+
+import java.util.regex.Pattern;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.junit.Test;
+
+import junit.framework.TestCase;
+
+public class WildCardToRegexTest extends TestCase {
+
+    @Test
+    public void test() throws HyracksDataException {
+        String result = patternToRegex("*?[abc]");
+        assertEquals(".*.[abc]", result);
+        Pattern.compile(result);
+
+        result = patternToRegex("*?[!@#$%^&*()+<>|=!{}.]");
+        assertEquals(".*.[^@#\\$%\\^&\\*\\(\\)\\+\\<\\>\\|\\=\\!\\{\\}\\.]", result);
+        Pattern.compile(result);
+
+        result = patternToRegex("**??[[a-z*0-9]]");
+        assertEquals(".*.*..[\\[a-z\\*0-9]\\]", result);
+        Pattern.compile(result);
+
+        result = patternToRegex("**??[[a-z*0-9]]");
+        assertEquals(".*.*..[\\[a-z\\*0-9]\\]", result);
+        Pattern.compile(result);
+
+        result = patternToRegex("*?[!abc]");
+        assertEquals(".*.[^abc]", result);
+        Pattern.compile(result);
+
+        result = patternToRegex("[!]abc");
+        assertEquals("\\[\\!\\]abc", result);
+        Pattern.compile(result);
+
+        result = patternToRegex("[!]abc]");
+        assertEquals("[^\\]abc]", result);
+        Pattern.compile(result);
+
+        result = patternToRegex("[]]");
+        assertEquals("[\\]]", result);
+        Pattern.compile(result);
+
+        result = patternToRegex("[]abcd");
+        assertEquals("\\[\\]abcd", result);
+        Pattern.compile(result);
+
+        result = patternToRegex("[]abcd]");
+        assertEquals("[\\]abcd]", result);
+        Pattern.compile(result);
+
+        result = patternToRegex("[^]");
+        assertEquals("[\\^]", result);
+        Pattern.compile(result);
+
+        result = patternToRegex("[^]]");
+        assertEquals("[\\^]\\]", result);
+        Pattern.compile(result);
+
+        result = patternToRegex("[!]");
+        assertEquals("\\[\\!\\]", result);
+        Pattern.compile(result);
+
+        result = patternToRegex("[!]]");
+        assertEquals("[^\\]]", result);
+        Pattern.compile(result);
+
+        result = patternToRegex("[][!][^]]]]*[![*a-zA--&&^$||0-9B$\\\\*&&]*&&[^a-b||0--9][[[");
+        assertEquals(
+                "[\\]\\[\\!][\\^]\\]\\]\\].*[^\\[\\*a-zA\\-\\-\\&\\&\\^\\$\\|\\|0-9B\\$\\\\\\\\\\*\\&\\&].*&&[\\^a-b\\|\\|0\\-\\-9]\\[\\[\\[",
+                result);
+        Pattern.compile(result);
+    }
+}


[asterixdb] 03/07: [ASTERIXDB-2757] Properly handle include/exclude pattern translation

Posted by mb...@apache.org.
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 8a64bf87cea9113a6b5ed0715c5e8d6ad8ff56c4
Author: Hussain Towaileb <Hu...@Gmail.com>
AuthorDate: Wed Jul 8 23:04:52 2020 +0300

    [ASTERIXDB-2757] Properly handle include/exclude pattern translation
    
    - user model changes: no
    - storage format changes: no
    - interface changes: no
    
    Details:
    - Handles properly handling the translation of a wildcard
      pattern to a regex to avoid potential failure in the query.
    - Added and updated some test cases.
    
    Change-Id: I661c1fad9b2c2692fa231e48f00eb5cf9d79ad5e
    Reviewed-on: https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/7143
    Tested-by: Jenkins <je...@fulliautomatix.ics.uci.edu>
    Integration-Tests: Jenkins <je...@fulliautomatix.ics.uci.edu>
    Reviewed-by: Ali Alsuliman <al...@gmail.com>
---
 .../include-exclude/include-11/test.000.ddl.sqlpp  |  39 +++++++
 .../include-11/test.001.query.sqlpp                |  21 ++++
 .../include-exclude/include-11/test.099.ddl.sqlpp  |  20 ++++
 .../include-exclude/include-12/test.000.ddl.sqlpp  |  41 +++++++
 .../include-12/test.001.query.sqlpp                |  21 ++++
 .../include-exclude/include-12/test.099.ddl.sqlpp  |  20 ++++
 .../s3/include-exclude/include-10/result.001.adm   |   1 +
 .../s3/include-exclude/include-11/result.001.adm   |   1 +
 .../s3/include-exclude/include-12/result.001.adm   |   1 +
 .../runtimets/testsuite_external_dataset.xml       |  11 +-
 .../record/reader/aws/AwsS3InputStreamFactory.java |   4 +-
 .../asterix/external/util/ExternalDataUtils.java   | 119 ++++++++++++---------
 .../evaluators/functions/StringEvaluatorUtils.java |   4 +-
 13 files changed, 247 insertions(+), 56 deletions(-)

diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/include-11/test.000.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/include-11/test.000.ddl.sqlpp
new file mode 100644
index 0000000..e4cf69b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/include-11/test.000.ddl.sqlpp
@@ -0,0 +1,39 @@
+/*
+ * 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"),
+("secretAccessKey"="dummySecretKey"),
+("region"="us-west-2"),
+("serviceEndpoint"="http://localhost:8001"),
+("container"="include-exclude"),
+("definition"="data/mixed/"),
+("format"="csv"),
+("header"=false),
+("include"="*.[a-c][a-z][a-z**||\\\\&&--~~]")
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/include-11/test.001.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/include-11/test.001.query.sqlpp
new file mode 100644
index 0000000..3306d5c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/include-11/test.001.query.sqlpp
@@ -0,0 +1,21 @@
+/*
+ * 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.
+ */
+
+use test;
+select count(*) as `count` from test;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/include-11/test.099.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/include-11/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/include-exclude/include-11/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/queries_sqlpp/external-dataset/aws/s3/include-exclude/include-12/test.000.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/include-12/test.000.ddl.sqlpp
new file mode 100644
index 0000000..47fbaef
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/include-12/test.000.ddl.sqlpp
@@ -0,0 +1,41 @@
+/*
+ * 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.
+ */
+
+ // This test case matches nothing for "include", but has extreme cases and complicated pattern, expected is to not fail
+
+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"),
+("secretAccessKey"="dummySecretKey"),
+("region"="us-west-2"),
+("serviceEndpoint"="http://localhost:8001"),
+("container"="include-exclude"),
+("definition"="data/mixed/"),
+("format"="csv"),
+("header"=false),
+("include"="[][!][^]]]]*[![*a-zA--&&^$||0-9B$\\*&&]*&&[^a-b||0--9][[[")
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/include-12/test.001.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/include-12/test.001.query.sqlpp
new file mode 100644
index 0000000..3306d5c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/include-12/test.001.query.sqlpp
@@ -0,0 +1,21 @@
+/*
+ * 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.
+ */
+
+use test;
+select count(*) as `count` from test;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/include-12/test.099.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/include-12/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/include-exclude/include-12/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/results/external-dataset/aws/s3/include-exclude/include-10/result.001.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/include-exclude/include-10/result.001.adm
new file mode 100644
index 0000000..c1a0ea2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/include-exclude/include-10/result.001.adm
@@ -0,0 +1 @@
+{ "count": 0 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/include-exclude/include-11/result.001.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/include-exclude/include-11/result.001.adm
new file mode 100644
index 0000000..95204aa
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/include-exclude/include-11/result.001.adm
@@ -0,0 +1 @@
+{ "count": 6 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/include-exclude/include-12/result.001.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/include-exclude/include-12/result.001.adm
new file mode 100644
index 0000000..c1a0ea2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/include-exclude/include-12/result.001.adm
@@ -0,0 +1 @@
+{ "count": 0 }
\ 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 ad17afe..3a4f03e 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
@@ -232,7 +232,16 @@
     <test-case FilePath="external-dataset">
       <compilation-unit name="aws/s3/include-exclude/include-10">
         <output-dir compare="Text">aws/s3/include-exclude/include-10</output-dir>
-        <expected-error>Invalid pattern *[abc][.*</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="external-dataset">
+      <compilation-unit name="aws/s3/include-exclude/include-11">
+        <output-dir compare="Text">aws/s3/include-exclude/include-11</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="external-dataset">
+      <compilation-unit name="aws/s3/include-exclude/include-12">
+        <output-dir compare="Text">aws/s3/include-exclude/include-12</output-dir>
       </compilation-unit>
     </test-case>
   </test-group>
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/aws/AwsS3InputStreamFactory.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/aws/AwsS3InputStreamFactory.java
index 67a7c93..f3a36ff 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/aws/AwsS3InputStreamFactory.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/aws/AwsS3InputStreamFactory.java
@@ -103,10 +103,10 @@ public class AwsS3InputStreamFactory implements IInputStreamFactory {
             for (Map.Entry<String, String> entry : configuration.entrySet()) {
                 if (entry.getKey().startsWith(KEY_INCLUDE)) {
                     pattern = entry.getValue();
-                    includeMatchers.add(Pattern.compile(ExternalDataUtils.wildcardToRegex(pattern)).matcher(""));
+                    includeMatchers.add(Pattern.compile(ExternalDataUtils.patternToRegex(pattern)).matcher(""));
                 } else if (entry.getKey().startsWith(KEY_EXCLUDE)) {
                     pattern = entry.getValue();
-                    excludeMatchers.add(Pattern.compile(ExternalDataUtils.wildcardToRegex(pattern)).matcher(""));
+                    excludeMatchers.add(Pattern.compile(ExternalDataUtils.patternToRegex(pattern)).matcher(""));
                 }
             }
         } catch (PatternSyntaxException ex) {
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 1300ac3..fc31286 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
@@ -23,10 +23,12 @@ import static org.apache.asterix.external.util.ExternalDataConstants.KEY_ESCAPE;
 import static org.apache.asterix.external.util.ExternalDataConstants.KEY_QUOTE;
 import static org.apache.asterix.external.util.ExternalDataConstants.KEY_RECORD_END;
 import static org.apache.asterix.external.util.ExternalDataConstants.KEY_RECORD_START;
+import static org.apache.asterix.runtime.evaluators.functions.StringEvaluatorUtils.RESERVED_REGEX_CHARS;
 
 import java.net.URI;
 import java.net.URISyntaxException;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.EnumMap;
 import java.util.List;
 import java.util.Map;
@@ -498,76 +500,91 @@ public class ExternalDataUtils {
     /**
      * Converts the wildcard to proper regex
      *
-     * @param wildcard wildcard pattern to convert
+     * @param pattern wildcard pattern to convert
      *
      * @return regex expression
      */
-    public static String wildcardToRegex(String wildcard) {
-        StringBuilder builder = new StringBuilder(wildcard.length());
-        builder.append('^');
+    public static String patternToRegex(String pattern) {
+        int charPosition = 0;
+        int patternLength = pattern.length();
+        StringBuilder stuffBuilder = new StringBuilder();
+        StringBuilder result = new StringBuilder();
 
-        // This keeps an eye on the presence inside or outside a sequence, everything inside a sequence is a literal
-        // e.g ("*" ===> ".*" while "[*]" ===> "[\*]"
-        boolean outsideBracketSequence = true;
+        while (charPosition < patternLength) {
+            char c = pattern.charAt(charPosition);
+            charPosition++;
 
-        for (int i = 0; i < wildcard.length(); i++) {
-            char c = wildcard.charAt(i);
             switch (c) {
                 case '*':
-                    builder.append(outsideBracketSequence ? "." : "\\").append(c);
+                    result.append(".*");
                     break;
                 case '?':
-                    builder.append(outsideBracketSequence ? "." : "\\?");
+                    result.append(".");
                     break;
                 case '[':
-                    if (outsideBracketSequence) {
-                        outsideBracketSequence = false;
-                        builder.append(c);
-                        if (i + 1 < wildcard.length()) {
-                            if (wildcard.charAt(i + 1) == '!') {
-                                i++;
-                                builder.append('^');
-                            }
-                        }
-                    } else {
-                        // escape the open bracket "[" if we are already inside a bracket sequence
-                        builder.append("\\").append(c);
+                    int closingBracketPosition = charPosition;
+                    if (closingBracketPosition < patternLength && pattern.charAt(closingBracketPosition) == '!') {
+                        closingBracketPosition++;
                     }
-                    break;
-                case ']':
-                    if (outsideBracketSequence) {
-                        // escape if we are outside bracket sequence
-                        builder.append("\\").append(c);
+
+                    // 2 cases can happen here:
+                    // 1- Empty character class [] which is invalid for java, so treat ] as literal and find another
+                    // closing bracket, if no closing bracket is found, the whole thing is a literal
+                    // 2- Negated empty class [!] converted to [^] which is invalid for java, so treat ] as literal and
+                    // find another closing bracket, if no closing bracket is found, the whole thing is a literal
+                    if (closingBracketPosition < patternLength && pattern.charAt(closingBracketPosition) == ']') {
+                        closingBracketPosition++;
+                    }
+
+                    // No [] and [!] cases, search for the closing bracket
+                    while (closingBracketPosition < patternLength && pattern.charAt(closingBracketPosition) != ']') {
+                        closingBracketPosition++;
+                    }
+
+                    // No closing bracket found (or [] or [!]), escape the opening bracket, treat it as literals
+                    if (closingBracketPosition >= patternLength) {
+                        result.append("\\[");
                     } else {
-                        // Inside bracket, close it and mark as outside bracket
-                        outsideBracketSequence = true;
-                        builder.append(c);
+                        // Found closing bracket, get the stuff in between the found the character class ("[" and "]")
+                        String stuff = pattern.substring(charPosition, closingBracketPosition);
+
+                        stuffBuilder.setLength(0);
+                        int stuffCharPos = 0;
+
+                        // If first character in the character class is "!" then convert it to "^"
+                        if (stuff.charAt(0) == '!') {
+                            stuffBuilder.append('^');
+                            stuffCharPos++; // ignore first character when escaping metacharacters next step
+                        }
+
+                        for (; stuffCharPos < stuff.length(); stuffCharPos++) {
+                            char stuffChar = stuff.charAt(stuffCharPos);
+                            if (stuffChar != '-' && Arrays.binarySearch(RESERVED_REGEX_CHARS, stuffChar) >= 0) {
+                                stuffBuilder.append("\\");
+                            }
+                            stuffBuilder.append(stuffChar);
+                        }
+
+                        String stuffEscaped = stuffBuilder.toString();
+
+                        // Escape the set operations
+                        stuffEscaped = stuffEscaped.replace("&&", "\\&\\&").replace("~~", "\\~\\~")
+                                .replace("||", "\\|\\|").replace("--", "\\-\\-");
+
+                        result.append("[").append(stuffEscaped).append("]");
+                        charPosition = closingBracketPosition + 1;
                     }
                     break;
-                // escape special regexp-characters
-                case '(':
-                case ')':
-                case '$':
-                case '^':
-                case '.':
-                case '{':
-                case '}':
-                case '|':
-                case '+':
-                case '=':
-                case '<':
-                case '>':
-                case '!':
-                case '\\':
-                    builder.append("\\").append(c);
-                    break;
                 default:
-                    builder.append(c);
+                    if (Arrays.binarySearch(RESERVED_REGEX_CHARS, c) >= 0) {
+                        result.append("\\");
+                    }
+                    result.append(c);
                     break;
             }
         }
-        builder.append('$');
-        return builder.toString();
+
+        return result.toString();
     }
 
     public static class AwsS3 {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringEvaluatorUtils.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringEvaluatorUtils.java
index 81ce832..492d64c 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringEvaluatorUtils.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringEvaluatorUtils.java
@@ -60,8 +60,8 @@ public final class StringEvaluatorUtils {
         return destString;
     }
 
-    static final char[] RESERVED_REGEX_CHARS = new char[] { '\\', '(', ')', '[', ']', '{', '}', '.', '^', '$', '*', '|',
-            '+', '?', '<', '>', '-', '=', '!' };
+    public static final char[] RESERVED_REGEX_CHARS = new char[] { '\\', '(', ')', '[', ']', '{', '}', '.', '^', '$',
+            '*', '|', '+', '?', '<', '>', '-', '=', '!' };
 
     static {
         Arrays.sort(RESERVED_REGEX_CHARS);


[asterixdb] 04/07: [NO ISSUE] Redact sensitive data from statement logging

Posted by mb...@apache.org.
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 b44f5c3cd4802cd215f51dcd7698370a8e45ea78
Author: Michael Blow <mi...@couchbase.com>
AuthorDate: Thu Jul 9 17:01:03 2020 -0400

    [NO ISSUE] Redact sensitive data from statement logging
    
    Change-Id: Ibd63ca9167c769eea4d03982dbf7fa543913dc67
    Reviewed-on: https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/7165
    Integration-Tests: Jenkins <je...@fulliautomatix.ics.uci.edu>
    Tested-by: Jenkins <je...@fulliautomatix.ics.uci.edu>
    Reviewed-by: Michael Blow <mb...@apache.org>
    Reviewed-by: Murtadha Hubail <mh...@apache.org>
---
 .../api/http/server/QueryServiceServlet.java       |  7 ++--
 .../message/ExecuteStatementRequestMessage.java    |  2 +-
 .../asterix/hyracks/bootstrap/CCApplication.java   |  3 ++
 .../asterix/hyracks/bootstrap/NCApplication.java   |  3 ++
 .../org/apache/asterix/utils/RedactionUtil.java    | 41 ++++++++++++----------
 .../java/org/apache/hyracks/util/ILogRedactor.java |  8 +++++
 .../org/apache/hyracks/util/LogRedactionUtil.java  |  9 +++++
 7 files changed, 50 insertions(+), 23 deletions(-)

diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryServiceServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryServiceServlet.java
index cb1d6cf..440b351 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryServiceServlet.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryServiceServlet.java
@@ -279,7 +279,7 @@ public class QueryServiceServlet extends AbstractQueryApiServlet {
             if (forceReadOnly) {
                 param.setReadOnly(true);
             }
-            LOGGER.info(() -> "handleRequest: " + LogRedactionUtil.userData(param.toString()));
+            LOGGER.info(() -> "handleRequest: " + LogRedactionUtil.statement(param.toString()));
             delivery = param.getMode();
             setSessionConfig(sessionOutput, param, delivery);
             final ResultProperties resultProperties = new ResultProperties(delivery, param.getMaxResultReads());
@@ -427,10 +427,11 @@ public class QueryServiceServlet extends AbstractQueryApiServlet {
         if (t instanceof org.apache.asterix.aqlplus.parser.TokenMgrError || t instanceof TokenMgrError
                 || t instanceof AlgebricksException) {
             if (LOGGER.isDebugEnabled()) {
-                LOGGER.debug("handleException: {}: {}", t.getMessage(), LogRedactionUtil.userData(param.toString()), t);
+                LOGGER.debug("handleException: {}: {}", t.getMessage(), LogRedactionUtil.statement(param.toString()),
+                        t);
             } else {
                 LOGGER.info(() -> "handleException: " + t.getMessage() + ": "
-                        + LogRedactionUtil.userData(param.toString()));
+                        + LogRedactionUtil.statement(param.toString()));
             }
             executionState.setStatus(ResultStatus.FATAL, HttpResponseStatus.BAD_REQUEST);
         } else if (t instanceof HyracksException) {
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/message/ExecuteStatementRequestMessage.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/message/ExecuteStatementRequestMessage.java
index 149ed33..2eced12 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/message/ExecuteStatementRequestMessage.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/message/ExecuteStatementRequestMessage.java
@@ -204,6 +204,6 @@ public final class ExecuteStatementRequestMessage implements ICcAddressedMessage
     @Override
     public String toString() {
         return String.format("%s(id=%s, from=%s): %s", getClass().getSimpleName(), requestMessageId, requestNodeId,
-                LogRedactionUtil.userData(statementsText));
+                LogRedactionUtil.statement(statementsText));
     }
 }
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/CCApplication.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/CCApplication.java
index fc912b0..0f0620c 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/CCApplication.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/CCApplication.java
@@ -86,6 +86,7 @@ import org.apache.asterix.runtime.utils.CcApplicationContext;
 import org.apache.asterix.translator.IStatementExecutorFactory;
 import org.apache.asterix.translator.Receptionist;
 import org.apache.asterix.util.MetadataBuiltinFunctions;
+import org.apache.asterix.utils.RedactionUtil;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.api.application.IServiceContext;
 import org.apache.hyracks.api.client.IHyracksClientConnection;
@@ -104,6 +105,7 @@ import org.apache.hyracks.http.server.HttpServerConfig;
 import org.apache.hyracks.http.server.HttpServerConfigBuilder;
 import org.apache.hyracks.http.server.WebManager;
 import org.apache.hyracks.ipc.impl.HyracksConnection;
+import org.apache.hyracks.util.LogRedactionUtil;
 import org.apache.hyracks.util.LoggingConfigUtil;
 import org.apache.logging.log4j.Level;
 import org.apache.logging.log4j.LogManager;
@@ -204,6 +206,7 @@ public class CCApplication extends BaseCCApplication {
     public void configureLoggingLevel(Level level) {
         super.configureLoggingLevel(level);
         LoggingConfigUtil.defaultIfMissing(GlobalConfig.ASTERIX_LOGGER_NAME, level);
+        LogRedactionUtil.setRedactor(RedactionUtil.LOG_REDACTOR);
     }
 
     protected List<AsterixExtension> getExtensions() throws Exception {
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/NCApplication.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/NCApplication.java
index 2e5c09c..1036fb2 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/NCApplication.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/NCApplication.java
@@ -72,6 +72,7 @@ import org.apache.asterix.messaging.NCMessageBroker;
 import org.apache.asterix.transaction.management.resource.PersistentLocalResourceRepository;
 import org.apache.asterix.translator.Receptionist;
 import org.apache.asterix.util.MetadataBuiltinFunctions;
+import org.apache.asterix.utils.RedactionUtil;
 import org.apache.hyracks.api.application.IServiceContext;
 import org.apache.hyracks.api.client.NodeStatus;
 import org.apache.hyracks.api.config.IConfigManager;
@@ -86,6 +87,7 @@ import org.apache.hyracks.http.server.HttpServer;
 import org.apache.hyracks.http.server.HttpServerConfig;
 import org.apache.hyracks.http.server.HttpServerConfigBuilder;
 import org.apache.hyracks.http.server.WebManager;
+import org.apache.hyracks.util.LogRedactionUtil;
 import org.apache.hyracks.util.LoggingConfigUtil;
 import org.apache.logging.log4j.Level;
 import org.apache.logging.log4j.LogManager;
@@ -184,6 +186,7 @@ public class NCApplication extends BaseNCApplication {
     public void configureLoggingLevel(Level level) {
         super.configureLoggingLevel(level);
         LoggingConfigUtil.defaultIfMissing(GlobalConfig.ASTERIX_LOGGER_NAME, level);
+        LogRedactionUtil.setRedactor(RedactionUtil.LOG_REDACTOR);
     }
 
     protected void configureServers() throws Exception {
diff --git a/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/LogRedactionUtil.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/utils/RedactionUtil.java
similarity index 53%
copy from hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/LogRedactionUtil.java
copy to asterixdb/asterix-app/src/main/java/org/apache/asterix/utils/RedactionUtil.java
index 89c957e..156b78a 100644
--- a/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/LogRedactionUtil.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/utils/RedactionUtil.java
@@ -16,36 +16,39 @@
  * specific language governing permissions and limitations
  * under the License.
  */
+package org.apache.asterix.utils;
 
-package org.apache.hyracks.util;
+import static java.util.regex.Pattern.CASE_INSENSITIVE;
+import static java.util.regex.Pattern.DOTALL;
+import static org.apache.asterix.external.util.ExternalDataConstants.AwsS3.SECRET_ACCESS_KEY_FIELD_NAME;
 
-public class LogRedactionUtil {
+import java.util.regex.Pattern;
 
-    private static final ILogRedactor DEFAULT_LOG_REDACTOR = new ILogRedactor() {
+import org.apache.hyracks.util.ILogRedactor;
+
+public class RedactionUtil {
+    private RedactionUtil() {
+        throw new AssertionError("do not instantiate");
+    }
+
+    private static final Pattern STATEMENT_PATTERN =
+            Pattern.compile("(" + SECRET_ACCESS_KEY_FIELD_NAME + ").*", CASE_INSENSITIVE | DOTALL);
+    private static final String STATEMENT_REPLACEMENT = "$1...<redacted sensitive data>";
+
+    public static final ILogRedactor LOG_REDACTOR = new ILogRedactor() {
         @Override
         public String userData(String text) {
             return text;
         }
 
         @Override
+        public String statement(String text) {
+            return STATEMENT_PATTERN.matcher(text).replaceFirst(STATEMENT_REPLACEMENT);
+        }
+
+        @Override
         public String unredactUserData(String text) {
             return text;
         }
     };
-    private static ILogRedactor redactor = DEFAULT_LOG_REDACTOR;
-
-    private LogRedactionUtil() {
-    }
-
-    public static void setRedactor(ILogRedactor redactor) {
-        LogRedactionUtil.redactor = redactor;
-    }
-
-    public static String userData(String text) {
-        return redactor.userData(text);
-    }
-
-    public static String unredactUserData(String text) {
-        return redactor.unredactUserData(text);
-    }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/ILogRedactor.java b/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/ILogRedactor.java
index b133894..d36c77b 100644
--- a/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/ILogRedactor.java
+++ b/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/ILogRedactor.java
@@ -29,6 +29,14 @@ public interface ILogRedactor {
     String userData(String text);
 
     /**
+     * Redacts statement argument.
+     *
+     * @param text statement to redact.
+     * @return redacted statement.
+     */
+    String statement(String text);
+
+    /**
      * Unredacts user data found in the argument.
      *
      * @param text text that contains some redacted user data.
diff --git a/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/LogRedactionUtil.java b/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/LogRedactionUtil.java
index 89c957e..fdce8c8 100644
--- a/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/LogRedactionUtil.java
+++ b/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/LogRedactionUtil.java
@@ -28,6 +28,11 @@ public class LogRedactionUtil {
         }
 
         @Override
+        public String statement(String text) {
+            return text;
+        }
+
+        @Override
         public String unredactUserData(String text) {
             return text;
         }
@@ -45,6 +50,10 @@ public class LogRedactionUtil {
         return redactor.userData(text);
     }
 
+    public static String statement(String text) {
+        return redactor.statement(text);
+    }
+
     public static String unredactUserData(String text) {
         return redactor.unredactUserData(text);
     }


[asterixdb] 01/07: [ASTERIXDB-2754][COMP] LoadRecordFieldsRule must clone expression

Posted by mb...@apache.org.
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 26e411e19e5d44146187471c02953de147cfcdb7
Author: Dmitry Lychagin <dm...@couchbase.com>
AuthorDate: Wed Jul 1 10:20:00 2020 -0700

    [ASTERIXDB-2754][COMP] LoadRecordFieldsRule must clone expression
    
    - user model changes: no
    - storage format changes: no
    - interface changes: no
    
    Details:
    - LoadRecordFieldsRule replaces a field access expression with
      another expression that it gets from a record constructor that
      created this field. It must clone the replacement expression
      because it's still used by the record constructor
    
    Change-Id: I4ba653bbb82a76783b295c3bc3cfbed897a3b4e6
    Reviewed-on: https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/7064
    Integration-Tests: Jenkins <je...@fulliautomatix.ics.uci.edu>
    Tested-by: Jenkins <je...@fulliautomatix.ics.uci.edu>
    Reviewed-by: Dmitry Lychagin <dm...@couchbase.com>
    Reviewed-by: Ali Alsuliman <al...@gmail.com>
---
 .../optimizer/rules/LoadRecordFieldsRule.java      |  2 +-
 .../optimizerts/queries/common-expr-01.sqlpp       | 65 +++++++++++++++++++++
 .../optimizerts/results/common-expr-01.plan        | 68 ++++++++++++++++++++++
 3 files changed, 134 insertions(+), 1 deletion(-)

diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/LoadRecordFieldsRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/LoadRecordFieldsRule.java
index 6762e77..ce1312f 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/LoadRecordFieldsRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/LoadRecordFieldsRule.java
@@ -292,7 +292,7 @@ public class LoadRecordFieldsRule implements IAlgebraicRewriteRule {
         VariableUtilities.getLiveVariables(assign, liveInputVars);
         usedVariables.removeAll(liveInputVars);
         if (usedVariables.isEmpty()) {
-            assign.getExpressions().get(0).setValue(fldExpr);
+            assign.getExpressions().get(0).setValue(fldExpr.cloneExpression());
             return true;
         } else {
             return false;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/common-expr-01.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/common-expr-01.sqlpp
new file mode 100644
index 0000000..30249d8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/common-expr-01.sqlpp
@@ -0,0 +1,65 @@
+/*
+ * 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.
+ */
+/*
+ * Test common expression extraction
+ */
+
+DROP DATAVERSE test IF EXISTS;
+CREATE DATAVERSE test;
+
+USE test;
+
+CREATE TYPE test.TestType AS
+{
+  id : INTEGER
+};
+
+CREATE DATASET t1(TestType) PRIMARY KEY id;
+
+WITH DT_PATTERN AS "YYYY-MM-DDThh:mm:ss"
+
+SELECT
+  v2.c11,
+  v1.c3,
+  SUM(v1.c4)
+FROM (
+  SELECT
+    c1,
+    c2,
+    GET_MONTH(PRINT_DATETIME(PARSE_DATETIME(c1,DT_PATTERN),DT_PATTERN)) AS c3,
+    (CASE WHEN CONTAINS(c2, "x") THEN 1 ELSE 0 END) AS c4
+  FROM test.t1
+) AS v1
+INNER JOIN (
+  SELECT
+    GET_HOUR(DURATION_FROM_INTERVAL(INTERVAL(
+      PARSE_DATETIME(PRINT_DATETIME(CURRENT_DATETIME(),DT_PATTERN),DT_PATTERN),
+      PARSE_DATETIME(MAX(PRINT_DATETIME(PARSE_DATETIME(t1.c5,DT_PATTERN),DT_PATTERN)),DT_PATTERN)
+    ))) AS c11
+  FROM test.t1
+) AS v2 ON TRUE
+WHERE
+  GET_DAY(DURATION_FROM_INTERVAL(INTERVAL(
+    PARSE_DATETIME(PRINT_DATETIME(CURRENT_DATETIME(),DT_PATTERN),DT_PATTERN),
+    PARSE_DATETIME(v1.c1,DT_PATTERN)
+  ))) <= 30
+  AND
+  (v1.c2 NOT IN ["a","b","c","d"])
+GROUP BY
+  v1.c3, v2.c11;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/common-expr-01.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/common-expr-01.plan
new file mode 100644
index 0000000..83fc968
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/common-expr-01.plan
@@ -0,0 +1,68 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- SORT_GROUP_BY[$$176, $$177]  |PARTITIONED|
+                  {
+                    -- AGGREGATE  |LOCAL|
+                      -- NESTED_TUPLE_SOURCE  |LOCAL|
+                  }
+            -- HASH_PARTITION_EXCHANGE [$$176, $$177]  |PARTITIONED|
+              -- SORT_GROUP_BY[$$159, $$160]  |PARTITIONED|
+                      {
+                        -- AGGREGATE  |LOCAL|
+                          -- NESTED_TUPLE_SOURCE  |LOCAL|
+                      }
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ASSIGN  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- NESTED_LOOP  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- STREAM_PROJECT  |PARTITIONED|
+                              -- ASSIGN  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
+                                  -- STREAM_SELECT  |PARTITIONED|
+                                    -- STREAM_PROJECT  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- PRE_CLUSTERED_GROUP_BY[$$163]  |PARTITIONED|
+                                                {
+                                                  -- AGGREGATE  |LOCAL|
+                                                    -- STREAM_SELECT  |LOCAL|
+                                                      -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                }
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- STABLE_SORT [$$163(ASC)]  |PARTITIONED|
+                                              -- HASH_PARTITION_EXCHANGE [$$163]  |PARTITIONED|
+                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- NESTED_LOOP  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- STREAM_SELECT  |PARTITIONED|
+                                                          -- STREAM_PROJECT  |PARTITIONED|
+                                                            -- ASSIGN  |PARTITIONED|
+                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                -- REPLICATE  |PARTITIONED|
+                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                    -- DATASOURCE_SCAN  |PARTITIONED|
+                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                        -- ASSIGN  |UNPARTITIONED|
+                                                          -- UNNEST  |UNPARTITIONED|
+                                                            -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+                          -- BROADCAST_EXCHANGE  |PARTITIONED|
+                            -- AGGREGATE  |UNPARTITIONED|
+                              -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                                -- AGGREGATE  |PARTITIONED|
+                                  -- STREAM_PROJECT  |PARTITIONED|
+                                    -- ASSIGN  |PARTITIONED|
+                                      -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ASSIGN  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- REPLICATE  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- DATASOURCE_SCAN  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
\ No newline at end of file


[asterixdb] 02/07: [NO ISSUE][RT] Support External Datasets Rebalance

Posted by mb...@apache.org.
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 9ac78167dd2424ca78bf0144bd1c5c70610d542e
Author: Murtadha Hubail <mh...@apache.org>
AuthorDate: Mon Jul 6 21:24:08 2020 +0300

    [NO ISSUE][RT] Support External Datasets Rebalance
    
    - user model changes: no
    - storage format changes: no
    - interface changes: no
    
    Details:
    - Do not attempt to rebalance external datasets files
      on rebalance since they do not have files on local
      storage.
    - Skip dropping external datasets files on rebalance
      since they have no files on local storage.
    
    Change-Id: I71f50ade98c8b798225baf0cb34dd6e70a5e923e
    Reviewed-on: https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/7123
    Tested-by: Jenkins <je...@fulliautomatix.ics.uci.edu>
    Integration-Tests: Jenkins <je...@fulliautomatix.ics.uci.edu>
    Reviewed-by: Murtadha Hubail <mh...@apache.org>
    Reviewed-by: Ali Alsuliman <al...@gmail.com>
---
 .../src/main/java/org/apache/asterix/utils/RebalanceUtil.java     | 8 +++++++-
 1 file changed, 7 insertions(+), 1 deletion(-)

diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/utils/RebalanceUtil.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/utils/RebalanceUtil.java
index 483987c..dff0b4a 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/utils/RebalanceUtil.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/utils/RebalanceUtil.java
@@ -19,6 +19,7 @@
 package org.apache.asterix.utils;
 
 import static org.apache.asterix.app.translator.QueryTranslator.abort;
+import static org.apache.asterix.common.config.DatasetConfig.DatasetType;
 import static org.apache.hyracks.storage.am.common.dataflow.IndexDropOperatorDescriptor.DropOption;
 
 import java.rmi.RemoteException;
@@ -128,7 +129,9 @@ public class RebalanceUtil {
                         sourceDataset.getDatasetName(), sourceDataset.getNodeGroupName(), sourceNodes,
                         targetDataset.getNodeGroupName(), targetNcNames);
                 // Rebalances the source dataset into the target dataset.
-                rebalance(sourceDataset, targetDataset, metadataProvider, hcc, datasetRebalanceCallback);
+                if (sourceDataset.getDatasetType() != DatasetType.EXTERNAL) {
+                    rebalance(sourceDataset, targetDataset, metadataProvider, hcc, datasetRebalanceCallback);
+                }
             } else {
                 targetDataset = null;
                 // if this the last NC in the cluster, just drop the dataset
@@ -351,6 +354,9 @@ public class RebalanceUtil {
     // Drops dataset files of a given dataset.
     private static void dropDatasetFiles(Dataset dataset, MetadataProvider metadataProvider,
             IHyracksClientConnection hcc) throws Exception {
+        if (dataset.getDatasetType() == DatasetType.EXTERNAL) {
+            return;
+        }
         List<JobSpecification> jobs = new ArrayList<>();
         List<Index> indexes = metadataProvider.getDatasetIndexes(dataset.getDataverseName(), dataset.getDatasetName());
         for (Index index : indexes) {