Yingyi Bu has uploaded a new change for review.

  https://asterix-gerrit.ics.uci.edu/1113

Change subject: ASTERIXDB-1577: fix error handling for field access over a 
dataset.
......................................................................

ASTERIXDB-1577: fix error handling for field access over a dataset.

- ASTERIXDB-159 is fixed as well.

Change-Id: Id0dc4db91a6251d55dafd734d9ea5bfb6c11c315
---
M 
asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/RuleCollections.java
A 
asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ListifyUnnestFunctionRule.java
M 
asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ResolveVariableRule.java
A 
asterixdb/asterix-app/src/test/resources/runtimets/queries/aggregate/query-ASTERIXDB-159/query-ASTERIXDB-159.1.ddl.aql
A 
asterixdb/asterix-app/src/test/resources/runtimets/queries/aggregate/query-ASTERIXDB-159/query-ASTERIXDB-159.2.update.aql
A 
asterixdb/asterix-app/src/test/resources/runtimets/queries/aggregate/query-ASTERIXDB-159/query-ASTERIXDB-159.3.query.aql
A 
asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/global-aggregate/q11/q11.1.ddl.sqlpp
A 
asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/global-aggregate/q11/q11.2.update.sqlpp
A 
asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/global-aggregate/q11/q11.3.query.sqlpp
A 
asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/list/list-dataset/list-dataset.1.ddl.sqlpp
A 
asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/list/list-dataset/list-dataset.2.update.sqlpp
A 
asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/list/list-dataset/list-dataset.3.query.sqlpp
A 
asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/list/list-range/list-range.1.query.sqlpp
A 
asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/query-ASTERIXDB-1577/query-ASTERIXDB-1577.1.ddl.sqlpp
A 
asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/query-ASTERIXDB-1577/query-ASTERIXDB-1577.2.update.sqlpp
A 
asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/query-ASTERIXDB-1577/query-ASTERIXDB-1577.3.query.sqlpp
A 
asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/query-ASTERIXDB-1577/query-ASTERIXDB-1577.4.ddl.sqlpp
A 
asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate/query-ASTERIXDB-159/query-ASTERIXDB-159.1.adm
A 
asterixdb/asterix-app/src/test/resources/runtimets/results/global-aggregate/q11/q11.1.adm
A 
asterixdb/asterix-app/src/test/resources/runtimets/results/list/list-dataset/list-dataset.1.adm
A 
asterixdb/asterix-app/src/test/resources/runtimets/results/list/list-range/list-range.1.adm
M asterixdb/asterix-app/src/test/resources/runtimets/testsuite.xml
M asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
M 
asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/AsterixBuiltinFunctions.java
24 files changed, 514 insertions(+), 3 deletions(-)


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

diff --git 
a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/RuleCollections.java
 
b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/RuleCollections.java
index b29f743..72f7af2 100644
--- 
a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/RuleCollections.java
+++ 
b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/RuleCollections.java
@@ -34,7 +34,6 @@
 import org.apache.asterix.optimizer.rules.CountVarToCountOneRule;
 import org.apache.asterix.optimizer.rules.DisjunctivePredicateToJoinRule;
 import org.apache.asterix.optimizer.rules.ExtractDistinctByExpressionsRule;
-import 
org.apache.hyracks.algebricks.rewriter.rules.ExtractGroupByDecorVariablesRule;
 import org.apache.asterix.optimizer.rules.ExtractOrderExpressionsRule;
 import org.apache.asterix.optimizer.rules.FeedScanCollectionToUnnest;
 import org.apache.asterix.optimizer.rules.FuzzyEqRule;
@@ -51,6 +50,7 @@
 import 
org.apache.asterix.optimizer.rules.IntroduceSecondaryIndexInsertDeleteRule;
 import org.apache.asterix.optimizer.rules.IntroduceStaticTypeCastForInsertRule;
 import 
org.apache.asterix.optimizer.rules.IntroduceUnnestForCollectionToSequenceRule;
+import org.apache.asterix.optimizer.rules.ListifyUnnestFunctionRule;
 import org.apache.asterix.optimizer.rules.LoadRecordFieldsRule;
 import org.apache.asterix.optimizer.rules.MetaFunctionToMetaVariableRule;
 import org.apache.asterix.optimizer.rules.NestGroupByRule;
