You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@asterixdb.apache.org by "Yingyi Bu (Code Review)" <do...@asterixdb.incubator.apache.org> on 2016/02/09 20:42:02 UTC

Change in asterixdb[master]: Fix for ASTERIXDB-1291: added regression tests.

Yingyi Bu has uploaded a new change for review.

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

Change subject: Fix for ASTERIXDB-1291: added regression tests.
......................................................................

Fix for ASTERIXDB-1291: added regression tests.

Change-Id: I1c8bbaf2522d421dff9a22af28c160f406dd2888
---
M asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/FuzzyJoinRule.java
M asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/InvertedIndexAccessMethod.java
M asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/InlineAllNtsInSubplanVisitor.java
A asterix-app/src/test/resources/optimizerts/queries/gby_inline.aql
A asterix-app/src/test/resources/optimizerts/results/gby_inline.plan
A asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/gby_inline/gby_inline.1.ddl.sqlpp
A asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/gby_inline/gby_inline.2.update.sqlpp
A asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/gby_inline/gby_inline.3.query.sqlpp
A asterix-app/src/test/resources/runtimets/results/subquery/gby_inline/gby_inline.1.adm
M asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
10 files changed, 437 insertions(+), 22 deletions(-)


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

diff --git a/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/FuzzyJoinRule.java b/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/FuzzyJoinRule.java
index 1741008..99a418e 100644
--- a/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/FuzzyJoinRule.java
+++ b/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/FuzzyJoinRule.java
@@ -276,7 +276,7 @@
         translator.addVariableToMetaScope(new Identifier("$$RIGHT"), rightInputVar);
         translator.addVariableToMetaScope(new Identifier("$$RIGHTPK"), rightPKVar);
 
-        translator.addOperatorToMetaScope(new Identifier("#LEFT_1"), deepCopyVisitor.deepCopy(leftInputOp, null));
+        translator.addOperatorToMetaScope(new Identifier("#LEFT_1"), deepCopyVisitor.deepCopy(leftInputOp));
         translator.addVariableToMetaScope(new Identifier("$$LEFT_1"), deepCopyVisitor.varCopy(leftInputVar));
         translator.addVariableToMetaScope(new Identifier("$$LEFTPK_1"), deepCopyVisitor.varCopy(leftPKVar));
         deepCopyVisitor.updatePrimaryKeys(context);
@@ -300,20 +300,20 @@
         // deepCopyVisitor.updatePrimaryKeys(context);
         // deepCopyVisitor.reset();
 
-        translator.addOperatorToMetaScope(new Identifier("#RIGHT_1"), deepCopyVisitor.deepCopy(rightInputOp, null));
+        translator.addOperatorToMetaScope(new Identifier("#RIGHT_1"), deepCopyVisitor.deepCopy(rightInputOp));
         translator.addVariableToMetaScope(new Identifier("$$RIGHT_1"), deepCopyVisitor.varCopy(rightInputVar));
         translator.addVariableToMetaScope(new Identifier("$$RIGHTPK_1"), deepCopyVisitor.varCopy(rightPKVar));
         deepCopyVisitor.updatePrimaryKeys(context);
         deepCopyVisitor.reset();
 
         // TODO pick side to run Stage 1, currently always picks RIGHT side
-        translator.addOperatorToMetaScope(new Identifier("#RIGHT_2"), deepCopyVisitor.deepCopy(rightInputOp, null));
+        translator.addOperatorToMetaScope(new Identifier("#RIGHT_2"), deepCopyVisitor.deepCopy(rightInputOp));
         translator.addVariableToMetaScope(new Identifier("$$RIGHT_2"), deepCopyVisitor.varCopy(rightInputVar));
         translator.addVariableToMetaScope(new Identifier("$$RIGHTPK_2"), deepCopyVisitor.varCopy(rightPKVar));
         deepCopyVisitor.updatePrimaryKeys(context);
         deepCopyVisitor.reset();
 
-        translator.addOperatorToMetaScope(new Identifier("#RIGHT_3"), deepCopyVisitor.deepCopy(rightInputOp, null));
+        translator.addOperatorToMetaScope(new Identifier("#RIGHT_3"), deepCopyVisitor.deepCopy(rightInputOp));
         translator.addVariableToMetaScope(new Identifier("$$RIGHT_3"), deepCopyVisitor.varCopy(rightInputVar));
         translator.addVariableToMetaScope(new Identifier("$$RIGHTPK_3"), deepCopyVisitor.varCopy(rightPKVar));
         deepCopyVisitor.updatePrimaryKeys(context);
diff --git a/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/InvertedIndexAccessMethod.java b/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/InvertedIndexAccessMethod.java
index 851264c..d587649 100644
--- a/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/InvertedIndexAccessMethod.java
+++ b/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/InvertedIndexAccessMethod.java
@@ -303,8 +303,9 @@
                         (IAType) AqlExpressionTypeComputer.INSTANCE.getType(arg3, null, null) });
         for (IOptimizableFuncExpr optFuncExpr : analysisCtx.matchedFuncExprs) {
             //avoid additional optFuncExpressions in case of a join
-            if (optFuncExpr.getFuncExpr().equals(funcExpr))
+            if (optFuncExpr.getFuncExpr().equals(funcExpr)) {
                 return true;
+            }
         }
         analysisCtx.matchedFuncExprs.add(newOptFuncExpr);
         return true;
@@ -604,13 +605,13 @@
         // Create first copy.
         LogicalOperatorDeepCopyWithNewVariablesVisitor firstDeepCopyVisitor = new LogicalOperatorDeepCopyWithNewVariablesVisitor(
                 context, newProbeSubTreeVarMap);
-        ILogicalOperator newProbeSubTree = firstDeepCopyVisitor.deepCopy(probeSubTree.root, null);
+        ILogicalOperator newProbeSubTree = firstDeepCopyVisitor.deepCopy(probeSubTree.root);
         inferTypes(newProbeSubTree, context);
         Mutable<ILogicalOperator> newProbeSubTreeRootRef = new MutableObject<ILogicalOperator>(newProbeSubTree);
         // Create second copy.
         LogicalOperatorDeepCopyWithNewVariablesVisitor secondDeepCopyVisitor = new LogicalOperatorDeepCopyWithNewVariablesVisitor(
                 context, joinInputSubTreeVarMap);
-        ILogicalOperator joinInputSubTree = secondDeepCopyVisitor.deepCopy(probeSubTree.root, null);
+        ILogicalOperator joinInputSubTree = secondDeepCopyVisitor.deepCopy(probeSubTree.root);
         inferTypes(joinInputSubTree, context);
         probeSubTree.rootRef.setValue(joinInputSubTree);
 
@@ -672,10 +673,11 @@
         // Create select ops for removing tuples that are filterable and not filterable, respectively.
         IVariableTypeEnvironment probeTypeEnv = context.getOutputTypeEnvironment(probeSubTree.root);
         IAType inputSearchVarType;
-        if (chosenIndex.isEnforcingKeyFileds())
+        if (chosenIndex.isEnforcingKeyFileds()) {
             inputSearchVarType = optFuncExpr.getFieldType(optFuncExpr.findLogicalVar(inputSearchVar));
-        else
+        } else {
             inputSearchVarType = (IAType) probeTypeEnv.getVarType(inputSearchVar);
+        }
         Mutable<ILogicalOperator> isFilterableSelectOpRef = new MutableObject<ILogicalOperator>();
         Mutable<ILogicalOperator> isNotFilterableSelectOpRef = new MutableObject<ILogicalOperator>();
         createIsFilterableSelectOps(replicateOp, inputSearchVar, inputSearchVarType, optFuncExpr, chosenIndex, context,
@@ -687,7 +689,7 @@
         // Copy the scan subtree in indexSubTree.
         LogicalOperatorDeepCopyWithNewVariablesVisitor deepCopyVisitor = new LogicalOperatorDeepCopyWithNewVariablesVisitor(
                 context);
-        ILogicalOperator scanSubTree = deepCopyVisitor.deepCopy(indexSubTree.root, null);
+        ILogicalOperator scanSubTree = deepCopyVisitor.deepCopy(indexSubTree.root);
 
         Map<LogicalVariable, LogicalVariable> copyVarMap = deepCopyVisitor.getInputToOutputVariableMapping();
         panicVarMap.putAll(copyVarMap);
@@ -871,10 +873,11 @@
     }
 
     private boolean isEditDistanceFuncJoinOptimizable(Index index, IOptimizableFuncExpr optFuncExpr) {
-        if (index.isEnforcingKeyFileds())
+        if (index.isEnforcingKeyFileds()) {
             return isEditDistanceFuncCompatible(index.getKeyFieldTypes().get(0).getTypeTag(), index.getIndexType());
-        else
+        } else {
             return isEditDistanceFuncCompatible(optFuncExpr.getFieldType(0).getTypeTag(), index.getIndexType());
+        }
     }
 
     private boolean isEditDistanceFuncCompatible(ATypeTag typeTag, IndexType indexType) {
@@ -972,11 +975,13 @@
         LogicalVariable targetVar = null;
         for (int i = 0; i < variableCount; i++) {
             subTree = optFuncExpr.getOperatorSubTree(i);
-            if (subTree == null)
+            if (subTree == null) {
                 continue;
+            }
             targetVar = optFuncExpr.getLogicalVar(i);
-            if (targetVar == null)
+            if (targetVar == null) {
                 continue;
+            }
             return isJaccardFuncCompatible(optFuncExpr.getFuncExpr().getArguments().get(i).getValue(),
                     optFuncExpr.getFieldType(i).getTypeTag(), index.getIndexType());
         }
@@ -1001,20 +1006,25 @@
         }
 
         for (AbstractLogicalOperator op : subTree.assignsAndUnnests) {
-            if (op.getOperatorTag() != LogicalOperatorTag.ASSIGN)
+            if (op.getOperatorTag() != LogicalOperatorTag.ASSIGN) {
                 continue;
+            }
             List<Mutable<ILogicalExpression>> exprList = ((AssignOperator) op).getExpressions();
             for (Mutable<ILogicalExpression> expr : exprList) {
-                if (expr.getValue().getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL)
+                if (expr.getValue().getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
                     continue;
+                }
                 AbstractFunctionCallExpression funcExpr = (AbstractFunctionCallExpression) expr.getValue();
-                if (funcExpr.getFunctionIdentifier() != funcId)
+                if (funcExpr.getFunctionIdentifier() != funcId) {
                     continue;
+                }
                 ILogicalExpression varExpr = funcExpr.getArguments().get(0).getValue();
-                if (varExpr.getExpressionTag() != LogicalExpressionTag.VARIABLE)
+                if (varExpr.getExpressionTag() != LogicalExpressionTag.VARIABLE) {
                     continue;
-                if (((VariableReferenceExpression) varExpr).getVariableReference() == targetVar)
+                }
+                if (((VariableReferenceExpression) varExpr).getVariableReference() == targetVar) {
                     continue;
+                }
                 return (ScalarFunctionCallExpression) funcExpr;
             }
         }
