You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@asterixdb.apache.org by bu...@apache.org on 2016/07/06 16:38:16 UTC

asterixdb git commit: Support SELECT star.

Repository: asterixdb
Updated Branches:
  refs/heads/master ea36aa65e -> a89fae64a


Support SELECT star.

- star is optional;
- a star can be mixed with other projection columns.

Change-Id: Ifa77431912670b55387fd5a722c2184341400a50
Reviewed-on: https://asterix-gerrit.ics.uci.edu/971
Sonar-Qube: Jenkins <je...@fulliautomatix.ics.uci.edu>
Tested-by: Jenkins <je...@fulliautomatix.ics.uci.edu>
Integration-Tests: Jenkins <je...@fulliautomatix.ics.uci.edu>
Reviewed-by: Till Westmann <ti...@apache.org>


Project: http://git-wip-us.apache.org/repos/asf/asterixdb/repo
Commit: http://git-wip-us.apache.org/repos/asf/asterixdb/commit/a89fae64
Tree: http://git-wip-us.apache.org/repos/asf/asterixdb/tree/a89fae64
Diff: http://git-wip-us.apache.org/repos/asf/asterixdb/diff/a89fae64

Branch: refs/heads/master
Commit: a89fae64ac21fb8eefde79f79d2dbe1a0e54c364
Parents: ea36aa6
Author: Yingyi Bu <yi...@couchbase.com>
Authored: Wed Jul 6 07:58:55 2016 -0700
Committer: Yingyi Bu <bu...@gmail.com>
Committed: Wed Jul 6 09:37:54 2016 -0700

----------------------------------------------------------------------
 .../SqlppExpressionToPlanTranslator.java        | 130 ++++++++++++++-----
 .../select-star/from/from.1.ddl.sqlpp           |  76 +++++++++++
 .../select-star/from/from.2.update.sqlpp        |  28 ++++
 .../select-star/from/from.3.query.sqlpp         |  32 +++++
 .../select-star/group_by/group_by.1.ddl.sqlpp   |  34 +++++
 .../group_by/group_by.2.update.sqlpp            |  24 ++++
 .../select-star/group_by/group_by.3.query.sqlpp |  27 ++++
 .../select-star/join/join.1.ddl.sqlpp           |  76 +++++++++++
 .../select-star/join/join.2.update.sqlpp        |  28 ++++
 .../select-star/join/join.3.query.sqlpp         |  32 +++++
 .../select-star/mixed/mixed.1.ddl.sqlpp         |  76 +++++++++++
 .../select-star/mixed/mixed.2.update.sqlpp      |  28 ++++
 .../select-star/mixed/mixed.3.query.sqlpp       |  32 +++++
 .../select-star/no_star/no_star.1.ddl.sqlpp     |  76 +++++++++++
 .../select-star/no_star/no_star.2.update.sqlpp  |  28 ++++
 .../select-star/no_star/no_star.3.query.sqlpp   |  32 +++++
 .../results/select-star/from/from.1.adm         |   3 +
 .../results/select-star/group_by/group_by.1.adm |   5 +
 .../results/select-star/join/join.1.adm         |   3 +
 .../results/select-star/mixed/mixed.1.adm       |   3 +
 .../results/select-star/no_star/no_star.1.adm   |   3 +
 .../resources/runtimets/testsuite_sqlpp.xml     |  27 ++++
 .../rewrites/SqlppFunctionBodyRewriter.java     |   6 +-
 .../lang/sqlpp/rewrites/SqlppQueryRewriter.java |  16 ++-
 .../visitor/InlineColumnAliasVisitor.java       |   3 +
 .../lang/sqlpp/util/SqlppVariableUtil.java      |  12 ++
 .../visitor/CheckSql92AggregateVisitor.java     |   3 +
 .../AbstractSqlppSimpleExpressionVisitor.java   |   4 +-
 .../src/main/javacc/SQLPP.html                  |   6 +-
 .../asterix-lang-sqlpp/src/main/javacc/SQLPP.jj |  16 ++-
 .../LogicalOperatorPrettyPrintVisitor.java      |  11 +-
 31 files changed, 823 insertions(+), 57 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/asterixdb/blob/a89fae64/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/SqlppExpressionToPlanTranslator.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/SqlppExpressionToPlanTranslator.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/SqlppExpressionToPlanTranslator.java
index 410da06..b7ed988 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/SqlppExpressionToPlanTranslator.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/SqlppExpressionToPlanTranslator.java
@@ -28,8 +28,10 @@ import org.apache.asterix.common.exceptions.AsterixException;
 import org.apache.asterix.lang.common.base.Clause.ClauseType;
 import org.apache.asterix.lang.common.base.Expression;
 import org.apache.asterix.lang.common.base.Expression.Kind;
+import org.apache.asterix.lang.common.clause.GroupbyClause;
 import org.apache.asterix.lang.common.clause.LetClause;
 import org.apache.asterix.lang.common.expression.FieldBinding;
+import org.apache.asterix.lang.common.expression.GbyVariableExpressionPair;
 import org.apache.asterix.lang.common.expression.LiteralExpr;
 import org.apache.asterix.lang.common.expression.RecordConstructor;
 import org.apache.asterix.lang.common.expression.VariableExpr;
@@ -52,6 +54,7 @@ import org.apache.asterix.lang.sqlpp.clause.UnnestClause;
 import org.apache.asterix.lang.sqlpp.expression.IndependentSubquery;
 import org.apache.asterix.lang.sqlpp.expression.SelectExpression;
 import org.apache.asterix.lang.sqlpp.optype.JoinType;
+import org.apache.asterix.lang.sqlpp.util.SqlppVariableUtil;
 import org.apache.asterix.lang.sqlpp.visitor.base.ISqlppVisitor;
 import org.apache.asterix.metadata.declared.AqlMetadataProvider;
 import org.apache.asterix.om.base.AInt32;
@@ -208,7 +211,7 @@ class SqlppExpressionToPlanTranslator extends LangExpressionToPlanTranslator imp
         if (selectBlock.hasHavingClause()) {
             currentOpRef = new MutableObject<>(selectBlock.getHavingClause().accept(this, currentOpRef).first);
         }
-        return selectBlock.getSelectClause().accept(this, currentOpRef);
+        return processSelectClause(selectBlock, currentOpRef);
     }
 
     @Override