@@ -92,6 +92,7 @@
 import 
org.apache.hyracks.algebricks.rewriter.rules.ExtractCommonExpressionsRule;
 import org.apache.hyracks.algebricks.rewriter.rules.ExtractCommonOperatorsRule;
 import org.apache.hyracks.algebricks.rewriter.rules.ExtractGbyExpressionsRule;
+import 
org.apache.hyracks.algebricks.rewriter.rules.ExtractGroupByDecorVariablesRule;
 import 
org.apache.hyracks.algebricks.rewriter.rules.FactorRedundantGroupAndDecorVarsRule;
 import org.apache.hyracks.algebricks.rewriter.rules.InferTypesRule;
 import 
org.apache.hyracks.algebricks.rewriter.rules.InlineAssignIntoAggregateRule;
@@ -158,6 +159,7 @@
             IAlgebraExtensionManager algebraExtensionManager) {
         List<IAlgebraicRewriteRule> normalization = new LinkedList<>();
         normalization.add(new ResolveVariableRule());
+        normalization.add(new ListifyUnnestFunctionRule());
         normalization.add(new IntroduceUnnestForCollectionToSequenceRule());
         normalization.add(new EliminateSubplanRule());
         normalization.add(new EnforceOrderByAfterSubplan());
diff --git 
a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ListifyUnnestFunctionRule.java
 
b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ListifyUnnestFunctionRule.java
new file mode 100644
index 0000000..583f499
--- /dev/null
+++ 
b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ListifyUnnestFunctionRule.java
@@ -0,0 +1,130 @@
+/*
+ * 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.optimizer.rules;
+
+import java.util.ArrayList;
+import java.util.Collections;
+
+import org.apache.asterix.lang.common.util.FunctionUtil;
+import org.apache.asterix.om.functions.AsterixBuiltinFunctions;
+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;
+import org.apache.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import 
org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
+import 
org.apache.hyracks.algebricks.core.algebra.expressions.AggregateFunctionCallExpression;
+import 
org.apache.hyracks.algebricks.core.algebra.expressions.UnnestingFunctionCallExpression;
+import 
org.apache.hyracks.algebricks.core.algebra.expressions.VariableReferenceExpression;
+import org.apache.hyracks.algebricks.core.algebra.functions.IFunctionInfo;
+import 
org.apache.hyracks.algebricks.core.algebra.operators.logical.AggregateOperator;
+import 
org.apache.hyracks.algebricks.core.algebra.operators.logical.EmptyTupleSourceOperator;
+import 
org.apache.hyracks.algebricks.core.algebra.operators.logical.SubplanOperator;
+import 
org.apache.hyracks.algebricks.core.algebra.operators.logical.UnnestOperator;
+import org.apache.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+
+/**
+ * In principle, an unnest operator produces a sequence items from a 
collection.
+ * However, the final result of an unnest is still a collection.
+ * </p>
+ * Hence, if an unnesting function expression is not called from a unnest 
operator, it is invalid and we need to extract
+ * it out into an unnest operator and then listify the unnested sequence items 
so as the listified collection
+ * can replace the original call of the unnest function. This rule performs 
the aforementioned transformations.
+ */
+public class ListifyUnnestFunctionRule implements IAlgebraicRewriteRule {
+
+    @Override
+    public boolean rewritePost(Mutable<ILogicalOperator> opRef, 
IOptimizationContext context)
+            throws AlgebricksException {
+        ILogicalOperator op = opRef.getValue();
+        if (op.getOperatorTag() == LogicalOperatorTag.UNNEST
+                || op.getOperatorTag() == 
LogicalOperatorTag.LEFT_OUTER_UNNEST) {
+            return false;
+        }
+        return op.acceptExpressionTransform(exprRef -> 
rewriteExpressionReference(op, exprRef, context));
+    }
+
+    // Recursively rewrites for an expression within an operator.
+    private boolean rewriteExpressionReference(ILogicalOperator op, 
Mutable<ILogicalExpression> exprRef,
+            IOptimizationContext context) throws AlgebricksException {
+        ILogicalExpression expr = exprRef.getValue();
+        if (expr.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
+            return false;
+        }
+        boolean changed = false;
+        AbstractFunctionCallExpression funcExpr = 
(AbstractFunctionCallExpression) expr;
+
+        // Rewrites arguments.
+        for (Mutable<ILogicalExpression> funcArgRef : funcExpr.getArguments()) 
{
+            if (rewriteExpressionReference(op, funcArgRef, context)) {
+                changed = true;
+            }
+        }
+
+        // Rewrites the current function expression.
+        return changed || listifyUnnestingFunction(op, exprRef, funcExpr, 
context);
+    }
+
+    // Performs the actual logical transformation.
+    private boolean listifyUnnestingFunction(ILogicalOperator op, 
Mutable<ILogicalExpression> exprRef,
+            AbstractFunctionCallExpression func, IOptimizationContext context) 
throws AlgebricksException {
+        IFunctionInfo functionInfo = func.getFunctionInfo();
+        if 
(!AsterixBuiltinFunctions.isBuiltinUnnestingFunction(functionInfo.getFunctionIdentifier()))
 {
+            return false;
+        }
+        SubplanOperator subplanOperator = new SubplanOperator();
+        // Creates an empty tuple source operator, as the subplan is not 
correlated with the outer data flow.
+        EmptyTupleSourceOperator ets = new EmptyTupleSourceOperator();
+        context.computeAndSetTypeEnvironmentForOperator(ets);
+
+        // Unnests the dataset.
+        LogicalVariable unnestVar = context.newVar();
+        ILogicalExpression unnestExpr = new 
UnnestingFunctionCallExpression(functionInfo, func.getArguments());
+        UnnestOperator unnestOperator = new UnnestOperator(unnestVar, new 
MutableObject<>(unnestExpr));
+        unnestOperator.getInputs().add(new MutableObject<>(ets));
+        context.computeAndSetTypeEnvironmentForOperator(unnestOperator);
+
+        // Listify the dataset into one collection.
+        LogicalVariable aggVar = context.newVar();
+        Mutable<ILogicalExpression> aggArgExprRef = new MutableObject<>(new 
VariableReferenceExpression(unnestVar));
+        ILogicalExpression aggExpr = new AggregateFunctionCallExpression(
+                FunctionUtil.getFunctionInfo(AsterixBuiltinFunctions.LISTIFY), 
false, new ArrayList<>(
+                        Collections.singletonList(aggArgExprRef)));
+        AggregateOperator aggregateOperator = new AggregateOperator(new 
ArrayList<>(Collections.singletonList(aggVar)),
+                new ArrayList<>(Collections.singletonList(new 
MutableObject<>(aggExpr))));
+        aggregateOperator.getInputs().add(new MutableObject<>(unnestOperator));
+        context.computeAndSetTypeEnvironmentForOperator(aggregateOperator);
+
+        // Adds the aggregate operator as the root of the subplan.
+        subplanOperator.setRootOp(new MutableObject<>(aggregateOperator));
+
+        // Sticks a subplan into the query plan.
+        subplanOperator.getInputs().add(op.getInputs().get(0));
+        op.getInputs().set(0, new MutableObject<>(subplanOperator));
+        exprRef.setValue(new VariableReferenceExpression(aggVar));
+        context.computeAndSetTypeEnvironmentForOperator(subplanOperator);
+        return true;
+    }
+
+}
diff --git 
a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ResolveVariableRule.java
 
b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ResolveVariableRule.java
index 2cce9b0..819e90d 100644
--- 
a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ResolveVariableRule.java
+++ 
b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ResolveVariableRule.java
@@ -120,7 +120,7 @@
         }
         ILogicalExpression arg = funcExpr.getArguments().get(0).getValue();
         String unresolvedVarName = extractConstantString(arg);