@@ -1077,10 +1087,11 @@
     }
 
     private boolean isContainsFuncJoinOptimizable(Index index, IOptimizableFuncExpr optFuncExpr) {
-        if (index.isEnforcingKeyFileds())
+        if (index.isEnforcingKeyFileds()) {
             return isContainsFuncCompatible(index.getKeyFieldTypes().get(0).getTypeTag(), index.getIndexType());
-        else
+        } else {
             return isContainsFuncCompatible(optFuncExpr.getFieldType(0).getTypeTag(), index.getIndexType());
+        }
     }
 
     private boolean isContainsFuncCompatible(ATypeTag typeTag, IndexType indexType) {
diff --git a/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/InlineAllNtsInSubplanVisitor.java b/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/InlineAllNtsInSubplanVisitor.java
index 3489982..42905ee 100644
--- a/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/InlineAllNtsInSubplanVisitor.java
+++ b/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/InlineAllNtsInSubplanVisitor.java
@@ -403,7 +403,7 @@
         }
         LogicalOperatorDeepCopyWithNewVariablesVisitor deepCopyVisitor = new LogicalOperatorDeepCopyWithNewVariablesVisitor(
                 context);
-        ILogicalOperator copiedInputOperator = deepCopyVisitor.deepCopy(subplanInputOperator, null);
+        ILogicalOperator copiedInputOperator = deepCopyVisitor.deepCopy(subplanInputOperator);
 
         // Updates the primary key info in the copied plan segment.
         Map<LogicalVariable, LogicalVariable> varMap = deepCopyVisitor.getInputToOutputVariableMapping();
