ASTERIXDB-1577: fix error handling for field access over a dataset.
- add a rewrite rule to handle unnesting functions that do not appear
in an unnest operator or left outer unnest operator;
- ASTERIXDB-159 is fixed as well.
Change-Id: Id0dc4db91a6251d55dafd734d9ea5bfb6c11c315
Reviewed-on: https://asterix-gerrit.ics.uci.edu/1113
Sonar-Qube: Jenkins <jenkins@fulliautomatix.ics.uci.edu>
Tested-by: Jenkins <jenkins@fulliautomatix.ics.uci.edu>
Integration-Tests: Jenkins <jenkins@fulliautomatix.ics.uci.edu>
Reviewed-by: Yingyi Bu <buyingyi@gmail.com>
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 1b11cde..cf36c05 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.ListifyUnnestingFunctionRule;
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;
@@ -178,6 +179,7 @@
// Let PushAggFuncIntoStandaloneAggregateRule run after ExtractCommonExpressionsRule
// so that PushAggFunc can happen in fewer places.
normalization.add(new PushAggFuncIntoStandaloneAggregateRule());
+ normalization.add(new ListifyUnnestingFunctionRule());
normalization.add(new ConstantFoldingRule());
normalization.add(new RemoveRedundantSelectRule());
normalization.add(new UnnestToDataScanRule(algebraExtensionManager));
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ListifyUnnestingFunctionRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ListifyUnnestingFunctionRule.java
new file mode 100644
index 0000000..caaddd4
--- /dev/null
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ListifyUnnestingFunctionRule.java
@@ -0,0 +1,150 @@
+/*
+ * 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.NestedTupleSourceOperator;
+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 of 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 or left outer unnest operator,
+ * it is invalid and we need to extract it out into an unnest operator and then listify the unnested sequence of items
+ * so that the listified collection can replace the original call of the unnesting function.
+ * <p/>
+ *
+ * Assuming FacebookUsers is a dataset. Example queries:
+ * <p/>
+ * COUNT(FacebookUsers);
+ * <p/>
+ * FacebookUsers;
+ * <p/>
+ * This rule performs the aforementioned transformations.
+ */
+public class ListifyUnnestingFunctionRule 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 || op.getInputs().size() > 1) {
+ 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()) {
+ changed = changed || rewriteExpressionReference(op, funcArgRef, context);
+ }
+
+ // 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();
+
+ // Checks if the function is an unnesting function.
+ if (!AsterixBuiltinFunctions.isBuiltinUnnestingFunction(functionInfo.getFunctionIdentifier())) {
+ return false;
+ }
+
+ // Generates the listified collection in a subplan.
+ SubplanOperator subplanOperator = new SubplanOperator();
+ // Creates a nested tuple source operator.
+ NestedTupleSourceOperator ntsOperator = new NestedTupleSourceOperator(new MutableObject<>(subplanOperator));
+
+ // 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<>(ntsOperator));
+
+ // 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));
+
+
+ // Adds the aggregate operator as the root of the subplan.
+ subplanOperator.setRootOp(new MutableObject<>(aggregateOperator));
+
+ // Sticks a subplan operator into the query plan.
+ // Note: given the way we compile JOINs, the unnesting function expression cannot appear in
+ // any binary operators.
+ // Example test queries:
+ // asterixdb/asterix-app/src/test/resources/runtimets/results/list/query-ASTERIXDB-159-2
+ // asterixdb/asterix-app/src/test/resources/runtimets/results/list/query-ASTERIXDB-159-3
+ subplanOperator.getInputs().add(op.getInputs().get(0));
+ op.getInputs().set(0, new MutableObject<>(subplanOperator));
+ exprRef.setValue(new VariableReferenceExpression(aggVar));
+
+ // Computes type environments for new operators.
+ context.computeAndSetTypeEnvironmentForOperator(ntsOperator);
+ context.computeAndSetTypeEnvironmentForOperator(unnestOperator);
+ context.computeAndSetTypeEnvironmentForOperator(aggregateOperator);
+ context.computeAndSetTypeEnvironmentForOperator(subplanOperator);
+ return true;
+ }
+
+}
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/aggregate/count-dataset.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/aggregate/count-dataset.sqlpp
new file mode 100644
index 0000000..a224428
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/aggregate/count-dataset.sqlpp
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop dataverse TinySocial if exists;
+create dataverse TinySocial;
+
+use TinySocial;
+
+
+create type TinySocial.FacebookUserType as
+ open {
+ id : int64
+}
+
+
+create dataset FacebookUsers(FacebookUserType) primary key id;
+
+COLL_COUNT(FacebookUsers);
+
+drop dataverse TinySocial;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/aggregate/list-dataset.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/aggregate/list-dataset.sqlpp
new file mode 100644
index 0000000..f738fad
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/aggregate/list-dataset.sqlpp
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop dataverse TinySocial if exists;
+create dataverse TinySocial;
+
+use TinySocial;
+
+
+create type TinySocial.FacebookUserType as
+ open {
+ id : int64
+}
+
+
+create dataset FacebookUsers(FacebookUserType) primary key id;
+
+FacebookUsers;
+
+drop dataverse TinySocial;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query-ASTERIXDB-159-3.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query-ASTERIXDB-159-3.sqlpp
new file mode 100644
index 0000000..5b87757
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query-ASTERIXDB-159-3.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 TinySocial if exists;
+create dataverse TinySocial;
+
+use TinySocial;
+
+create type TinySocial.FacebookUserType as
+ open {
+ id : int64
+}
+
+create dataset FacebookUsers(FacebookUserType) primary key id;
+
+SELECT fu1.id id1, fu2.id id2
+FROM FacebookUsers fu1
+LEFT OUTER JOIN FacebookUsers fu2
+ON COLL_COUNT(RANGE(fu1.id, fu2.id))>2 AND COLL_COUNT(RANGE(fu1.id, fu2.id))<5
+ORDER BY fu1.id DESC, fu2.id DESC
+LIMIT 5;
+
+drop dataverse TinySocial;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/aggregate/count-dataset.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/aggregate/count-dataset.plan
new file mode 100644
index 0000000..81efcc2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/aggregate/count-dataset.plan
@@ -0,0 +1,10 @@
+-- DISTRIBUTE_RESULT |UNPARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ -- AGGREGATE |UNPARTITIONED|
+ -- RANDOM_MERGE_EXCHANGE |PARTITIONED|
+ -- AGGREGATE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/aggregate/list-dataset.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/aggregate/list-dataset.plan
new file mode 100644
index 0000000..0740dea
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/aggregate/list-dataset.plan
@@ -0,0 +1,9 @@
+-- DISTRIBUTE_RESULT |UNPARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ -- AGGREGATE |UNPARTITIONED|
+ -- RANDOM_MERGE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-ASTERIXDB-159-3.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-ASTERIXDB-159-3.plan
new file mode 100644
index 0000000..7699b8e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-ASTERIXDB-159-3.plan
@@ -0,0 +1,63 @@
+-- DISTRIBUTE_RESULT |UNPARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ -- STREAM_LIMIT |UNPARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$29(DESC), $$30(DESC) ] |PARTITIONED|
+ -- STREAM_LIMIT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [topK: 5] [$$29(DESC), $$30(DESC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$29][$$37] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- SPLIT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- SPLIT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$37] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- SUBPLAN |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- UNNEST |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- NESTED_LOOP |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- SPLIT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- SPLIT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- SPLIT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
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/aggregate/query-ASTERIXDB-159/query-ASTERIXDB-159.4.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/aggregate/query-ASTERIXDB-159/query-ASTERIXDB-159.4.ddl.aql
new file mode 100644
index 0000000..2228161
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/aggregate/query-ASTERIXDB-159/query-ASTERIXDB-159.4.ddl.aql
@@ -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 TinySocial if exists;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/global-aggregate/query-ASTERIXDB-159/query-ASTERIXDB-159.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/global-aggregate/query-ASTERIXDB-159/query-ASTERIXDB-159.1.ddl.sqlpp
new file mode 100644
index 0000000..78ce34a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/global-aggregate/query-ASTERIXDB-159/query-ASTERIXDB-159.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/query-ASTERIXDB-159/query-ASTERIXDB-159.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/global-aggregate/query-ASTERIXDB-159/query-ASTERIXDB-159.2.update.sqlpp
new file mode 100644
index 0000000..9f4f824
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/global-aggregate/query-ASTERIXDB-159/query-ASTERIXDB-159.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/query-ASTERIXDB-159/query-ASTERIXDB-159.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/global-aggregate/query-ASTERIXDB-159/query-ASTERIXDB-159.3.query.sqlpp
new file mode 100644
index 0000000..2340771
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/global-aggregate/query-ASTERIXDB-159/query-ASTERIXDB-159.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/global-aggregate/query-ASTERIXDB-159/query-ASTERIXDB-159.4.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/global-aggregate/query-ASTERIXDB-159/query-ASTERIXDB-159.4.ddl.sqlpp
new file mode 100644
index 0000000..3509f58
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/global-aggregate/query-ASTERIXDB-159/query-ASTERIXDB-159.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 TinySocial if exists;
\ No newline at end of file
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/list/query-ASTERIXDB-159-2/query-ASTERIXDB-159-2.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/list/query-ASTERIXDB-159-2/query-ASTERIXDB-159-2.1.ddl.sqlpp
new file mode 100644
index 0000000..78ce34a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/list/query-ASTERIXDB-159-2/query-ASTERIXDB-159-2.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/query-ASTERIXDB-159-2/query-ASTERIXDB-159-2.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/list/query-ASTERIXDB-159-2/query-ASTERIXDB-159-2.2.update.sqlpp
new file mode 100644
index 0000000..9f4f824
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/list/query-ASTERIXDB-159-2/query-ASTERIXDB-159-2.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/query-ASTERIXDB-159-2/query-ASTERIXDB-159-2.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/list/query-ASTERIXDB-159-2/query-ASTERIXDB-159-2.3.query.sqlpp
new file mode 100644
index 0000000..0954239
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/list/query-ASTERIXDB-159-2/query-ASTERIXDB-159-2.3.query.sqlpp
@@ -0,0 +1,27 @@
+/*
+ * 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;
+
+SELECT fu1.id id1, fu2.id id2
+FROM FacebookUsers fu1
+JOIN FacebookUsers fu2
+ON COLL_COUNT(RANGE(fu1.id, fu2.id))<5
+ORDER BY fu1.id, fu2.id
+LIMIT 5;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/list/query-ASTERIXDB-159-2/query-ASTERIXDB-159-2.4.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/list/query-ASTERIXDB-159-2/query-ASTERIXDB-159-2.4.ddl.sqlpp
new file mode 100644
index 0000000..3509f58
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/list/query-ASTERIXDB-159-2/query-ASTERIXDB-159-2.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 TinySocial if exists;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/list/query-ASTERIXDB-159-3/query-ASTERIXDB-159-3.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/list/query-ASTERIXDB-159-3/query-ASTERIXDB-159-3.1.ddl.sqlpp
new file mode 100644
index 0000000..78ce34a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/list/query-ASTERIXDB-159-3/query-ASTERIXDB-159-3.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/query-ASTERIXDB-159-3/query-ASTERIXDB-159-3.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/list/query-ASTERIXDB-159-3/query-ASTERIXDB-159-3.2.update.sqlpp
new file mode 100644
index 0000000..9f4f824
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/list/query-ASTERIXDB-159-3/query-ASTERIXDB-159-3.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/query-ASTERIXDB-159-3/query-ASTERIXDB-159-3.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/list/query-ASTERIXDB-159-3/query-ASTERIXDB-159-3.3.query.sqlpp
new file mode 100644
index 0000000..a932f2a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/list/query-ASTERIXDB-159-3/query-ASTERIXDB-159-3.3.query.sqlpp
@@ -0,0 +1,27 @@
+/*
+ * 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;
+
+SELECT fu1.id id1, fu2.id id2
+FROM FacebookUsers fu1
+LEFT OUTER JOIN FacebookUsers fu2
+ON COLL_COUNT(RANGE(fu1.id, fu2.id))>2 AND COLL_COUNT(RANGE(fu1.id, fu2.id))<5
+ORDER BY fu1.id DESC, fu2.id DESC
+LIMIT 5;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/list/query-ASTERIXDB-159-3/query-ASTERIXDB-159-3.4.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/list/query-ASTERIXDB-159-3/query-ASTERIXDB-159-3.4.ddl.sqlpp
new file mode 100644
index 0000000..3509f58
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/list/query-ASTERIXDB-159-3/query-ASTERIXDB-159-3.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 TinySocial if exists;
\ No newline at end of file
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/query-ASTERIXDB-159/query-ASTERIXDB-159.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/global-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/global-aggregate/query-ASTERIXDB-159/query-ASTERIXDB-159.1.adm
@@ -0,0 +1 @@
+10
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/results/list/query-ASTERIXDB-159-2/query-ASTERIXDB-159-2.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/list/query-ASTERIXDB-159-2/query-ASTERIXDB-159-2.1.adm
new file mode 100644
index 0000000..3c73115
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/list/query-ASTERIXDB-159-2/query-ASTERIXDB-159-2.1.adm
@@ -0,0 +1,5 @@
+{ "id1": 1, "id2": 1 }
+{ "id1": 1, "id2": 2 }
+{ "id1": 1, "id2": 3 }
+{ "id1": 1, "id2": 4 }
+{ "id1": 2, "id2": 1 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/list/query-ASTERIXDB-159-3/query-ASTERIXDB-159-3.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/list/query-ASTERIXDB-159-3/query-ASTERIXDB-159-3.1.adm
new file mode 100644
index 0000000..f335ba7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/list/query-ASTERIXDB-159-3/query-ASTERIXDB-159-3.1.adm
@@ -0,0 +1,5 @@
+{ "id1": 10 }
+{ "id1": 9 }
+{ "id1": 8, "id2": 10 }
+{ "id1": 7, "id2": 10 }
+{ "id1": 7, "id2": 9 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite.xml b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite.xml
index cd54be8..9e4a3c4 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 608923e..b48a99c 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
@@ -2460,6 +2460,11 @@
<expected-error>The parameter * can only be used in COUNT().</expected-error>
</compilation-unit>
</test-case>
+ <test-case FilePath="global-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="group-by">
<test-case FilePath="group-by">
@@ -2899,6 +2904,11 @@
</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>
@@ -2964,6 +2974,16 @@
<output-dir compare="Text">query-issue428</output-dir>
</compilation-unit>
</test-case>
+ <test-case FilePath="list">
+ <compilation-unit name="query-ASTERIXDB-159-2">
+ <output-dir compare="Text">query-ASTERIXDB-159-2</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="list">
+ <compilation-unit name="query-ASTERIXDB-159-3">
+ <output-dir compare="Text">query-ASTERIXDB-159-3</output-dir>
+ </compilation-unit>
+ </test-case>
</test-group>
<test-group name="misc">
<test-case FilePath="misc">
@@ -3093,6 +3113,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">