@@ -453,38 +456,7 @@ class SqlppExpressionToPlanTranslator extends LangExpressionToPlanTranslator imp
     @Override
     public Pair<ILogicalOperator, LogicalVariable> visit(SelectClause selectClause, Mutable<ILogicalOperator> tupSrc)
             throws AsterixException {
-        Expression returnExpr;
-        if (selectClause.selectElement()) {
-            returnExpr = selectClause.getSelectElement().getExpression();
-        } else {
-            List<Projection> projections = selectClause.getSelectRegular().getProjections();
-            List<FieldBinding> fieldBindings = new ArrayList<>();
-            for (Projection projection : projections) {
-                fieldBindings.add(new FieldBinding(new LiteralExpr(new StringLiteral(projection.getName())),
-                        projection.getExpression()));
-            }
-            returnExpr = new RecordConstructor(fieldBindings);
-        }
-        Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo = langExprToAlgExpression(returnExpr, tupSrc);
-        LogicalVariable returnVar;
-        ILogicalOperator returnOperator;
-        if (returnExpr.getKind() == Kind.VARIABLE_EXPRESSION) {
-            VariableExpr varExpr = (VariableExpr) returnExpr;
-            returnOperator = eo.second.getValue();
-            returnVar = context.getVar(varExpr.getVar().getId());
-        } else {
-            returnVar = context.newVar();
-            returnOperator = new AssignOperator(returnVar, new MutableObject<ILogicalExpression>(eo.first));
-            returnOperator.getInputs().add(eo.second);
-        }
-        if (selectClause.distinct()) {
-            DistinctOperator distinctOperator = new DistinctOperator(mkSingletonArrayList(
-                    new MutableObject<ILogicalExpression>(new VariableReferenceExpression(returnVar))));
-            distinctOperator.getInputs().add(new MutableObject<ILogicalOperator>(returnOperator));
-            return new Pair<>(distinctOperator, returnVar);
-        } else {
-            return new Pair<>(returnOperator, returnVar);
-        }
+        throw new UnsupportedOperationException(ERR_MSG);
     }
 
     @Override
@@ -502,7 +474,7 @@ class SqlppExpressionToPlanTranslator extends LangExpressionToPlanTranslator imp
     @Override
     public Pair<ILogicalOperator, LogicalVariable> visit(Projection projection, Mutable<ILogicalOperator> arg)
             throws AsterixException {
-        throw new IllegalStateException();
+        throw new UnsupportedOperationException(ERR_MSG);
     }
 
     private Pair<ILogicalOperator, LogicalVariable> produceSelectPlan(boolean isSubquery,
@@ -542,4 +514,94 @@ class SqlppExpressionToPlanTranslator extends LangExpressionToPlanTranslator imp
         }
     }
 
