You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@tajo.apache.org by ji...@apache.org on 2014/02/16 08:18:20 UTC

git commit: TAJO-554: LogicalPlanner should allow additional expressions with asterisk in select list. (jihoon)

Repository: incubator-tajo
Updated Branches:
  refs/heads/master 519a16a23 -> cd7bbae0d


TAJO-554: LogicalPlanner should allow additional expressions with asterisk in select list. (jihoon)


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

Branch: refs/heads/master
Commit: cd7bbae0d60395fd312fbd153c69f9a8d480af34
Parents: 519a16a
Author: Jihoon Son <ji...@apache.org>
Authored: Sun Feb 16 16:16:55 2014 +0900
Committer: Jihoon Son <ji...@apache.org>
Committed: Sun Feb 16 16:16:55 2014 +0900

----------------------------------------------------------------------
 CHANGES.txt                                     |   3 +
 .../java/org/apache/tajo/algebra/OpType.java    |   1 +
 .../org/apache/tajo/algebra/Projection.java     |  13 +-
 .../tajo/algebra/QualifiedAsteriskExpr.java     |  64 ++++++++++
 .../org/apache/tajo/engine/parser/SQLParser.g4  |  17 ++-
 .../tajo/engine/parser/HiveConverter.java       |  74 +++--------
 .../apache/tajo/engine/parser/SQLAnalyzer.java  |  31 +++--
 .../tajo/engine/planner/AlgebraVisitor.java     |   1 +
 .../tajo/engine/planner/BaseAlgebraVisitor.java |  14 +-
 .../engine/planner/LogicalPlanPreprocessor.java | 128 +++++++++++++++++--
 .../tajo/engine/planner/LogicalPlanner.java     |  37 ++----
 .../tajo/engine/parser/TestHiveConverter.java   |  32 +++++
 .../tajo/engine/parser/TestSQLAnalyzer.java     |  24 ++++
 .../tajo/engine/planner/TestLogicalPlanner.java |  25 +++-
 .../apache/tajo/engine/query/TestJoinQuery.java |  32 +++++
 .../tajo/engine/query/TestSelectQuery.java      |   8 ++
 .../apache/tajo/engine/query/TestSortQuery.java |   9 +-
 .../testCrossJoinWithAsterisk1.sql              |   1 +
 .../testCrossJoinWithAsterisk2.sql              |   1 +
 .../testCrossJoinWithAsterisk3.sql              |   1 +
 .../testCrossJoinWithAsterisk4.sql              |   1 +
 .../TestSelectQuery/testSelectAsterisk4.sql     |   1 +
 .../queries/TestSortQuery/testAsterisk.sql      |   1 +
 .../resources/queries/default/asterisk_1.sql    |   1 +
 .../resources/queries/default/asterisk_2.sql    |   1 +
 .../resources/queries/default/asterisk_3.sql    |   1 +
 .../resources/queries/default/asterisk_4.sql    |   1 +
 .../testCrossJoinWithAsterisk1.result           |  27 ++++
 .../testCrossJoinWithAsterisk2.result           |  27 ++++
 .../testCrossJoinWithAsterisk3.result           |  27 ++++
 .../testCrossJoinWithAsterisk4.result           |  27 ++++
 .../TestSelectQuery/testSelectAsterisk4.result  |   7 +
 .../results/TestSortQuery/testAsterisk.result   |   7 +
 33 files changed, 514 insertions(+), 131 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/cd7bbae0/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index e5dde69..2ef8284 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -252,6 +252,9 @@ Release 0.8.0 - unreleased
 
   BUG FIXES
 
