You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@asterixdb.apache.org by AsterixDB Code Review <do...@asterix-gerrit.ics.uci.edu> on 2022/12/02 23:00:50 UTC

Change in asterixdb[neo]: [ASTERIXDB-3092][COMP] Consider only field-access functions in LoadRe...

From Wail Alkowaileet <wa...@gmail.com>:

Wail Alkowaileet has uploaded this change for review. ( https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/17298 )


Change subject: [ASTERIXDB-3092][COMP] Consider only field-access functions in LoadRecordFieldsRule
......................................................................

[ASTERIXDB-3092][COMP] Consider only field-access functions in LoadRecordFieldsRule

- user model changes: no
- storage format changes: no
- interface changes: no

Details:
The compiler rule LoadRecordFieldsRule fails when it encounters
a non-field-access function. This patch fixes this issue by ensuring
that only field accesses are considered for this rule.

Change-Id: I88f72fd51716dd8152e709c841489e87af0a5137
---
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/load-record-fields/load-record-fields.1.ddl.sqlpp
M asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/ObjectsQueries.xml
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/load-record-fields/load-record-fields.3.query.sqlpp
M asterixdb/asterix-app/src/test/resources/runtimets/only_sqlpp.xml
A asterixdb/asterix-app/src/test/resources/runtimets/results/objects/load-record-fields/load-record-fields.3.adm
A asterixdb/asterix-app/src/test/resources/runtimets/results/objects/load-record-fields/load-record-fields.4.plan
M asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/LoadRecordFieldsRule.java
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/load-record-fields/load-record-fields.4.query.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/load-record-fields/load-record-fields.2.update.sqlpp
9 files changed, 193 insertions(+), 10 deletions(-)



  git pull ssh://asterix-gerrit.ics.uci.edu:29418/asterixdb refs/changes/98/17298/1

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 b9d512b..5377770 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
@@ -18,6 +18,9 @@
  */
 package org.apache.asterix.optimizer.rules;
 
+import org.apache.commons.lang3.mutable.Mutable;
+import org.apache.commons.lang3.mutable.MutableObject;
+
 import java.util.ArrayList;
 import java.util.HashSet;
 import java.util.Iterator;
@@ -34,8 +37,6 @@
 import org.apache.asterix.om.types.ARecordType;
 import org.apache.asterix.om.utils.ConstantExpressionUtil;
 import org.apache.asterix.optimizer.base.AnalysisUtil;