diff --git a/asterix-app/src/test/resources/optimizerts/queries/gby_inline.aql b/asterix-app/src/test/resources/optimizerts/queries/gby_inline.aql
new file mode 100644
index 0000000..bb29875
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/queries/gby_inline.aql
@@ -0,0 +1,84 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/**
+ * This query is to verify the fix for ASTERIXDB-1291.
+ */
+
+drop dataverse tpch if exists;
+create dataverse tpch;
+use dataverse tpch;
+
+create type LineItemType as closed {
+  l_orderkey: int32,
+  l_partkey: int32,
+  l_suppkey: int32,
+  l_linenumber: int32,
+  l_quantity: double,
+  l_extendedprice: double,
+  l_discount: double,
+  l_tax: double,
+  l_returnflag: string,
+  l_linestatus: string,
+  l_shipdate: string,
+  l_commitdate: string,
+  l_receiptdate: string,
+  l_shipinstruct: string,
+  l_shipmode: string,
+  l_comment: string
+}
+
+create type OrderType as closed {
+  o_orderkey: int32,
+  o_custkey: int32,
+  o_orderstatus: string,
+  o_totalprice: double,
+  o_orderdate: string,
+  o_orderpriority: string,
+  o_clerk: string,
+  o_shippriority: int32,
+  o_comment: string
+}
+
+create type CustomerType as closed {
+  c_custkey: int32,
+  c_name: string,
+  c_address: string,
+  c_nationkey: int32,
+  c_phone: string,
+  c_acctbal: double,
+  c_mktsegment: string,
+  c_comment: string
+}
+
+create dataset Orders(OrderType)
+  primary key o_orderkey;
+create dataset Customers(CustomerType)
+  primary key c_custkey;
+
+for $c in dataset('Customers')
+group by $ccustkey := $c.c_custkey, $cnationkey := $c.c_nationkey with $c
+let $orders :=
+  for $o in dataset('Orders')
+  where $cnationkey = 5 and $o.o_$o.o_custkey = $ccustkey
+  return $o
+return {
+  "cust": $c,
+  "orders": $orders
+}
diff --git a/asterix-app/src/test/resources/optimizerts/results/gby_inline.plan b/asterix-app/src/test/resources/optimizerts/results/gby_inline.plan
new file mode 100644
index 0000000..b3f4ac2
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/results/gby_inline.plan
@@ -0,0 +1,58 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- PRE_CLUSTERED_GROUP_BY[$$1]  |PARTITIONED|
+                    {
+                      -- AGGREGATE  |LOCAL|
+                        -- STREAM_SELECT  |LOCAL|
+                          -- NESTED_TUPLE_SOURCE  |LOCAL|
+                    }
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- STABLE_SORT [$$1(ASC)]  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- HYBRID_HASH_JOIN [$$1, $$2][$$24, $$26]  |PARTITIONED|
+                            -- HASH_PARTITION_EXCHANGE [$$1]  |PARTITIONED|
+                              -- PRE_CLUSTERED_GROUP_BY[$$20]  |PARTITIONED|
+                                      {
+                                        -- AGGREGATE  |LOCAL|
+                                          -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                      }
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- ASSIGN  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- DATASOURCE_SCAN  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- ASSIGN  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- HYBRID_HASH_JOIN [$$24][$$22]  |PARTITIONED|
+                                      -- HASH_PARTITION_EXCHANGE [$$24]  |PARTITIONED|
+                                        -- STREAM_SELECT  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- PRE_CLUSTERED_GROUP_BY[$$25]  |PARTITIONED|
+                                                    {
+                                                      -- AGGREGATE  |LOCAL|
+                                                        -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                    }
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                  -- ASSIGN  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- DATASOURCE_SCAN  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                      -- HASH_PARTITION_EXCHANGE [$$22]  |PARTITIONED|
+                                        -- ASSIGN  |PARTITIONED|
+                                          -- STREAM_PROJECT  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- DATASOURCE_SCAN  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/gby_inline/gby_inline.1.ddl.sqlpp b/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/gby_inline/gby_inline.1.ddl.sqlpp
new file mode 100644
index 0000000..2a24740
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/gby_inline/gby_inline.1.ddl.sqlpp
@@ -0,0 +1,55 @@
+/*
+ * 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 DATABASE test IF EXISTS;
+CREATE DATABASE test;
+USE test;
+
+
+CREATE TYPE OrderType AS CLOSED {
+  o_orderkey: int32,
+  o_custkey: int32,
+  o_orderstatus: string,
+  o_totalprice: double,
+  o_orderdate: string,
+  o_orderpriority: string,
+  o_clerk: string,
+  o_shippriority: int32,
+  o_comment: string
+}
+
+CREATE TYPE CustomerType AS CLOSED {
+  c_custkey: int32,
+  c_name: string,
+  c_address: string,
+  c_nationkey: int32,
+  c_phone: string,
+  c_acctbal: double,
+  c_mktsegment: string,
+  c_comment: string
+}
+
+
+CREATE EXTERNAL TABLE Customers(CustomerType) USING "localfs"
+(("path"="asterix_nc1://data/tpch0.001/customer.tbl"),
+("input-format"="text-input-format"),("format"="delimited-text"),("delimiter"="|"));
+
+CREATE EXTERNAL TABLE Orders(OrderType) USING "localfs"
+(("path"="asterix_nc1://data/tpch0.001/orders.tbl"),
+("input-format"="text-input-format"),("format"="delimited-text"),("delimiter"="|"));
diff --git a/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/gby_inline/gby_inline.2.update.sqlpp b/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/gby_inline/gby_inline.2.update.sqlpp
new file mode 100644
index 0000000..7220975
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/gby_inline/gby_inline.2.update.sqlpp
@@ -0,0 +1,18 @@
+/*
+ * 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.
+ */
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/gby_inline/gby_inline.3.query.sqlpp b/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/gby_inline/gby_inline.3.query.sqlpp
new file mode 100644
index 0000000..549377e
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/gby_inline/gby_inline.3.query.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/**
+ * This query is to verify the fix of ASTERIXDB-1291.
+ */
+
+USE test;
+
+FROM Customers c
+GROUP BY c.c_custkey AS ccustkey, c.c_nationkey AS cnationkey
+LET orders = (
+    FROM Orders o
+    WHERE cnationkey = 5 and o.o_custkey = ccustkey
+    SELECT ELEMENT o
+  )
+SELECT ccustkey customer_name, orders orders;
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/results/subquery/gby_inline/gby_inline.1.adm b/asterix-app/src/test/resources/runtimets/results/subquery/gby_inline/gby_inline.1.adm
new file mode 100644
index 0000000..f90f5db
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/subquery/gby_inline/gby_inline.1.adm
@@ -0,0 +1,150 @@
+{ "customer_name": 6i32, "orders": [  ] }
+{ "customer_name": 11i32, "orders": [  ] }
+{ "customer_name": 12i32, "orders": [  ] }
+{ "customer_name": 14i32, "orders": [  ] }
+{ "customer_name": 21i32, "orders": [  ] }
+{ "customer_name": 23i32, "orders": [  ] }
+{ "customer_name": 26i32, "orders": [  ] }
+{ "customer_name": 30i32, "orders": [  ] }
+{ "customer_name": 33i32, "orders": [  ] }
+{ "customer_name": 38i32, "orders": [  ] }
+{ "customer_name": 45i32, "orders": [  ] }
+{ "customer_name": 47i32, "orders": [  ] }
+{ "customer_name": 49i32, "orders": [  ] }
+{ "customer_name": 51i32, "orders": [  ] }
+{ "customer_name": 58i32, "orders": [  ] }
+{ "customer_name": 60i32, "orders": [  ] }
+{ "customer_name": 70i32, "orders": [  ] }
+{ "customer_name": 72i32, "orders": [  ] }
+{ "customer_name": 77i32, "orders": [  ] }
+{ "customer_name": 88i32, "orders": [  ] }
+{ "customer_name": 89i32, "orders": [  ] }
+{ "customer_name": 92i32, "orders": [  ] }
+{ "customer_name": 93i32, "orders": [  ] }
+{ "customer_name": 103i32, "orders": [  ] }
+{ "customer_name": 105i32, "orders": [  ] }
+{ "customer_name": 109i32, "orders": [  ] }
+{ "customer_name": 115i32, "orders": [  ] }
+{ "customer_name": 118i32, "orders": [  ] }
+{ "customer_name": 125i32, "orders": [  ] }
+{ "customer_name": 126i32, "orders": [  ] }
+{ "customer_name": 135i32, "orders": [  ] }
+{ "customer_name": 138i32, "orders": [  ] }
+{ "customer_name": 141i32, "orders": [  ] }
+{ "customer_name": 147i32, "orders": [  ] }
+{ "customer_name": 1i32, "orders": [  ] }
+{ "customer_name": 2i32, "orders": [  ] }
+{ "customer_name": 4i32, "orders": [  ] }
+{ "customer_name": 13i32, "orders": [  ] }
+{ "customer_name": 15i32, "orders": [  ] }
+{ "customer_name": 16i32, "orders": [  ] }
+{ "customer_name": 19i32, "orders": [  ] }
+{ "customer_name": 20i32, "orders": [  ] }
+{ "customer_name": 22i32, "orders": [  ] }
+{ "customer_name": 24i32, "orders": [  ] }
+{ "customer_name": 29i32, "orders": [  ] }
+{ "customer_name": 31i32, "orders": [  ] }
+{ "customer_name": 35i32, "orders": [  ] }
+{ "customer_name": 46i32, "orders": [  ] }
+{ "customer_name": 48i32, "orders": [  ] }
+{ "customer_name": 52i32, "orders": [  ] }
+{ "customer_name": 55i32, "orders": [  ] }
+{ "customer_name": 56i32, "orders": [  ] }
+{ "customer_name": 57i32, "orders": [  ] }
+{ "customer_name": 59i32, "orders": [  ] }
+{ "customer_name": 62i32, "orders": [  ] }
+{ "customer_name": 63i32, "orders": [  ] }
+{ "customer_name": 64i32, "orders": [  ] }
+{ "customer_name": 65i32, "orders": [  ] }
+{ "customer_name": 68i32, "orders": [  ] }
+{ "customer_name": 71i32, "orders": [  ] }
+{ "customer_name": 73i32, "orders": [  ] }
+{ "customer_name": 75i32, "orders": [  ] }
+{ "customer_name": 81i32, "orders": [  ] }
+{ "customer_name": 91i32, "orders": [  ] }
+{ "customer_name": 97i32, "orders": [  ] }
+{ "customer_name": 98i32, "orders": [  ] }
+{ "customer_name": 100i32, "orders": [  ] }
+{ "customer_name": 106i32, "orders": [  ] }
+{ "customer_name": 114i32, "orders": [  ] }
+{ "customer_name": 119i32, "orders": [  ] }
+{ "customer_name": 121i32, "orders": [  ] }
+{ "customer_name": 122i32, "orders": [  ] }
+{ "customer_name": 127i32, "orders": [  ] }
+{ "customer_name": 129i32, "orders": [  ] }
+{ "customer_name": 130i32, "orders": [  ] }
+{ "customer_name": 131i32, "orders": [  ] }
+{ "customer_name": 140i32, "orders": [  ] }
+{ "customer_name": 143i32, "orders": [  ] }
+{ "customer_name": 144i32, "orders": [  ] }
+{ "customer_name": 149i32, "orders": [  ] }
+{ "customer_name": 8i32, "orders": [  ] }
+{ "customer_name": 9i32, "orders": [  ] }
+{ "customer_name": 10i32, "orders": [ { "o_orderkey": 227i32, "o_custkey": 10i32, "o_orderstatus": "O", "o_totalprice": 46076.46d, "o_orderdate": "1995-11-10", "o_orderpriority": "5-LOW", "o_clerk": "Clerk#000000919", "o_shippriority": 0i32, "o_comment": " express instructions. slyly regul" }, { "o_orderkey": 517i32, "o_custkey": 10i32, "o_orderstatus": "O", "o_totalprice": 82197.79d, "o_orderdate": "1997-04-07", "o_orderpriority": "5-LOW", "o_clerk": "Clerk#000000359", "o_shippriority": 0i32, "o_comment": "slyly pending deposits cajole quickly packages. furiou" }, { "o_orderkey": 902i32, "o_custkey": 10i32, "o_orderstatus": "F", "o_totalprice": 37348.62d, "o_orderdate": "1994-07-27", "o_orderpriority": "4-NOT SPECIFIED", "o_clerk": "Clerk#000000811", "o_shippriority": 0i32, "o_comment": "yly final requests over the furiously regula" }, { "o_orderkey": 1218i32, "o_custkey": 10i32, "o_orderstatus": "F", "o_totalprice": 99834.47d, "o_orderdate": "1994-06-20", "o_orderpriority"
 : "4-NOT SPECIFIED", "o_clerk": "Clerk#000000994", "o_shippriority": 0i32, "o_comment": "s cajole. special, silent deposits about the theo" }, { "o_orderkey": 1222i32, "o_custkey": 10i32, "o_orderstatus": "F", "o_totalprice": 47623.94d, "o_orderdate": "1993-02-05", "o_orderpriority": "3-MEDIUM", "o_clerk": "Clerk#000000811", "o_shippriority": 0i32, "o_comment": "theodolites use quickly even accounts. carefully final asympto" }, { "o_orderkey": 1223i32, "o_custkey": 10i32, "o_orderstatus": "O", "o_totalprice": 26714.67d, "o_orderdate": "1996-05-25", "o_orderpriority": "4-NOT SPECIFIED", "o_clerk": "Clerk#000000238", "o_shippriority": 0i32, "o_comment": "posits was blithely fr" }, { "o_orderkey": 1860i32, "o_custkey": 10i32, "o_orderstatus": "O", "o_totalprice": 9103.4d, "o_orderdate": "1996-04-04", "o_orderpriority": "3-MEDIUM", "o_clerk": "Clerk#000000556", "o_shippriority": 0i32, "o_comment": "osits. quickly bold deposits according to " }, { "o_orderkey": 1890i32, "o_custke
 y": 10i32, "o_orderstatus": "O", "o_totalprice": 202364.58d, "o_orderdate": "1996-12-18", "o_orderpriority": "4-NOT SPECIFIED", "o_clerk": "Clerk#000000627", "o_shippriority": 0i32, "o_comment": "romise final, regular deposits. regular fox" }, { "o_orderkey": 2562i32, "o_custkey": 10i32, "o_orderstatus": "F", "o_totalprice": 136360.37d, "o_orderdate": "1992-08-01", "o_orderpriority": "1-URGENT", "o_clerk": "Clerk#000000467", "o_shippriority": 0i32, "o_comment": "elets. pending dolphins promise slyly. bo" }, { "o_orderkey": 3204i32, "o_custkey": 10i32, "o_orderstatus": "F", "o_totalprice": 41573.42d, "o_orderdate": "1992-12-26", "o_orderpriority": "1-URGENT", "o_clerk": "Clerk#000000693", "o_shippriority": 0i32, "o_comment": "ess somas cajole slyly. pending accounts cajole" }, { "o_orderkey": 3428i32, "o_custkey": 10i32, "o_orderstatus": "O", "o_totalprice": 88047.04d, "o_orderdate": "1996-04-07", "o_orderpriority": "5-LOW", "o_clerk": "Clerk#000000953", "o_shippriority": 0i3
 2, "o_comment": "lar excuses. slyly pending ideas detect p" }, { "o_orderkey": 3618i32, "o_custkey": 10i32, "o_orderstatus": "O", "o_totalprice": 136954.81d, "o_orderdate": "1997-12-13", "o_orderpriority": "3-MEDIUM", "o_clerk": "Clerk#000000894", "o_shippriority": 0i32, "o_comment": ". ideas run carefully. thin, pending " }, { "o_orderkey": 3751i32, "o_custkey": 10i32, "o_orderstatus": "F", "o_totalprice": 202917.72d, "o_orderdate": "1994-04-27", "o_orderpriority": "4-NOT SPECIFIED", "o_clerk": "Clerk#000000925", "o_shippriority": 0i32, "o_comment": "sheaves. express, unusual t" }, { "o_orderkey": 3843i32, "o_custkey": 10i32, "o_orderstatus": "O", "o_totalprice": 34035.17d, "o_orderdate": "1997-01-04", "o_orderpriority": "4-NOT SPECIFIED", "o_clerk": "Clerk#000000693", "o_shippriority": 0i32, "o_comment": "eodolites; slyly unusual accounts nag boldly " }, { "o_orderkey": 3911i32, "o_custkey": 10i32, "o_orderstatus": "P", "o_totalprice": 35019.95d, "o_orderdate": "1995-03-17
 ", "o_orderpriority": "4-NOT SPECIFIED", "o_clerk": "Clerk#000000818", "o_shippriority": 0i32, "o_comment": "he fluffily final forges haggle slyly according to the blithely" }, { "o_orderkey": 4032i32, "o_custkey": 10i32, "o_orderstatus": "O", "o_totalprice": 62497.51d, "o_orderdate": "1998-02-26", "o_orderpriority": "3-MEDIUM", "o_clerk": "Clerk#000000686", "o_shippriority": 0i32, "o_comment": "iresias sleep slyly regular ideas. quickly unusual" }, { "o_orderkey": 4097i32, "o_custkey": 10i32, "o_orderstatus": "O", "o_totalprice": 134308.04d, "o_orderdate": "1996-05-24", "o_orderpriority": "1-URGENT", "o_clerk": "Clerk#000000475", "o_shippriority": 0i32, "o_comment": "ickly under the even accounts. even packages after the furiously express" }, { "o_orderkey": 4388i32, "o_custkey": 10i32, "o_orderstatus": "O", "o_totalprice": 69668.22d, "o_orderdate": "1996-03-28", "o_orderpriority": "2-HIGH", "o_clerk": "Clerk#000000715", "o_shippriority": 0i32, "o_comment": "ts wake against
  the carefully final accounts. sly" }, { "o_orderkey": 4421i32, "o_custkey": 10i32, "o_orderstatus": "O", "o_totalprice": 258779.02d, "o_orderdate": "1997-04-04", "o_orderpriority": "3-MEDIUM", "o_clerk": "Clerk#000000246", "o_shippriority": 0i32, "o_comment": "t the pending warhorses. express waters a" }, { "o_orderkey": 4449i32, "o_custkey": 10i32, "o_orderstatus": "O", "o_totalprice": 48206.14d, "o_orderdate": "1998-02-08", "o_orderpriority": "5-LOW", "o_clerk": "Clerk#000000035", "o_shippriority": 0i32, "o_comment": "ourts are carefully even deposits. pending " }, { "o_orderkey": 4867i32, "o_custkey": 10i32, "o_orderstatus": "F", "o_totalprice": 9741.03d, "o_orderdate": "1992-05-21", "o_orderpriority": "1-URGENT", "o_clerk": "Clerk#000000891", "o_shippriority": 0i32, "o_comment": "ss the slyly regular dependencies. fluffily regular deposits within the car" }, { "o_orderkey": 5123i32, "o_custkey": 10i32, "o_orderstatus": "O", "o_totalprice": 11850.45d, "o_orderdate": "199
 8-02-10", "o_orderpriority": "1-URGENT", "o_clerk": "Clerk#000000776", "o_shippriority": 0i32, "o_comment": "ic requests. furiously ironic packages grow above the express, ironic inst" }, { "o_orderkey": 5220i32, "o_custkey": 10i32, "o_orderstatus": "F", "o_totalprice": 24844.39d, "o_orderdate": "1992-07-30", "o_orderpriority": "2-HIGH", "o_clerk": "Clerk#000000051", "o_shippriority": 0i32, "o_comment": " final packages. ideas detect slyly around" } ] }
+{ "customer_name": 25i32, "orders": [  ] }
+{ "customer_name": 28i32, "orders": [  ] }
+{ "customer_name": 34i32, "orders": [  ] }
+{ "customer_name": 37i32, "orders": [  ] }
+{ "customer_name": 40i32, "orders": [  ] }
+{ "customer_name": 41i32, "orders": [  ] }
+{ "customer_name": 43i32, "orders": [  ] }
+{ "customer_name": 66i32, "orders": [  ] }
+{ "customer_name": 67i32, "orders": [  ] }
+{ "customer_name": 69i32, "orders": [  ] }
+{ "customer_name": 76i32, "orders": [  ] }
+{ "customer_name": 79i32, "orders": [  ] }
+{ "customer_name": 84i32, "orders": [  ] }
+{ "customer_name": 85i32, "orders": [ { "o_orderkey": 69i32, "o_custkey": 85i32, "o_orderstatus": "F", "o_totalprice": 162176.23d, "o_orderdate": "1994-06-04", "o_orderpriority": "4-NOT SPECIFIED", "o_clerk": "Clerk#000000330", "o_shippriority": 0i32, "o_comment": " depths atop the slyly thin deposits detect among the furiously silent accou" }, { "o_orderkey": 704i32, "o_custkey": 85i32, "o_orderstatus": "O", "o_totalprice": 56210.26d, "o_orderdate": "1996-11-21", "o_orderpriority": "3-MEDIUM", "o_clerk": "Clerk#000000682", "o_shippriority": 0i32, "o_comment": "blithely pending platelets wake alongside of the final, iron" }, { "o_orderkey": 1699i32, "o_custkey": 85i32, "o_orderstatus": "F", "o_totalprice": 66408.29d, "o_orderdate": "1993-12-30", "o_orderpriority": "1-URGENT", "o_clerk": "Clerk#000000125", "o_shippriority": 0i32, "o_comment": "jole blithely. furiously un" }, { "o_orderkey": 2437i32, "o_custkey": 85i32, "o_orderstatus": "F", "o_totalprice": 143411.69d, "o_orde
 rdate": "1993-04-21", "o_orderpriority": "4-NOT SPECIFIED", "o_clerk": "Clerk#000000578", "o_shippriority": 0i32, "o_comment": ". theodolites wake slyly-- ironic, pending platelets above the carefully exp" }, { "o_orderkey": 2630i32, "o_custkey": 85i32, "o_orderstatus": "F", "o_totalprice": 127132.51d, "o_orderdate": "1992-10-24", "o_orderpriority": "5-LOW", "o_clerk": "Clerk#000000712", "o_shippriority": 0i32, "o_comment": "inal theodolites. ironic instructions s" }, { "o_orderkey": 2982i32, "o_custkey": 85i32, "o_orderstatus": "F", "o_totalprice": 55582.94d, "o_orderdate": "1995-03-19", "o_orderpriority": "2-HIGH", "o_clerk": "Clerk#000000402", "o_shippriority": 0i32, "o_comment": "lyly. express theodolites affix slyly after the slyly speci" }, { "o_orderkey": 3108i32, "o_custkey": 85i32, "o_orderstatus": "F", "o_totalprice": 63278.0d, "o_orderdate": "1993-08-05", "o_orderpriority": "1-URGENT", "o_clerk": "Clerk#000000574", "o_shippriority": 0i32, "o_comment": "s packages 
 haggle furiously am" }, { "o_orderkey": 3776i32, "o_custkey": 85i32, "o_orderstatus": "F", "o_totalprice": 150349.92d, "o_orderdate": "1992-11-20", "o_orderpriority": "2-HIGH", "o_clerk": "Clerk#000000698", "o_shippriority": 0i32, "o_comment": "efully even platelets slee" }, { "o_orderkey": 4480i32, "o_custkey": 85i32, "o_orderstatus": "F", "o_totalprice": 28658.26d, "o_orderdate": "1994-03-31", "o_orderpriority": "4-NOT SPECIFIED", "o_clerk": "Clerk#000000534", "o_shippriority": 0i32, "o_comment": "press, bold deposits boost blit" }, { "o_orderkey": 4513i32, "o_custkey": 85i32, "o_orderstatus": "O", "o_totalprice": 119820.38d, "o_orderdate": "1996-03-15", "o_orderpriority": "5-LOW", "o_clerk": "Clerk#000000154", "o_shippriority": 0i32, "o_comment": "ests. final, final ideas" }, { "o_orderkey": 4708i32, "o_custkey": 85i32, "o_orderstatus": "F", "o_totalprice": 56998.36d, "o_orderdate": "1994-10-01", "o_orderpriority": "1-URGENT", "o_clerk": "Clerk#000000383", "o_shippriority
 ": 0i32, "o_comment": "ly thinly even accounts. unusu" }, { "o_orderkey": 4865i32, "o_custkey": 85i32, "o_orderstatus": "O", "o_totalprice": 162113.46d, "o_orderdate": "1997-06-07", "o_orderpriority": "3-MEDIUM", "o_clerk": "Clerk#000000418", "o_shippriority": 0i32, "o_comment": "sits boost stealthily above the bl" }, { "o_orderkey": 4896i32, "o_custkey": 85i32, "o_orderstatus": "F", "o_totalprice": 93206.35d, "o_orderdate": "1992-08-22", "o_orderpriority": "1-URGENT", "o_clerk": "Clerk#000000622", "o_shippriority": 0i32, "o_comment": "sly pending deposits. final accounts boost above the sly, even" }, { "o_orderkey": 4999i32, "o_custkey": 85i32, "o_orderstatus": "F", "o_totalprice": 98643.17d, "o_orderdate": "1993-06-26", "o_orderpriority": "2-HIGH", "o_clerk": "Clerk#000000504", "o_shippriority": 0i32, "o_comment": " dolphins cajole blithely above the sly " }, { "o_orderkey": 5184i32, "o_custkey": 85i32, "o_orderstatus": "O", "o_totalprice": 209155.48d, "o_orderdate": "1998
 -07-20", "o_orderpriority": "5-LOW", "o_clerk": "Clerk#000000250", "o_shippriority": 0i32, "o_comment": "nding accounts detect final, even" }, { "o_orderkey": 5830i32, "o_custkey": 85i32, "o_orderstatus": "F", "o_totalprice": 28223.57d, "o_orderdate": "1993-03-25", "o_orderpriority": "3-MEDIUM", "o_clerk": "Clerk#000000233", "o_shippriority": 0i32, "o_comment": "lites haggle. ironic, ironic instructions maintain blit" } ] }
+{ "customer_name": 86i32, "orders": [  ] }
+{ "customer_name": 94i32, "orders": [  ] }
+{ "customer_name": 95i32, "orders": [  ] }
+{ "customer_name": 96i32, "orders": [  ] }
+{ "customer_name": 99i32, "orders": [  ] }
+{ "customer_name": 102i32, "orders": [  ] }
+{ "customer_name": 108i32, "orders": [  ] }
+{ "customer_name": 111i32, "orders": [  ] }
+{ "customer_name": 113i32, "orders": [  ] }
+{ "customer_name": 116i32, "orders": [  ] }
+{ "customer_name": 124i32, "orders": [  ] }
+{ "customer_name": 128i32, "orders": [  ] }
+{ "customer_name": 132i32, "orders": [  ] }
+{ "customer_name": 133i32, "orders": [  ] }
+{ "customer_name": 134i32, "orders": [  ] }
+{ "customer_name": 136i32, "orders": [  ] }
+{ "customer_name": 139i32, "orders": [  ] }
+{ "customer_name": 150i32, "orders": [  ] }
+{ "customer_name": 3i32, "orders": [  ] }
+{ "customer_name": 5i32, "orders": [  ] }
+{ "customer_name": 7i32, "orders": [  ] }
+{ "customer_name": 17i32, "orders": [  ] }
+{ "customer_name": 18i32, "orders": [  ] }
+{ "customer_name": 27i32, "orders": [  ] }
+{ "customer_name": 32i32, "orders": [  ] }
+{ "customer_name": 36i32, "orders": [  ] }
+{ "customer_name": 39i32, "orders": [  ] }
+{ "customer_name": 42i32, "orders": [  ] }
+{ "customer_name": 44i32, "orders": [  ] }
+{ "customer_name": 50i32, "orders": [  ] }
+{ "customer_name": 53i32, "orders": [  ] }
+{ "customer_name": 54i32, "orders": [  ] }
+{ "customer_name": 61i32, "orders": [  ] }
+{ "customer_name": 74i32, "orders": [  ] }
+{ "customer_name": 78i32, "orders": [  ] }
+{ "customer_name": 80i32, "orders": [  ] }
+{ "customer_name": 82i32, "orders": [  ] }
+{ "customer_name": 83i32, "orders": [  ] }
+{ "customer_name": 87i32, "orders": [  ] }
+{ "customer_name": 90i32, "orders": [  ] }
+{ "customer_name": 101i32, "orders": [  ] }
+{ "customer_name": 104i32, "orders": [  ] }
+{ "customer_name": 107i32, "orders": [  ] }
+{ "customer_name": 110i32, "orders": [  ] }
+{ "customer_name": 112i32, "orders": [  ] }
+{ "customer_name": 117i32, "orders": [  ] }
+{ "customer_name": 120i32, "orders": [  ] }
+{ "customer_name": 123i32, "orders": [  ] }
+{ "customer_name": 137i32, "orders": [  ] }
+{ "customer_name": 142i32, "orders": [  ] }
+{ "customer_name": 145i32, "orders": [  ] }
+{ "customer_name": 146i32, "orders": [  ] }
+{ "customer_name": 148i32, "orders": [  ] }
diff --git a/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml b/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
index ba36184..7f3fa95 100644
--- a/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
+++ b/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
@@ -4951,6 +4951,13 @@
             </compilation-unit>
         </test-case>
     </test-group>
+    <test-group name="subquery">
+        <test-case FilePath="subquery">
+            <compilation-unit name="gby_inline">
+                <output-dir compare="Text">gby_inline</output-dir>
+            </compilation-unit>
+        </test-case>
+    </test-group>
     <test-group name="subset-collection">
         <test-case FilePath="subset-collection">
             <compilation-unit name="01">

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

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


Change in asterixdb[master]: Fix for ASTERIXDB-1291: added regression tests.

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: Fix for ASTERIXDB-1291: added regression tests.
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-notopic/808/

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I1c8bbaf2522d421dff9a22af28c160f406dd2888
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: Fix for ASTERIXDB-1291: added regression tests.

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: Fix for ASTERIXDB-1291: added regression tests.
......................................................................


Patch Set 1: Verified+1

Build Successful 

https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-topic/655/ : SUCCESS

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I1c8bbaf2522d421dff9a22af28c160f406dd2888
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Wenhai Li <lw...@yahoo.com>
Gerrit-HasComments: No

Change in asterixdb[master]: Fix for ASTERIXDB-1291: added regression tests.

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: Fix for ASTERIXDB-1291: added regression tests.
......................................................................


Patch Set 1: -Verified

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-topic/655/

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I1c8bbaf2522d421dff9a22af28c160f406dd2888
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: Fix for ASTERIXDB-1291: added regression tests.

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: Fix for ASTERIXDB-1291: added regression tests.
......................................................................


Patch Set 1: Verified-1

Build Failed 

https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-notopic/808/ : ABORTED

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I1c8bbaf2522d421dff9a22af28c160f406dd2888
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: Fix for ASTERIXDB-1291: added regression tests.

Posted by "Yingyi Bu (Code Review)" <do...@asterixdb.incubator.apache.org>.
Yingyi Bu has submitted this change and it was merged.

Change subject: Fix for ASTERIXDB-1291: added regression tests.
......................................................................


Fix for ASTERIXDB-1291: added regression tests.

Change-Id: I1c8bbaf2522d421dff9a22af28c160f406dd2888
Reviewed-on: https://asterix-gerrit.ics.uci.edu/623
Tested-by: Jenkins <je...@fulliautomatix.ics.uci.edu>
Reviewed-by: Wenhai Li <lw...@yahoo.com>
Reviewed-by: Taewoo Kim <wa...@gmail.com>
---
M asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/FuzzyJoinRule.java
M asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/InvertedIndexAccessMethod.java
M asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/InlineAllNtsInSubplanVisitor.java
A asterix-app/src/test/resources/optimizerts/queries/gby_inline.aql
A asterix-app/src/test/resources/optimizerts/results/gby_inline.plan
A asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/gby_inline/gby_inline.1.ddl.sqlpp
A asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/gby_inline/gby_inline.2.update.sqlpp
A asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/gby_inline/gby_inline.3.query.sqlpp
A asterix-app/src/test/resources/runtimets/results/subquery/gby_inline/gby_inline.1.adm
M asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
10 files changed, 437 insertions(+), 22 deletions(-)

Approvals:
  Taewoo Kim: Looks good to me, approved
  Wenhai Li: Looks good to me, but someone else must approve
  Jenkins: Verified



diff --git a/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/FuzzyJoinRule.java b/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/FuzzyJoinRule.java
index 1741008..99a418e 100644
--- a/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/FuzzyJoinRule.java
+++ b/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/FuzzyJoinRule.java
@@ -276,7 +276,7 @@
         translator.addVariableToMetaScope(new Identifier("$$RIGHT"), rightInputVar);
         translator.addVariableToMetaScope(new Identifier("$$RIGHTPK"), rightPKVar);
 
-        translator.addOperatorToMetaScope(new Identifier("#LEFT_1"), deepCopyVisitor.deepCopy(leftInputOp, null));
+        translator.addOperatorToMetaScope(new Identifier("#LEFT_1"), deepCopyVisitor.deepCopy(leftInputOp));
         translator.addVariableToMetaScope(new Identifier("$$LEFT_1"), deepCopyVisitor.varCopy(leftInputVar));
         translator.addVariableToMetaScope(new Identifier("$$LEFTPK_1"), deepCopyVisitor.varCopy(leftPKVar));
         deepCopyVisitor.updatePrimaryKeys(context);
@@ -300,20 +300,20 @@
         // deepCopyVisitor.updatePrimaryKeys(context);
         // deepCopyVisitor.reset();
 
-        translator.addOperatorToMetaScope(new Identifier("#RIGHT_1"), deepCopyVisitor.deepCopy(rightInputOp, null));
+        translator.addOperatorToMetaScope(new Identifier("#RIGHT_1"), deepCopyVisitor.deepCopy(rightInputOp));
         translator.addVariableToMetaScope(new Identifier("$$RIGHT_1"), deepCopyVisitor.varCopy(rightInputVar));
         translator.addVariableToMetaScope(new Identifier("$$RIGHTPK_1"), deepCopyVisitor.varCopy(rightPKVar));
         deepCopyVisitor.updatePrimaryKeys(context);
         deepCopyVisitor.reset();
 
         // TODO pick side to run Stage 1, currently always picks RIGHT side
-        translator.addOperatorToMetaScope(new Identifier("#RIGHT_2"), deepCopyVisitor.deepCopy(rightInputOp, null));
+        translator.addOperatorToMetaScope(new Identifier("#RIGHT_2"), deepCopyVisitor.deepCopy(rightInputOp));
         translator.addVariableToMetaScope(new Identifier("$$RIGHT_2"), deepCopyVisitor.varCopy(rightInputVar));
         translator.addVariableToMetaScope(new Identifier("$$RIGHTPK_2"), deepCopyVisitor.varCopy(rightPKVar));
         deepCopyVisitor.updatePrimaryKeys(context);
         deepCopyVisitor.reset();
 
-        translator.addOperatorToMetaScope(new Identifier("#RIGHT_3"), deepCopyVisitor.deepCopy(rightInputOp, null));
+        translator.addOperatorToMetaScope(new Identifier("#RIGHT_3"), deepCopyVisitor.deepCopy(rightInputOp));
         translator.addVariableToMetaScope(new Identifier("$$RIGHT_3"), deepCopyVisitor.varCopy(rightInputVar));
         translator.addVariableToMetaScope(new Identifier("$$RIGHTPK_3"), deepCopyVisitor.varCopy(rightPKVar));
         deepCopyVisitor.updatePrimaryKeys(context);
diff --git a/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/InvertedIndexAccessMethod.java b/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/InvertedIndexAccessMethod.java
index 851264c..d587649 100644
--- a/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/InvertedIndexAccessMethod.java
+++ b/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/InvertedIndexAccessMethod.java
@@ -303,8 +303,9 @@
                         (IAType) AqlExpressionTypeComputer.INSTANCE.getType(arg3, null, null) });
         for (IOptimizableFuncExpr optFuncExpr : analysisCtx.matchedFuncExprs) {
             //avoid additional optFuncExpressions in case of a join
-            if (optFuncExpr.getFuncExpr().equals(funcExpr))
+            if (optFuncExpr.getFuncExpr().equals(funcExpr)) {
                 return true;
+            }
         }
         analysisCtx.matchedFuncExprs.add(newOptFuncExpr);
         return true;
@@ -604,13 +605,13 @@
         // Create first copy.
         LogicalOperatorDeepCopyWithNewVariablesVisitor firstDeepCopyVisitor = new LogicalOperatorDeepCopyWithNewVariablesVisitor(
                 context, newProbeSubTreeVarMap);
-        ILogicalOperator newProbeSubTree = firstDeepCopyVisitor.deepCopy(probeSubTree.root, null);
+        ILogicalOperator newProbeSubTree = firstDeepCopyVisitor.deepCopy(probeSubTree.root);
         inferTypes(newProbeSubTree, context);
         Mutable<ILogicalOperator> newProbeSubTreeRootRef = new MutableObject<ILogicalOperator>(newProbeSubTree);
         // Create second copy.
         LogicalOperatorDeepCopyWithNewVariablesVisitor secondDeepCopyVisitor = new LogicalOperatorDeepCopyWithNewVariablesVisitor(
                 context, joinInputSubTreeVarMap);
-        ILogicalOperator joinInputSubTree = secondDeepCopyVisitor.deepCopy(probeSubTree.root, null);
+        ILogicalOperator joinInputSubTree = secondDeepCopyVisitor.deepCopy(probeSubTree.root);
         inferTypes(joinInputSubTree, context);
         probeSubTree.rootRef.setValue(joinInputSubTree);
 
@@ -672,10 +673,11 @@
         // Create select ops for removing tuples that are filterable and not filterable, respectively.
         IVariableTypeEnvironment probeTypeEnv = context.getOutputTypeEnvironment(probeSubTree.root);
         IAType inputSearchVarType;
-        if (chosenIndex.isEnforcingKeyFileds())
+        if (chosenIndex.isEnforcingKeyFileds()) {
             inputSearchVarType = optFuncExpr.getFieldType(optFuncExpr.findLogicalVar(inputSearchVar));
-        else
+        } else {
             inputSearchVarType = (IAType) probeTypeEnv.getVarType(inputSearchVar);
+        }
         Mutable<ILogicalOperator> isFilterableSelectOpRef = new MutableObject<ILogicalOperator>();
         Mutable<ILogicalOperator> isNotFilterableSelectOpRef = new MutableObject<ILogicalOperator>();
         createIsFilterableSelectOps(replicateOp, inputSearchVar, inputSearchVarType, optFuncExpr, chosenIndex, context,
@@ -687,7 +689,7 @@
         // Copy the scan subtree in indexSubTree.
         LogicalOperatorDeepCopyWithNewVariablesVisitor deepCopyVisitor = new LogicalOperatorDeepCopyWithNewVariablesVisitor(
                 context);
-        ILogicalOperator scanSubTree = deepCopyVisitor.deepCopy(indexSubTree.root, null);
+        ILogicalOperator scanSubTree = deepCopyVisitor.deepCopy(indexSubTree.root);
 
         Map<LogicalVariable, LogicalVariable> copyVarMap = deepCopyVisitor.getInputToOutputVariableMapping();
         panicVarMap.putAll(copyVarMap);
@@ -871,10 +873,11 @@
     }
 
     private boolean isEditDistanceFuncJoinOptimizable(Index index, IOptimizableFuncExpr optFuncExpr) {
-        if (index.isEnforcingKeyFileds())
+        if (index.isEnforcingKeyFileds()) {
             return isEditDistanceFuncCompatible(index.getKeyFieldTypes().get(0).getTypeTag(), index.getIndexType());
-        else
+        } else {
             return isEditDistanceFuncCompatible(optFuncExpr.getFieldType(0).getTypeTag(), index.getIndexType());
+        }
     }
 
     private boolean isEditDistanceFuncCompatible(ATypeTag typeTag, IndexType indexType) {
@@ -972,11 +975,13 @@
         LogicalVariable targetVar = null;
         for (int i = 0; i < variableCount; i++) {
             subTree = optFuncExpr.getOperatorSubTree(i);
-            if (subTree == null)
+            if (subTree == null) {
                 continue;
+            }
             targetVar = optFuncExpr.getLogicalVar(i);
-            if (targetVar == null)
+            if (targetVar == null) {
                 continue;
+            }
             return isJaccardFuncCompatible(optFuncExpr.getFuncExpr().getArguments().get(i).getValue(),
                     optFuncExpr.getFieldType(i).getTypeTag(), index.getIndexType());
         }
@@ -1001,20 +1006,25 @@
         }
 
         for (AbstractLogicalOperator op : subTree.assignsAndUnnests) {
-            if (op.getOperatorTag() != LogicalOperatorTag.ASSIGN)
+            if (op.getOperatorTag() != LogicalOperatorTag.ASSIGN) {
                 continue;
+            }
             List<Mutable<ILogicalExpression>> exprList = ((AssignOperator) op).getExpressions();
             for (Mutable<ILogicalExpression> expr : exprList) {
-                if (expr.getValue().getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL)
+                if (expr.getValue().getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
                     continue;
+                }
                 AbstractFunctionCallExpression funcExpr = (AbstractFunctionCallExpression) expr.getValue();
-                if (funcExpr.getFunctionIdentifier() != funcId)
+                if (funcExpr.getFunctionIdentifier() != funcId) {
                     continue;
+                }
                 ILogicalExpression varExpr = funcExpr.getArguments().get(0).getValue();
-                if (varExpr.getExpressionTag() != LogicalExpressionTag.VARIABLE)
+                if (varExpr.getExpressionTag() != LogicalExpressionTag.VARIABLE) {
                     continue;
-                if (((VariableReferenceExpression) varExpr).getVariableReference() == targetVar)
+                }
+                if (((VariableReferenceExpression) varExpr).getVariableReference() == targetVar) {
                     continue;
+                }
                 return (ScalarFunctionCallExpression) funcExpr;
             }
         }
@@ -1077,10 +1087,11 @@
     }
 
     private boolean isContainsFuncJoinOptimizable(Index index, IOptimizableFuncExpr optFuncExpr) {
-        if (index.isEnforcingKeyFileds())
+        if (index.isEnforcingKeyFileds()) {
             return isContainsFuncCompatible(index.getKeyFieldTypes().get(0).getTypeTag(), index.getIndexType());
-        else
+        } else {
             return isContainsFuncCompatible(optFuncExpr.getFieldType(0).getTypeTag(), index.getIndexType());
+        }
     }
 
     private boolean isContainsFuncCompatible(ATypeTag typeTag, IndexType indexType) {
diff --git a/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/InlineAllNtsInSubplanVisitor.java b/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/InlineAllNtsInSubplanVisitor.java
index 3489982..42905ee 100644
--- a/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/InlineAllNtsInSubplanVisitor.java
+++ b/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/InlineAllNtsInSubplanVisitor.java
@@ -403,7 +403,7 @@
         }
         LogicalOperatorDeepCopyWithNewVariablesVisitor deepCopyVisitor = new LogicalOperatorDeepCopyWithNewVariablesVisitor(
                 context);
-        ILogicalOperator copiedInputOperator = deepCopyVisitor.deepCopy(subplanInputOperator, null);
+        ILogicalOperator copiedInputOperator = deepCopyVisitor.deepCopy(subplanInputOperator);
 
         // Updates the primary key info in the copied plan segment.
         Map<LogicalVariable, LogicalVariable> varMap = deepCopyVisitor.getInputToOutputVariableMapping();
diff --git a/asterix-app/src/test/resources/optimizerts/queries/gby_inline.aql b/asterix-app/src/test/resources/optimizerts/queries/gby_inline.aql
new file mode 100644
index 0000000..bb29875
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/queries/gby_inline.aql
@@ -0,0 +1,84 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/**
+ * This query is to verify the fix for ASTERIXDB-1291.
+ */
+
+drop dataverse tpch if exists;
+create dataverse tpch;
+use dataverse tpch;
+
+create type LineItemType as closed {
+  l_orderkey: int32,
+  l_partkey: int32,
+  l_suppkey: int32,
+  l_linenumber: int32,
+  l_quantity: double,
+  l_extendedprice: double,
+  l_discount: double,
+  l_tax: double,
+  l_returnflag: string,
+  l_linestatus: string,
+  l_shipdate: string,
+  l_commitdate: string,
+  l_receiptdate: string,
+  l_shipinstruct: string,
+  l_shipmode: string,
+  l_comment: string
+}
+
+create type OrderType as closed {
+  o_orderkey: int32,
+  o_custkey: int32,
+  o_orderstatus: string,
+  o_totalprice: double,
+  o_orderdate: string,
+  o_orderpriority: string,
+  o_clerk: string,
+  o_shippriority: int32,
+  o_comment: string
+}
+
+create type CustomerType as closed {
+  c_custkey: int32,
+  c_name: string,
+  c_address: string,
+  c_nationkey: int32,
+  c_phone: string,
+  c_acctbal: double,
+  c_mktsegment: string,
+  c_comment: string
+}
+
+create dataset Orders(OrderType)
+  primary key o_orderkey;
+create dataset Customers(CustomerType)
+  primary key c_custkey;
+
+for $c in dataset('Customers')
+group by $ccustkey := $c.c_custkey, $cnationkey := $c.c_nationkey with $c
+let $orders :=
+  for $o in dataset('Orders')
+  where $cnationkey = 5 and $o.o_$o.o_custkey = $ccustkey
+  return $o
+return {
+  "cust": $c,
+  "orders": $orders
+}
diff --git a/asterix-app/src/test/resources/optimizerts/results/gby_inline.plan b/asterix-app/src/test/resources/optimizerts/results/gby_inline.plan
new file mode 100644
index 0000000..b3f4ac2
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/results/gby_inline.plan
@@ -0,0 +1,58 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- PRE_CLUSTERED_GROUP_BY[$$1]  |PARTITIONED|
+                    {
+                      -- AGGREGATE  |LOCAL|
+                        -- STREAM_SELECT  |LOCAL|
+                          -- NESTED_TUPLE_SOURCE  |LOCAL|
+                    }
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- STABLE_SORT [$$1(ASC)]  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- HYBRID_HASH_JOIN [$$1, $$2][$$24, $$26]  |PARTITIONED|
+                            -- HASH_PARTITION_EXCHANGE [$$1]  |PARTITIONED|
+                              -- PRE_CLUSTERED_GROUP_BY[$$20]  |PARTITIONED|
+                                      {
+                                        -- AGGREGATE  |LOCAL|
+                                          -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                      }
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- ASSIGN  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- DATASOURCE_SCAN  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- ASSIGN  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- HYBRID_HASH_JOIN [$$24][$$22]  |PARTITIONED|
+                                      -- HASH_PARTITION_EXCHANGE [$$24]  |PARTITIONED|
+                                        -- STREAM_SELECT  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- PRE_CLUSTERED_GROUP_BY[$$25]  |PARTITIONED|
+                                                    {
+                                                      -- AGGREGATE  |LOCAL|
+                                                        -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                    }
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                  -- ASSIGN  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- DATASOURCE_SCAN  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                      -- HASH_PARTITION_EXCHANGE [$$22]  |PARTITIONED|
+                                        -- ASSIGN  |PARTITIONED|
+                                          -- STREAM_PROJECT  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- DATASOURCE_SCAN  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/gby_inline/gby_inline.1.ddl.sqlpp b/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/gby_inline/gby_inline.1.ddl.sqlpp
new file mode 100644
index 0000000..2a24740
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/gby_inline/gby_inline.1.ddl.sqlpp
@@ -0,0 +1,55 @@
+/*
+ * 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 DATABASE test IF EXISTS;
+CREATE DATABASE test;
+USE test;
+
+
+CREATE TYPE OrderType AS CLOSED {
+  o_orderkey: int32,
+  o_custkey: int32,
+  o_orderstatus: string,
+  o_totalprice: double,
+  o_orderdate: string,
+  o_orderpriority: string,
+  o_clerk: string,
+  o_shippriority: int32,
+  o_comment: string
+}
+
+CREATE TYPE CustomerType AS CLOSED {
+  c_custkey: int32,
+  c_name: string,
+  c_address: string,
+  c_nationkey: int32,
+  c_phone: string,
+  c_acctbal: double,
+  c_mktsegment: string,
+  c_comment: string
+}
+
+
+CREATE EXTERNAL TABLE Customers(CustomerType) USING "localfs"
+(("path"="asterix_nc1://data/tpch0.001/customer.tbl"),
+("input-format"="text-input-format"),("format"="delimited-text"),("delimiter"="|"));
+
+CREATE EXTERNAL TABLE Orders(OrderType) USING "localfs"
+(("path"="asterix_nc1://data/tpch0.001/orders.tbl"),
+("input-format"="text-input-format"),("format"="delimited-text"),("delimiter"="|"));
diff --git a/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/gby_inline/gby_inline.2.update.sqlpp b/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/gby_inline/gby_inline.2.update.sqlpp
new file mode 100644
index 0000000..7220975
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/gby_inline/gby_inline.2.update.sqlpp
@@ -0,0 +1,18 @@
+/*
+ * 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.
+ */
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/gby_inline/gby_inline.3.query.sqlpp b/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/gby_inline/gby_inline.3.query.sqlpp
new file mode 100644
index 0000000..549377e
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/gby_inline/gby_inline.3.query.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/**
+ * This query is to verify the fix of ASTERIXDB-1291.
+ */
+
+USE test;
+
+FROM Customers c
+GROUP BY c.c_custkey AS ccustkey, c.c_nationkey AS cnationkey
+LET orders = (
+    FROM Orders o
+    WHERE cnationkey = 5 and o.o_custkey = ccustkey
+    SELECT ELEMENT o
+  )
+SELECT ccustkey customer_name, orders orders;
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/results/subquery/gby_inline/gby_inline.1.adm b/asterix-app/src/test/resources/runtimets/results/subquery/gby_inline/gby_inline.1.adm
new file mode 100644
index 0000000..f90f5db
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/subquery/gby_inline/gby_inline.1.adm
@@ -0,0 +1,150 @@
+{ "customer_name": 6i32, "orders": [  ] }
+{ "customer_name": 11i32, "orders": [  ] }
+{ "customer_name": 12i32, "orders": [  ] }
+{ "customer_name": 14i32, "orders": [  ] }
+{ "customer_name": 21i32, "orders": [  ] }
+{ "customer_name": 23i32, "orders": [  ] }
+{ "customer_name": 26i32, "orders": [  ] }
+{ "customer_name": 30i32, "orders": [  ] }
+{ "customer_name": 33i32, "orders": [  ] }
+{ "customer_name": 38i32, "orders": [  ] }
+{ "customer_name": 45i32, "orders": [  ] }
+{ "customer_name": 47i32, "orders": [  ] }
+{ "customer_name": 49i32, "orders": [  ] }
+{ "customer_name": 51i32, "orders": [  ] }
+{ "customer_name": 58i32, "orders": [  ] }
+{ "customer_name": 60i32, "orders": [  ] }
+{ "customer_name": 70i32, "orders": [  ] }
+{ "customer_name": 72i32, "orders": [  ] }
+{ "customer_name": 77i32, "orders": [  ] }
+{ "customer_name": 88i32, "orders": [  ] }
+{ "customer_name": 89i32, "orders": [  ] }
+{ "customer_name": 92i32, "orders": [  ] }
+{ "customer_name": 93i32, "orders": [  ] }
+{ "customer_name": 103i32, "orders": [  ] }
+{ "customer_name": 105i32, "orders": [  ] }
+{ "customer_name": 109i32, "orders": [  ] }
+{ "customer_name": 115i32, "orders": [  ] }
+{ "customer_name": 118i32, "orders": [  ] }
+{ "customer_name": 125i32, "orders": [  ] }
+{ "customer_name": 126i32, "orders": [  ] }
+{ "customer_name": 135i32, "orders": [  ] }
+{ "customer_name": 138i32, "orders": [  ] }
+{ "customer_name": 141i32, "orders": [  ] }
+{ "customer_name": 147i32, "orders": [  ] }
+{ "customer_name": 1i32, "orders": [  ] }
+{ "customer_name": 2i32, "orders": [  ] }
+{ "customer_name": 4i32, "orders": [  ] }
+{ "customer_name": 13i32, "orders": [  ] }
+{ "customer_name": 15i32, "orders": [  ] }
+{ "customer_name": 16i32, "orders": [  ] }
+{ "customer_name": 19i32, "orders": [  ] }
+{ "customer_name": 20i32, "orders": [  ] }
+{ "customer_name": 22i32, "orders": [  ] }
+{ "customer_name": 24i32, "orders": [  ] }
+{ "customer_name": 29i32, "orders": [  ] }
+{ "customer_name": 31i32, "orders": [  ] }
+{ "customer_name": 35i32, "orders": [  ] }
+{ "customer_name": 46i32, "orders": [  ] }
+{ "customer_name": 48i32, "orders": [  ] }
+{ "customer_name": 52i32, "orders": [  ] }
+{ "customer_name": 55i32, "orders": [  ] }
+{ "customer_name": 56i32, "orders": [  ] }
+{ "customer_name": 57i32, "orders": [  ] }
+{ "customer_name": 59i32, "orders": [  ] }
+{ "customer_name": 62i32, "orders": [  ] }
+{ "customer_name": 63i32, "orders": [  ] }
+{ "customer_name": 64i32, "orders": [  ] }
+{ "customer_name": 65i32, "orders": [  ] }
+{ "customer_name": 68i32, "orders": [  ] }
+{ "customer_name": 71i32, "orders": [  ] }
+{ "customer_name": 73i32, "orders": [  ] }
+{ "customer_name": 75i32, "orders": [  ] }
+{ "customer_name": 81i32, "orders": [  ] }
+{ "customer_name": 91i32, "orders": [  ] }
+{ "customer_name": 97i32, "orders": [  ] }
+{ "customer_name": 98i32, "orders": [  ] }
+{ "customer_name": 100i32, "orders": [  ] }
+{ "customer_name": 106i32, "orders": [  ] }
+{ "customer_name": 114i32, "orders": [  ] }
+{ "customer_name": 119i32, "orders": [  ] }
+{ "customer_name": 121i32, "orders": [  ] }
+{ "customer_name": 122i32, "orders": [  ] }
+{ "customer_name": 127i32, "orders": [  ] }
+{ "customer_name": 129i32, "orders": [  ] }
+{ "customer_name": 130i32, "orders": [  ] }
+{ "customer_name": 131i32, "orders": [  ] }
+{ "customer_name": 140i32, "orders": [  ] }
+{ "customer_name": 143i32, "orders": [  ] }
+{ "customer_name": 144i32, "orders": [  ] }
+{ "customer_name": 149i32, "orders": [  ] }
+{ "customer_name": 8i32, "orders": [  ] }
+{ "customer_name": 9i32, "orders": [  ] }
+{ "customer_name": 10i32, "orders": [ { "o_orderkey": 227i32, "o_custkey": 10i32, "o_orderstatus": "O", "o_totalprice": 46076.46d, "o_orderdate": "1995-11-10", "o_orderpriority": "5-LOW", "o_clerk": "Clerk#000000919", "o_shippriority": 0i32, "o_comment": " express instructions. slyly regul" }, { "o_orderkey": 517i32, "o_custkey": 10i32, "o_orderstatus": "O", "o_totalprice": 82197.79d, "o_orderdate": "1997-04-07", "o_orderpriority": "5-LOW", "o_clerk": "Clerk#000000359", "o_shippriority": 0i32, "o_comment": "slyly pending deposits cajole quickly packages. furiou" }, { "o_orderkey": 902i32, "o_custkey": 10i32, "o_orderstatus": "F", "o_totalprice": 37348.62d, "o_orderdate": "1994-07-27", "o_orderpriority": "4-NOT SPECIFIED", "o_clerk": "Clerk#000000811", "o_shippriority": 0i32, "o_comment": "yly final requests over the furiously regula" }, { "o_orderkey": 1218i32, "o_custkey": 10i32, "o_orderstatus": "F", "o_totalprice": 99834.47d, "o_orderdate": "1994-06-20", "o_orderpriority"
 : "4-NOT SPECIFIED", "o_clerk": "Clerk#000000994", "o_shippriority": 0i32, "o_comment": "s cajole. special, silent deposits about the theo" }, { "o_orderkey": 1222i32, "o_custkey": 10i32, "o_orderstatus": "F", "o_totalprice": 47623.94d, "o_orderdate": "1993-02-05", "o_orderpriority": "3-MEDIUM", "o_clerk": "Clerk#000000811", "o_shippriority": 0i32, "o_comment": "theodolites use quickly even accounts. carefully final asympto" }, { "o_orderkey": 1223i32, "o_custkey": 10i32, "o_orderstatus": "O", "o_totalprice": 26714.67d, "o_orderdate": "1996-05-25", "o_orderpriority": "4-NOT SPECIFIED", "o_clerk": "Clerk#000000238", "o_shippriority": 0i32, "o_comment": "posits was blithely fr" }, { "o_orderkey": 1860i32, "o_custkey": 10i32, "o_orderstatus": "O", "o_totalprice": 9103.4d, "o_orderdate": "1996-04-04", "o_orderpriority": "3-MEDIUM", "o_clerk": "Clerk#000000556", "o_shippriority": 0i32, "o_comment": "osits. quickly bold deposits according to " }, { "o_orderkey": 1890i32, "o_custke
 y": 10i32, "o_orderstatus": "O", "o_totalprice": 202364.58d, "o_orderdate": "1996-12-18", "o_orderpriority": "4-NOT SPECIFIED", "o_clerk": "Clerk#000000627", "o_shippriority": 0i32, "o_comment": "romise final, regular deposits. regular fox" }, { "o_orderkey": 2562i32, "o_custkey": 10i32, "o_orderstatus": "F", "o_totalprice": 136360.37d, "o_orderdate": "1992-08-01", "o_orderpriority": "1-URGENT", "o_clerk": "Clerk#000000467", "o_shippriority": 0i32, "o_comment": "elets. pending dolphins promise slyly. bo" }, { "o_orderkey": 3204i32, "o_custkey": 10i32, "o_orderstatus": "F", "o_totalprice": 41573.42d, "o_orderdate": "1992-12-26", "o_orderpriority": "1-URGENT", "o_clerk": "Clerk#000000693", "o_shippriority": 0i32, "o_comment": "ess somas cajole slyly. pending accounts cajole" }, { "o_orderkey": 3428i32, "o_custkey": 10i32, "o_orderstatus": "O", "o_totalprice": 88047.04d, "o_orderdate": "1996-04-07", "o_orderpriority": "5-LOW", "o_clerk": "Clerk#000000953", "o_shippriority": 0i3
 2, "o_comment": "lar excuses. slyly pending ideas detect p" }, { "o_orderkey": 3618i32, "o_custkey": 10i32, "o_orderstatus": "O", "o_totalprice": 136954.81d, "o_orderdate": "1997-12-13", "o_orderpriority": "3-MEDIUM", "o_clerk": "Clerk#000000894", "o_shippriority": 0i32, "o_comment": ". ideas run carefully. thin, pending " }, { "o_orderkey": 3751i32, "o_custkey": 10i32, "o_orderstatus": "F", "o_totalprice": 202917.72d, "o_orderdate": "1994-04-27", "o_orderpriority": "4-NOT SPECIFIED", "o_clerk": "Clerk#000000925", "o_shippriority": 0i32, "o_comment": "sheaves. express, unusual t" }, { "o_orderkey": 3843i32, "o_custkey": 10i32, "o_orderstatus": "O", "o_totalprice": 34035.17d, "o_orderdate": "1997-01-04", "o_orderpriority": "4-NOT SPECIFIED", "o_clerk": "Clerk#000000693", "o_shippriority": 0i32, "o_comment": "eodolites; slyly unusual accounts nag boldly " }, { "o_orderkey": 3911i32, "o_custkey": 10i32, "o_orderstatus": "P", "o_totalprice": 35019.95d, "o_orderdate": "1995-03-17
 ", "o_orderpriority": "4-NOT SPECIFIED", "o_clerk": "Clerk#000000818", "o_shippriority": 0i32, "o_comment": "he fluffily final forges haggle slyly according to the blithely" }, { "o_orderkey": 4032i32, "o_custkey": 10i32, "o_orderstatus": "O", "o_totalprice": 62497.51d, "o_orderdate": "1998-02-26", "o_orderpriority": "3-MEDIUM", "o_clerk": "Clerk#000000686", "o_shippriority": 0i32, "o_comment": "iresias sleep slyly regular ideas. quickly unusual" }, { "o_orderkey": 4097i32, "o_custkey": 10i32, "o_orderstatus": "O", "o_totalprice": 134308.04d, "o_orderdate": "1996-05-24", "o_orderpriority": "1-URGENT", "o_clerk": "Clerk#000000475", "o_shippriority": 0i32, "o_comment": "ickly under the even accounts. even packages after the furiously express" }, { "o_orderkey": 4388i32, "o_custkey": 10i32, "o_orderstatus": "O", "o_totalprice": 69668.22d, "o_orderdate": "1996-03-28", "o_orderpriority": "2-HIGH", "o_clerk": "Clerk#000000715", "o_shippriority": 0i32, "o_comment": "ts wake against
  the carefully final accounts. sly" }, { "o_orderkey": 4421i32, "o_custkey": 10i32, "o_orderstatus": "O", "o_totalprice": 258779.02d, "o_orderdate": "1997-04-04", "o_orderpriority": "3-MEDIUM", "o_clerk": "Clerk#000000246", "o_shippriority": 0i32, "o_comment": "t the pending warhorses. express waters a" }, { "o_orderkey": 4449i32, "o_custkey": 10i32, "o_orderstatus": "O", "o_totalprice": 48206.14d, "o_orderdate": "1998-02-08", "o_orderpriority": "5-LOW", "o_clerk": "Clerk#000000035", "o_shippriority": 0i32, "o_comment": "ourts are carefully even deposits. pending " }, { "o_orderkey": 4867i32, "o_custkey": 10i32, "o_orderstatus": "F", "o_totalprice": 9741.03d, "o_orderdate": "1992-05-21", "o_orderpriority": "1-URGENT", "o_clerk": "Clerk#000000891", "o_shippriority": 0i32, "o_comment": "ss the slyly regular dependencies. fluffily regular deposits within the car" }, { "o_orderkey": 5123i32, "o_custkey": 10i32, "o_orderstatus": "O", "o_totalprice": 11850.45d, "o_orderdate": "199
 8-02-10", "o_orderpriority": "1-URGENT", "o_clerk": "Clerk#000000776", "o_shippriority": 0i32, "o_comment": "ic requests. furiously ironic packages grow above the express, ironic inst" }, { "o_orderkey": 5220i32, "o_custkey": 10i32, "o_orderstatus": "F", "o_totalprice": 24844.39d, "o_orderdate": "1992-07-30", "o_orderpriority": "2-HIGH", "o_clerk": "Clerk#000000051", "o_shippriority": 0i32, "o_comment": " final packages. ideas detect slyly around" } ] }
+{ "customer_name": 25i32, "orders": [  ] }
+{ "customer_name": 28i32, "orders": [  ] }
+{ "customer_name": 34i32, "orders": [  ] }
+{ "customer_name": 37i32, "orders": [  ] }
+{ "customer_name": 40i32, "orders": [  ] }
+{ "customer_name": 41i32, "orders": [  ] }
+{ "customer_name": 43i32, "orders": [  ] }
+{ "customer_name": 66i32, "orders": [  ] }
+{ "customer_name": 67i32, "orders": [  ] }
+{ "customer_name": 69i32, "orders": [  ] }
+{ "customer_name": 76i32, "orders": [  ] }
+{ "customer_name": 79i32, "orders": [  ] }
+{ "customer_name": 84i32, "orders": [  ] }
+{ "customer_name": 85i32, "orders": [ { "o_orderkey": 69i32, "o_custkey": 85i32, "o_orderstatus": "F", "o_totalprice": 162176.23d, "o_orderdate": "1994-06-04", "o_orderpriority": "4-NOT SPECIFIED", "o_clerk": "Clerk#000000330", "o_shippriority": 0i32, "o_comment": " depths atop the slyly thin deposits detect among the furiously silent accou" }, { "o_orderkey": 704i32, "o_custkey": 85i32, "o_orderstatus": "O", "o_totalprice": 56210.26d, "o_orderdate": "1996-11-21", "o_orderpriority": "3-MEDIUM", "o_clerk": "Clerk#000000682", "o_shippriority": 0i32, "o_comment": "blithely pending platelets wake alongside of the final, iron" }, { "o_orderkey": 1699i32, "o_custkey": 85i32, "o_orderstatus": "F", "o_totalprice": 66408.29d, "o_orderdate": "1993-12-30", "o_orderpriority": "1-URGENT", "o_clerk": "Clerk#000000125", "o_shippriority": 0i32, "o_comment": "jole blithely. furiously un" }, { "o_orderkey": 2437i32, "o_custkey": 85i32, "o_orderstatus": "F", "o_totalprice": 143411.69d, "o_orde
 rdate": "1993-04-21", "o_orderpriority": "4-NOT SPECIFIED", "o_clerk": "Clerk#000000578", "o_shippriority": 0i32, "o_comment": ". theodolites wake slyly-- ironic, pending platelets above the carefully exp" }, { "o_orderkey": 2630i32, "o_custkey": 85i32, "o_orderstatus": "F", "o_totalprice": 127132.51d, "o_orderdate": "1992-10-24", "o_orderpriority": "5-LOW", "o_clerk": "Clerk#000000712", "o_shippriority": 0i32, "o_comment": "inal theodolites. ironic instructions s" }, { "o_orderkey": 2982i32, "o_custkey": 85i32, "o_orderstatus": "F", "o_totalprice": 55582.94d, "o_orderdate": "1995-03-19", "o_orderpriority": "2-HIGH", "o_clerk": "Clerk#000000402", "o_shippriority": 0i32, "o_comment": "lyly. express theodolites affix slyly after the slyly speci" }, { "o_orderkey": 3108i32, "o_custkey": 85i32, "o_orderstatus": "F", "o_totalprice": 63278.0d, "o_orderdate": "1993-08-05", "o_orderpriority": "1-URGENT", "o_clerk": "Clerk#000000574", "o_shippriority": 0i32, "o_comment": "s packages 
 haggle furiously am" }, { "o_orderkey": 3776i32, "o_custkey": 85i32, "o_orderstatus": "F", "o_totalprice": 150349.92d, "o_orderdate": "1992-11-20", "o_orderpriority": "2-HIGH", "o_clerk": "Clerk#000000698", "o_shippriority": 0i32, "o_comment": "efully even platelets slee" }, { "o_orderkey": 4480i32, "o_custkey": 85i32, "o_orderstatus": "F", "o_totalprice": 28658.26d, "o_orderdate": "1994-03-31", "o_orderpriority": "4-NOT SPECIFIED", "o_clerk": "Clerk#000000534", "o_shippriority": 0i32, "o_comment": "press, bold deposits boost blit" }, { "o_orderkey": 4513i32, "o_custkey": 85i32, "o_orderstatus": "O", "o_totalprice": 119820.38d, "o_orderdate": "1996-03-15", "o_orderpriority": "5-LOW", "o_clerk": "Clerk#000000154", "o_shippriority": 0i32, "o_comment": "ests. final, final ideas" }, { "o_orderkey": 4708i32, "o_custkey": 85i32, "o_orderstatus": "F", "o_totalprice": 56998.36d, "o_orderdate": "1994-10-01", "o_orderpriority": "1-URGENT", "o_clerk": "Clerk#000000383", "o_shippriority
 ": 0i32, "o_comment": "ly thinly even accounts. unusu" }, { "o_orderkey": 4865i32, "o_custkey": 85i32, "o_orderstatus": "O", "o_totalprice": 162113.46d, "o_orderdate": "1997-06-07", "o_orderpriority": "3-MEDIUM", "o_clerk": "Clerk#000000418", "o_shippriority": 0i32, "o_comment": "sits boost stealthily above the bl" }, { "o_orderkey": 4896i32, "o_custkey": 85i32, "o_orderstatus": "F", "o_totalprice": 93206.35d, "o_orderdate": "1992-08-22", "o_orderpriority": "1-URGENT", "o_clerk": "Clerk#000000622", "o_shippriority": 0i32, "o_comment": "sly pending deposits. final accounts boost above the sly, even" }, { "o_orderkey": 4999i32, "o_custkey": 85i32, "o_orderstatus": "F", "o_totalprice": 98643.17d, "o_orderdate": "1993-06-26", "o_orderpriority": "2-HIGH", "o_clerk": "Clerk#000000504", "o_shippriority": 0i32, "o_comment": " dolphins cajole blithely above the sly " }, { "o_orderkey": 5184i32, "o_custkey": 85i32, "o_orderstatus": "O", "o_totalprice": 209155.48d, "o_orderdate": "1998
 -07-20", "o_orderpriority": "5-LOW", "o_clerk": "Clerk#000000250", "o_shippriority": 0i32, "o_comment": "nding accounts detect final, even" }, { "o_orderkey": 5830i32, "o_custkey": 85i32, "o_orderstatus": "F", "o_totalprice": 28223.57d, "o_orderdate": "1993-03-25", "o_orderpriority": "3-MEDIUM", "o_clerk": "Clerk#000000233", "o_shippriority": 0i32, "o_comment": "lites haggle. ironic, ironic instructions maintain blit" } ] }
+{ "customer_name": 86i32, "orders": [  ] }
+{ "customer_name": 94i32, "orders": [  ] }
+{ "customer_name": 95i32, "orders": [  ] }
+{ "customer_name": 96i32, "orders": [  ] }
+{ "customer_name": 99i32, "orders": [  ] }
+{ "customer_name": 102i32, "orders": [  ] }
+{ "customer_name": 108i32, "orders": [  ] }
+{ "customer_name": 111i32, "orders": [  ] }
+{ "customer_name": 113i32, "orders": [  ] }
+{ "customer_name": 116i32, "orders": [  ] }
+{ "customer_name": 124i32, "orders": [  ] }
+{ "customer_name": 128i32, "orders": [  ] }
+{ "customer_name": 132i32, "orders": [  ] }
+{ "customer_name": 133i32, "orders": [  ] }
+{ "customer_name": 134i32, "orders": [  ] }
+{ "customer_name": 136i32, "orders": [  ] }
+{ "customer_name": 139i32, "orders": [  ] }
+{ "customer_name": 150i32, "orders": [  ] }
+{ "customer_name": 3i32, "orders": [  ] }
+{ "customer_name": 5i32, "orders": [  ] }
+{ "customer_name": 7i32, "orders": [  ] }
+{ "customer_name": 17i32, "orders": [  ] }
+{ "customer_name": 18i32, "orders": [  ] }
+{ "customer_name": 27i32, "orders": [  ] }
+{ "customer_name": 32i32, "orders": [  ] }
+{ "customer_name": 36i32, "orders": [  ] }
+{ "customer_name": 39i32, "orders": [  ] }
+{ "customer_name": 42i32, "orders": [  ] }
+{ "customer_name": 44i32, "orders": [  ] }
+{ "customer_name": 50i32, "orders": [  ] }
+{ "customer_name": 53i32, "orders": [  ] }
+{ "customer_name": 54i32, "orders": [  ] }
+{ "customer_name": 61i32, "orders": [  ] }
+{ "customer_name": 74i32, "orders": [  ] }
+{ "customer_name": 78i32, "orders": [  ] }
+{ "customer_name": 80i32, "orders": [  ] }
+{ "customer_name": 82i32, "orders": [  ] }
+{ "customer_name": 83i32, "orders": [  ] }
+{ "customer_name": 87i32, "orders": [  ] }
+{ "customer_name": 90i32, "orders": [  ] }
+{ "customer_name": 101i32, "orders": [  ] }
+{ "customer_name": 104i32, "orders": [  ] }
+{ "customer_name": 107i32, "orders": [  ] }
+{ "customer_name": 110i32, "orders": [  ] }
+{ "customer_name": 112i32, "orders": [  ] }
+{ "customer_name": 117i32, "orders": [  ] }
+{ "customer_name": 120i32, "orders": [  ] }
+{ "customer_name": 123i32, "orders": [  ] }
+{ "customer_name": 137i32, "orders": [  ] }
+{ "customer_name": 142i32, "orders": [  ] }
+{ "customer_name": 145i32, "orders": [  ] }
+{ "customer_name": 146i32, "orders": [  ] }
+{ "customer_name": 148i32, "orders": [  ] }
diff --git a/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml b/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
index ba36184..7f3fa95 100644
--- a/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
+++ b/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
@@ -4951,6 +4951,13 @@
             </compilation-unit>
         </test-case>
     </test-group>
+    <test-group name="subquery">
+        <test-case FilePath="subquery">
+            <compilation-unit name="gby_inline">
+                <output-dir compare="Text">gby_inline</output-dir>
+            </compilation-unit>
+        </test-case>
+    </test-group>
     <test-group name="subset-collection">
         <test-case FilePath="subset-collection">
             <compilation-unit name="01">

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

Gerrit-MessageType: merged
Gerrit-Change-Id: I1c8bbaf2522d421dff9a22af28c160f406dd2888
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>


Change in asterixdb[master]: Fix for ASTERIXDB-1291: added regression tests.

Posted by "Wenhai Li (Code Review)" <do...@asterixdb.incubator.apache.org>.
Wenhai Li has posted comments on this change.

Change subject: Fix for ASTERIXDB-1291: added regression tests.
......................................................................


Patch Set 1: Code-Review+1

Tests passed!

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I1c8bbaf2522d421dff9a22af28c160f406dd2888
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Wenhai Li <lw...@yahoo.com>
Gerrit-HasComments: No

Change in asterixdb[master]: Fix for ASTERIXDB-1291: added regression tests.

Posted by "Taewoo Kim (Code Review)" <do...@asterixdb.incubator.apache.org>.
Taewoo Kim has posted comments on this change.

Change subject: Fix for ASTERIXDB-1291: added regression tests.
......................................................................


Patch Set 1: Code-Review+2

+2 on behalf of Wenhai

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I1c8bbaf2522d421dff9a22af28c160f406dd2888
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Wenhai Li <lw...@yahoo.com>
Gerrit-HasComments: No