-        return resolveInternal(exprRef, 
hasMatchedDatasetForVariableName(unresolvedVarName, context),
+        return resolveInternal(op, exprRef, 
hasMatchedDatasetForVariableName(unresolvedVarName, context),
                 findCandidatePaths(op, extractExprs(funcExpr.getArguments()), 
unresolvedVarName, context),
                 unresolvedVarName, 
fullyQualifiedDatasetPathCandidateFromParent, parentFuncRef, context);
     }
@@ -138,7 +138,8 @@
 
     // Resolves an undefined name to a dataset or a fully qualified 
variable/field-access path
     // based on the given information of dataset matches and candidate paths.
-    private boolean resolveInternal(Mutable<ILogicalExpression> funcRef, 
boolean hasMatchedDataset,
+    private boolean resolveInternal(ILogicalOperator op, 
Mutable<ILogicalExpression> funcRef,
+            boolean hasMatchedDataset,
             Collection<Pair<ILogicalExpression, List<String>>> 
varAccessCandidates, String unresolvedVarName,
             Triple<Boolean, String, String> 
fullyQualifiedDatasetPathCandidateFromParent,
             Mutable<ILogicalExpression> parentFuncRef, IOptimizationContext 
context) throws AlgebricksException {
diff --git 
a/asterixdb/asterix-app/src/test/resources/runtimets/queries/aggregate/query-ASTERIXDB-159/query-ASTERIXDB-159.1.ddl.aql
 
b/asterixdb/asterix-app/src/test/resources/runtimets/queries/aggregate/query-ASTERIXDB-159/query-ASTERIXDB-159.1.ddl.aql
new file mode 100644
index 0000000..3eb0ba1
--- /dev/null
+++ 
b/asterixdb/asterix-app/src/test/resources/runtimets/queries/aggregate/query-ASTERIXDB-159/query-ASTERIXDB-159.1.ddl.aql
@@ -0,0 +1,31 @@
+/*
+ * 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 TinySocial if exists;
+create  dataverse TinySocial;
+
+use dataverse TinySocial;
+
+create type TinySocial.FacebookUserType as
+ open {
+  id : int64
+}
+
+create  dataset FacebookUsers(FacebookUserType) primary key id;
+
diff --git 
a/asterixdb/asterix-app/src/test/resources/runtimets/queries/aggregate/query-ASTERIXDB-159/query-ASTERIXDB-159.2.update.aql
 
b/asterixdb/asterix-app/src/test/resources/runtimets/queries/aggregate/query-ASTERIXDB-159/query-ASTERIXDB-159.2.update.aql
new file mode 100644
index 0000000..61449a7
--- /dev/null
+++ 
b/asterixdb/asterix-app/src/test/resources/runtimets/queries/aggregate/query-ASTERIXDB-159/query-ASTERIXDB-159.2.update.aql
@@ -0,0 +1,22 @@
+/*
+ * 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 dataverse TinySocial;
+
+load  dataset FacebookUsers using localfs 
(("path"="asterix_nc1://data/tinysocial/fbu.adm"),("format"="adm"));
diff --git 
a/asterixdb/asterix-app/src/test/resources/runtimets/queries/aggregate/query-ASTERIXDB-159/query-ASTERIXDB-159.3.query.aql
 
b/asterixdb/asterix-app/src/test/resources/runtimets/queries/aggregate/query-ASTERIXDB-159/query-ASTERIXDB-159.3.query.aql
new file mode 100644
index 0000000..d3fa032
--- /dev/null
+++ 
b/asterixdb/asterix-app/src/test/resources/runtimets/queries/aggregate/query-ASTERIXDB-159/query-ASTERIXDB-159.3.query.aql
@@ -0,0 +1,22 @@
+/*
+ * 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 dataverse TinySocial;
+
+count(dataset("FacebookUsers"));
diff --git 
a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/global-aggregate/q11/q11.1.ddl.sqlpp
 
b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/global-aggregate/q11/q11.1.ddl.sqlpp
new file mode 100644
index 0000000..78ce34a
--- /dev/null
+++ 
b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/global-aggregate/q11/q11.1.ddl.sqlpp
@@ -0,0 +1,31 @@
+/*
+ * 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 TinySocial if exists;
+create  dataverse TinySocial;
+
+use TinySocial;
+
+create type TinySocial.FacebookUserType as
+ open {
+  id : int64
+}
+
+create  dataset FacebookUsers(FacebookUserType) primary key id;
+
diff --git 
a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/global-aggregate/q11/q11.2.update.sqlpp
 
b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/global-aggregate/q11/q11.2.update.sqlpp
new file mode 100644
index 0000000..9f4f824
--- /dev/null
+++ 
b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/global-aggregate/q11/q11.2.update.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * 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 TinySocial;
+
+
+load  dataset FacebookUsers using localfs 
((`path`=`asterix_nc1://data/tinysocial/fbu.adm`),(`format`=`adm`));
diff --git 
a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/global-aggregate/q11/q11.3.query.sqlpp
 
b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/global-aggregate/q11/q11.3.query.sqlpp
new file mode 100644
index 0000000..2340771
--- /dev/null
+++ 
b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/global-aggregate/q11/q11.3.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * 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 TinySocial;
+
+COLL_COUNT(FacebookUsers);
diff --git 
a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/list/list-dataset/list-dataset.1.ddl.sqlpp
 
b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/list/list-dataset/list-dataset.1.ddl.sqlpp
new file mode 100644
index 0000000..78ce34a
--- /dev/null
+++ 
b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/list/list-dataset/list-dataset.1.ddl.sqlpp
@@ -0,0 +1,31 @@
+/*
+ * 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 TinySocial if exists;
+create  dataverse TinySocial;
+
+use TinySocial;
+
+create type TinySocial.FacebookUserType as
+ open {
+  id : int64
+}
+
+create  dataset FacebookUsers(FacebookUserType) primary key id;
+
diff --git 
a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/list/list-dataset/list-dataset.2.update.sqlpp
 
b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/list/list-dataset/list-dataset.2.update.sqlpp
new file mode 100644
index 0000000..9f4f824
--- /dev/null
+++ 
b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/list/list-dataset/list-dataset.2.update.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * 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 TinySocial;
+
+
+load  dataset FacebookUsers using localfs 
((`path`=`asterix_nc1://data/tinysocial/fbu.adm`),(`format`=`adm`));
diff --git 
a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/list/list-dataset/list-dataset.3.query.sqlpp
 
b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/list/list-dataset/list-dataset.3.query.sqlpp
new file mode 100644
index 0000000..b80e6a5
--- /dev/null
+++ 
b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/list/list-dataset/list-dataset.3.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * 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 TinySocial;
+
+FacebookUsers;
diff --git 
a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/list/list-range/list-range.1.query.sqlpp
 
b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/list/list-range/list-range.1.query.sqlpp
new file mode 100644
index 0000000..b0da262
--- /dev/null
+++ 
b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/list/list-range/list-range.1.query.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.
+ */
+
+range(0, 5);
diff --git 
a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/query-ASTERIXDB-1577/query-ASTERIXDB-1577.1.ddl.sqlpp
 
b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/query-ASTERIXDB-1577/query-ASTERIXDB-1577.1.ddl.sqlpp
new file mode 100644
index 0000000..5b1d0dd
--- /dev/null
+++ 
b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/query-ASTERIXDB-1577/query-ASTERIXDB-1577.1.ddl.sqlpp
@@ -0,0 +1,31 @@
+/*
+ * 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 sampdb if exists;
+create dataverse sampdb;
+use sampdb;
+
+drop dataset samptable if exists;
+drop type samptabletype if exists;
+
+create type samptabletype as closed {
+  id: int8
+};
+
+create dataset samptable1(samptabletype) primary key id;
diff --git 
a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/query-ASTERIXDB-1577/query-ASTERIXDB-1577.2.update.sqlpp
 
b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/query-ASTERIXDB-1577/query-ASTERIXDB-1577.2.update.sqlpp
new file mode 100644
index 0000000..5d4bfea
--- /dev/null
+++ 
b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/query-ASTERIXDB-1577/query-ASTERIXDB-1577.2.update.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * 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 sampdb;
+
+insert into samptable1 ({'id' : 0});
+
+insert into samptable1 ({'id' : 1});
diff --git 
a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/query-ASTERIXDB-1577/query-ASTERIXDB-1577.3.query.sqlpp
 
b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/query-ASTERIXDB-1577/query-ASTERIXDB-1577.3.query.sqlpp
new file mode 100644
index 0000000..adf07cd
--- /dev/null
+++ 
b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/query-ASTERIXDB-1577/query-ASTERIXDB-1577.3.query.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * 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 sampdb;
+
+select id
+from samptable1 s2
+where samptable1.id = 1;
diff --git 
a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/query-ASTERIXDB-1577/query-ASTERIXDB-1577.4.ddl.sqlpp
 
b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/query-ASTERIXDB-1577/query-ASTERIXDB-1577.4.ddl.sqlpp
new file mode 100644
index 0000000..e866e65
--- /dev/null
+++ 
b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/query-ASTERIXDB-1577/query-ASTERIXDB-1577.4.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 sampdb if exists;
diff --git 
a/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate/query-ASTERIXDB-159/query-ASTERIXDB-159.1.adm
 
b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate/query-ASTERIXDB-159/query-ASTERIXDB-159.1.adm
new file mode 100644
index 0000000..f599e28
--- /dev/null
+++ 
b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate/query-ASTERIXDB-159/query-ASTERIXDB-159.1.adm
@@ -0,0 +1 @@
+10
diff --git 
a/asterixdb/asterix-app/src/test/resources/runtimets/results/global-aggregate/q11/q11.1.adm
 
b/asterixdb/asterix-app/src/test/resources/runtimets/results/global-aggregate/q11/q11.1.adm
new file mode 100644
index 0000000..f599e28
--- /dev/null
+++ 
b/asterixdb/asterix-app/src/test/resources/runtimets/results/global-aggregate/q11/q11.1.adm
@@ -0,0 +1 @@
+10
diff --git 
a/asterixdb/asterix-app/src/test/resources/runtimets/results/list/list-dataset/list-dataset.1.adm
 
b/asterixdb/asterix-app/src/test/resources/runtimets/results/list/list-dataset/list-dataset.1.adm
new file mode 100644
index 0000000..ca2abdf
--- /dev/null
+++ 
b/asterixdb/asterix-app/src/test/resources/runtimets/results/list/list-dataset/list-dataset.1.adm
@@ -0,0 +1 @@
+[ { "id": 6, "alias": "Willis", "name": "WillisWynne", "user-since": 
datetime("2005-01-17T10:10:00.000Z"), "friend-ids": {{ 1, 3, 7 }}, 
"employment": [ { "organization-name": "jaydax", "start-date": 
date("2009-05-15") } ] }, { "id": 1, "alias": "Margarita", "name": 
"MargaritaStoddard", "user-since": datetime("2012-08-20T10:10:00.000Z"), 
"friend-ids": {{ 2, 3, 6, 10 }}, "employment": [ { "organization-name": 
"Codetechno", "start-date": date("2006-08-06") } ] }, { "id": 2, "alias": 
"Isbel", "name": "IsbelDull", "user-since": 
datetime("2011-01-22T10:10:00.000Z"), "friend-ids": {{ 1, 4 }}, "employment": [ 
{ "organization-name": "Hexviafind", "start-date": date("2010-04-27") } ] }, { 
"id": 4, "alias": "Nicholas", "name": "NicholasStroh", "user-since": 
datetime("2010-12-27T10:10:00.000Z"), "friend-ids": {{ 2 }}, "employment": [ { 
"organization-name": "Zamcorporation", "start-date": date("2010-06-08") } ] }, 
{ "id": 8, "alias": "Nila", "name": "NilaMilliron", "user-since": datetime
 ("2008-01-01T10:10:00.000Z"), "friend-ids": {{ 3 }}, "employment": [ { 
"organization-name": "Plexlane", "start-date": date("2010-02-28") } ] }, { 
"id": 9, "alias": "Woodrow", "name": "WoodrowNehling", "user-since": 
datetime("2005-09-20T10:10:00.000Z"), "friend-ids": {{ 3, 10 }}, "employment": 
[ { "organization-name": "Zuncan", "start-date": date("2003-04-22"), 
"end-date": date("2009-12-13") } ] }, { "id": 10, "alias": "Bram", "name": 
"BramHatch", "user-since": datetime("2010-10-16T10:10:00.000Z"), "friend-ids": 
{{ 1, 5, 9 }}, "employment": [ { "organization-name": "physcane", "start-date": 
date("2007-06-05"), "end-date": date("2011-11-05") } ] }, { "id": 3, "alias": 
"Emory", "name": "EmoryUnk", "user-since": 
datetime("2012-07-10T10:10:00.000Z"), "friend-ids": {{ 1, 5, 8, 9 }}, 
"employment": [ { "organization-name": "geomedia", "start-date": 
date("2010-06-17"), "end-date": date("2010-01-26") } ] }, { "id": 5, "alias": 
"Von", "name": "VonKemble", "user-since": datetime("2010-0
 1-05T10:10:00.000Z"), "friend-ids": {{ 3, 6, 10 }}, "employment": [ { 
"organization-name": "Kongreen", "start-date": date("2010-11-27") } ] }, { 
"id": 7, "alias": "Suzanna", "name": "SuzannaTillson", "user-since": 
datetime("2012-08-07T10:10:00.000Z"), "friend-ids": {{ 6 }}, "employment": [ { 
"organization-name": "Labzatron", "start-date": date("2011-04-19") } ] } ]
diff --git 
a/asterixdb/asterix-app/src/test/resources/runtimets/results/list/list-range/list-range.1.adm
 
b/asterixdb/asterix-app/src/test/resources/runtimets/results/list/list-range/list-range.1.adm
new file mode 100644
index 0000000..18dde9f
--- /dev/null
+++ 
b/asterixdb/asterix-app/src/test/resources/runtimets/results/list/list-range/list-range.1.adm
@@ -0,0 +1 @@
+[ 0, 1, 2, 3, 4, 5 ]
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite.xml 
b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite.xml
index e5710bc..0838299 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite.xml
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite.xml
@@ -868,6 +868,11 @@
         <output-dir compare="Text">query-ASTERIXDB-1230</output-dir>
       </compilation-unit>
     </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="query-ASTERIXDB-159">
+        <output-dir compare="Text">query-ASTERIXDB-159</output-dir>
+      </compilation-unit>
+    </test-case>
   </test-group>
   <test-group name="aggregate-sql">
     <test-case FilePath="aggregate-sql">
diff --git 
a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml 
b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
index 9548259..d083666 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
@@ -2439,6 +2439,11 @@
         <output-dir compare="Text">q09</output-dir>
       </compilation-unit>
     </test-case>
+    <test-case FilePath="global-aggregate">
+      <compilation-unit name="q11">
+        <output-dir compare="Text">q11</output-dir>
+      </compilation-unit>
+    </test-case>
   </test-group>
   <test-group name="group-by">
     <test-case FilePath="group-by">
@@ -2878,6 +2883,16 @@
       </compilation-unit>
     </test-case>
     <test-case FilePath="list">
+      <compilation-unit name="list-dataset">
+        <output-dir compare="Text">list-dataset</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="list">
+      <compilation-unit name="list-range">
+        <output-dir compare="Text">list-range</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="list">
       <compilation-unit name="listify_01">
         <output-dir compare="Text">listify_01</output-dir>
       </compilation-unit>
@@ -3072,6 +3087,12 @@
         <output-dir compare="Text">query-ASTERIXDB-1531</output-dir>
       </compilation-unit>
     </test-case>
+    <test-case FilePath="misc">
+      <compilation-unit name="query-ASTERIXDB-1577">
+        <output-dir compare="Text">query-ASTERIXDB-1577</output-dir>
+        <expected-error>The first argument of a field access should be a 
RECORD, but it is</expected-error>
+      </compilation-unit>
+    </test-case>
   </test-group>
   <test-group name="open-index-enforced">
     <test-group FilePath="open-index-enforced/error-checking">
diff --git 
a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/AsterixBuiltinFunctions.java
 
b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/AsterixBuiltinFunctions.java
index d17be1a..5ea52aa 100644
--- 
a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/AsterixBuiltinFunctions.java
+++ 
b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/AsterixBuiltinFunctions.java
@@ -287,6 +287,8 @@
             "string-join", 2);
 
     public static final FunctionIdentifier DATASET = new 
FunctionIdentifier(FunctionConstants.ASTERIX_NS, "dataset", 1);
+    public static final FunctionIdentifier LISTIFY_DATASET = new 
FunctionIdentifier(FunctionConstants.ASTERIX_NS,
+            "listify-dataset", 1);
     public static final FunctionIdentifier FEED_COLLECT = new 
FunctionIdentifier(FunctionConstants.ASTERIX_NS,
             "feed-collect", 6);
     public static final FunctionIdentifier FEED_INTERCEPT = new 
FunctionIdentifier(FunctionConstants.ASTERIX_NS,

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/1113
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: newchange
Gerrit-Change-Id: Id0dc4db91a6251d55dafd734d9ea5bfb6c11c315
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Yingyi Bu <buyin...@gmail.com>

Reply via email to