-import org.apache.commons.lang3.mutable.Mutable;
-import org.apache.commons.lang3.mutable.MutableObject;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalOperator;
@@ -64,7 +65,7 @@
 
 public class LoadRecordFieldsRule implements IAlgebraicRewriteRule {
 
-    private ExtractFieldLoadExpressionVisitor exprVisitor = new ExtractFieldLoadExpressionVisitor();
+    private final ExtractFieldLoadExpressionVisitor exprVisitor = new ExtractFieldLoadExpressionVisitor();
 
     @Override
     public boolean rewritePre(Mutable<ILogicalOperator> opRef, IOptimizationContext context) {
@@ -98,13 +99,15 @@
             // checking if we can annotate a Selection as using just one field
             // access
             SelectOperator sigma = (SelectOperator) op1;
-            LinkedList<LogicalVariable> vars = new LinkedList<LogicalVariable>();
+            List<LogicalVariable> vars = new ArrayList<>();
             VariableUtilities.getUsedVariables(sigma, vars);
             if (vars.size() == 1) {
                 // we can annotate Selection
                 AssignOperator assign1 = (AssignOperator) op1.getInputs().get(0).getValue();
-                AbstractLogicalExpression expr1 = (AbstractLogicalExpression) getFirstExpr(assign1);
-                if (expr1.getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL) {
+                ILogicalExpression expr1 = getFirstExpr(assign1);
+                FunctionIdentifier fid = getFunctionIdentifier(expr1);
+                if (BuiltinFunctions.FIELD_ACCESS_BY_INDEX.equals(fid) || BuiltinFunctions.FIELD_ACCESS_BY_NAME.equals(
+                        fid) || BuiltinFunctions.FIELD_ACCESS_NESTED.equals(fid)) {
                     AbstractFunctionCallExpression f = (AbstractFunctionCallExpression) expr1;
                     // f should be a call to a field/data access kind of
                     // function
@@ -141,7 +144,7 @@
                     }
                     // create an assign
                     LogicalVariable v = context.newVar();
-                    AssignOperator a2 = new AssignOperator(v, new MutableObject<ILogicalExpression>(f));
+                    AssignOperator a2 = new AssignOperator(v, new MutableObject<>(f));
                     a2.setSourceLocation(expr.getSourceLocation());
                     pushFieldAssign(a2, topOp, context);
                     context.computeAndSetTypeEnvironmentForOperator(a2);
@@ -151,7 +154,7 @@
                         LogicalVariable var = ref.getVariableReference();
                         List<LogicalVariable> keys = context.findPrimaryKey(var);
                         if (keys != null) {
-                            List<LogicalVariable> tail = new ArrayList<LogicalVariable>();
+                            List<LogicalVariable> tail = new ArrayList<>();
                             tail.add(v);
                             FunctionalDependency pk = new FunctionalDependency(keys, tail);
                             context.addPrimaryKey(pk);
@@ -381,8 +384,9 @@
         Integer fieldIndex = (Integer) index;
         ARecordType recordType = (ARecordType) typeEnvironment.getType(fce);
         String[] closedFieldNames = recordType.getFieldNames();
-        return closedFieldNames.length > fieldIndex
-                ? findFieldByNameFromRecordConstructor(closedFieldNames[fieldIndex], fce) : null;
+        return closedFieldNames.length > fieldIndex ?
+                findFieldByNameFromRecordConstructor(closedFieldNames[fieldIndex], fce) :
+                null;
     }
 
     private final class ExtractFieldLoadExpressionVisitor implements ILogicalExpressionReferenceTransform {
@@ -409,4 +413,12 @@
         return assign.getExpressions().get(0).getValue();
     }
 
+    private static FunctionIdentifier getFunctionIdentifier(ILogicalExpression expression) {
+        if (expression.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
+            return null;
+        }
+        AbstractFunctionCallExpression funcExpr = (AbstractFunctionCallExpression) expression;
+        return funcExpr.getFunctionIdentifier();
+    }
+
 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/only_sqlpp.xml b/asterixdb/asterix-app/src/test/resources/runtimets/only_sqlpp.xml
index 334dd52..8ad0b4b 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/only_sqlpp.xml
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/only_sqlpp.xml
@@ -19,5 +19,10 @@
  !-->
 <test-suite xmlns="urn:xml.testframework.asterix.apache.org" ResultOffsetPath="results" QueryOffsetPath="queries_sqlpp" QueryFileExtension=".sqlpp">
   <test-group name="failed">
+    <test-case FilePath="objects">
+      <compilation-unit name="load-record-fields">
+        <output-dir compare="Text">load-record-fields</output-dir>
+      </compilation-unit>
+    </test-case>
   </test-group>
 </test-suite>
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/ObjectsQueries.xml b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/ObjectsQueries.xml
index 623237b..8532fe9 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/ObjectsQueries.xml
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/ObjectsQueries.xml
@@ -244,4 +244,9 @@
       <expected-warn>Duplicate field name 'fname1' (in line 25, at column 45)</expected-warn>
     </compilation-unit>
   </test-case>
+  <test-case FilePath="objects">
+    <compilation-unit name="load-record-fields">
+      <output-dir compare="Text">load-record-fields</output-dir>
+    </compilation-unit>
+  </test-case>
 </test-group>
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/load-record-fields/load-record-fields.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/load-record-fields/load-record-fields.1.ddl.sqlpp
new file mode 100644
index 0000000..99898f9
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/load-record-fields/load-record-fields.1.ddl.sqlpp
@@ -0,0 +1,30 @@
+/*
+ * 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;
+
+CREATE TYPE OpenType AS {
+    id: int
+};
+
+CREATE DATASET MyDataset(OpenType)
+PRIMARY KEY id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/load-record-fields/load-record-fields.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/load-record-fields/load-record-fields.2.update.sqlpp
new file mode 100644
index 0000000..1f80ae1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/load-record-fields/load-record-fields.2.update.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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;
+
+INSERT INTO MyDataset (
+    {"id": 1, "name": "Alice"},
+    {"id": 2, "name": "Bob"}
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/load-record-fields/load-record-fields.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/load-record-fields/load-record-fields.3.query.sqlpp
new file mode 100644
index 0000000..278f5d2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/load-record-fields/load-record-fields.3.query.sqlpp
@@ -0,0 +1,30 @@
+/*
+ * 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;
+
+-- Disabled for a simpler plan
+SET `compiler.sort.parallel` "false";
+
+
+SELECT VALUE md.name
+FROM MyDataset md
+LET currentData = {"myDate": current_date()}
+WHERE currentData.myDate = current_date()
+ORDER BY md.id
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/load-record-fields/load-record-fields.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/load-record-fields/load-record-fields.4.query.sqlpp
new file mode 100644
index 0000000..f44921f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/load-record-fields/load-record-fields.4.query.sqlpp
@@ -0,0 +1,30 @@
+/*
+ * 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;
+
+-- Disabled for a simpler plan
+SET `compiler.sort.parallel` "false";
+
+EXPLAIN
+SELECT VALUE md.name
+FROM MyDataset md
+LET currentData = {"myDate": current_date()}
+WHERE currentData.myDate = current_date()
+ORDER BY md.id
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/load-record-fields/load-record-fields.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/load-record-fields/load-record-fields.3.adm
new file mode 100644
index 0000000..ac2dc97
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/load-record-fields/load-record-fields.3.adm
@@ -0,0 +1,2 @@
+"Alice"
+"Bob"
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/load-record-fields/load-record-fields.4.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/load-record-fields/load-record-fields.4.plan
new file mode 100644
index 0000000..5ac69f3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/load-record-fields/load-record-fields.4.plan
@@ -0,0 +1,26 @@
+distribute result [$$28]
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  exchange
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    project ([$$28])
+    -- STREAM_PROJECT  |PARTITIONED|
+      exchange
+      -- SORT_MERGE_EXCHANGE [$$30(ASC) ]  |PARTITIONED|
+        project ([$$28, $$30])
+        -- STREAM_PROJECT  |PARTITIONED|
+          select (eq($$31, current-date()))
+          -- STREAM_SELECT  |PARTITIONED|
+            assign [$$31] <- [current-date()]
+            -- ASSIGN  |PARTITIONED|
+              project ([$$30, $$28])
+              -- STREAM_PROJECT  |PARTITIONED|
+                assign [$$28] <- [$$md.getField("name")]
+                -- ASSIGN  |PARTITIONED|
+                  exchange
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    data-scan []<-[$$30, $$md] <- test.MyDataset
+                    -- DATASOURCE_SCAN  |PARTITIONED|
+                      exchange
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        empty-tuple-source
+                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/17298
To unsubscribe, or for help writing mail filters, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-Project: asterixdb
Gerrit-Branch: neo
Gerrit-Change-Id: I88f72fd51716dd8152e709c841489e87af0a5137
Gerrit-Change-Number: 17298
Gerrit-PatchSet: 1
Gerrit-Owner: Wail Alkowaileet <wa...@gmail.com>
Gerrit-MessageType: newchange