+    TAJO-554: LogicalPlanner should allow additional expressions with asterisk 
+    in select list. (jihoon)
+
     TAJO-593: outer groupby and groupby in derived table causes only one
     shuffle output number. (hyunsik)
 

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/cd7bbae0/tajo-algebra/src/main/java/org/apache/tajo/algebra/OpType.java
----------------------------------------------------------------------
diff --git a/tajo-algebra/src/main/java/org/apache/tajo/algebra/OpType.java b/tajo-algebra/src/main/java/org/apache/tajo/algebra/OpType.java
index 84f784e..91f322b 100644
--- a/tajo-algebra/src/main/java/org/apache/tajo/algebra/OpType.java
+++ b/tajo-algebra/src/main/java/org/apache/tajo/algebra/OpType.java
@@ -87,6 +87,7 @@ public enum OpType {
   Column(ColumnReferenceExpr.class),
   Target(NamedExpr.class),
   Function(FunctionExpr.class),
+  Asterisk(QualifiedAsteriskExpr.class),
 
   // Set Functions
   CountRowsFunction(CountRowsFunctionExpr.class),

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/cd7bbae0/tajo-algebra/src/main/java/org/apache/tajo/algebra/Projection.java
----------------------------------------------------------------------
diff --git a/tajo-algebra/src/main/java/org/apache/tajo/algebra/Projection.java b/tajo-algebra/src/main/java/org/apache/tajo/algebra/Projection.java
index 3d9f8a6..ecf92dc 100644
--- a/tajo-algebra/src/main/java/org/apache/tajo/algebra/Projection.java
+++ b/tajo-algebra/src/main/java/org/apache/tajo/algebra/Projection.java
@@ -22,7 +22,6 @@ import com.google.common.base.Objects;
 import org.apache.tajo.util.TUtil;
 
 public class Projection extends UnaryOperator implements Cloneable {
-  private boolean all;
   private boolean distinct = false;
 
   private NamedExpr[] targets;
@@ -43,14 +42,6 @@ public class Projection extends UnaryOperator implements Cloneable {
     distinct = true;
   }
 
-  public void setAll() {
-    all = true;
-  }
-
-  public boolean isAllProjected() {
-    return all;
-  }
-	
 	public NamedExpr[] getNamedExprs() {
 	  return this.targets;
 	}
@@ -60,13 +51,13 @@ public class Projection extends UnaryOperator implements Cloneable {
   }
 
   public int hashCode() {
-    return Objects.hashCode(all, distinct, Objects.hashCode(targets), getChild());
+    return Objects.hashCode(distinct, Objects.hashCode(targets), getChild());
   }
 
   @Override
   boolean equalsTo(Expr expr) {
     Projection another = (Projection) expr;
-    return TUtil.checkEquals(all, another.all) && distinct == another.distinct &&
+    return distinct == another.distinct &&
         TUtil.checkEquals(targets, another.targets);
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/cd7bbae0/tajo-algebra/src/main/java/org/apache/tajo/algebra/QualifiedAsteriskExpr.java
----------------------------------------------------------------------
diff --git a/tajo-algebra/src/main/java/org/apache/tajo/algebra/QualifiedAsteriskExpr.java b/tajo-algebra/src/main/java/org/apache/tajo/algebra/QualifiedAsteriskExpr.java
new file mode 100644
index 0000000..ab8e677
--- /dev/null
+++ b/tajo-algebra/src/main/java/org/apache/tajo/algebra/QualifiedAsteriskExpr.java
@@ -0,0 +1,64 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.algebra;
+
+import com.google.common.base.Objects;
+import org.apache.tajo.util.TUtil;
+
+public class QualifiedAsteriskExpr extends Expr {
+  private final static String ASTERISK = "*";
+  private String qualifier;
+
+  public QualifiedAsteriskExpr() {
+    super(OpType.Asterisk);
+  }
+
+  public QualifiedAsteriskExpr(String qualifier) {
+    this();
+    setQualifier(qualifier);
+  }
+
+  public boolean hasQualifier() {
+    return this.qualifier != null;
+  }
+
+  public void setQualifier(String qualifier) {
+    this.qualifier = qualifier.toLowerCase();
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hashCode(qualifier, ASTERISK);
+  }
+
+  @Override
+  boolean equalsTo(Expr expr) {
+    QualifiedAsteriskExpr another = (QualifiedAsteriskExpr) expr;
+    return TUtil.checkEquals(this.qualifier, another.qualifier);
+  }
+
+  public String getQualifier() {
+    return qualifier;
+  }
+
+  @Override
+  public String toString() {
+    return hasQualifier() ? qualifier + "." + ASTERISK : ASTERISK;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/cd7bbae0/tajo-core/tajo-core-backend/src/main/antlr4/org/apache/tajo/engine/parser/SQLParser.g4
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/antlr4/org/apache/tajo/engine/parser/SQLParser.g4 b/tajo-core/tajo-core-backend/src/main/antlr4/org/apache/tajo/engine/parser/SQLParser.g4
index ab6bff6..db04d4b 100644
--- a/tajo-core/tajo-core-backend/src/main/antlr4/org/apache/tajo/engine/parser/SQLParser.g4
+++ b/tajo-core/tajo-core-backend/src/main/antlr4/org/apache/tajo/engine/parser/SQLParser.g4
@@ -1040,13 +1040,20 @@ query_specification
   ;
 
 select_list
-  : MULTIPLY
-  | select_sublist (COMMA select_sublist)*
+  : select_sublist (COMMA select_sublist)*
   ;
 
 select_sublist
   : derived_column
-  | asterisked_qualifier=identifier DOT MULTIPLY
+  | qualified_asterisk
+  ;
+
+derived_column
+  : value_expression as_clause?
+  ;
+
+qualified_asterisk
+  : (tb_name=Identifier DOT)? MULTIPLY
   ;
 
 set_qualifier
@@ -1054,10 +1061,6 @@ set_qualifier
   | ALL
   ;
 
-derived_column
-  : value_expression as_clause?
-  ;
-
 column_reference
   : (tb_name=identifier DOT)? name=identifier
   ;

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/cd7bbae0/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/parser/HiveConverter.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/parser/HiveConverter.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/parser/HiveConverter.java
index a8a555b..a762720 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/parser/HiveConverter.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/parser/HiveConverter.java
@@ -27,12 +27,13 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.tajo.algebra.*;
 import org.apache.tajo.common.TajoDataTypes;
+import org.apache.tajo.engine.parser.HiveParser.TableAllColumnsContext;
 
 import java.math.BigInteger;
 import java.util.ArrayList;
 import java.util.HashMap;
-import java.util.Map;
 import java.util.List;
+import java.util.Map;
 
 public class HiveConverter extends HiveParserBaseVisitor<Expr> {
   private static final Log LOG = LogFactory.getLog(HiveConverter.class.getName());
@@ -90,7 +91,6 @@ public class HiveConverter extends HiveParserBaseVisitor<Expr> {
       select = (Projection) visitSelectClause(ctx.selectClause());
       if (ctx.selectClause().KW_DISTINCT() != null) {
         select.setDistinct();
-        ;
       }
 
     }
@@ -158,9 +158,6 @@ public class HiveConverter extends HiveParserBaseVisitor<Expr> {
     if (current != null)
       projection.setChild(current);
 
-    if (select.isAllProjected())
-      projection.setAll();
-
     if (select.isDistinct())
       projection.setDistinct();
 
@@ -193,7 +190,6 @@ public class HiveConverter extends HiveParserBaseVisitor<Expr> {
         select = (Projection) visitSelectClause(ctx.selectClause());
         if (ctx.selectClause().KW_DISTINCT() != null) {
           select.setDistinct();
-          ;
         }
 
       }
@@ -258,9 +254,6 @@ public class HiveConverter extends HiveParserBaseVisitor<Expr> {
       if (current != null)
         projection.setChild(current);
 
-      if (select.isAllProjected())
-        projection.setAll();
-
       if (select.isDistinct())
         projection.setDistinct();
 
@@ -312,7 +305,6 @@ public class HiveConverter extends HiveParserBaseVisitor<Expr> {
 
     if (ctx.selectClause().KW_DISTINCT() != null) {
       select.setDistinct();
-      ;
     }
 
     Expr from = visitFromClause(ctx.fromClause());
@@ -373,9 +365,6 @@ public class HiveConverter extends HiveParserBaseVisitor<Expr> {
     if (current != null)
       projection.setChild(current);
 
-    if (select.isAllProjected())
-      projection.setAll();
-
     if (select.isDistinct())
       projection.setDistinct();
 
@@ -498,7 +487,7 @@ public class HiveConverter extends HiveParserBaseVisitor<Expr> {
       String tableAlias = "";
       for (int i = 0; i < ctx.subQuerySource().getChildCount(); i++) {
         if (ctx.subQuerySource().getChild(i) instanceof HiveParser.IdentifierContext) {
-          tableAlias = ((HiveParser.IdentifierContext) ctx.subQuerySource().getChild(i)).getText();
+          tableAlias = (ctx.subQuerySource().getChild(i)).getText();
         }
       }
 
@@ -547,56 +536,23 @@ public class HiveConverter extends HiveParserBaseVisitor<Expr> {
       targets[i] = visitSelectItem(ctx.selectItem(i));
     }
 
-    if (targets.length == 1) {
-      if (targets[0].getExpr().getType().equals(OpType.Column)) {
-        ColumnReferenceExpr columnReferenceExprs = (ColumnReferenceExpr) targets[0].getExpr();
-        if (columnReferenceExprs.getQualifier() == null && columnReferenceExprs.getName().equals("*"))
-          projection.setAll();
-        ;
-      }
-    }
-
-    if (!projection.isAllProjected())
-      projection.setNamedExprs(targets);
-
+    projection.setNamedExprs(targets);
     current = projection;
     return current;
   }
 
   @Override
   public NamedExpr visitSelectItem(HiveParser.SelectItemContext ctx) {
-    ColumnReferenceExpr columnReference;
     NamedExpr target = null;
 
-    String tableName = "", itemName = "", alias = "";
-
-    String[] selectItem = ctx.getText().split("\\.");
-
-    if (selectItem.length == 2) {
-      tableName = selectItem[0];
-      itemName = selectItem[1];
-    } else if (selectItem.length == 1) {
-      itemName = selectItem[0];
-    } else {
-      itemName = ctx.getText();
-    }
-
-    columnReference = new ColumnReferenceExpr(itemName);
-    if (!tableName.equals(""))
-      columnReference.setQualifier(tableName);
-
     if (ctx.selectExpression() != null) {
-      if (ctx.selectExpression().expression() != null) {
-        target = new NamedExpr(visitSelectExpression(ctx.selectExpression()));
-      } else {
-        target = new NamedExpr(columnReference);
-      }
+      target = new NamedExpr(visitSelectExpression(ctx.selectExpression()));
+    } else if (ctx.window_specification() != null) {
+      // TODO: if there is a window specification clause, we should handle it properly.
     }
 
-    if (ctx.identifier().size() > 0) {
-      alias = ctx.identifier(0).getText();
-      if (!alias.equals(itemName))
-        target.setAlias(alias);
+    if (ctx.identifier().size() > 0 && target != null) {
+      target.setAlias(ctx.identifier(0).getText());
     }
     return target;
   }
@@ -606,7 +562,7 @@ public class HiveConverter extends HiveParserBaseVisitor<Expr> {
     Expr current = null;
 
     if (ctx.tableAllColumns() != null) {
-      visitTableAllColumns(ctx.tableAllColumns());
+      current = visitTableAllColumns(ctx.tableAllColumns());
     } else {
       if (ctx.expression() != null) {
         current = visitExpression(ctx.expression());
@@ -617,6 +573,16 @@ public class HiveConverter extends HiveParserBaseVisitor<Expr> {
   }
 
   @Override
+  public Expr visitTableAllColumns(TableAllColumnsContext ctx) {
+    QualifiedAsteriskExpr target = new QualifiedAsteriskExpr();
+    if (ctx.tableName() != null) {
+      target.setQualifier(ctx.tableName().getText());
+    }
+
+    return target;
+  }
+
+  @Override
   public Expr visitExpression(HiveParser.ExpressionContext ctx) {
     Expr current = visitPrecedenceOrExpression(ctx.precedenceOrExpression());
     return current;

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/cd7bbae0/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/parser/SQLAnalyzer.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/parser/SQLAnalyzer.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/parser/SQLAnalyzer.java
index 493f892..29e4d43 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/parser/SQLAnalyzer.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/parser/SQLAnalyzer.java
@@ -211,8 +211,7 @@ public class SQLAnalyzer extends SQLParserBaseVisitor<Expr> {
   /**
    * <pre>
    *   select_list
-   *   : MULTIPLY
-   *   | select_sublist (COMMA select_sublist)*
+   *   : select_sublist (COMMA select_sublist)*
    *   ;
    * </pre>
    * @param ctx
@@ -221,15 +220,11 @@ public class SQLAnalyzer extends SQLParserBaseVisitor<Expr> {
   @Override
   public Projection visitSelect_list(SQLParser.Select_listContext ctx) {
     Projection projection = new Projection();
-    if (ctx.MULTIPLY() != null) {
-      projection.setAll();
-    } else {
-      NamedExpr[] targets = new NamedExpr[ctx.select_sublist().size()];
-      for (int i = 0; i < targets.length; i++) {
-        targets[i] = visitSelect_sublist(ctx.select_sublist(i));
-      }
-      projection.setNamedExprs(targets);
+    NamedExpr[] targets = new NamedExpr[ctx.select_sublist().size()];
+    for (int i = 0; i < targets.length; i++) {
+      targets[i] = visitSelect_sublist(ctx.select_sublist(i));
     }
+    projection.setNamedExprs(targets);
 
     return projection;
   }
@@ -238,7 +233,7 @@ public class SQLAnalyzer extends SQLParserBaseVisitor<Expr> {
    * <pre>
    *   select_sublist
    *   : derived_column
-   *   | asterisked_qualifier=Identifier DOT MULTIPLY
+   *   | asterisked_qualifier
    *   ;
    * </pre>
    * @param ctx
@@ -246,8 +241,8 @@ public class SQLAnalyzer extends SQLParserBaseVisitor<Expr> {
    */
   @Override
   public NamedExpr visitSelect_sublist(SQLParser.Select_sublistContext ctx) {
-    if (ctx.asterisked_qualifier != null) {
-      return new NamedExpr(new ColumnReferenceExpr(ctx.asterisked_qualifier.getText(), "*"));
+    if (ctx.qualified_asterisk() != null) {
+      return visitQualified_asterisk(ctx.qualified_asterisk());
     } else {
       return visitDerived_column(ctx.derived_column());
     }
@@ -861,7 +856,7 @@ public class SQLAnalyzer extends SQLParserBaseVisitor<Expr> {
   }
 
   @Override
-  public NamedExpr visitDerived_column(SQLParser.Derived_columnContext ctx) {
+  public NamedExpr visitDerived_column(Derived_columnContext ctx) {
     NamedExpr target = new NamedExpr(visitValue_expression(ctx.value_expression()));
     if (ctx.as_clause() != null) {
       target.setAlias(ctx.as_clause().identifier().getText());
@@ -869,7 +864,15 @@ public class SQLAnalyzer extends SQLParserBaseVisitor<Expr> {
     return target;
   }
 
+  @Override
+  public NamedExpr visitQualified_asterisk(Qualified_asteriskContext ctx) {
+    QualifiedAsteriskExpr target = new QualifiedAsteriskExpr();
+    if (ctx.tb_name != null) {
+      target.setQualifier(ctx.tb_name.getText());
+    }
 
+    return new NamedExpr(target);
+  }
 
   @Override
   public Expr visitCharacter_string_type(SQLParser.Character_string_typeContext ctx) {

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/cd7bbae0/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/AlgebraVisitor.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/AlgebraVisitor.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/AlgebraVisitor.java
index 1c710dc..acb9008 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/AlgebraVisitor.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/AlgebraVisitor.java
@@ -86,6 +86,7 @@ public interface AlgebraVisitor<CONTEXT, RESULT> {
   RESULT visitColumnReference(CONTEXT ctx, Stack<Expr> stack, ColumnReferenceExpr expr) throws PlanningException;
   RESULT visitTargetExpr(CONTEXT ctx, Stack<Expr> stack, NamedExpr expr) throws PlanningException;
   RESULT visitFunction(CONTEXT ctx, Stack<Expr> stack, FunctionExpr expr) throws PlanningException;
+  RESULT visitQualifiedAsterisk(CONTEXT ctx, Stack<Expr> stack, QualifiedAsteriskExpr expr) throws PlanningException;
 
   // set functions
   RESULT visitCountRowsFunction(CONTEXT ctx, Stack<Expr> stack, CountRowsFunctionExpr expr) throws PlanningException;

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/cd7bbae0/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/BaseAlgebraVisitor.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/BaseAlgebraVisitor.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/BaseAlgebraVisitor.java
index 25ee316..5e4c893 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/BaseAlgebraVisitor.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/BaseAlgebraVisitor.java
@@ -201,6 +201,9 @@ public class BaseAlgebraVisitor<CONTEXT, RESULT> implements AlgebraVisitor<CONTE
     case Function:
       current = visitFunction(ctx, stack, (FunctionExpr) expr);
       break;
+    case Asterisk:
+      current = visitQualifiedAsterisk(ctx, stack, (QualifiedAsteriskExpr) expr);
+      break;
 
 
     case CountRowsFunction:
@@ -270,10 +273,8 @@ public class BaseAlgebraVisitor<CONTEXT, RESULT> implements AlgebraVisitor<CONTE
   public RESULT visitProjection(CONTEXT ctx, Stack<Expr> stack, Projection expr) throws PlanningException {
     stack.push(expr);
     try {
-      if (!expr.isAllProjected()) {
-        for (NamedExpr target : expr.getNamedExprs()) {
-          visit(ctx, stack, target);
-        }
+      for (NamedExpr target : expr.getNamedExprs()) {
+        visit(ctx, stack, target);
       }
       if (expr.hasChild()) {
         return visit(ctx, stack, expr.getChild());
@@ -622,6 +623,11 @@ public class BaseAlgebraVisitor<CONTEXT, RESULT> implements AlgebraVisitor<CONTE
     return result;
   }
 
+  @Override
+  public RESULT visitQualifiedAsterisk(CONTEXT ctx, Stack<Expr> stack, QualifiedAsteriskExpr expr) throws PlanningException {
+    return null;
+  }
+
   ///////////////////////////////////////////////////////////////////////////////////////////////////////////
   // General Set Section
   ///////////////////////////////////////////////////////////////////////////////////////////////////////////

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/cd7bbae0/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/LogicalPlanPreprocessor.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/LogicalPlanPreprocessor.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/LogicalPlanPreprocessor.java
index 099e462..df9b952 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/LogicalPlanPreprocessor.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/LogicalPlanPreprocessor.java
@@ -20,14 +20,22 @@ package org.apache.tajo.engine.planner;
 
 import org.apache.tajo.algebra.*;
 import org.apache.tajo.catalog.CatalogService;
+import org.apache.tajo.catalog.Column;
 import org.apache.tajo.catalog.Schema;
 import org.apache.tajo.catalog.TableDesc;
 import org.apache.tajo.engine.eval.EvalNode;
 import org.apache.tajo.engine.eval.EvalType;
 import org.apache.tajo.engine.eval.FieldEval;
+import org.apache.tajo.engine.exception.NoSuchColumnException;
+import org.apache.tajo.engine.planner.LogicalPlan.QueryBlock;
 import org.apache.tajo.engine.planner.logical.*;
 import org.apache.tajo.engine.utils.SchemaUtil;
+import org.apache.tajo.util.TUtil;
 
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.List;
 import java.util.Stack;
 
 /**
@@ -77,6 +85,86 @@ class LogicalPlanPreprocessor extends BaseAlgebraVisitor<LogicalPlanPreprocessor
     return result;
   }
 
+  /**
+   * Get all columns of the relations correspondent to the asterisk expression.
+   * @param ctx
+   * @param asteriskExpr
+   * @return array of columns
+   * @throws PlanningException
+   */
+  public static Column[] getColumns(PreprocessContext ctx, QualifiedAsteriskExpr asteriskExpr)
+      throws PlanningException {
+    RelationNode relationOp = null;
+    QueryBlock block = ctx.currentBlock;
+    Collection<QueryBlock> queryBlocks = ctx.plan.getQueryBlocks();
+    if (asteriskExpr.hasQualifier()) {
+      relationOp = block.getRelation(asteriskExpr.getQualifier());
+
+      // if a column name is outside of this query block
+      if (relationOp == null) {
+        // TODO - nested query can only refer outer query block? or not?
+        for (QueryBlock eachBlock : queryBlocks) {
+          if (eachBlock.existsRelation(asteriskExpr.getQualifier())) {
+            relationOp = eachBlock.getRelation(asteriskExpr.getQualifier());
+          }
+        }
+      }
+
+      // If we cannot find any relation against a qualified column name
+      if (relationOp == null) {
+        throw new NoSuchColumnException(asteriskExpr.toString());
+      }
+
+      Schema schema = relationOp.getTableSchema();
+      Column[] resolvedColumns = new Column[schema.getColumnNum()];
+      return schema.getColumns().toArray(resolvedColumns);
+    } else { // if a column reference is not qualified
+      // columns of every relation should be resolved.
+      Iterator<RelationNode> iterator = block.getRelations().iterator();
+      Schema schema;
+      List<Column> resolvedColumns = TUtil.newList();
+
+      while (iterator.hasNext()) {
+        relationOp = iterator.next();
+        schema = relationOp.getTableSchema();
+        resolvedColumns.addAll(schema.getColumns());
+      }
+
+      if (resolvedColumns.size() == 0) {
+        throw new NoSuchColumnException(asteriskExpr.toString());
+      }
+
+      return resolvedColumns.toArray(new Column[resolvedColumns.size()]);
+    }
+  }
+
+  /**
+   * Resolve an asterisk expression to the real column reference expressions.
+   * @param ctx context
+   * @param asteriskExpr asterisk expression
+   * @return a list of NamedExpr each of which has ColumnReferenceExprs as its child
+   * @throws PlanningException
+   */
+  private static List<NamedExpr> resolveAsterisk(PreprocessContext ctx, QualifiedAsteriskExpr asteriskExpr)
+      throws PlanningException {
+    Column[] columns = getColumns(ctx, asteriskExpr);
+    List<NamedExpr> newTargetExprs = new ArrayList<NamedExpr>(columns.length);
+    int i;
+    for (i = 0; i < columns.length; i++) {
+      newTargetExprs.add(new NamedExpr(new ColumnReferenceExpr(columns[i].getQualifier(), columns[i].getColumnName())));
+    }
+    return newTargetExprs;
+  }
+
+  private static boolean hasAsterisk(Projection projection) {
+    for (NamedExpr eachTarget : projection.getNamedExprs()) {
+      if (eachTarget.getExpr().getType() == OpType.Asterisk) {
+        return true;
+      }
+    }
+    return false;
+  }
+
   @Override
   public LogicalNode visitProjection(PreprocessContext ctx, Stack<Expr> stack, Projection expr) throws PlanningException {
     // If Non-from statement, it immediately returns.
@@ -87,23 +175,35 @@ class LogicalPlanPreprocessor extends BaseAlgebraVisitor<LogicalPlanPreprocessor
     stack.push(expr); // <--- push
     LogicalNode child = visit(ctx, stack, expr.getChild());
 
+    // Resolve the asterisk expression
+    if (hasAsterisk(expr)) {
+      List<NamedExpr> rewrittenTargets = TUtil.newList();
+      for (NamedExpr originTarget : expr.getNamedExprs()) {
+        if (originTarget.getExpr().getType() == OpType.Asterisk) {
+          // rewrite targets
+          rewrittenTargets.addAll(resolveAsterisk(ctx, (QualifiedAsteriskExpr) originTarget.getExpr()));
+        } else {
+          rewrittenTargets.add(originTarget);
+        }
+      }
+      expr.setNamedExprs(rewrittenTargets.toArray(new NamedExpr[rewrittenTargets.size()]));
+    }
+
+    NamedExpr[] projectTargetExprs = expr.getNamedExprs();
+
     Target [] targets;
-    if (expr.isAllProjected()) {
-      targets = PlannerUtil.schemaToTargets(child.getOutSchema());
-    } else {
-      targets = new Target[expr.getNamedExprs().length];
+    targets = new Target[projectTargetExprs.length];
 
-      for (int i = 0; i < expr.getNamedExprs().length; i++) {
-        NamedExpr namedExpr = expr.getNamedExprs()[i];
-        EvalNode evalNode = annotator.createEvalNode(ctx.plan, ctx.currentBlock, namedExpr.getExpr());
+    for (int i = 0; i < expr.getNamedExprs().length; i++) {
+      NamedExpr namedExpr = expr.getNamedExprs()[i];
+      EvalNode evalNode = annotator.createEvalNode(ctx.plan, ctx.currentBlock, namedExpr.getExpr());
 
-        if (namedExpr.hasAlias()) {
-          targets[i] = new Target(evalNode, namedExpr.getAlias());
-        } else if (evalNode.getType() == EvalType.FIELD) {
-          targets[i] = new Target((FieldEval) evalNode);
-        } else {
-          targets[i] = new Target(evalNode, "?name_" + i);
-        }
+      if (namedExpr.hasAlias()) {
+        targets[i] = new Target(evalNode, namedExpr.getAlias());
+      } else if (evalNode.getType() == EvalType.FIELD) {
+        targets[i] = new Target((FieldEval) evalNode);
+      } else {
+        targets[i] = new Target(evalNode, "?name_" + i);
       }
     }
     stack.pop(); // <--- Pop

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/cd7bbae0/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/LogicalPlanner.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/LogicalPlanner.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/LogicalPlanner.java
index d929218..9a8fd51 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/LogicalPlanner.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/LogicalPlanner.java
@@ -183,13 +183,9 @@ public class LogicalPlanner extends BaseAlgebraVisitor<LogicalPlanner.PlanContex
     }
 
     String [] referenceNames;
-    if (projection.isAllProjected()) {
-      referenceNames = null;
-    } else {
-      // in prephase, insert all target list into NamedExprManagers.
-      // Then it gets reference names, each of which points an expression in target list.
-      referenceNames = doProjectionPrephase(context, projection);
-    }
+    // in prephase, insert all target list into NamedExprManagers.
+    // Then it gets reference names, each of which points an expression in target list.
+    referenceNames = doProjectionPrephase(context, projection);
 
     ////////////////////////////////////////////////////////
     // Visit and Build Child Plan
@@ -206,12 +202,7 @@ public class LogicalPlanner extends BaseAlgebraVisitor<LogicalPlanner.PlanContex
 
     ProjectionNode projectionNode;
     Target [] targets;
-    if (projection.isAllProjected()) {
-      // should takes all columns except for generated columns whose names are prefixed with '$'.
-      targets = PlannerUtil.schemaToTargetsWithGeneratedFields(child.getOutSchema());
-    } else {
-      targets = buildTargets(plan, block, referenceNames);
-    }
+    targets = buildTargets(plan, block, referenceNames);
 
     // Set ProjectionNode
     projectionNode = context.queryBlock.getNodeFromExpr(projection);
@@ -242,19 +233,15 @@ public class LogicalPlanner extends BaseAlgebraVisitor<LogicalPlanner.PlanContex
     LogicalPlan plan = context.plan;
     QueryBlock block = context.queryBlock;
 
-    if (projection.isAllProjected()) {
-      block.setRawTargets(targets);
-    } else {
-      // It's for debugging or unit tests.
-      Target [] rawTargets = new Target[projection.getNamedExprs().length];
-      for (int i = 0; i < targets.length; i++) {
-        NamedExpr namedExpr = projection.getNamedExprs()[i];
-        EvalNode evalNode = exprAnnotator.createEvalNode(plan, block, namedExpr.getExpr());
-        rawTargets[i] = new Target(evalNode, referenceNames[i]);
-      }
-      // it's for debugging or unit testing
-      block.setRawTargets(rawTargets);
+    // It's for debugging or unit tests.
+    Target [] rawTargets = new Target[projection.getNamedExprs().length];
+    for (int i = 0; i < projection.getNamedExprs().length; i++) {
+      NamedExpr namedExpr = projection.getNamedExprs()[i];
+      EvalNode evalNode = exprAnnotator.createEvalNode(plan, block, namedExpr.getExpr());
+      rawTargets[i] = new Target(evalNode, referenceNames[i]);
     }
+    // it's for debugging or unit testing
+    block.setRawTargets(rawTargets);
   }
 
   private void insertDistinctOperator(PlanContext context, ProjectionNode projectionNode, LogicalNode child,

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/cd7bbae0/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/parser/TestHiveConverter.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/parser/TestHiveConverter.java b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/parser/TestHiveConverter.java
index e090c89..e445c92 100644
--- a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/parser/TestHiveConverter.java
+++ b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/parser/TestHiveConverter.java
@@ -171,6 +171,38 @@ public class TestHiveConverter {
   }
 
   @Test
+  public void testAsterisk1() throws IOException {
+    String sql = FileUtil.readTextFile(new File("src/test/resources/queries/default/asterisk_1.sql"));
+    Expr expr = parseQuery(sql);
+    Expr hiveExpr = parseHiveQL(sql);
+    compareJsonResult(expr, hiveExpr);
+  }
+
+  @Test
+  public void testAsterisk2() throws IOException {
+    String sql = FileUtil.readTextFile(new File("src/test/resources/queries/default/asterisk_2.sql"));
+    Expr expr = parseQuery(sql);
+    Expr hiveExpr = parseHiveQL(sql);
+    compareJsonResult(expr, hiveExpr);
+  }
+
+  @Test
+  public void testAsterisk3() throws IOException {
+    String sql = FileUtil.readTextFile(new File("src/test/resources/queries/default/asterisk_3.sql"));
+    Expr expr = parseQuery(sql);
+    Expr hiveExpr = parseHiveQL(sql);
+    compareJsonResult(expr, hiveExpr);
+  }
+
+  @Test
+  public void testAsterisk4() throws IOException {
+    String sql = FileUtil.readTextFile(new File("src/test/resources/queries/default/asterisk_4.sql"));
+    Expr expr = parseQuery(sql);
+    Expr hiveExpr = parseHiveQL(sql);
+    compareJsonResult(expr, hiveExpr);
+  }
+
+  @Test
   public void testGroupby1() throws IOException {
     String sql = FileUtil.readTextFile(new File("src/test/resources/queries/default/groupby_1.sql"));
     Expr expr = parseQuery(sql);

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/cd7bbae0/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/parser/TestSQLAnalyzer.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/parser/TestSQLAnalyzer.java b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/parser/TestSQLAnalyzer.java
index 8d7cb94..9462c75 100644
--- a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/parser/TestSQLAnalyzer.java
+++ b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/parser/TestSQLAnalyzer.java
@@ -84,6 +84,30 @@ public class TestSQLAnalyzer {
   }
 
   @Test
+  public void testAsterisk1() throws IOException {
+    String sql = FileUtil.readTextFile(new File("src/test/resources/queries/default/asterisk_1.sql"));
+    parseQuery(sql);
+  }
+
+  @Test
+  public void testAsterisk2() throws IOException {
+    String sql = FileUtil.readTextFile(new File("src/test/resources/queries/default/asterisk_2.sql"));
+    parseQuery(sql);
+  }
+
+  @Test
+  public void testAsterisk3() throws IOException {
+    String sql = FileUtil.readTextFile(new File("src/test/resources/queries/default/asterisk_3.sql"));
+    parseQuery(sql);
+  }
+
+  @Test
+  public void testAsterisk4() throws IOException {
+    String sql = FileUtil.readTextFile(new File("src/test/resources/queries/default/asterisk_4.sql"));
+    parseQuery(sql);
+  }
+
+  @Test
   public void testGroupby1() throws IOException {
     String sql = FileUtil.readTextFile(new File("src/test/resources/queries/default/groupby_1.sql"));
     parseQuery(sql);

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/cd7bbae0/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/TestLogicalPlanner.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/TestLogicalPlanner.java b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/TestLogicalPlanner.java
index 81f57d4..f9cbcf9 100644
--- a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/TestLogicalPlanner.java
+++ b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/TestLogicalPlanner.java
@@ -126,7 +126,8 @@ public class TestLogicalPlanner {
       "select deptName, sumtest(score) from score group by deptName having sumtest(score) > 30", // 9
       "select 7 + 8 as res1, 8 * 9 as res2, 10 * 10 as res3", // 10
       "create index idx_employee on employee using bitmap (name null first, empId desc) with ('fillfactor' = 70)", // 11
-      "select name, score from employee, score order by score limit 3" // 12
+      "select name, score from employee, score order by score limit 3", // 12
+      "select length(name), length(deptname), *, empid+10 from employee where empId > 500", // 13
   };
 
   @Test
@@ -568,6 +569,28 @@ public class TestLogicalPlanner {
     assertEquals("res3", col.getColumnName());
   }
 
+  @Test
+  public final void testAsterisk() throws CloneNotSupportedException, PlanningException {
+    Expr expr = sqlAnalyzer.parse(QUERIES[13]);
+    LogicalPlan planNode = planner.createPlan(expr);
+    LogicalNode plan = planNode.getRootBlock().getRoot();
+    assertEquals(NodeType.ROOT, plan.getType());
+    TestLogicalNode.testCloneLogicalNode(plan);
+    LogicalRootNode root = (LogicalRootNode) plan;
+    testJsonSerDerObject(root);
+
+    assertEquals(NodeType.PROJECTION, root.getChild().getType());
+    ProjectionNode projNode = root.getChild();
+    assertEquals(6, projNode.getOutSchema().getColumnNum());
+
+    assertEquals(NodeType.SELECTION, projNode.getChild().getType());
+    SelectionNode selNode = projNode.getChild();
+
+    assertEquals(NodeType.SCAN, selNode.getChild().getType());
+    ScanNode scanNode = selNode.getChild();
+    assertEquals("employee", scanNode.getTableName());
+  }
+
   static final String ALIAS [] = {
     "select deptName, sum(score) as total from score group by deptName",
     "select em.empId as id, sum(score) as total from employee as em inner join score using (em.deptName) group by id"

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/cd7bbae0/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/query/TestJoinQuery.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/query/TestJoinQuery.java b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/query/TestJoinQuery.java
index dc951cf..5860b38 100644
--- a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/query/TestJoinQuery.java
+++ b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/query/TestJoinQuery.java
@@ -160,4 +160,36 @@ public class TestJoinQuery extends QueryTestCaseBase {
     assertResultSet(res);
     cleanupQuery(res);
   }
+
+  @Test
+  public void testCrossJoinWithAsterisk1() throws Exception {
+    // select region.*, customer.* from region, customer;
+    ResultSet res = executeQuery();
+    assertResultSet(res);
+    cleanupQuery(res);
+  }
+
+  @Test
+   public void testCrossJoinWithAsterisk2() throws Exception {
+    // select region.*, customer.* from customer, region;
+    ResultSet res = executeQuery();
+    assertResultSet(res);
+    cleanupQuery(res);
+  }
+
+  @Test
+  public void testCrossJoinWithAsterisk3() throws Exception {
+    // select * from customer, region
+    ResultSet res = executeQuery();
+    assertResultSet(res);
+    cleanupQuery(res);
+  }
+
+  @Test
+  public void testCrossJoinWithAsterisk4() throws Exception {
+    // select length(r_regionkey), *, c_custkey*10 from customer, region
+    ResultSet res = executeQuery();
+    assertResultSet(res);
+    cleanupQuery(res);
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/cd7bbae0/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/query/TestSelectQuery.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/query/TestSelectQuery.java b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/query/TestSelectQuery.java
index 4d852fc..6c8581f 100644
--- a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/query/TestSelectQuery.java
+++ b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/query/TestSelectQuery.java
@@ -112,6 +112,14 @@ public class TestSelectQuery extends QueryTestCaseBase {
   }
 
   @Test
+  public final void testSelectAsterisk4() throws Exception {
+    // select length(l_comment), l_extendedprice * l_discount, *, l_tax * 10 from lineitem;
+    ResultSet res = executeQuery();
+    assertResultSet(res);
+    cleanupQuery(res);
+  }
+
+  @Test
   public final void testSelectDistinct() throws Exception {
     ResultSet res = executeQuery();
     assertResultSet(res);

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/cd7bbae0/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/query/TestSortQuery.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/query/TestSortQuery.java b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/query/TestSortQuery.java
index b6e439a..23d7bdb 100644
--- a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/query/TestSortQuery.java
+++ b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/query/TestSortQuery.java
@@ -105,6 +105,14 @@ public class TestSortQuery extends QueryTestCaseBase {
   }
 
   @Test
+  public final void testAsterisk() throws Exception {
+    //select *, length(l_comment) as len_comment from lineitem order by len_comment;
+    ResultSet res = executeQuery();
+    assertResultSet(res);
+    cleanupQuery(res);
+  }
+
+  @Test
   public final void testSortWithAscDescKeys() throws Exception {
     executeDDL("create_table_with_asc_desc_keys.sql", "table2.tbl");
 
@@ -112,5 +120,4 @@ public class TestSortQuery extends QueryTestCaseBase {
     System.out.println(resultSetToString(res));
     cleanupQuery(res);
   }
-
 }

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/cd7bbae0/tajo-core/tajo-core-backend/src/test/resources/queries/TestJoinQuery/testCrossJoinWithAsterisk1.sql
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/resources/queries/TestJoinQuery/testCrossJoinWithAsterisk1.sql b/tajo-core/tajo-core-backend/src/test/resources/queries/TestJoinQuery/testCrossJoinWithAsterisk1.sql
new file mode 100644
index 0000000..910138f
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/test/resources/queries/TestJoinQuery/testCrossJoinWithAsterisk1.sql
@@ -0,0 +1 @@
+select region.*, customer.* from region, customer;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/cd7bbae0/tajo-core/tajo-core-backend/src/test/resources/queries/TestJoinQuery/testCrossJoinWithAsterisk2.sql
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/resources/queries/TestJoinQuery/testCrossJoinWithAsterisk2.sql b/tajo-core/tajo-core-backend/src/test/resources/queries/TestJoinQuery/testCrossJoinWithAsterisk2.sql
new file mode 100644
index 0000000..a68e40a
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/test/resources/queries/TestJoinQuery/testCrossJoinWithAsterisk2.sql
@@ -0,0 +1 @@
+select region.*, customer.* from customer, region;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/cd7bbae0/tajo-core/tajo-core-backend/src/test/resources/queries/TestJoinQuery/testCrossJoinWithAsterisk3.sql
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/resources/queries/TestJoinQuery/testCrossJoinWithAsterisk3.sql b/tajo-core/tajo-core-backend/src/test/resources/queries/TestJoinQuery/testCrossJoinWithAsterisk3.sql
new file mode 100644
index 0000000..f75c8dc
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/test/resources/queries/TestJoinQuery/testCrossJoinWithAsterisk3.sql
@@ -0,0 +1 @@
+select * from customer, region;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/cd7bbae0/tajo-core/tajo-core-backend/src/test/resources/queries/TestJoinQuery/testCrossJoinWithAsterisk4.sql
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/resources/queries/TestJoinQuery/testCrossJoinWithAsterisk4.sql b/tajo-core/tajo-core-backend/src/test/resources/queries/TestJoinQuery/testCrossJoinWithAsterisk4.sql
new file mode 100644
index 0000000..100be80
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/test/resources/queries/TestJoinQuery/testCrossJoinWithAsterisk4.sql
@@ -0,0 +1 @@
+select length(r_comment), *, c_custkey*10 from customer, region;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/cd7bbae0/tajo-core/tajo-core-backend/src/test/resources/queries/TestSelectQuery/testSelectAsterisk4.sql
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/resources/queries/TestSelectQuery/testSelectAsterisk4.sql b/tajo-core/tajo-core-backend/src/test/resources/queries/TestSelectQuery/testSelectAsterisk4.sql
new file mode 100644
index 0000000..71580c8
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/test/resources/queries/TestSelectQuery/testSelectAsterisk4.sql
@@ -0,0 +1 @@
+select length(l_comment), l_extendedprice * l_discount, *, l_tax * 10 from lineitem;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/cd7bbae0/tajo-core/tajo-core-backend/src/test/resources/queries/TestSortQuery/testAsterisk.sql
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/resources/queries/TestSortQuery/testAsterisk.sql b/tajo-core/tajo-core-backend/src/test/resources/queries/TestSortQuery/testAsterisk.sql
new file mode 100644
index 0000000..e88ca43
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/test/resources/queries/TestSortQuery/testAsterisk.sql
@@ -0,0 +1 @@
+select *, length(l_comment) as len_comment from lineitem order by len_comment;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/cd7bbae0/tajo-core/tajo-core-backend/src/test/resources/queries/default/asterisk_1.sql
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/resources/queries/default/asterisk_1.sql b/tajo-core/tajo-core-backend/src/test/resources/queries/default/asterisk_1.sql
new file mode 100644
index 0000000..2bbe245
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/test/resources/queries/default/asterisk_1.sql
@@ -0,0 +1 @@
+select * from people
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/cd7bbae0/tajo-core/tajo-core-backend/src/test/resources/queries/default/asterisk_2.sql
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/resources/queries/default/asterisk_2.sql b/tajo-core/tajo-core-backend/src/test/resources/queries/default/asterisk_2.sql
new file mode 100644
index 0000000..d646044
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/test/resources/queries/default/asterisk_2.sql
@@ -0,0 +1 @@
+select p.*, s.* from people cross join students
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/cd7bbae0/tajo-core/tajo-core-backend/src/test/resources/queries/default/asterisk_3.sql
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/resources/queries/default/asterisk_3.sql b/tajo-core/tajo-core-backend/src/test/resources/queries/default/asterisk_3.sql
new file mode 100644
index 0000000..1566230
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/test/resources/queries/default/asterisk_3.sql
@@ -0,0 +1 @@
+select * from people cross join students
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/cd7bbae0/tajo-core/tajo-core-backend/src/test/resources/queries/default/asterisk_4.sql
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/resources/queries/default/asterisk_4.sql b/tajo-core/tajo-core-backend/src/test/resources/queries/default/asterisk_4.sql
new file mode 100644
index 0000000..22faf44
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/test/resources/queries/default/asterisk_4.sql
@@ -0,0 +1 @@
+select age-10, *, length(name) from people
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/cd7bbae0/tajo-core/tajo-core-backend/src/test/resources/results/TestJoinQuery/testCrossJoinWithAsterisk1.result
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/resources/results/TestJoinQuery/testCrossJoinWithAsterisk1.result b/tajo-core/tajo-core-backend/src/test/resources/results/TestJoinQuery/testCrossJoinWithAsterisk1.result
new file mode 100644
index 0000000..f4e8c88
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/test/resources/results/TestJoinQuery/testCrossJoinWithAsterisk1.result
@@ -0,0 +1,27 @@
+r_regionkey,r_name,r_comment,c_custkey,c_name,c_address,c_nationkey,c_phone,c_acctbal,c_mktsegment,c_comment
+-------------------------------
+0,AFRICA,lar deposits. blithely final packages cajole. regular waters are final requests. regular accounts are according to ,1,Customer#000000001,IVhzIApeRb ot,c,E,15,25-989-741-2988,711.56,BUILDING,to the even, regular platelets. regular, ironic epitaphs nag e
+0,AFRICA,lar deposits. blithely final packages cajole. regular waters are final requests. regular accounts are according to ,2,Customer#000000002,XSTf4,NCwDVaWNe6tEgvwfmRchLXak,13,23-768-687-3665,121.65,AUTOMOBILE,l accounts. blithely ironic theodolites integrate boldly: caref
+0,AFRICA,lar deposits. blithely final packages cajole. regular waters are final requests. regular accounts are according to ,3,Customer#000000003,MG9kdTD2WBHm,1,11-719-748-3364,7498.12,AUTOMOBILE, deposits eat slyly ironic, even instructions. express foxes detect slyly. blithely even accounts abov
+0,AFRICA,lar deposits. blithely final packages cajole. regular waters are final requests. regular accounts are according to ,4,Customer#000000004,XxVSJsLAGtn,4,14-128-190-5944,2866.83,MACHINERY, requests. final, regular ideas sleep final accou
+0,AFRICA,lar deposits. blithely final packages cajole. regular waters are final requests. regular accounts are according to ,5,Customer#000000005,KvpyuHCplrB84WgAiGV6sYpZq7Tj,3,13-750-942-6364,794.47,HOUSEHOLD,n accounts will have to unwind. foxes cajole accor
+1,AMERICA,hs use ironic, even requests. s,1,Customer#000000001,IVhzIApeRb ot,c,E,15,25-989-741-2988,711.56,BUILDING,to the even, regular platelets. regular, ironic epitaphs nag e
+1,AMERICA,hs use ironic, even requests. s,2,Customer#000000002,XSTf4,NCwDVaWNe6tEgvwfmRchLXak,13,23-768-687-3665,121.65,AUTOMOBILE,l accounts. blithely ironic theodolites integrate boldly: caref
+1,AMERICA,hs use ironic, even requests. s,3,Customer#000000003,MG9kdTD2WBHm,1,11-719-748-3364,7498.12,AUTOMOBILE, deposits eat slyly ironic, even instructions. express foxes detect slyly. blithely even accounts abov
+1,AMERICA,hs use ironic, even requests. s,4,Customer#000000004,XxVSJsLAGtn,4,14-128-190-5944,2866.83,MACHINERY, requests. final, regular ideas sleep final accou
+1,AMERICA,hs use ironic, even requests. s,5,Customer#000000005,KvpyuHCplrB84WgAiGV6sYpZq7Tj,3,13-750-942-6364,794.47,HOUSEHOLD,n accounts will have to unwind. foxes cajole accor
+2,ASIA,ges. thinly even pinto beans ca,1,Customer#000000001,IVhzIApeRb ot,c,E,15,25-989-741-2988,711.56,BUILDING,to the even, regular platelets. regular, ironic epitaphs nag e
+2,ASIA,ges. thinly even pinto beans ca,2,Customer#000000002,XSTf4,NCwDVaWNe6tEgvwfmRchLXak,13,23-768-687-3665,121.65,AUTOMOBILE,l accounts. blithely ironic theodolites integrate boldly: caref
+2,ASIA,ges. thinly even pinto beans ca,3,Customer#000000003,MG9kdTD2WBHm,1,11-719-748-3364,7498.12,AUTOMOBILE, deposits eat slyly ironic, even instructions. express foxes detect slyly. blithely even accounts abov
+2,ASIA,ges. thinly even pinto beans ca,4,Customer#000000004,XxVSJsLAGtn,4,14-128-190-5944,2866.83,MACHINERY, requests. final, regular ideas sleep final accou
+2,ASIA,ges. thinly even pinto beans ca,5,Customer#000000005,KvpyuHCplrB84WgAiGV6sYpZq7Tj,3,13-750-942-6364,794.47,HOUSEHOLD,n accounts will have to unwind. foxes cajole accor
+3,EUROPE,ly final courts cajole furiously final excuse,1,Customer#000000001,IVhzIApeRb ot,c,E,15,25-989-741-2988,711.56,BUILDING,to the even, regular platelets. regular, ironic epitaphs nag e
+3,EUROPE,ly final courts cajole furiously final excuse,2,Customer#000000002,XSTf4,NCwDVaWNe6tEgvwfmRchLXak,13,23-768-687-3665,121.65,AUTOMOBILE,l accounts. blithely ironic theodolites integrate boldly: caref
+3,EUROPE,ly final courts cajole furiously final excuse,3,Customer#000000003,MG9kdTD2WBHm,1,11-719-748-3364,7498.12,AUTOMOBILE, deposits eat slyly ironic, even instructions. express foxes detect slyly. blithely even accounts abov
+3,EUROPE,ly final courts cajole furiously final excuse,4,Customer#000000004,XxVSJsLAGtn,4,14-128-190-5944,2866.83,MACHINERY, requests. final, regular ideas sleep final accou
+3,EUROPE,ly final courts cajole furiously final excuse,5,Customer#000000005,KvpyuHCplrB84WgAiGV6sYpZq7Tj,3,13-750-942-6364,794.47,HOUSEHOLD,n accounts will have to unwind. foxes cajole accor
+4,MIDDLE EAST,uickly special accounts cajole carefully blithely close requests. carefully final asymptotes haggle furiousl,1,Customer#000000001,IVhzIApeRb ot,c,E,15,25-989-741-2988,711.56,BUILDING,to the even, regular platelets. regular, ironic epitaphs nag e
+4,MIDDLE EAST,uickly special accounts cajole carefully blithely close requests. carefully final asymptotes haggle furiousl,2,Customer#000000002,XSTf4,NCwDVaWNe6tEgvwfmRchLXak,13,23-768-687-3665,121.65,AUTOMOBILE,l accounts. blithely ironic theodolites integrate boldly: caref
+4,MIDDLE EAST,uickly special accounts cajole carefully blithely close requests. carefully final asymptotes haggle furiousl,3,Customer#000000003,MG9kdTD2WBHm,1,11-719-748-3364,7498.12,AUTOMOBILE, deposits eat slyly ironic, even instructions. express foxes detect slyly. blithely even accounts abov
+4,MIDDLE EAST,uickly special accounts cajole carefully blithely close requests. carefully final asymptotes haggle furiousl,4,Customer#000000004,XxVSJsLAGtn,4,14-128-190-5944,2866.83,MACHINERY, requests. final, regular ideas sleep final accou
+4,MIDDLE EAST,uickly special accounts cajole carefully blithely close requests. carefully final asymptotes haggle furiousl,5,Customer#000000005,KvpyuHCplrB84WgAiGV6sYpZq7Tj,3,13-750-942-6364,794.47,HOUSEHOLD,n accounts will have to unwind. foxes cajole accor
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/cd7bbae0/tajo-core/tajo-core-backend/src/test/resources/results/TestJoinQuery/testCrossJoinWithAsterisk2.result
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/resources/results/TestJoinQuery/testCrossJoinWithAsterisk2.result b/tajo-core/tajo-core-backend/src/test/resources/results/TestJoinQuery/testCrossJoinWithAsterisk2.result
new file mode 100644
index 0000000..f4e8c88
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/test/resources/results/TestJoinQuery/testCrossJoinWithAsterisk2.result
@@ -0,0 +1,27 @@
+r_regionkey,r_name,r_comment,c_custkey,c_name,c_address,c_nationkey,c_phone,c_acctbal,c_mktsegment,c_comment
+-------------------------------
+0,AFRICA,lar deposits. blithely final packages cajole. regular waters are final requests. regular accounts are according to ,1,Customer#000000001,IVhzIApeRb ot,c,E,15,25-989-741-2988,711.56,BUILDING,to the even, regular platelets. regular, ironic epitaphs nag e
+0,AFRICA,lar deposits. blithely final packages cajole. regular waters are final requests. regular accounts are according to ,2,Customer#000000002,XSTf4,NCwDVaWNe6tEgvwfmRchLXak,13,23-768-687-3665,121.65,AUTOMOBILE,l accounts. blithely ironic theodolites integrate boldly: caref
+0,AFRICA,lar deposits. blithely final packages cajole. regular waters are final requests. regular accounts are according to ,3,Customer#000000003,MG9kdTD2WBHm,1,11-719-748-3364,7498.12,AUTOMOBILE, deposits eat slyly ironic, even instructions. express foxes detect slyly. blithely even accounts abov
+0,AFRICA,lar deposits. blithely final packages cajole. regular waters are final requests. regular accounts are according to ,4,Customer#000000004,XxVSJsLAGtn,4,14-128-190-5944,2866.83,MACHINERY, requests. final, regular ideas sleep final accou
+0,AFRICA,lar deposits. blithely final packages cajole. regular waters are final requests. regular accounts are according to ,5,Customer#000000005,KvpyuHCplrB84WgAiGV6sYpZq7Tj,3,13-750-942-6364,794.47,HOUSEHOLD,n accounts will have to unwind. foxes cajole accor
+1,AMERICA,hs use ironic, even requests. s,1,Customer#000000001,IVhzIApeRb ot,c,E,15,25-989-741-2988,711.56,BUILDING,to the even, regular platelets. regular, ironic epitaphs nag e
+1,AMERICA,hs use ironic, even requests. s,2,Customer#000000002,XSTf4,NCwDVaWNe6tEgvwfmRchLXak,13,23-768-687-3665,121.65,AUTOMOBILE,l accounts. blithely ironic theodolites integrate boldly: caref
+1,AMERICA,hs use ironic, even requests. s,3,Customer#000000003,MG9kdTD2WBHm,1,11-719-748-3364,7498.12,AUTOMOBILE, deposits eat slyly ironic, even instructions. express foxes detect slyly. blithely even accounts abov
+1,AMERICA,hs use ironic, even requests. s,4,Customer#000000004,XxVSJsLAGtn,4,14-128-190-5944,2866.83,MACHINERY, requests. final, regular ideas sleep final accou
+1,AMERICA,hs use ironic, even requests. s,5,Customer#000000005,KvpyuHCplrB84WgAiGV6sYpZq7Tj,3,13-750-942-6364,794.47,HOUSEHOLD,n accounts will have to unwind. foxes cajole accor
+2,ASIA,ges. thinly even pinto beans ca,1,Customer#000000001,IVhzIApeRb ot,c,E,15,25-989-741-2988,711.56,BUILDING,to the even, regular platelets. regular, ironic epitaphs nag e
+2,ASIA,ges. thinly even pinto beans ca,2,Customer#000000002,XSTf4,NCwDVaWNe6tEgvwfmRchLXak,13,23-768-687-3665,121.65,AUTOMOBILE,l accounts. blithely ironic theodolites integrate boldly: caref
+2,ASIA,ges. thinly even pinto beans ca,3,Customer#000000003,MG9kdTD2WBHm,1,11-719-748-3364,7498.12,AUTOMOBILE, deposits eat slyly ironic, even instructions. express foxes detect slyly. blithely even accounts abov
+2,ASIA,ges. thinly even pinto beans ca,4,Customer#000000004,XxVSJsLAGtn,4,14-128-190-5944,2866.83,MACHINERY, requests. final, regular ideas sleep final accou
+2,ASIA,ges. thinly even pinto beans ca,5,Customer#000000005,KvpyuHCplrB84WgAiGV6sYpZq7Tj,3,13-750-942-6364,794.47,HOUSEHOLD,n accounts will have to unwind. foxes cajole accor
+3,EUROPE,ly final courts cajole furiously final excuse,1,Customer#000000001,IVhzIApeRb ot,c,E,15,25-989-741-2988,711.56,BUILDING,to the even, regular platelets. regular, ironic epitaphs nag e
+3,EUROPE,ly final courts cajole furiously final excuse,2,Customer#000000002,XSTf4,NCwDVaWNe6tEgvwfmRchLXak,13,23-768-687-3665,121.65,AUTOMOBILE,l accounts. blithely ironic theodolites integrate boldly: caref
+3,EUROPE,ly final courts cajole furiously final excuse,3,Customer#000000003,MG9kdTD2WBHm,1,11-719-748-3364,7498.12,AUTOMOBILE, deposits eat slyly ironic, even instructions. express foxes detect slyly. blithely even accounts abov
+3,EUROPE,ly final courts cajole furiously final excuse,4,Customer#000000004,XxVSJsLAGtn,4,14-128-190-5944,2866.83,MACHINERY, requests. final, regular ideas sleep final accou
+3,EUROPE,ly final courts cajole furiously final excuse,5,Customer#000000005,KvpyuHCplrB84WgAiGV6sYpZq7Tj,3,13-750-942-6364,794.47,HOUSEHOLD,n accounts will have to unwind. foxes cajole accor
+4,MIDDLE EAST,uickly special accounts cajole carefully blithely close requests. carefully final asymptotes haggle furiousl,1,Customer#000000001,IVhzIApeRb ot,c,E,15,25-989-741-2988,711.56,BUILDING,to the even, regular platelets. regular, ironic epitaphs nag e
+4,MIDDLE EAST,uickly special accounts cajole carefully blithely close requests. carefully final asymptotes haggle furiousl,2,Customer#000000002,XSTf4,NCwDVaWNe6tEgvwfmRchLXak,13,23-768-687-3665,121.65,AUTOMOBILE,l accounts. blithely ironic theodolites integrate boldly: caref
+4,MIDDLE EAST,uickly special accounts cajole carefully blithely close requests. carefully final asymptotes haggle furiousl,3,Customer#000000003,MG9kdTD2WBHm,1,11-719-748-3364,7498.12,AUTOMOBILE, deposits eat slyly ironic, even instructions. express foxes detect slyly. blithely even accounts abov
+4,MIDDLE EAST,uickly special accounts cajole carefully blithely close requests. carefully final asymptotes haggle furiousl,4,Customer#000000004,XxVSJsLAGtn,4,14-128-190-5944,2866.83,MACHINERY, requests. final, regular ideas sleep final accou
+4,MIDDLE EAST,uickly special accounts cajole carefully blithely close requests. carefully final asymptotes haggle furiousl,5,Customer#000000005,KvpyuHCplrB84WgAiGV6sYpZq7Tj,3,13-750-942-6364,794.47,HOUSEHOLD,n accounts will have to unwind. foxes cajole accor
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/cd7bbae0/tajo-core/tajo-core-backend/src/test/resources/results/TestJoinQuery/testCrossJoinWithAsterisk3.result
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/resources/results/TestJoinQuery/testCrossJoinWithAsterisk3.result b/tajo-core/tajo-core-backend/src/test/resources/results/TestJoinQuery/testCrossJoinWithAsterisk3.result
new file mode 100644
index 0000000..f4e8c88
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/test/resources/results/TestJoinQuery/testCrossJoinWithAsterisk3.result
@@ -0,0 +1,27 @@
+r_regionkey,r_name,r_comment,c_custkey,c_name,c_address,c_nationkey,c_phone,c_acctbal,c_mktsegment,c_comment
+-------------------------------
+0,AFRICA,lar deposits. blithely final packages cajole. regular waters are final requests. regular accounts are according to ,1,Customer#000000001,IVhzIApeRb ot,c,E,15,25-989-741-2988,711.56,BUILDING,to the even, regular platelets. regular, ironic epitaphs nag e
+0,AFRICA,lar deposits. blithely final packages cajole. regular waters are final requests. regular accounts are according to ,2,Customer#000000002,XSTf4,NCwDVaWNe6tEgvwfmRchLXak,13,23-768-687-3665,121.65,AUTOMOBILE,l accounts. blithely ironic theodolites integrate boldly: caref
+0,AFRICA,lar deposits. blithely final packages cajole. regular waters are final requests. regular accounts are according to ,3,Customer#000000003,MG9kdTD2WBHm,1,11-719-748-3364,7498.12,AUTOMOBILE, deposits eat slyly ironic, even instructions. express foxes detect slyly. blithely even accounts abov
+0,AFRICA,lar deposits. blithely final packages cajole. regular waters are final requests. regular accounts are according to ,4,Customer#000000004,XxVSJsLAGtn,4,14-128-190-5944,2866.83,MACHINERY, requests. final, regular ideas sleep final accou
+0,AFRICA,lar deposits. blithely final packages cajole. regular waters are final requests. regular accounts are according to ,5,Customer#000000005,KvpyuHCplrB84WgAiGV6sYpZq7Tj,3,13-750-942-6364,794.47,HOUSEHOLD,n accounts will have to unwind. foxes cajole accor
+1,AMERICA,hs use ironic, even requests. s,1,Customer#000000001,IVhzIApeRb ot,c,E,15,25-989-741-2988,711.56,BUILDING,to the even, regular platelets. regular, ironic epitaphs nag e
+1,AMERICA,hs use ironic, even requests. s,2,Customer#000000002,XSTf4,NCwDVaWNe6tEgvwfmRchLXak,13,23-768-687-3665,121.65,AUTOMOBILE,l accounts. blithely ironic theodolites integrate boldly: caref
+1,AMERICA,hs use ironic, even requests. s,3,Customer#000000003,MG9kdTD2WBHm,1,11-719-748-3364,7498.12,AUTOMOBILE, deposits eat slyly ironic, even instructions. express foxes detect slyly. blithely even accounts abov
+1,AMERICA,hs use ironic, even requests. s,4,Customer#000000004,XxVSJsLAGtn,4,14-128-190-5944,2866.83,MACHINERY, requests. final, regular ideas sleep final accou
+1,AMERICA,hs use ironic, even requests. s,5,Customer#000000005,KvpyuHCplrB84WgAiGV6sYpZq7Tj,3,13-750-942-6364,794.47,HOUSEHOLD,n accounts will have to unwind. foxes cajole accor
+2,ASIA,ges. thinly even pinto beans ca,1,Customer#000000001,IVhzIApeRb ot,c,E,15,25-989-741-2988,711.56,BUILDING,to the even, regular platelets. regular, ironic epitaphs nag e
+2,ASIA,ges. thinly even pinto beans ca,2,Customer#000000002,XSTf4,NCwDVaWNe6tEgvwfmRchLXak,13,23-768-687-3665,121.65,AUTOMOBILE,l accounts. blithely ironic theodolites integrate boldly: caref
+2,ASIA,ges. thinly even pinto beans ca,3,Customer#000000003,MG9kdTD2WBHm,1,11-719-748-3364,7498.12,AUTOMOBILE, deposits eat slyly ironic, even instructions. express foxes detect slyly. blithely even accounts abov
+2,ASIA,ges. thinly even pinto beans ca,4,Customer#000000004,XxVSJsLAGtn,4,14-128-190-5944,2866.83,MACHINERY, requests. final, regular ideas sleep final accou
+2,ASIA,ges. thinly even pinto beans ca,5,Customer#000000005,KvpyuHCplrB84WgAiGV6sYpZq7Tj,3,13-750-942-6364,794.47,HOUSEHOLD,n accounts will have to unwind. foxes cajole accor
+3,EUROPE,ly final courts cajole furiously final excuse,1,Customer#000000001,IVhzIApeRb ot,c,E,15,25-989-741-2988,711.56,BUILDING,to the even, regular platelets. regular, ironic epitaphs nag e
+3,EUROPE,ly final courts cajole furiously final excuse,2,Customer#000000002,XSTf4,NCwDVaWNe6tEgvwfmRchLXak,13,23-768-687-3665,121.65,AUTOMOBILE,l accounts. blithely ironic theodolites integrate boldly: caref
+3,EUROPE,ly final courts cajole furiously final excuse,3,Customer#000000003,MG9kdTD2WBHm,1,11-719-748-3364,7498.12,AUTOMOBILE, deposits eat slyly ironic, even instructions. express foxes detect slyly. blithely even accounts abov
+3,EUROPE,ly final courts cajole furiously final excuse,4,Customer#000000004,XxVSJsLAGtn,4,14-128-190-5944,2866.83,MACHINERY, requests. final, regular ideas sleep final accou
+3,EUROPE,ly final courts cajole furiously final excuse,5,Customer#000000005,KvpyuHCplrB84WgAiGV6sYpZq7Tj,3,13-750-942-6364,794.47,HOUSEHOLD,n accounts will have to unwind. foxes cajole accor
+4,MIDDLE EAST,uickly special accounts cajole carefully blithely close requests. carefully final asymptotes haggle furiousl,1,Customer#000000001,IVhzIApeRb ot,c,E,15,25-989-741-2988,711.56,BUILDING,to the even, regular platelets. regular, ironic epitaphs nag e
+4,MIDDLE EAST,uickly special accounts cajole carefully blithely close requests. carefully final asymptotes haggle furiousl,2,Customer#000000002,XSTf4,NCwDVaWNe6tEgvwfmRchLXak,13,23-768-687-3665,121.65,AUTOMOBILE,l accounts. blithely ironic theodolites integrate boldly: caref
+4,MIDDLE EAST,uickly special accounts cajole carefully blithely close requests. carefully final asymptotes haggle furiousl,3,Customer#000000003,MG9kdTD2WBHm,1,11-719-748-3364,7498.12,AUTOMOBILE, deposits eat slyly ironic, even instructions. express foxes detect slyly. blithely even accounts abov
+4,MIDDLE EAST,uickly special accounts cajole carefully blithely close requests. carefully final asymptotes haggle furiousl,4,Customer#000000004,XxVSJsLAGtn,4,14-128-190-5944,2866.83,MACHINERY, requests. final, regular ideas sleep final accou
+4,MIDDLE EAST,uickly special accounts cajole carefully blithely close requests. carefully final asymptotes haggle furiousl,5,Customer#000000005,KvpyuHCplrB84WgAiGV6sYpZq7Tj,3,13-750-942-6364,794.47,HOUSEHOLD,n accounts will have to unwind. foxes cajole accor
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/cd7bbae0/tajo-core/tajo-core-backend/src/test/resources/results/TestJoinQuery/testCrossJoinWithAsterisk4.result
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/resources/results/TestJoinQuery/testCrossJoinWithAsterisk4.result b/tajo-core/tajo-core-backend/src/test/resources/results/TestJoinQuery/testCrossJoinWithAsterisk4.result
new file mode 100644
index 0000000..95b5bc3
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/test/resources/results/TestJoinQuery/testCrossJoinWithAsterisk4.result
@@ -0,0 +1,27 @@
+?length,r_regionkey,r_name,r_comment,c_custkey,c_name,c_address,c_nationkey,c_phone,c_acctbal,c_mktsegment,c_comment,?multiply_1
+-------------------------------
+115,0,AFRICA,lar deposits. blithely final packages cajole. regular waters are final requests. regular accounts are according to ,1,Customer#000000001,IVhzIApeRb ot,c,E,15,25-989-741-2988,711.56,BUILDING,to the even, regular platelets. regular, ironic epitaphs nag e,10
+115,0,AFRICA,lar deposits. blithely final packages cajole. regular waters are final requests. regular accounts are according to ,2,Customer#000000002,XSTf4,NCwDVaWNe6tEgvwfmRchLXak,13,23-768-687-3665,121.65,AUTOMOBILE,l accounts. blithely ironic theodolites integrate boldly: caref,20
+115,0,AFRICA,lar deposits. blithely final packages cajole. regular waters are final requests. regular accounts are according to ,3,Customer#000000003,MG9kdTD2WBHm,1,11-719-748-3364,7498.12,AUTOMOBILE, deposits eat slyly ironic, even instructions. express foxes detect slyly. blithely even accounts abov,30
+115,0,AFRICA,lar deposits. blithely final packages cajole. regular waters are final requests. regular accounts are according to ,4,Customer#000000004,XxVSJsLAGtn,4,14-128-190-5944,2866.83,MACHINERY, requests. final, regular ideas sleep final accou,40
+115,0,AFRICA,lar deposits. blithely final packages cajole. regular waters are final requests. regular accounts are according to ,5,Customer#000000005,KvpyuHCplrB84WgAiGV6sYpZq7Tj,3,13-750-942-6364,794.47,HOUSEHOLD,n accounts will have to unwind. foxes cajole accor,50
+31,1,AMERICA,hs use ironic, even requests. s,1,Customer#000000001,IVhzIApeRb ot,c,E,15,25-989-741-2988,711.56,BUILDING,to the even, regular platelets. regular, ironic epitaphs nag e,10
+31,1,AMERICA,hs use ironic, even requests. s,2,Customer#000000002,XSTf4,NCwDVaWNe6tEgvwfmRchLXak,13,23-768-687-3665,121.65,AUTOMOBILE,l accounts. blithely ironic theodolites integrate boldly: caref,20
+31,1,AMERICA,hs use ironic, even requests. s,3,Customer#000000003,MG9kdTD2WBHm,1,11-719-748-3364,7498.12,AUTOMOBILE, deposits eat slyly ironic, even instructions. express foxes detect slyly. blithely even accounts abov,30
+31,1,AMERICA,hs use ironic, even requests. s,4,Customer#000000004,XxVSJsLAGtn,4,14-128-190-5944,2866.83,MACHINERY, requests. final, regular ideas sleep final accou,40
+31,1,AMERICA,hs use ironic, even requests. s,5,Customer#000000005,KvpyuHCplrB84WgAiGV6sYpZq7Tj,3,13-750-942-6364,794.47,HOUSEHOLD,n accounts will have to unwind. foxes cajole accor,50
+31,2,ASIA,ges. thinly even pinto beans ca,1,Customer#000000001,IVhzIApeRb ot,c,E,15,25-989-741-2988,711.56,BUILDING,to the even, regular platelets. regular, ironic epitaphs nag e,10
+31,2,ASIA,ges. thinly even pinto beans ca,2,Customer#000000002,XSTf4,NCwDVaWNe6tEgvwfmRchLXak,13,23-768-687-3665,121.65,AUTOMOBILE,l accounts. blithely ironic theodolites integrate boldly: caref,20
+31,2,ASIA,ges. thinly even pinto beans ca,3,Customer#000000003,MG9kdTD2WBHm,1,11-719-748-3364,7498.12,AUTOMOBILE, deposits eat slyly ironic, even instructions. express foxes detect slyly. blithely even accounts abov,30
+31,2,ASIA,ges. thinly even pinto beans ca,4,Customer#000000004,XxVSJsLAGtn,4,14-128-190-5944,2866.83,MACHINERY, requests. final, regular ideas sleep final accou,40
+31,2,ASIA,ges. thinly even pinto beans ca,5,Customer#000000005,KvpyuHCplrB84WgAiGV6sYpZq7Tj,3,13-750-942-6364,794.47,HOUSEHOLD,n accounts will have to unwind. foxes cajole accor,50
+45,3,EUROPE,ly final courts cajole furiously final excuse,1,Customer#000000001,IVhzIApeRb ot,c,E,15,25-989-741-2988,711.56,BUILDING,to the even, regular platelets. regular, ironic epitaphs nag e,10
+45,3,EUROPE,ly final courts cajole furiously final excuse,2,Customer#000000002,XSTf4,NCwDVaWNe6tEgvwfmRchLXak,13,23-768-687-3665,121.65,AUTOMOBILE,l accounts. blithely ironic theodolites integrate boldly: caref,20
+45,3,EUROPE,ly final courts cajole furiously final excuse,3,Customer#000000003,MG9kdTD2WBHm,1,11-719-748-3364,7498.12,AUTOMOBILE, deposits eat slyly ironic, even instructions. express foxes detect slyly. blithely even accounts abov,30
+45,3,EUROPE,ly final courts cajole furiously final excuse,4,Customer#000000004,XxVSJsLAGtn,4,14-128-190-5944,2866.83,MACHINERY, requests. final, regular ideas sleep final accou,40
+45,3,EUROPE,ly final courts cajole furiously final excuse,5,Customer#000000005,KvpyuHCplrB84WgAiGV6sYpZq7Tj,3,13-750-942-6364,794.47,HOUSEHOLD,n accounts will have to unwind. foxes cajole accor,50
+108,4,MIDDLE EAST,uickly special accounts cajole carefully blithely close requests. carefully final asymptotes haggle furiousl,1,Customer#000000001,IVhzIApeRb ot,c,E,15,25-989-741-2988,711.56,BUILDING,to the even, regular platelets. regular, ironic epitaphs nag e,10
+108,4,MIDDLE EAST,uickly special accounts cajole carefully blithely close requests. carefully final asymptotes haggle furiousl,2,Customer#000000002,XSTf4,NCwDVaWNe6tEgvwfmRchLXak,13,23-768-687-3665,121.65,AUTOMOBILE,l accounts. blithely ironic theodolites integrate boldly: caref,20
+108,4,MIDDLE EAST,uickly special accounts cajole carefully blithely close requests. carefully final asymptotes haggle furiousl,3,Customer#000000003,MG9kdTD2WBHm,1,11-719-748-3364,7498.12,AUTOMOBILE, deposits eat slyly ironic, even instructions. express foxes detect slyly. blithely even accounts abov,30
+108,4,MIDDLE EAST,uickly special accounts cajole carefully blithely close requests. carefully final asymptotes haggle furiousl,4,Customer#000000004,XxVSJsLAGtn,4,14-128-190-5944,2866.83,MACHINERY, requests. final, regular ideas sleep final accou,40
+108,4,MIDDLE EAST,uickly special accounts cajole carefully blithely close requests. carefully final asymptotes haggle furiousl,5,Customer#000000005,KvpyuHCplrB84WgAiGV6sYpZq7Tj,3,13-750-942-6364,794.47,HOUSEHOLD,n accounts will have to unwind. foxes cajole accor,50
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/cd7bbae0/tajo-core/tajo-core-backend/src/test/resources/results/TestSelectQuery/testSelectAsterisk4.result
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/resources/results/TestSelectQuery/testSelectAsterisk4.result b/tajo-core/tajo-core-backend/src/test/resources/results/TestSelectQuery/testSelectAsterisk4.result
new file mode 100644
index 0000000..9c6bcb9
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/test/resources/results/TestSelectQuery/testSelectAsterisk4.result
@@ -0,0 +1,7 @@
+?length,?multiply_1,l_orderkey,l_partkey,l_suppkey,l_linenumber,l_quantity,l_extendedprice,l_discount,l_tax,l_returnflag,l_linestatus,l_shipdate,l_commitdate,l_receiptdate,l_shipinstruct,l_shipmode,l_comment,?multiply_2
+-------------------------------
+23,846.7292,1,1,7706,1,17.0,21168.23,0.04,0.02,N,O,1996-03-13,1996-02-12,1996-03-22,DELIVER IN PERSON,TRUCK,egular courts above the,0.2
+34,4138.4844,1,1,7311,2,36.0,45983.16,0.09,0.06,N,O,1996-04-12,1996-02-28,1996-04-20,TAKE BACK RETURN,MAIL,ly final dependencies: slyly bold ,0.6
+31,0.0,2,2,1191,1,38.0,44694.46,0.0,0.05,N,O,1997-01-28,1997-01-14,1997-02-02,TAKE BACK RETURN,RAIL,ven requests. deposits breach a,0.5
+35,3243.483,3,2,1798,1,45.0,54058.05,0.06,0.0,R,F,1994-02-02,1994-01-04,1994-02-23,NONE,AIR,ongside of the furiously brave acco,0.0
+22,4679.647,3,3,6540,2,49.0,46796.47,0.1,0.0,R,F,1993-11-09,1993-12-20,1993-11-24,TAKE BACK RETURN,RAIL, unusual accounts. eve,0.0
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/cd7bbae0/tajo-core/tajo-core-backend/src/test/resources/results/TestSortQuery/testAsterisk.result
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/resources/results/TestSortQuery/testAsterisk.result b/tajo-core/tajo-core-backend/src/test/resources/results/TestSortQuery/testAsterisk.result
new file mode 100644
index 0000000..f2e3c84
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/test/resources/results/TestSortQuery/testAsterisk.result
@@ -0,0 +1,7 @@
+l_orderkey,l_partkey,l_suppkey,l_linenumber,l_quantity,l_extendedprice,l_discount,l_tax,l_returnflag,l_linestatus,l_shipdate,l_commitdate,l_receiptdate,l_shipinstruct,l_shipmode,l_comment,len_comment
+-------------------------------
+3,3,6540,2,49.0,46796.47,0.1,0.0,R,F,1993-11-09,1993-12-20,1993-11-24,TAKE BACK RETURN,RAIL, unusual accounts. eve,22
+1,1,7706,1,17.0,21168.23,0.04,0.02,N,O,1996-03-13,1996-02-12,1996-03-22,DELIVER IN PERSON,TRUCK,egular courts above the,23
+2,2,1191,1,38.0,44694.46,0.0,0.05,N,O,1997-01-28,1997-01-14,1997-02-02,TAKE BACK RETURN,RAIL,ven requests. deposits breach a,31
+1,1,7311,2,36.0,45983.16,0.09,0.06,N,O,1996-04-12,1996-02-28,1996-04-20,TAKE BACK RETURN,MAIL,ly final dependencies: slyly bold ,34
+3,2,1798,1,45.0,54058.05,0.06,0.0,R,F,1994-02-02,1994-01-04,1994-02-23,NONE,AIR,ongside of the furiously brave acco,35
\ No newline at end of file