+    // Generates the return expression for a select clause.
+    private Pair<ILogicalOperator, LogicalVariable> processSelectClause(SelectBlock selectBlock,
+            Mutable<ILogicalOperator> tupSrc) throws AsterixException {
+        SelectClause selectClause = selectBlock.getSelectClause();
+        Expression returnExpr;
+        if (selectClause.selectElement()) {
+            returnExpr = selectClause.getSelectElement().getExpression();
+        } else {
+            returnExpr = generateReturnExpr(selectClause, selectBlock);
+        }
+        Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo = langExprToAlgExpression(returnExpr, tupSrc);
+        LogicalVariable returnVar;
+        ILogicalOperator returnOperator;
+        if (returnExpr.getKind() == Kind.VARIABLE_EXPRESSION) {
+            VariableExpr varExpr = (VariableExpr) returnExpr;
+            returnOperator = eo.second.getValue();
+            returnVar = context.getVar(varExpr.getVar().getId());
+        } else {
+            returnVar = context.newVar();
+            returnOperator = new AssignOperator(returnVar, new MutableObject<ILogicalExpression>(eo.first));
+            returnOperator.getInputs().add(eo.second);
+        }
+        if (selectClause.distinct()) {
+            DistinctOperator distinctOperator = new DistinctOperator(mkSingletonArrayList(
+                    new MutableObject<ILogicalExpression>(new VariableReferenceExpression(returnVar))));
+            distinctOperator.getInputs().add(new MutableObject<ILogicalOperator>(returnOperator));
+            return new Pair<>(distinctOperator, returnVar);
+        } else {
+            return new Pair<>(returnOperator, returnVar);
+        }
+    }
+
+    // Generates the return expression for a select clause.
+    private Expression generateReturnExpr(SelectClause selectClause, SelectBlock selectBlock) {
+        SelectRegular selectRegular = selectClause.getSelectRegular();
+        List<FieldBinding> fieldBindings = new ArrayList<>();
+        List<Projection> projections = selectRegular.getProjections();
+        for (Projection projection : projections) {
+            if (projection.star()) {
+                if (selectBlock.hasGroupbyClause()) {
+                    fieldBindings.addAll(getGroupBindings(selectBlock.getGroupbyClause()));
+                } else if (selectBlock.hasFromClause()) {
+                    fieldBindings.addAll(getFromBindings(selectBlock.getFromClause()));
+                }
+            } else {
+                fieldBindings.add(new FieldBinding(new LiteralExpr(new StringLiteral(projection.getName())),
+                        projection.getExpression()));
+            }
+        }
+        return new RecordConstructor(fieldBindings);
+    }
+
+    // Generates all field bindings according to the from clause.
+    private List<FieldBinding> getFromBindings(FromClause fromClause) {
+        List<FieldBinding> fieldBindings = new ArrayList<>();
+        for (FromTerm fromTerm : fromClause.getFromTerms()) {
+            fieldBindings.add(getFieldBinding(fromTerm.getLeftVariable()));
+            if (fromTerm.hasPositionalVariable()) {
+                fieldBindings.add(getFieldBinding(fromTerm.getPositionalVariable()));
+            }
+            if (!fromTerm.hasCorrelateClauses()) {
+                continue;
+            }
+            for (AbstractBinaryCorrelateClause correlateClause : fromTerm.getCorrelateClauses()) {
+                fieldBindings.add(getFieldBinding(correlateClause.getRightVariable()));
+                if (correlateClause.hasPositionalVariable()) {
+                    fieldBindings.add(getFieldBinding(correlateClause.getPositionalVariable()));
+                }
+            }
+        }
+        return fieldBindings;
+    }
+
+    // Generates all field bindings according to the from clause.
+    private List<FieldBinding> getGroupBindings(GroupbyClause groupbyClause) {
+        List<FieldBinding> fieldBindings = new ArrayList<>();
+        for (GbyVariableExpressionPair pair : groupbyClause.getGbyPairList()) {
+            fieldBindings.add(getFieldBinding(pair.getVar()));
+        }
+        fieldBindings.add(getFieldBinding(groupbyClause.getGroupVar()));
+        return fieldBindings;
+    }
+
+    // Generates a field binding for a variable.
+    private FieldBinding getFieldBinding(VariableExpr var) {
+        LiteralExpr fieldName = new LiteralExpr(
+                new StringLiteral(SqlppVariableUtil.variableNameToDisplayedFieldName(var.getVar().getValue())));
+        return new FieldBinding(fieldName, var);
+    }
+
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/a89fae64/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-star/from/from.1.ddl.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-star/from/from.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-star/from/from.1.ddl.sqlpp
new file mode 100644
index 0000000..dba927c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-star/from/from.1.ddl.sqlpp
@@ -0,0 +1,76 @@
+/*
+ * 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 tpch if exists;
+create  database tpch;
+
+use tpch;
+
+
+create type tpch.LineItemType as
+ closed {
+  l_orderkey : int64,
+  l_partkey : int64,
+  l_suppkey : int64,
+  l_linenumber : int64,
+  l_quantity : int64,
+  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 tpch.OrderType as
+ closed {
+  o_orderkey : int64,
+  o_custkey : int64,
+  o_orderstatus : string,
+  o_totalprice : double,
+  o_orderdate : string,
+  o_orderpriority : string,
+  o_clerk : string,
+  o_shippriority : int64,
+  o_comment : string
+}
+
+create type tpch.CustomerType as
+ closed {
+  c_custkey : int64,
+  c_name : string,
+  c_address : string,
+  c_nationkey : int64,
+  c_phone : string,
+  c_acctbal : double,
+  c_mktsegment : string,
+  c_comment : string
+}
+
+create  table LineItem(LineItemType) primary key l_orderkey,l_linenumber;
+
+create  table Orders(OrderType) primary key o_orderkey;
+
+create  table Customer(CustomerType) primary key c_custkey;
+

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/a89fae64/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-star/from/from.2.update.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-star/from/from.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-star/from/from.2.update.sqlpp
new file mode 100644
index 0000000..dd4aa2f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-star/from/from.2.update.sqlpp
@@ -0,0 +1,28 @@
+/*
+ * 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 tpch;
+
+
+load  table LineItem using localfs ((`path`=`asterix_nc1://data/tpch0.001/lineitem.tbl`),(`format`=`delimited-text`),(`delimiter`=`|`)) pre-sorted;
+
+load  table Orders using localfs ((`path`=`asterix_nc1://data/tpch0.001/orders.tbl`),(`format`=`delimited-text`),(`delimiter`=`|`)) pre-sorted;
+
+load  table Customer using localfs ((`path`=`asterix_nc1://data/tpch0.001/customer.tbl`),(`format`=`delimited-text`),(`delimiter`=`|`)) pre-sorted;
+

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/a89fae64/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-star/from/from.3.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-star/from/from.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-star/from/from.3.query.sqlpp
new file mode 100644
index 0000000..53fb0cf
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-star/from/from.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.
+ */
+
+USE tpch;
+
+
+SELECT *
+FROM  Customer c,
+      Orders o,
+      LineItem l
+WHERE c_mktsegment = 'BUILDING' AND c_custkey = o_custkey
+      AND l_orderkey = o_orderkey AND o_orderdate < '1995-03-15'
+      AND l_shipdate > '1995-03-15'
+ORDER BY l_linenumber, l_orderkey
+LIMIT 3;
+

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/a89fae64/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-star/group_by/group_by.1.ddl.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-star/group_by/group_by.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-star/group_by/group_by.1.ddl.sqlpp
new file mode 100644
index 0000000..581b684
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-star/group_by/group_by.1.ddl.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * 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 tpch if exists;
+create  database tpch;
+
+use tpch;
+
+
+create type tpch.RegionType as
+ closed {
+  r_regionkey : int32,
+  r_name : string,
+  r_comment : string
+}
+
+create  table Regions_group_no_agg(RegionType) primary key r_regionkey;
+

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/a89fae64/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-star/group_by/group_by.2.update.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-star/group_by/group_by.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-star/group_by/group_by.2.update.sqlpp
new file mode 100644
index 0000000..7283894
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-star/group_by/group_by.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 tpch;
+
+
+load  table Regions_group_no_agg using localfs ((`path`=`asterix_nc1://data/tpch0.001/region.tbl`),(`format`=`delimited-text`),(`delimiter`=`|`)) pre-sorted;
+

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/a89fae64/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-star/group_by/group_by.3.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-star/group_by/group_by.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-star/group_by/group_by.3.query.sqlpp
new file mode 100644
index 0000000..4f0ddf4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-star/group_by/group_by.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 tpch;
+
+
+SELECT *
+FROM  Regions_group_no_agg AS r
+GROUP BY r_name AS name
+ORDER BY name
+;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/a89fae64/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-star/join/join.1.ddl.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-star/join/join.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-star/join/join.1.ddl.sqlpp
new file mode 100644
index 0000000..dba927c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-star/join/join.1.ddl.sqlpp
@@ -0,0 +1,76 @@
+/*
+ * 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 tpch if exists;
+create  database tpch;
+
+use tpch;
+
+
+create type tpch.LineItemType as
+ closed {
+  l_orderkey : int64,
+  l_partkey : int64,
+  l_suppkey : int64,
+  l_linenumber : int64,
+  l_quantity : int64,
+  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 tpch.OrderType as
+ closed {
+  o_orderkey : int64,
+  o_custkey : int64,
+  o_orderstatus : string,
+  o_totalprice : double,
+  o_orderdate : string,
+  o_orderpriority : string,
+  o_clerk : string,
+  o_shippriority : int64,
+  o_comment : string
+}
+
+create type tpch.CustomerType as
+ closed {
+  c_custkey : int64,
+  c_name : string,
+  c_address : string,
+  c_nationkey : int64,
+  c_phone : string,
+  c_acctbal : double,
+  c_mktsegment : string,
+  c_comment : string
+}
+
+create  table LineItem(LineItemType) primary key l_orderkey,l_linenumber;
+
+create  table Orders(OrderType) primary key o_orderkey;
+
+create  table Customer(CustomerType) primary key c_custkey;
+

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/a89fae64/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-star/join/join.2.update.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-star/join/join.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-star/join/join.2.update.sqlpp
new file mode 100644
index 0000000..dd4aa2f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-star/join/join.2.update.sqlpp
@@ -0,0 +1,28 @@
+/*
+ * 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 tpch;
+
+
+load  table LineItem using localfs ((`path`=`asterix_nc1://data/tpch0.001/lineitem.tbl`),(`format`=`delimited-text`),(`delimiter`=`|`)) pre-sorted;
+
+load  table Orders using localfs ((`path`=`asterix_nc1://data/tpch0.001/orders.tbl`),(`format`=`delimited-text`),(`delimiter`=`|`)) pre-sorted;
+
+load  table Customer using localfs ((`path`=`asterix_nc1://data/tpch0.001/customer.tbl`),(`format`=`delimited-text`),(`delimiter`=`|`)) pre-sorted;
+

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/a89fae64/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-star/join/join.3.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-star/join/join.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-star/join/join.3.query.sqlpp
new file mode 100644
index 0000000..5eeee7c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-star/join/join.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.
+ */
+
+USE tpch;
+
+
+SELECT *
+FROM  Customer c
+JOIN  Orders o ON c_custkey = o_custkey
+JOIN  LineItem l ON l_orderkey = o_orderkey
+WHERE c_mktsegment = 'BUILDING'
+      AND o_orderdate < '1995-03-15'
+      AND l_shipdate > '1995-03-15'
+ORDER BY l_linenumber, l_orderkey
+LIMIT 3;
+

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/a89fae64/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-star/mixed/mixed.1.ddl.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-star/mixed/mixed.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-star/mixed/mixed.1.ddl.sqlpp
new file mode 100644
index 0000000..dba927c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-star/mixed/mixed.1.ddl.sqlpp
@@ -0,0 +1,76 @@
+/*
+ * 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 tpch if exists;
+create  database tpch;
+
+use tpch;
+
+
+create type tpch.LineItemType as
+ closed {
+  l_orderkey : int64,
+  l_partkey : int64,
+  l_suppkey : int64,
+  l_linenumber : int64,
+  l_quantity : int64,
+  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 tpch.OrderType as
+ closed {
+  o_orderkey : int64,
+  o_custkey : int64,
+  o_orderstatus : string,
+  o_totalprice : double,
+  o_orderdate : string,
+  o_orderpriority : string,
+  o_clerk : string,
+  o_shippriority : int64,
+  o_comment : string
+}
+
+create type tpch.CustomerType as
+ closed {
+  c_custkey : int64,
+  c_name : string,
+  c_address : string,
+  c_nationkey : int64,
+  c_phone : string,
+  c_acctbal : double,
+  c_mktsegment : string,
+  c_comment : string
+}
+
+create  table LineItem(LineItemType) primary key l_orderkey,l_linenumber;
+
+create  table Orders(OrderType) primary key o_orderkey;
+
+create  table Customer(CustomerType) primary key c_custkey;
+

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/a89fae64/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-star/mixed/mixed.2.update.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-star/mixed/mixed.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-star/mixed/mixed.2.update.sqlpp
new file mode 100644
index 0000000..dd4aa2f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-star/mixed/mixed.2.update.sqlpp
@@ -0,0 +1,28 @@
+/*
+ * 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 tpch;
+
+
+load  table LineItem using localfs ((`path`=`asterix_nc1://data/tpch0.001/lineitem.tbl`),(`format`=`delimited-text`),(`delimiter`=`|`)) pre-sorted;
+
+load  table Orders using localfs ((`path`=`asterix_nc1://data/tpch0.001/orders.tbl`),(`format`=`delimited-text`),(`delimiter`=`|`)) pre-sorted;
+
+load  table Customer using localfs ((`path`=`asterix_nc1://data/tpch0.001/customer.tbl`),(`format`=`delimited-text`),(`delimiter`=`|`)) pre-sorted;
+

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/a89fae64/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-star/mixed/mixed.3.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-star/mixed/mixed.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-star/mixed/mixed.3.query.sqlpp
new file mode 100644
index 0000000..95ac04e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-star/mixed/mixed.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.
+ */
+
+USE tpch;
+
+
+SELECT  c_custkey, *, o_orderkey
+FROM  Customer c,
+      Orders o,
+      LineItem l
+WHERE c_mktsegment = 'BUILDING' AND c_custkey = o_custkey
+      AND l_orderkey = o_orderkey AND o_orderdate < '1995-03-15'
+      AND l_shipdate > '1995-03-15'
+ORDER BY l_linenumber, l_orderkey
+LIMIT 3;
+

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/a89fae64/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-star/no_star/no_star.1.ddl.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-star/no_star/no_star.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-star/no_star/no_star.1.ddl.sqlpp
new file mode 100644
index 0000000..dba927c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-star/no_star/no_star.1.ddl.sqlpp
@@ -0,0 +1,76 @@
+/*
+ * 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 tpch if exists;
+create  database tpch;
+
+use tpch;
+
+
+create type tpch.LineItemType as
+ closed {
+  l_orderkey : int64,
+  l_partkey : int64,
+  l_suppkey : int64,
+  l_linenumber : int64,
+  l_quantity : int64,
+  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 tpch.OrderType as
+ closed {
+  o_orderkey : int64,
+  o_custkey : int64,
+  o_orderstatus : string,
+  o_totalprice : double,
+  o_orderdate : string,
+  o_orderpriority : string,
+  o_clerk : string,
+  o_shippriority : int64,
+  o_comment : string
+}
+
+create type tpch.CustomerType as
+ closed {
+  c_custkey : int64,
+  c_name : string,
+  c_address : string,
+  c_nationkey : int64,
+  c_phone : string,
+  c_acctbal : double,
+  c_mktsegment : string,
+  c_comment : string
+}
+
+create  table LineItem(LineItemType) primary key l_orderkey,l_linenumber;
+
+create  table Orders(OrderType) primary key o_orderkey;
+
+create  table Customer(CustomerType) primary key c_custkey;
+

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/a89fae64/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-star/no_star/no_star.2.update.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-star/no_star/no_star.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-star/no_star/no_star.2.update.sqlpp
new file mode 100644
index 0000000..dd4aa2f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-star/no_star/no_star.2.update.sqlpp
@@ -0,0 +1,28 @@
+/*
+ * 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 tpch;
+
+
+load  table LineItem using localfs ((`path`=`asterix_nc1://data/tpch0.001/lineitem.tbl`),(`format`=`delimited-text`),(`delimiter`=`|`)) pre-sorted;
+
+load  table Orders using localfs ((`path`=`asterix_nc1://data/tpch0.001/orders.tbl`),(`format`=`delimited-text`),(`delimiter`=`|`)) pre-sorted;
+
+load  table Customer using localfs ((`path`=`asterix_nc1://data/tpch0.001/customer.tbl`),(`format`=`delimited-text`),(`delimiter`=`|`)) pre-sorted;
+

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/a89fae64/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-star/no_star/no_star.3.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-star/no_star/no_star.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-star/no_star/no_star.3.query.sqlpp
new file mode 100644
index 0000000..14ae8d1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-star/no_star/no_star.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.
+ */
+
+USE tpch;
+
+
+SELECT
+FROM  Customer c,
+      Orders o,
+      LineItem l
+WHERE c_mktsegment = 'BUILDING' AND c_custkey = o_custkey
+      AND l_orderkey = o_orderkey AND o_orderdate < '1995-03-15'
+      AND l_shipdate > '1995-03-15'
+ORDER BY l_linenumber, l_orderkey
+LIMIT 3;
+

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/a89fae64/asterixdb/asterix-app/src/test/resources/runtimets/results/select-star/from/from.1.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/select-star/from/from.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/select-star/from/from.1.adm
new file mode 100644
index 0000000..fa3c0bf
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/select-star/from/from.1.adm
@@ -0,0 +1,3 @@
+{ "c": { "c_custkey": 73, "c_name": "Customer#000000073", "c_address": "8IhIxreu4Ug6tt5mog4", "c_nationkey": 0, "c_phone": "10-473-439-3214", "c_acctbal": 4288.5, "c_mktsegment": "BUILDING", "c_comment": "usual, unusual packages sleep busily along the furiou" }, "o": { "o_orderkey": 1637, "o_custkey": 73, "o_orderstatus": "F", "o_totalprice": 180912.15, "o_orderdate": "1995-02-08", "o_orderpriority": "4-NOT SPECIFIED", "o_clerk": "Clerk#000000189", "o_shippriority": 0, "o_comment": " final accounts. blithely silent ideas cajole bravely. carefully express " }, "l": { "l_orderkey": 1637, "l_partkey": 86, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 49, "l_extendedprice": 48317.92, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "F", "l_shipdate": "1995-06-08", "l_commitdate": "1995-04-19", "l_receiptdate": "1995-07-01", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": ". blithely i" } }
+{ "c": { "c_custkey": 64, "c_name": "Customer#000000064", "c_address": "MbCeGY20kaKK3oalJD,OT", "c_nationkey": 3, "c_phone": "13-558-731-7204", "c_acctbal": -646.64, "c_mktsegment": "BUILDING", "c_comment": "structions after the quietly ironic theodolites cajole be" }, "o": { "o_orderkey": 4423, "o_custkey": 64, "o_orderstatus": "F", "o_totalprice": 4913.06, "o_orderdate": "1995-02-17", "o_orderpriority": "5-LOW", "o_clerk": "Clerk#000000888", "o_shippriority": 0, "o_comment": "excuses are ruthless" }, "l": { "l_orderkey": 4423, "l_partkey": 150, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 3, "l_extendedprice": 3150.45, "l_discount": 0.03, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-03-22", "l_commitdate": "1995-04-06", "l_receiptdate": "1995-04-19", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": " final theodolites nag after the bli" } }
+{ "c": { "c_custkey": 32, "c_name": "Customer#000000032", "c_address": "jD2xZzi UmId,DCtNBLXKj9q0Tlp2iQ6ZcO3J", "c_nationkey": 15, "c_phone": "25-430-914-2194", "c_acctbal": 3471.53, "c_mktsegment": "BUILDING", "c_comment": "cial ideas. final, furious requests across the e" }, "o": { "o_orderkey": 998, "o_custkey": 32, "o_orderstatus": "F", "o_totalprice": 65269.38, "o_orderdate": "1994-11-26", "o_orderpriority": "4-NOT SPECIFIED", "o_clerk": "Clerk#000000956", "o_shippriority": 0, "o_comment": "ronic dolphins. ironic, bold ideas haggle furiously furious" }, "l": { "l_orderkey": 998, "l_partkey": 181, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 7, "l_extendedprice": 7568.26, "l_discount": 0.1, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-03-24", "l_commitdate": "1995-01-18", "l_receiptdate": "1995-04-03", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "nic deposits. even asym" } }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/a89fae64/asterixdb/asterix-app/src/test/resources/runtimets/results/select-star/group_by/group_by.1.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/select-star/group_by/group_by.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/select-star/group_by/group_by.1.adm
new file mode 100644
index 0000000..a26aeb8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/select-star/group_by/group_by.1.adm
@@ -0,0 +1,5 @@
+{ "name": "AFRICA", "$1": [ { "r": { "r_regionkey": 0, "r_name": "AFRICA", "r_comment": "lar deposits. blithely final packages cajole. regular waters are final requests. regular accounts are according to " } } ] }
+{ "name": "AMERICA", "$1": [ { "r": { "r_regionkey": 1, "r_name": "AMERICA", "r_comment": "hs use ironic, even requests. s" } } ] }
+{ "name": "ASIA", "$1": [ { "r": { "r_regionkey": 2, "r_name": "ASIA", "r_comment": "ges. thinly even pinto beans ca" } } ] }
+{ "name": "EUROPE", "$1": [ { "r": { "r_regionkey": 3, "r_name": "EUROPE", "r_comment": "ly final courts cajole furiously final excuse" } } ] }
+{ "name": "MIDDLE EAST", "$1": [ { "r": { "r_regionkey": 4, "r_name": "MIDDLE EAST", "r_comment": "uickly special accounts cajole carefully blithely close requests. carefully final asymptotes haggle furiousl" } } ] }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/a89fae64/asterixdb/asterix-app/src/test/resources/runtimets/results/select-star/join/join.1.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/select-star/join/join.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/select-star/join/join.1.adm
new file mode 100644
index 0000000..fa3c0bf
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/select-star/join/join.1.adm
@@ -0,0 +1,3 @@
+{ "c": { "c_custkey": 73, "c_name": "Customer#000000073", "c_address": "8IhIxreu4Ug6tt5mog4", "c_nationkey": 0, "c_phone": "10-473-439-3214", "c_acctbal": 4288.5, "c_mktsegment": "BUILDING", "c_comment": "usual, unusual packages sleep busily along the furiou" }, "o": { "o_orderkey": 1637, "o_custkey": 73, "o_orderstatus": "F", "o_totalprice": 180912.15, "o_orderdate": "1995-02-08", "o_orderpriority": "4-NOT SPECIFIED", "o_clerk": "Clerk#000000189", "o_shippriority": 0, "o_comment": " final accounts. blithely silent ideas cajole bravely. carefully express " }, "l": { "l_orderkey": 1637, "l_partkey": 86, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 49, "l_extendedprice": 48317.92, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "F", "l_shipdate": "1995-06-08", "l_commitdate": "1995-04-19", "l_receiptdate": "1995-07-01", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": ". blithely i" } }
+{ "c": { "c_custkey": 64, "c_name": "Customer#000000064", "c_address": "MbCeGY20kaKK3oalJD,OT", "c_nationkey": 3, "c_phone": "13-558-731-7204", "c_acctbal": -646.64, "c_mktsegment": "BUILDING", "c_comment": "structions after the quietly ironic theodolites cajole be" }, "o": { "o_orderkey": 4423, "o_custkey": 64, "o_orderstatus": "F", "o_totalprice": 4913.06, "o_orderdate": "1995-02-17", "o_orderpriority": "5-LOW", "o_clerk": "Clerk#000000888", "o_shippriority": 0, "o_comment": "excuses are ruthless" }, "l": { "l_orderkey": 4423, "l_partkey": 150, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 3, "l_extendedprice": 3150.45, "l_discount": 0.03, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-03-22", "l_commitdate": "1995-04-06", "l_receiptdate": "1995-04-19", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": " final theodolites nag after the bli" } }
+{ "c": { "c_custkey": 32, "c_name": "Customer#000000032", "c_address": "jD2xZzi UmId,DCtNBLXKj9q0Tlp2iQ6ZcO3J", "c_nationkey": 15, "c_phone": "25-430-914-2194", "c_acctbal": 3471.53, "c_mktsegment": "BUILDING", "c_comment": "cial ideas. final, furious requests across the e" }, "o": { "o_orderkey": 998, "o_custkey": 32, "o_orderstatus": "F", "o_totalprice": 65269.38, "o_orderdate": "1994-11-26", "o_orderpriority": "4-NOT SPECIFIED", "o_clerk": "Clerk#000000956", "o_shippriority": 0, "o_comment": "ronic dolphins. ironic, bold ideas haggle furiously furious" }, "l": { "l_orderkey": 998, "l_partkey": 181, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 7, "l_extendedprice": 7568.26, "l_discount": 0.1, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-03-24", "l_commitdate": "1995-01-18", "l_receiptdate": "1995-04-03", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "nic deposits. even asym" } }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/a89fae64/asterixdb/asterix-app/src/test/resources/runtimets/results/select-star/mixed/mixed.1.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/select-star/mixed/mixed.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/select-star/mixed/mixed.1.adm
new file mode 100644
index 0000000..38aff12
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/select-star/mixed/mixed.1.adm
@@ -0,0 +1,3 @@
+{ "c_custkey": 73, "c": { "c_custkey": 73, "c_name": "Customer#000000073", "c_address": "8IhIxreu4Ug6tt5mog4", "c_nationkey": 0, "c_phone": "10-473-439-3214", "c_acctbal": 4288.5, "c_mktsegment": "BUILDING", "c_comment": "usual, unusual packages sleep busily along the furiou" }, "o": { "o_orderkey": 1637, "o_custkey": 73, "o_orderstatus": "F", "o_totalprice": 180912.15, "o_orderdate": "1995-02-08", "o_orderpriority": "4-NOT SPECIFIED", "o_clerk": "Clerk#000000189", "o_shippriority": 0, "o_comment": " final accounts. blithely silent ideas cajole bravely. carefully express " }, "l": { "l_orderkey": 1637, "l_partkey": 86, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 49, "l_extendedprice": 48317.92, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "F", "l_shipdate": "1995-06-08", "l_commitdate": "1995-04-19", "l_receiptdate": "1995-07-01", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": ". blithely i" }, "o_orderkey": 1637 }
+{ "c_custkey": 64, "c": { "c_custkey": 64, "c_name": "Customer#000000064", "c_address": "MbCeGY20kaKK3oalJD,OT", "c_nationkey": 3, "c_phone": "13-558-731-7204", "c_acctbal": -646.64, "c_mktsegment": "BUILDING", "c_comment": "structions after the quietly ironic theodolites cajole be" }, "o": { "o_orderkey": 4423, "o_custkey": 64, "o_orderstatus": "F", "o_totalprice": 4913.06, "o_orderdate": "1995-02-17", "o_orderpriority": "5-LOW", "o_clerk": "Clerk#000000888", "o_shippriority": 0, "o_comment": "excuses are ruthless" }, "l": { "l_orderkey": 4423, "l_partkey": 150, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 3, "l_extendedprice": 3150.45, "l_discount": 0.03, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-03-22", "l_commitdate": "1995-04-06", "l_receiptdate": "1995-04-19", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": " final theodolites nag after the bli" }, "o_orderkey": 4423 }
+{ "c_custkey": 32, "c": { "c_custkey": 32, "c_name": "Customer#000000032", "c_address": "jD2xZzi UmId,DCtNBLXKj9q0Tlp2iQ6ZcO3J", "c_nationkey": 15, "c_phone": "25-430-914-2194", "c_acctbal": 3471.53, "c_mktsegment": "BUILDING", "c_comment": "cial ideas. final, furious requests across the e" }, "o": { "o_orderkey": 998, "o_custkey": 32, "o_orderstatus": "F", "o_totalprice": 65269.38, "o_orderdate": "1994-11-26", "o_orderpriority": "4-NOT SPECIFIED", "o_clerk": "Clerk#000000956", "o_shippriority": 0, "o_comment": "ronic dolphins. ironic, bold ideas haggle furiously furious" }, "l": { "l_orderkey": 998, "l_partkey": 181, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 7, "l_extendedprice": 7568.26, "l_discount": 0.1, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-03-24", "l_commitdate": "1995-01-18", "l_receiptdate": "1995-04-03", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "nic deposits. even asym" }, "o_orderkey": 998 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/a89fae64/asterixdb/asterix-app/src/test/resources/runtimets/results/select-star/no_star/no_star.1.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/select-star/no_star/no_star.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/select-star/no_star/no_star.1.adm
new file mode 100644
index 0000000..fa3c0bf
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/select-star/no_star/no_star.1.adm
@@ -0,0 +1,3 @@
+{ "c": { "c_custkey": 73, "c_name": "Customer#000000073", "c_address": "8IhIxreu4Ug6tt5mog4", "c_nationkey": 0, "c_phone": "10-473-439-3214", "c_acctbal": 4288.5, "c_mktsegment": "BUILDING", "c_comment": "usual, unusual packages sleep busily along the furiou" }, "o": { "o_orderkey": 1637, "o_custkey": 73, "o_orderstatus": "F", "o_totalprice": 180912.15, "o_orderdate": "1995-02-08", "o_orderpriority": "4-NOT SPECIFIED", "o_clerk": "Clerk#000000189", "o_shippriority": 0, "o_comment": " final accounts. blithely silent ideas cajole bravely. carefully express " }, "l": { "l_orderkey": 1637, "l_partkey": 86, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 49, "l_extendedprice": 48317.92, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "F", "l_shipdate": "1995-06-08", "l_commitdate": "1995-04-19", "l_receiptdate": "1995-07-01", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": ". blithely i" } }
+{ "c": { "c_custkey": 64, "c_name": "Customer#000000064", "c_address": "MbCeGY20kaKK3oalJD,OT", "c_nationkey": 3, "c_phone": "13-558-731-7204", "c_acctbal": -646.64, "c_mktsegment": "BUILDING", "c_comment": "structions after the quietly ironic theodolites cajole be" }, "o": { "o_orderkey": 4423, "o_custkey": 64, "o_orderstatus": "F", "o_totalprice": 4913.06, "o_orderdate": "1995-02-17", "o_orderpriority": "5-LOW", "o_clerk": "Clerk#000000888", "o_shippriority": 0, "o_comment": "excuses are ruthless" }, "l": { "l_orderkey": 4423, "l_partkey": 150, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 3, "l_extendedprice": 3150.45, "l_discount": 0.03, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-03-22", "l_commitdate": "1995-04-06", "l_receiptdate": "1995-04-19", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": " final theodolites nag after the bli" } }
+{ "c": { "c_custkey": 32, "c_name": "Customer#000000032", "c_address": "jD2xZzi UmId,DCtNBLXKj9q0Tlp2iQ6ZcO3J", "c_nationkey": 15, "c_phone": "25-430-914-2194", "c_acctbal": 3471.53, "c_mktsegment": "BUILDING", "c_comment": "cial ideas. final, furious requests across the e" }, "o": { "o_orderkey": 998, "o_custkey": 32, "o_orderstatus": "F", "o_totalprice": 65269.38, "o_orderdate": "1994-11-26", "o_orderpriority": "4-NOT SPECIFIED", "o_clerk": "Clerk#000000956", "o_shippriority": 0, "o_comment": "ronic dolphins. ironic, bold ideas haggle furiously furious" }, "l": { "l_orderkey": 998, "l_partkey": 181, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 7, "l_extendedprice": 7568.26, "l_discount": 0.1, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-03-24", "l_commitdate": "1995-01-18", "l_receiptdate": "1995-04-03", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "nic deposits. even asym" } }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/a89fae64/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
----------------------------------------------------------------------
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 944565c..195906b 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
@@ -4494,6 +4494,33 @@
         </test-case>
         -->
   </test-group>
+  <test-group name="select-star">
+    <test-case FilePath="select-star">
+      <compilation-unit name="group_by">
+        <output-dir compare="Text">group_by</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="select-star">
+      <compilation-unit name="from">
+        <output-dir compare="Text">from</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="select-star">
+      <compilation-unit name="join">
+        <output-dir compare="Text">join</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="select-star">
+      <compilation-unit name="mixed">
+        <output-dir compare="Text">mixed</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="select-star">
+      <compilation-unit name="no_star">
+        <output-dir compare="Text">no_star</output-dir>
+      </compilation-unit>
+    </test-case>
+  </test-group>
   <test-group name="semistructured">
     <test-case FilePath="semistructured">
       <compilation-unit name="count-nullable">

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/a89fae64/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/SqlppFunctionBodyRewriter.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/SqlppFunctionBodyRewriter.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/SqlppFunctionBodyRewriter.java
index 0189711..49823e7 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/SqlppFunctionBodyRewriter.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/SqlppFunctionBodyRewriter.java
@@ -40,15 +40,15 @@ class SqlppFunctionBodyRewriter extends SqlppQueryRewriter {
         // Inlines WITH expressions.
         inlineWithExpressions();
 
-        // Rewrites like/not-like expressions.
-        rewriteOperatorExpression();
-
         // Rewrites SQL-92 global aggregations.
         rewriteGlobalAggregations();
 
         // Group-by core/sugar rewrites.
         rewriteGroupBys();
 
+        // Rewrites like/not-like expressions.
+        rewriteOperatorExpression();
+
         // Generates ids for variables (considering scopes) but DOES NOT replace unbounded variable access with the dataset function.
         // An unbounded variable within a function could be a bounded variable in the top-level query.
         variableCheckAndRewrite(false);

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/a89fae64/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/SqlppQueryRewriter.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/SqlppQueryRewriter.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/SqlppQueryRewriter.java
index daffb97..5a62e1e 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/SqlppQueryRewriter.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/SqlppQueryRewriter.java
@@ -100,15 +100,15 @@ class SqlppQueryRewriter implements IQueryRewriter {
         // Inlines WITH expressions.
         inlineWithExpressions();
 
-        // Rewrites like/not-like expressions.
-        rewriteOperatorExpression();
-
         // Rewrites SQL-92 global aggregations.
         rewriteGlobalAggregations();
 
         // Group-by core/sugar rewrites.
         rewriteGroupBys();
 
+        // Rewrites like/not-like expressions.
+        rewriteOperatorExpression();
+
         // Generate ids for variables (considering scopes) and replace global variable access with the dataset function.
         variableCheckAndRewrite(true);
 
@@ -165,9 +165,9 @@ class SqlppQueryRewriter implements IQueryRewriter {
         if (topExpr == null) {
             return;
         }
-        // Rewrites like/not-like operators into function call expressions.
-        OperatorExpressionVisitor likeExpressionVisitor = new OperatorExpressionVisitor(context);
-        likeExpressionVisitor.visit(topExpr, null);
+        // Rewrites like/not-like/in/not-in operators into function call expressions.
+        OperatorExpressionVisitor operatorExpressionVisitor = new OperatorExpressionVisitor(context);
+        operatorExpressionVisitor.visit(topExpr, null);
     }
 
     protected void inlineColumnAlias() throws AsterixException {
@@ -317,7 +317,9 @@ class SqlppQueryRewriter implements IQueryRewriter {
 
         @Override
         public Void visit(Projection projection, Void arg) throws AsterixException {
-            projection.getExpression().accept(this, arg);
+            if (!projection.star()) {
+                projection.getExpression().accept(this, arg);
+            }
             return null;
         }
 

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/a89fae64/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/InlineColumnAliasVisitor.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/InlineColumnAliasVisitor.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/InlineColumnAliasVisitor.java
index 2918a90..6b74d3f 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/InlineColumnAliasVisitor.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/InlineColumnAliasVisitor.java
@@ -130,6 +130,9 @@ public class InlineColumnAliasVisitor extends AbstractSqlppQueryExpressionVisito
 
     @Override
     public Void visit(Projection projection, Boolean overwriteWithGbyKeyVarRefs) throws AsterixException {
+        if (projection.star()) {
+            return null;
+        }
         projection.getExpression().accept(this, overwriteWithGbyKeyVarRefs);
         VariableExpr columnAlias = new VariableExpr(
                 SqlppVariableUtil.toInternalVariableIdentifier(projection.getName()));

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/a89fae64/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/util/SqlppVariableUtil.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/util/SqlppVariableUtil.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/util/SqlppVariableUtil.java
index 12a2d20..a3e8447 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/util/SqlppVariableUtil.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/util/SqlppVariableUtil.java
@@ -56,6 +56,18 @@ public class SqlppVariableUtil {
         return new VarIdentifier(varName);
     }
 
+    public static String variableNameToDisplayedFieldName(String varName) {
+        if (varName.startsWith(USER_VAR_PREFIX)) {
+            return varName.substring(1);
+        } else {
+            // We use prefix "$" for user-defined variables and "#" for system-generated variables.
+            // However, in displayed query results, "$" is the prefix for
+            // system-generated variables/alias. Therefore we need to replace the prefix
+            // "#" with "$" if the system-generated alias needs to present in the final result.
+            return USER_VAR_PREFIX + varName.substring(1);
+        }
+    }
+
     public static String toUserDefinedName(String varName) {
         if (varName.startsWith(USER_VAR_PREFIX)) {
             return varName.substring(1);

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/a89fae64/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/CheckSql92AggregateVisitor.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/CheckSql92AggregateVisitor.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/CheckSql92AggregateVisitor.java
index 7c46d73..669b0a8 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/CheckSql92AggregateVisitor.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/CheckSql92AggregateVisitor.java
@@ -201,6 +201,9 @@ public class CheckSql92AggregateVisitor extends AbstractSqlppQueryExpressionVisi
 
     @Override
     public Boolean visit(Projection projection, ILangExpression parentSelectBlock) throws AsterixException {
+        if (projection.star()) {
+            return false;
+        }
         return projection.getExpression().accept(this, parentSelectBlock);
     }
 

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/a89fae64/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/base/AbstractSqlppSimpleExpressionVisitor.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/base/AbstractSqlppSimpleExpressionVisitor.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/base/AbstractSqlppSimpleExpressionVisitor.java
index 262b260..2098156 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/base/AbstractSqlppSimpleExpressionVisitor.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/base/AbstractSqlppSimpleExpressionVisitor.java
@@ -107,7 +107,9 @@ public class AbstractSqlppSimpleExpressionVisitor
 
     @Override
     public Expression visit(Projection projection, ILangExpression arg) throws AsterixException {
-        projection.setExpression(projection.getExpression().accept(this, arg));
+        if (!projection.star()) {
+            projection.setExpression(projection.getExpression().accept(this, arg));
+        }
         return null;
     }
 

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/a89fae64/asterixdb/asterix-lang-sqlpp/src/main/javacc/SQLPP.html
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/javacc/SQLPP.html b/asterixdb/asterix-lang-sqlpp/src/main/javacc/SQLPP.html
index 34f576f..b80aa61 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/javacc/SQLPP.html
+++ b/asterixdb/asterix-lang-sqlpp/src/main/javacc/SQLPP.html
@@ -19,6 +19,7 @@
 | &lt;ASC: "asc"&gt;
 | &lt;AT: "at"&gt;
 | &lt;AUTOGENERATED: "autogenerated"&gt;
+| &lt;BETWEEN: "between"&gt;
 | &lt;BTREE: "btree"&gt;
 | &lt;BY: "by"&gt;
 | &lt;CASE: "case"&gt;
@@ -72,6 +73,7 @@
 | &lt;LEFT: "left"&gt;
 | &lt;LETTING: "letting"&gt;
 | &lt;LET: "let"&gt;
+| &lt;LIKE: "like"&gt;
 | &lt;LIMIT: "limit"&gt;
 | &lt;LOAD: "load"&gt;
 | &lt;NEST: "nest"&gt;
@@ -688,7 +690,7 @@
 <TR>
 <TD ALIGN=RIGHT VALIGN=BASELINE><A NAME="prod67">RelExpr</A></TD>
 <TD ALIGN=CENTER VALIGN=BASELINE>::=</TD>
-<TD ALIGN=LEFT VALIGN=BASELINE><A HREF="#prod68">IsExpr</A> ( ( &lt;LT&gt; | &lt;GT&gt; | &lt;LE&gt; | &lt;GE&gt; | &lt;EQ&gt; | &lt;NE&gt; | &lt;SIMILAR&gt; ) <A HREF="#prod68">IsExpr</A> )?</TD>
+<TD ALIGN=LEFT VALIGN=BASELINE><A HREF="#prod68">IsExpr</A> ( ( &lt;LT&gt; | &lt;GT&gt; | &lt;LE&gt; | &lt;GE&gt; | &lt;EQ&gt; | &lt;NE&gt; | &lt;SIMILAR&gt; | ( &lt;NOT&gt; )? ( &lt;LIKE&gt; | &lt;IN&gt; ) ) <A HREF="#prod68">IsExpr</A> )?</TD>
 </TR>
 <TR>
 <TD ALIGN=RIGHT VALIGN=BASELINE><A NAME="prod68">IsExpr</A></TD>
@@ -818,7 +820,7 @@
 <TR>
 <TD ALIGN=RIGHT VALIGN=BASELINE><A NAME="prod93">SelectClause</A></TD>
 <TD ALIGN=CENTER VALIGN=BASELINE>::=</TD>
-<TD ALIGN=LEFT VALIGN=BASELINE>&lt;SELECT&gt; ( &lt;ALL&gt; | &lt;DISTINCT&gt; )? ( <A HREF="#prod98">SelectRegular</A> | <A HREF="#prod99">SelectElement</A> )</TD>
+<TD ALIGN=LEFT VALIGN=BASELINE>&lt;SELECT&gt; ( &lt;ALL&gt; | &lt;DISTINCT&gt; )? ( <A HREF="#prod98">SelectRegular</A> | <A HREF="#prod99">SelectElement</A> )?</TD>
 </TR>
 <TR>
 <TD ALIGN=RIGHT VALIGN=BASELINE><A NAME="prod98">SelectRegular</A></TD>

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/a89fae64/asterixdb/asterix-lang-sqlpp/src/main/javacc/SQLPP.jj
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/javacc/SQLPP.jj b/asterixdb/asterix-lang-sqlpp/src/main/javacc/SQLPP.jj
index ebcfeea..9b0c71c 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/javacc/SQLPP.jj
+++ b/asterixdb/asterix-lang-sqlpp/src/main/javacc/SQLPP.jj
@@ -2315,8 +2315,14 @@ SelectClause SelectClause() throws ParseException: {
     selectRegular = SelectRegular()
     |
     selectElement = SelectElement()
-  )
+  )?
   {
+    if(selectRegular == null && selectElement == null){
+        Projection projection = new Projection(null, null, true, false);
+        List<Projection> projections = new ArrayList<Projection>();
+        projections.add(projection);
+        selectRegular = new SelectRegular(projections);
+    }
     return new SelectClause(selectElement, selectRegular, distinct);
   }
 }
@@ -2329,9 +2335,9 @@ SelectRegular SelectRegular() throws ParseException: {
       Projection projection = null;
    }
    projection = Projection() { projections.add(projection); }
-   ( LOOKAHEAD(2) <COMMA>
-      projection = Projection() {projections.add(projection);}
-   )*
+    ( LOOKAHEAD(2) <COMMA>
+       projection = Projection() {projections.add(projection);}
+    )*
   {
     return new SelectRegular(projections);
   }
@@ -2363,7 +2369,7 @@ Projection Projection() throws ParseException: {
     | <MUL> {star = true; }
   )
   {
-    if(name == null){
+    if(!star && name == null){
        name = SqlppVariableUtil.toUserDefinedName(ExpressionToVariableUtil.getGeneratedIdentifier(expr));
     }
     return new Projection(expr, name, star, exprStar);

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/a89fae64/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/prettyprint/LogicalOperatorPrettyPrintVisitor.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/prettyprint/LogicalOperatorPrettyPrintVisitor.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/prettyprint/LogicalOperatorPrettyPrintVisitor.java
index 647c9ee..f961bd9 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/prettyprint/LogicalOperatorPrettyPrintVisitor.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/prettyprint/LogicalOperatorPrettyPrintVisitor.java
@@ -143,7 +143,7 @@ public class LogicalOperatorPrettyPrintVisitor implements ILogicalOperatorVisito
     @Override
     public String visitNestedTupleSourceOperator(NestedTupleSourceOperator op, Integer indent) {
         StringBuilder buffer = new StringBuilder();
-        addIndent(buffer, indent).append("nested tuple source" + op.getDataSourceReference().getValue());
+        addIndent(buffer, indent).append("nested tuple source");
         return buffer.toString();
     }
 
@@ -387,10 +387,11 @@ public class LogicalOperatorPrettyPrintVisitor implements ILogicalOperatorVisito
         buffer.append(" partitioned by ");
         pprintExprList(op.getPrimaryKeyExpressions(), buffer, indent);
         if (op.getOperation() == Kind.UPSERT) {
-            buffer.append(" out: ([record-before-upsert:" + op.getPrevRecordVar()
-                    + ((op.getPrevAdditionalNonFilteringVars() != null)
-                            ? (", additional-before-upsert: " + op.getPrevAdditionalNonFilteringVars()) : "")
-                    + "]) ");
+            buffer.append(
+                    " out: ([record-before-upsert:" + op.getPrevRecordVar()
+                            + ((op.getPrevAdditionalNonFilteringVars() != null)
+                                    ? (", additional-before-upsert: " + op.getPrevAdditionalNonFilteringVars()) : "")
+                            + "]) ");
         }
         if (op.isBulkload()) {
             buffer.append(" [bulkload]");