You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@phoenix.apache.org by ja...@apache.org on 2015/02/10 03:37:37 UTC

[3/5] phoenix git commit: PHOENIX-1646 Views and functional index expressions may lose information when stringified

PHOENIX-1646 Views and functional index expressions may lose information when stringified


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

Branch: refs/heads/master
Commit: abeaa74ad35e145fcae40f239437e1b5964bcd72
Parents: 2d5913b
Author: James Taylor <jt...@salesforce.com>
Authored: Mon Feb 9 16:36:34 2015 -0800
Committer: James Taylor <jt...@salesforce.com>
Committed: Mon Feb 9 18:37:14 2015 -0800

----------------------------------------------------------------------
 .../java/org/apache/phoenix/end2end/ViewIT.java |  27 ++
 .../phoenix/compile/CreateTableCompiler.java    |   4 +-
 .../phoenix/compile/ExpressionCompiler.java     |  22 +-
 .../expression/ComparisonExpression.java        |  12 +-
 .../phoenix/jdbc/PhoenixDatabaseMetaData.java   |  37 +--
 .../org/apache/phoenix/parse/AddParseNode.java  |   6 +
 .../AggregateFunctionWithinGroupParseNode.java  |  52 +++
 .../org/apache/phoenix/parse/AliasedNode.java   |  35 ++
 .../org/apache/phoenix/parse/AndParseNode.java  |  14 +
 .../phoenix/parse/ArithmeticParseNode.java      |  15 +
 .../parse/ArrayAllAnyComparisonNode.java        |  49 +++
 .../phoenix/parse/ArrayAllComparisonNode.java   |   3 +-
 .../phoenix/parse/ArrayAnyComparisonNode.java   |   3 +-
 .../phoenix/parse/ArrayConstructorNode.java     |  17 +
 .../apache/phoenix/parse/ArrayElemRefNode.java  |  11 +
 .../apache/phoenix/parse/BetweenParseNode.java  |  18 +-
 .../org/apache/phoenix/parse/BindParseNode.java |  12 +-
 .../org/apache/phoenix/parse/BindTableNode.java |   8 +
 .../org/apache/phoenix/parse/CaseParseNode.java |  20 ++
 .../org/apache/phoenix/parse/CastParseNode.java |  58 ++--
 .../org/apache/phoenix/parse/ColumnDef.java     |  26 +-
 .../apache/phoenix/parse/ColumnParseNode.java   |  47 ++-
 .../phoenix/parse/ComparisonParseNode.java      |  10 +
 .../apache/phoenix/parse/CompoundParseNode.java |   5 -
 .../apache/phoenix/parse/ConcreteTableNode.java |  19 ++
 .../apache/phoenix/parse/DerivedTableNode.java  |  27 ++
 .../phoenix/parse/DistinctCountParseNode.java   |  16 +
 .../apache/phoenix/parse/DivideParseNode.java   |   7 +
 .../apache/phoenix/parse/ExistsParseNode.java   |   9 +
 .../phoenix/parse/FamilyWildcardParseNode.java  |   8 +
 .../apache/phoenix/parse/FunctionParseNode.java |  36 ++-
 .../java/org/apache/phoenix/parse/HintNode.java |  36 +++
 .../apache/phoenix/parse/InListParseNode.java   |  19 ++
 .../org/apache/phoenix/parse/InParseNode.java   |  11 +
 .../apache/phoenix/parse/IsNullParseNode.java   |  10 +
 .../org/apache/phoenix/parse/JoinTableNode.java |  51 +++
 .../org/apache/phoenix/parse/LikeParseNode.java |  12 +
 .../org/apache/phoenix/parse/LimitNode.java     |  29 ++
 .../apache/phoenix/parse/LiteralParseNode.java  |  28 +-
 .../apache/phoenix/parse/ModulusParseNode.java  |   6 +
 .../apache/phoenix/parse/MultiplyParseNode.java |   6 +
 .../org/apache/phoenix/parse/NamedNode.java     |   6 +-
 .../apache/phoenix/parse/NamedParseNode.java    |  17 +-
 .../apache/phoenix/parse/NamedTableNode.java    |  38 +++
 .../org/apache/phoenix/parse/NotParseNode.java  |   9 +
 .../org/apache/phoenix/parse/OrParseNode.java   |  15 +
 .../org/apache/phoenix/parse/OrderByNode.java   |  34 +-
 .../phoenix/parse/OuterJoinParseNode.java       |  47 ---
 .../org/apache/phoenix/parse/ParseNode.java     |  11 +
 .../apache/phoenix/parse/ParseNodeFactory.java  |  34 +-
 .../parse/RowValueConstructorParseNode.java     |  16 +
 .../apache/phoenix/parse/SelectStatement.java   |  99 ++++++
 .../phoenix/parse/SequenceValueParseNode.java   |  10 +
 .../phoenix/parse/StringConcatParseNode.java    |  14 +
 .../apache/phoenix/parse/SubqueryParseNode.java |   8 +
 .../apache/phoenix/parse/SubtractParseNode.java |   7 +
 .../org/apache/phoenix/parse/TableName.java     |   4 +-
 .../org/apache/phoenix/parse/TableNode.java     |  10 +
 .../phoenix/parse/TableWildcardParseNode.java   |   7 +
 .../apache/phoenix/parse/WildcardParseNode.java |  16 +-
 .../apache/phoenix/schema/MetaDataClient.java   |  10 +-
 .../org/apache/phoenix/schema/types/PDate.java  |   5 +-
 .../apache/phoenix/schema/types/PVarchar.java   |   3 +-
 .../java/org/apache/phoenix/util/IndexUtil.java |   7 +-
 .../java/org/apache/phoenix/util/QueryUtil.java |  19 +-
 .../org/apache/phoenix/util/StringUtil.java     |   5 +
 .../phoenix/compile/WhereCompilerTest.java      |  84 ++---
 .../phoenix/compile/WhereOptimizerTest.java     |   3 +-
 .../apache/phoenix/parse/QueryParserTest.java   | 318 +++++++++++--------
 .../query/BaseConnectionlessQueryTest.java      |  23 ++
 .../phoenix/schema/types/PDataTypeTest.java     |   7 +
 71 files changed, 1357 insertions(+), 370 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/abeaa74a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ViewIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ViewIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ViewIT.java
index 9a89531..db1e58f 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ViewIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ViewIT.java
@@ -272,6 +272,33 @@ public class ViewIT extends BaseViewIT {
     }
     
     @Test
+    public void testViewWithCurrentDate() throws Exception {
+        Connection conn = DriverManager.getConnection(getUrl());
+        String ddl = "CREATE TABLE t (k INTEGER NOT NULL PRIMARY KEY, v1 INTEGER, v2 DATE)";
+        conn.createStatement().execute(ddl);
+        ddl = "CREATE VIEW v (v VARCHAR) AS SELECT * FROM t WHERE v2 > CURRENT_DATE()-5 AND v2 > DATE '2010-01-01'";
+        conn.createStatement().execute(ddl);
+        try {
+            conn.createStatement().execute("UPSERT INTO v VALUES(1)");
+            fail();
+        } catch (ReadOnlyTableException e) {
+            
+        }
+        for (int i = 0; i < 10; i++) {
+            conn.createStatement().execute("UPSERT INTO t VALUES(" + i + ", " + (i+10) + ",CURRENT_DATE()-" + i + ")");
+        }
+        conn.commit();
+        
+        int count = 0;
+        ResultSet rs = conn.createStatement().executeQuery("SELECT k FROM v");
+        while (rs.next()) {
+            assertEquals(count, rs.getInt(1));
+            count++;
+        }
+        assertEquals(5, count);
+    }
+
+    @Test
     public void testViewAndTableInDifferentSchemas() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
         String ddl = "CREATE TABLE s1.t (k INTEGER NOT NULL PRIMARY KEY, v1 DATE)";

http://git-wip-us.apache.org/repos/asf/phoenix/blob/abeaa74a/phoenix-core/src/main/java/org/apache/phoenix/compile/CreateTableCompiler.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/CreateTableCompiler.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/CreateTableCompiler.java
index a95cd86..edee788 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/CreateTableCompiler.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/CreateTableCompiler.java
@@ -106,7 +106,9 @@ public class CreateTableCompiler {
                 Expression where = whereNode.accept(expressionCompiler);
                 if (where != null && !LiteralExpression.isTrue(where)) {
                     TableName baseTableName = create.getBaseTableName();
-                    viewStatementToBe = QueryUtil.getViewStatement(baseTableName.getSchemaName(), baseTableName.getTableName(), where);
+                    StringBuilder buf = new StringBuilder();
+                    whereNode.toSQL(resolver, buf);
+                    viewStatementToBe = QueryUtil.getViewStatement(baseTableName.getSchemaName(), baseTableName.getTableName(), buf.toString());
                 }
                 if (viewTypeToBe != ViewType.MAPPED) {
                     Long scn = connection.getSCN();

http://git-wip-us.apache.org/repos/asf/phoenix/blob/abeaa74a/phoenix-core/src/main/java/org/apache/phoenix/compile/ExpressionCompiler.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/ExpressionCompiler.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/ExpressionCompiler.java
index 97818e6..81e4059 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/ExpressionCompiler.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/ExpressionCompiler.java
@@ -66,6 +66,8 @@ import org.apache.phoenix.expression.TimestampSubtractExpression;
 import org.apache.phoenix.expression.function.ArrayAllComparisonExpression;
 import org.apache.phoenix.expression.function.ArrayAnyComparisonExpression;
 import org.apache.phoenix.expression.function.ArrayElemRefExpression;
+import org.apache.phoenix.expression.function.RoundDecimalExpression;
+import org.apache.phoenix.expression.function.RoundTimestampExpression;
 import org.apache.phoenix.parse.AddParseNode;
 import org.apache.phoenix.parse.AndParseNode;
 import org.apache.phoenix.parse.ArithmeticParseNode;
@@ -534,6 +536,24 @@ public class ExpressionCompiler extends UnsupportedAllParseNodeVisitor<Expressio
         return true;
     }
 
+    // TODO: don't repeat this ugly cast logic (maybe use isCastable in the last else block.
+    private static Expression convertToRoundExpressionIfNeeded(PDataType fromDataType, PDataType targetDataType, List<Expression> expressions) throws SQLException {
+        Expression firstChildExpr = expressions.get(0);
+        if(fromDataType == targetDataType) {
+            return firstChildExpr;
+        } else if((fromDataType == PDecimal.INSTANCE || fromDataType == PTimestamp.INSTANCE || fromDataType == PUnsignedTimestamp.INSTANCE) && targetDataType.isCoercibleTo(
+          PLong.INSTANCE)) {
+            return RoundDecimalExpression.create(expressions);
+        } else if((fromDataType == PDecimal.INSTANCE || fromDataType == PTimestamp.INSTANCE || fromDataType == PUnsignedTimestamp.INSTANCE) && targetDataType.isCoercibleTo(
+          PDate.INSTANCE)) {
+            return RoundTimestampExpression.create(expressions);
+        } else if(fromDataType.isCastableTo(targetDataType)) {
+            return firstChildExpr;
+        } else {
+            throw TypeMismatchException.newException(fromDataType, targetDataType, firstChildExpr.toString());
+        }
+    }
+
     @Override
     public Expression visitLeave(CastParseNode node, List<Expression> children) throws SQLException {
         ParseNode childNode = node.getChildren().get(0);
@@ -553,7 +573,7 @@ public class ExpressionCompiler extends UnsupportedAllParseNodeVisitor<Expressio
              * end up creating a RoundExpression. 
              */
             if (context.getResolver().getTables().get(0).getTable().getType() != PTableType.INDEX) {
-                expr =  CastParseNode.convertToRoundExpressionIfNeeded(fromDataType, targetDataType, children);
+                expr =  convertToRoundExpressionIfNeeded(fromDataType, targetDataType, children);
             }
         }
         return wrapGroupByExpression(CoerceExpression.create(expr, targetDataType, SortOrder.getDefault(), expr.getMaxLength()));  

http://git-wip-us.apache.org/repos/asf/phoenix/blob/abeaa74a/phoenix-core/src/main/java/org/apache/phoenix/expression/ComparisonExpression.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/ComparisonExpression.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/ComparisonExpression.java
index 4bfa0e9..b9190e2 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/expression/ComparisonExpression.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/ComparisonExpression.java
@@ -44,6 +44,7 @@ import org.apache.phoenix.schema.types.PUnsignedInt;
 import org.apache.phoenix.schema.types.PUnsignedLong;
 import org.apache.phoenix.util.ByteUtil;
 import org.apache.phoenix.util.ExpressionUtil;
+import org.apache.phoenix.util.QueryUtil;
 import org.apache.phoenix.util.StringUtil;
 
 import com.google.common.collect.Lists;
@@ -57,15 +58,6 @@ import com.google.common.collect.Lists;
  */
 public class ComparisonExpression extends BaseCompoundExpression {
     private CompareOp op;
-    private static final String[] CompareOpString = new String[CompareOp.values().length];
-    static {
-        CompareOpString[CompareOp.EQUAL.ordinal()] = " = ";
-        CompareOpString[CompareOp.NOT_EQUAL.ordinal()] = " != ";
-        CompareOpString[CompareOp.GREATER.ordinal()] = " > ";
-        CompareOpString[CompareOp.LESS.ordinal()] = " < ";
-        CompareOpString[CompareOp.GREATER_OR_EQUAL.ordinal()] = " >= ";
-        CompareOpString[CompareOp.LESS_OR_EQUAL.ordinal()] = " <= ";
-    }
     
     private static void addEqualityExpression(Expression lhs, Expression rhs, List<Expression> andNodes, ImmutableBytesWritable ptr) throws SQLException {
         boolean isLHSNull = ExpressionUtil.isNull(lhs, ptr);
@@ -370,7 +362,7 @@ public class ComparisonExpression extends BaseCompoundExpression {
     }
     
     public static String toString(CompareOp op, List<Expression> children) {
-        return (children.get(0) + CompareOpString[op.ordinal()] + children.get(1));
+        return (children.get(0) + " " + QueryUtil.toSQL(op) + " " + children.get(1));
     }
     
     @Override

http://git-wip-us.apache.org/repos/asf/phoenix/blob/abeaa74a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixDatabaseMetaData.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixDatabaseMetaData.java b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixDatabaseMetaData.java
index 034c40a..154fef7 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixDatabaseMetaData.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixDatabaseMetaData.java
@@ -28,7 +28,6 @@ import java.util.Collection;
 import java.util.Collections;
 import java.util.List;
 
-import org.apache.commons.lang.StringEscapeUtils;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.client.Result;
@@ -366,10 +365,6 @@ public class PhoenixDatabaseMetaData implements DatabaseMetaData, org.apache.pho
         return emptyResultSet;
     }
 
-    private static String escapePattern(String pattern) {
-        return StringEscapeUtils.escapeSql(pattern); // Need to escape double quotes
-    }
-
     public static final String GLOBAL_TENANANTS_ONLY = "null";
 
     private void addTenantIdFilter(StringBuilder buf, String tenantIdPattern) {
@@ -378,16 +373,16 @@ public class PhoenixDatabaseMetaData implements DatabaseMetaData, org.apache.pho
             if (tenantId != null) {
                 appendConjunction(buf);
                 buf.append(" (" + TENANT_ID + " IS NULL " +
-                        " OR " + TENANT_ID + " = '" + escapePattern(tenantId.getString()) + "') ");
+                        " OR " + TENANT_ID + " = '" + StringUtil.escapeStringConstant(tenantId.getString()) + "') ");
             }
         } else if (tenantIdPattern.length() == 0) {
                 appendConjunction(buf);
                 buf.append(TENANT_ID + " IS NULL ");
         } else {
             appendConjunction(buf);
-            buf.append(" TENANT_ID LIKE '" + escapePattern(tenantIdPattern) + "' ");
+            buf.append(" TENANT_ID LIKE '" + StringUtil.escapeStringConstant(tenantIdPattern) + "' ");
             if (tenantId != null) {
-                buf.append(" and TENANT_ID + = '" + escapePattern(tenantId.getString()) + "' ");
+                buf.append(" and TENANT_ID + = '" + StringUtil.escapeStringConstant(tenantId.getString()) + "' ");
             }
         }
     }
@@ -433,11 +428,11 @@ public class PhoenixDatabaseMetaData implements DatabaseMetaData, org.apache.pho
         addTenantIdFilter(where, catalog);
         if (schemaPattern != null) {
             appendConjunction(where);
-            where.append(TABLE_SCHEM + (schemaPattern.length() == 0 ? " is null" : " like '" + escapePattern(schemaPattern) + "'" ));
+            where.append(TABLE_SCHEM + (schemaPattern.length() == 0 ? " is null" : " like '" + StringUtil.escapeStringConstant(schemaPattern) + "'" ));
         }
         if (tableNamePattern != null && tableNamePattern.length() > 0) {
             appendConjunction(where);
-            where.append(TABLE_NAME + " like '" + escapePattern(tableNamePattern) + "'" );
+            where.append(TABLE_NAME + " like '" + StringUtil.escapeStringConstant(tableNamePattern) + "'" );
         }
         // Allow a "." in columnNamePattern for column family match
         String colPattern = null;
@@ -455,11 +450,11 @@ public class PhoenixDatabaseMetaData implements DatabaseMetaData, org.apache.pho
             if (cfPattern != null && cfPattern.length() > 0) { // if null or empty, will pick up all columns
                 // Will pick up only KV columns
                 appendConjunction(where);
-                where.append(COLUMN_FAMILY + " like '" + escapePattern(cfPattern) + "'" );
+                where.append(COLUMN_FAMILY + " like '" + StringUtil.escapeStringConstant(cfPattern) + "'" );
             }
             if (colPattern != null && colPattern.length() > 0) {
                 appendConjunction(where);
-                where.append(COLUMN_NAME + " like '" + escapePattern(colPattern) + "'" );
+                where.append(COLUMN_NAME + " like '" + StringUtil.escapeStringConstant(colPattern) + "'" );
             }
         }
         if (colPattern == null) {
@@ -680,8 +675,8 @@ public class PhoenixDatabaseMetaData implements DatabaseMetaData, org.apache.pho
                 ARRAY_SIZE +
                 "\nfrom " + SYSTEM_CATALOG +
                 "\nwhere ");
-        buf.append(TABLE_SCHEM + (schema == null || schema.length() == 0 ? " is null" : " = '" + escapePattern(schema) + "'" ));
-        buf.append("\nand " + DATA_TABLE_NAME + " = '" + escapePattern(table) + "'" );
+        buf.append(TABLE_SCHEM + (schema == null || schema.length() == 0 ? " is null" : " = '" + StringUtil.escapeStringConstant(schema) + "'" ));
+        buf.append("\nand " + DATA_TABLE_NAME + " = '" + StringUtil.escapeStringConstant(table) + "'" );
         buf.append("\nand " + COLUMN_NAME + " is not null" );
         addTenantIdFilter(buf, catalog);
         buf.append("\norder by INDEX_NAME," + ORDINAL_POSITION);
@@ -825,8 +820,8 @@ public class PhoenixDatabaseMetaData implements DatabaseMetaData, org.apache.pho
                 VIEW_CONSTANT +
                 " from " + SYSTEM_CATALOG + " " + SYSTEM_CATALOG_ALIAS +
                 " where ");
-        buf.append(TABLE_SCHEM + (schema == null || schema.length() == 0 ? " is null" : " = '" + escapePattern(schema) + "'" ));
-        buf.append(" and " + TABLE_NAME + " = '" + escapePattern(table) + "'" );
+        buf.append(TABLE_SCHEM + (schema == null || schema.length() == 0 ? " is null" : " = '" + StringUtil.escapeStringConstant(schema) + "'" ));
+        buf.append(" and " + TABLE_NAME + " = '" + StringUtil.escapeStringConstant(table) + "'" );
         buf.append(" and " + COLUMN_NAME + " is not null");
         buf.append(" and " + COLUMN_FAMILY + " is null");
         addTenantIdFilter(buf, catalog);
@@ -891,7 +886,7 @@ public class PhoenixDatabaseMetaData implements DatabaseMetaData, org.apache.pho
                 " where " + COLUMN_NAME + " is null");
         this.addTenantIdFilter(buf, catalog);
         if (schemaPattern != null) {
-            buf.append(" and " + TABLE_SCHEM + " like '" + escapePattern(schemaPattern) + "'");
+            buf.append(" and " + TABLE_SCHEM + " like '" + StringUtil.escapeStringConstant(schemaPattern) + "'");
         }
         Statement stmt = connection.createStatement();
         return stmt.executeQuery(buf.toString());
@@ -919,10 +914,10 @@ public class PhoenixDatabaseMetaData implements DatabaseMetaData, org.apache.pho
                 " and " + LINK_TYPE + " = " + LinkType.PHYSICAL_TABLE.getSerializedValue());
         addTenantIdFilter(buf, catalog);
         if (schemaPattern != null) {
-            buf.append(" and " + TABLE_SCHEM + (schemaPattern.length() == 0 ? " is null" : " like '" + escapePattern(schemaPattern) + "'" ));
+            buf.append(" and " + TABLE_SCHEM + (schemaPattern.length() == 0 ? " is null" : " like '" + StringUtil.escapeStringConstant(schemaPattern) + "'" ));
         }
         if (tableNamePattern != null) {
-            buf.append(" and " + TABLE_NAME + " like '" + escapePattern(tableNamePattern) + "'" );
+            buf.append(" and " + TABLE_NAME + " like '" + StringUtil.escapeStringConstant(tableNamePattern) + "'" );
         }
         buf.append(" order by " + TENANT_ID + "," + TABLE_SCHEM + "," +TABLE_NAME + "," + SUPERTABLE_NAME);
         Statement stmt = connection.createStatement();
@@ -1017,10 +1012,10 @@ public class PhoenixDatabaseMetaData implements DatabaseMetaData, org.apache.pho
                 " and " + COLUMN_FAMILY + " is null");
         addTenantIdFilter(buf, catalog);
         if (schemaPattern != null) {
-            buf.append(" and " + TABLE_SCHEM + (schemaPattern.length() == 0 ? " is null" : " like '" + escapePattern(schemaPattern) + "'" ));
+            buf.append(" and " + TABLE_SCHEM + (schemaPattern.length() == 0 ? " is null" : " like '" + StringUtil.escapeStringConstant(schemaPattern) + "'" ));
         }
         if (tableNamePattern != null) {
-            buf.append(" and " + TABLE_NAME + " like '" + escapePattern(tableNamePattern) + "'" );
+            buf.append(" and " + TABLE_NAME + " like '" + StringUtil.escapeStringConstant(tableNamePattern) + "'" );
         }
         if (types != null && types.length > 0) {
             buf.append(" and " + TABLE_TYPE + " IN (");

http://git-wip-us.apache.org/repos/asf/phoenix/blob/abeaa74a/phoenix-core/src/main/java/org/apache/phoenix/parse/AddParseNode.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/AddParseNode.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/AddParseNode.java
index f855ada..fa04a55 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/parse/AddParseNode.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/AddParseNode.java
@@ -31,7 +31,13 @@ import java.util.List;
  * @since 0.1
  */
 public class AddParseNode extends ArithmeticParseNode {
+    public static final String OPERATOR = "+";
 
+    @Override
+    public String getOperator() {
+        return OPERATOR;
+    }
+    
     AddParseNode(List<ParseNode> children) {
         super(children);
     }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/abeaa74a/phoenix-core/src/main/java/org/apache/phoenix/parse/AggregateFunctionWithinGroupParseNode.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/AggregateFunctionWithinGroupParseNode.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/AggregateFunctionWithinGroupParseNode.java
new file mode 100644
index 0000000..5c32908
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/AggregateFunctionWithinGroupParseNode.java
@@ -0,0 +1,52 @@
+/*
+ * 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.phoenix.parse;
+
+import java.util.List;
+
+import org.apache.phoenix.compile.ColumnResolver;
+
+public class AggregateFunctionWithinGroupParseNode extends AggregateFunctionParseNode {
+
+    public AggregateFunctionWithinGroupParseNode(String name, List<ParseNode> children, BuiltInFunctionInfo info) {
+        super(name, children, info);
+    }
+
+
+    @Override
+    public void toSQL(ColumnResolver resolver, StringBuilder buf) {
+        buf.append(' ');
+        buf.append(getName());
+        buf.append('(');
+        List<ParseNode> children = getChildren();
+        List<ParseNode> args = children.subList(2, children.size());
+        if (!args.isEmpty()) {
+            for (ParseNode child : args) {
+                child.toSQL(resolver, buf);
+                buf.append(',');
+            }
+            buf.setLength(buf.length()-1);
+        }
+        buf.append(')');
+        
+        buf.append(" WITHIN GROUP (ORDER BY ");
+        children.get(0).toSQL(resolver, buf);
+        buf.append(" " + (LiteralParseNode.TRUE.equals(children.get(1)) ? "ASC" : "DESC"));
+        buf.append(')');
+    }
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/abeaa74a/phoenix-core/src/main/java/org/apache/phoenix/parse/AliasedNode.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/AliasedNode.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/AliasedNode.java
index f5dec8d..807a01f 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/parse/AliasedNode.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/AliasedNode.java
@@ -17,6 +17,7 @@
  */
 package org.apache.phoenix.parse;
 
+import org.apache.phoenix.compile.ColumnResolver;
 import org.apache.phoenix.util.SchemaUtil;
 
 /**
@@ -45,6 +46,40 @@ public class AliasedNode {
         return node;
     }
 
+    public void toSQL(ColumnResolver resolver, StringBuilder buf) {
+        node.toSQL(resolver, buf);
+        if (alias != null) {
+            buf.append(' ');
+            if (isCaseSensitve) buf.append('"');
+            buf.append(alias);
+            if (isCaseSensitve) buf.append('"');
+        }
+    }
+    
+    @Override
+    public int hashCode() {
+        final int prime = 31;
+        int result = 1;
+        result = prime * result + ((alias == null) ? 0 : alias.hashCode());
+        result = prime * result + ((node == null) ? 0 : node.hashCode());
+        return result;
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+        if (this == obj) return true;
+        if (obj == null) return false;
+        if (getClass() != obj.getClass()) return false;
+        AliasedNode other = (AliasedNode)obj;
+        if (alias == null) {
+            if (other.alias != null) return false;
+        } else if (!alias.equals(other.alias)) return false;
+        if (node == null) {
+            if (other.node != null) return false;
+        } else if (!node.equals(other.node)) return false;
+        return true;
+    }
+
     public boolean isCaseSensitve() {
         return isCaseSensitve;
     }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/abeaa74a/phoenix-core/src/main/java/org/apache/phoenix/parse/AndParseNode.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/AndParseNode.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/AndParseNode.java
index e8c6138..3c333c4 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/parse/AndParseNode.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/AndParseNode.java
@@ -21,6 +21,8 @@ import java.sql.SQLException;
 import java.util.Collections;
 import java.util.List;
 
+import org.apache.phoenix.compile.ColumnResolver;
+
 
 
 
@@ -32,6 +34,7 @@ import java.util.List;
  * @since 0.1
  */
 public class AndParseNode extends CompoundParseNode {
+    public static final String NAME = "AND";
 
     AndParseNode(List<ParseNode> children) {
         super(children);
@@ -46,4 +49,15 @@ public class AndParseNode extends CompoundParseNode {
         return visitor.visitLeave(this, l);
     }
     
+    @Override
+    public void toSQL(ColumnResolver resolver, StringBuilder buf) {
+        buf.append('(');
+        List<ParseNode> children = getChildren();
+        children.get(0).toSQL(resolver, buf);
+        for (int i = 1 ; i < children.size(); i++) {
+            buf.append(" " + NAME + " ");
+            children.get(i).toSQL(resolver, buf);
+        }
+        buf.append(')');
+    }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/abeaa74a/phoenix-core/src/main/java/org/apache/phoenix/parse/ArithmeticParseNode.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/ArithmeticParseNode.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/ArithmeticParseNode.java
index ca4b5f2..1a2f170 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/parse/ArithmeticParseNode.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/ArithmeticParseNode.java
@@ -19,10 +19,25 @@ package org.apache.phoenix.parse;
 
 import java.util.List;
 
+import org.apache.phoenix.compile.ColumnResolver;
+
 public abstract class ArithmeticParseNode extends CompoundParseNode {
 
     public ArithmeticParseNode(List<ParseNode> children) {
         super(children);
     }
 
+    public abstract String getOperator();
+    
+    @Override
+    public void toSQL(ColumnResolver resolver, StringBuilder buf) {
+        buf.append('(');
+        List<ParseNode> children = getChildren();
+        children.get(0).toSQL(resolver, buf);
+        for (int i = 1 ; i < children.size(); i++) {
+            buf.append(" " + getOperator() + " ");
+            children.get(i).toSQL(resolver, buf);
+        }
+        buf.append(')');
+    }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/abeaa74a/phoenix-core/src/main/java/org/apache/phoenix/parse/ArrayAllAnyComparisonNode.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/ArrayAllAnyComparisonNode.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/ArrayAllAnyComparisonNode.java
new file mode 100644
index 0000000..bdb50f9
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/ArrayAllAnyComparisonNode.java
@@ -0,0 +1,49 @@
+/*
+ * 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.phoenix.parse;
+
+import java.util.List;
+
+import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
+import org.apache.phoenix.compile.ColumnResolver;
+import org.apache.phoenix.util.QueryUtil;
+
+public abstract class ArrayAllAnyComparisonNode extends CompoundParseNode {
+
+    public ArrayAllAnyComparisonNode(List<ParseNode> children) {
+        super(children);
+    }
+
+    public abstract String getType();
+
+    @Override
+    public void toSQL(ColumnResolver resolver, StringBuilder buf) {
+        List<ParseNode> children = getChildren();
+        ParseNode rhs = children.get(0);
+        ComparisonParseNode comp = (ComparisonParseNode)children.get(1);
+        ParseNode lhs = comp.getLHS();
+        CompareOp op = comp.getFilterOp();
+        buf.append(' ');
+        lhs.toSQL(resolver, buf);
+        buf.append(" " + QueryUtil.toSQL(op) + " ");
+        buf.append(getType());
+        buf.append('(');
+        rhs.toSQL(resolver, buf);
+        buf.append(')');
+    }
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/abeaa74a/phoenix-core/src/main/java/org/apache/phoenix/parse/ArrayAllComparisonNode.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/ArrayAllComparisonNode.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/ArrayAllComparisonNode.java
index b31b3ae..98371a5 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/parse/ArrayAllComparisonNode.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/ArrayAllComparisonNode.java
@@ -25,12 +25,13 @@ import java.util.List;
  * The Expression a = ALL(b) where b is of type array is rewritten in this
  * node as ALL(a = b(n))
  */
-public class ArrayAllComparisonNode extends CompoundParseNode {
+public class ArrayAllComparisonNode extends ArrayAllAnyComparisonNode {
 
     ArrayAllComparisonNode(ParseNode rhs, ComparisonParseNode compareNode) {
         super(Arrays.<ParseNode>asList(rhs, compareNode));
     }
     
+    @Override
     public String getType() {
         return "ALL";
     }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/abeaa74a/phoenix-core/src/main/java/org/apache/phoenix/parse/ArrayAnyComparisonNode.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/ArrayAnyComparisonNode.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/ArrayAnyComparisonNode.java
index daca86d..a4662b5 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/parse/ArrayAnyComparisonNode.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/ArrayAnyComparisonNode.java
@@ -25,12 +25,13 @@ import java.util.List;
  * The Expression a = ANY(b) where b is of type array is rewritten in this
  * node as ANY(a = b(n))
  */
-public class ArrayAnyComparisonNode extends CompoundParseNode {
+public class ArrayAnyComparisonNode extends ArrayAllAnyComparisonNode {
 
     ArrayAnyComparisonNode(ParseNode rhs, ComparisonParseNode compareNode) {
         super(Arrays.<ParseNode>asList(rhs, compareNode));
     }
     
+    @Override
     public String getType() {
         return "ANY";
     }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/abeaa74a/phoenix-core/src/main/java/org/apache/phoenix/parse/ArrayConstructorNode.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/ArrayConstructorNode.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/ArrayConstructorNode.java
index a959ba7..9b6a6be 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/parse/ArrayConstructorNode.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/ArrayConstructorNode.java
@@ -21,6 +21,9 @@ import java.sql.SQLException;
 import java.util.Collections;
 import java.util.List;
 
+import org.apache.phoenix.compile.ColumnResolver;
+import org.apache.phoenix.schema.types.PArrayDataType;
+
 /**
  * Holds the list of array elements that will be used by the upsert stmt with ARRAY column 
  *
@@ -39,4 +42,18 @@ public class ArrayConstructorNode extends CompoundParseNode {
         }
         return visitor.visitLeave(this, l);
 	}
+    
+    @Override
+    public void toSQL(ColumnResolver resolver, StringBuilder buf) {
+        buf.append(' ');
+        buf.append(PArrayDataType.ARRAY_TYPE_SUFFIX);
+        buf.append('[');
+        List<ParseNode> children = getChildren();
+        children.get(0).toSQL(resolver, buf);
+        for (int i = 1 ; i < children.size(); i++) {
+            buf.append(',');
+            children.get(i).toSQL(resolver, buf);
+        }
+        buf.append(']');
+    }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/abeaa74a/phoenix-core/src/main/java/org/apache/phoenix/parse/ArrayElemRefNode.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/ArrayElemRefNode.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/ArrayElemRefNode.java
index da69de2..b3c4ad9 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/parse/ArrayElemRefNode.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/ArrayElemRefNode.java
@@ -21,6 +21,8 @@ import java.sql.SQLException;
 import java.util.Collections;
 import java.util.List;
 
+import org.apache.phoenix.compile.ColumnResolver;
+
 public class ArrayElemRefNode extends CompoundParseNode {
 
     public ArrayElemRefNode(List<ParseNode> parseNode) {
@@ -35,4 +37,13 @@ public class ArrayElemRefNode extends CompoundParseNode {
         }
         return visitor.visitLeave(this, l);
     }
+    
+    @Override
+    public void toSQL(ColumnResolver resolver, StringBuilder buf) {
+        List<ParseNode> children = getChildren();
+        children.get(0).toSQL(resolver, buf);
+        buf.append('[');
+        children.get(1).toSQL(resolver, buf);
+        buf.append(']');
+    }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/abeaa74a/phoenix-core/src/main/java/org/apache/phoenix/parse/BetweenParseNode.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/BetweenParseNode.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/BetweenParseNode.java
index 961af20..6d82117 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/parse/BetweenParseNode.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/BetweenParseNode.java
@@ -18,7 +18,11 @@
 package org.apache.phoenix.parse;
 
 import java.sql.SQLException;
-import java.util.*;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.phoenix.compile.ColumnResolver;
 
 
 
@@ -71,4 +75,16 @@ public class BetweenParseNode extends CompoundParseNode {
 			return false;
 		return true;
 	}
+
+    
+    @Override
+    public void toSQL(ColumnResolver resolver, StringBuilder buf) {
+        List<ParseNode> children = getChildren();
+        children.get(0).toSQL(resolver, buf);
+        if (negate) buf.append(" NOT");
+        buf.append(" BETWEEN ");
+        children.get(1).toSQL(resolver, buf);
+        buf.append(" AND ");
+        children.get(2).toSQL(resolver, buf);
+    }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/abeaa74a/phoenix-core/src/main/java/org/apache/phoenix/parse/BindParseNode.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/BindParseNode.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/BindParseNode.java
index 5f649de..42e42bf 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/parse/BindParseNode.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/BindParseNode.java
@@ -19,6 +19,8 @@ package org.apache.phoenix.parse;
 
 import java.sql.SQLException;
 
+import org.apache.phoenix.compile.ColumnResolver;
+
 
 
 /**
@@ -51,11 +53,6 @@ public class BindParseNode extends NamedParseNode {
         return true;
     }
     
-    @Override
-    public String toString() {
-        return ":" + index;
-    }
-
 	@Override
 	public int hashCode() {
 		final int prime = 31;
@@ -78,4 +75,9 @@ public class BindParseNode extends NamedParseNode {
 		return true;
 	}
 
+    @Override
+    public void toSQL(ColumnResolver resolver, StringBuilder buf) {
+        buf.append(':');
+        buf.append(index);
+    }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/abeaa74a/phoenix-core/src/main/java/org/apache/phoenix/parse/BindTableNode.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/BindTableNode.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/BindTableNode.java
index 52a8948..3895dd1 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/parse/BindTableNode.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/BindTableNode.java
@@ -19,6 +19,8 @@ package org.apache.phoenix.parse;
 
 import java.sql.SQLException;
 
+import org.apache.phoenix.compile.ColumnResolver;
+
 
 
 /**
@@ -39,5 +41,11 @@ public class BindTableNode extends ConcreteTableNode {
         return visitor.visit(this);
     }
 
+    @Override
+    public void toSQL(ColumnResolver resolver, StringBuilder buf) {
+        buf.append(this.getName().toString());
+        if (this.getAlias() != null) buf.append(" " + this.getAlias());
+        buf.append(' ');
+    }
 }
 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/abeaa74a/phoenix-core/src/main/java/org/apache/phoenix/parse/CaseParseNode.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/CaseParseNode.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/CaseParseNode.java
index 111b9c6..9467e68 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/parse/CaseParseNode.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/CaseParseNode.java
@@ -21,6 +21,8 @@ import java.sql.SQLException;
 import java.util.Collections;
 import java.util.List;
 
+import org.apache.phoenix.compile.ColumnResolver;
+
 
 
 /**
@@ -44,4 +46,22 @@ public class CaseParseNode extends CompoundParseNode {
         }
         return visitor.visitLeave(this, l);
     }
+
+    
+    @Override
+    public void toSQL(ColumnResolver resolver, StringBuilder buf) {
+        buf.append("CASE ");
+        List<ParseNode> children = getChildren();
+        for (int i = 0; i < children.size() - 1; i+=2) {
+            buf.append("WHEN ");
+            children.get(i+1).toSQL(resolver, buf);
+            buf.append(" THEN ");
+            children.get(i).toSQL(resolver, buf);
+        }
+        if (children.size() % 2 != 0) { // has ELSE
+            buf.append(" ELSE ");
+            children.get(children.size()-1).toSQL(resolver, buf);
+        }
+        buf.append(" END ");
+    }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/abeaa74a/phoenix-core/src/main/java/org/apache/phoenix/parse/CastParseNode.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/CastParseNode.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/CastParseNode.java
index 598a190..78be616 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/parse/CastParseNode.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/CastParseNode.java
@@ -21,16 +21,8 @@ import java.sql.SQLException;
 import java.util.Collections;
 import java.util.List;
 
-import org.apache.phoenix.expression.Expression;
-import org.apache.phoenix.expression.function.RoundDecimalExpression;
-import org.apache.phoenix.expression.function.RoundTimestampExpression;
-import org.apache.phoenix.schema.types.PDate;
-import org.apache.phoenix.schema.types.PDecimal;
+import org.apache.phoenix.compile.ColumnResolver;
 import org.apache.phoenix.schema.types.PDataType;
-import org.apache.phoenix.schema.types.PLong;
-import org.apache.phoenix.schema.types.PTimestamp;
-import org.apache.phoenix.schema.types.PUnsignedTimestamp;
-import org.apache.phoenix.schema.TypeMismatchException;
 import org.apache.phoenix.util.SchemaUtil;
 
 /**
@@ -42,7 +34,6 @@ import org.apache.phoenix.util.SchemaUtil;
  *
  */
 public class CastParseNode extends UnaryParseNode {
-	
 	private final PDataType dt;
     private final Integer maxLength;
     private final Integer scale;
@@ -83,28 +74,6 @@ public class CastParseNode extends UnaryParseNode {
         return scale;
     }
 
-    // TODO: don't repeat this ugly cast logic (maybe use isCastable in the last else block.
-    public static Expression convertToRoundExpressionIfNeeded(PDataType fromDataType, PDataType targetDataType, List<Expression> expressions) throws SQLException {
-	    Expression firstChildExpr = expressions.get(0);
-	    if(fromDataType == targetDataType) {
-	        return firstChildExpr;
-//        } else if((fromDataType == PDataType.DATE || fromDataType == PDataType.UNSIGNED_DATE) && targetDataType.isCoercibleTo(PDataType.LONG)) {
-//            return firstChildExpr;
-//        } else if(fromDataType.isCoercibleTo(PDataType.LONG) && (targetDataType == PDataType.DATE || targetDataType == PDataType.UNSIGNED_DATE)) {
-//            return firstChildExpr;
-	    } else if((fromDataType == PDecimal.INSTANCE || fromDataType == PTimestamp.INSTANCE || fromDataType == PUnsignedTimestamp.INSTANCE) && targetDataType.isCoercibleTo(
-          PLong.INSTANCE)) {
-	        return RoundDecimalExpression.create(expressions);
-	    } else if((fromDataType == PDecimal.INSTANCE || fromDataType == PTimestamp.INSTANCE || fromDataType == PUnsignedTimestamp.INSTANCE) && targetDataType.isCoercibleTo(
-          PDate.INSTANCE)) {
-	        return RoundTimestampExpression.create(expressions);
-	    } else if(fromDataType.isCastableTo(targetDataType)) {
-	        return firstChildExpr;
-        } else {
-            throw TypeMismatchException.newException(fromDataType, targetDataType, firstChildExpr.toString());
-	    }
-	}
-
 	@Override
 	public int hashCode() {
 		final int prime = 31;
@@ -142,4 +111,29 @@ public class CastParseNode extends UnaryParseNode {
 			return false;
 		return true;
 	}
+
+    @Override
+    public void toSQL(ColumnResolver resolver, StringBuilder buf) {
+        List<ParseNode> children = getChildren();
+        buf.append(" CAST(");
+        children.get(0).toSQL(resolver, buf);
+        buf.append(" AS ");
+        boolean isArray = dt.isArrayType();
+        PDataType type = isArray ? PDataType.arrayBaseType(dt) : dt;
+        buf.append(type.getSqlTypeName());
+        if (maxLength != null) {
+            buf.append('(');
+            buf.append(maxLength);
+            if (scale != null) {
+              buf.append(',');
+              buf.append(scale); // has both max length and scale. For ex- decimal(10,2)
+            }       
+            buf.append(')');
+       }
+        if (isArray) {
+            buf.append(' ');
+            buf.append(PDataType.ARRAY_TYPE_SUFFIX);
+            buf.append(' ');
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/abeaa74a/phoenix-core/src/main/java/org/apache/phoenix/parse/ColumnDef.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/ColumnDef.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/ColumnDef.java
index 8032ba5..cde3e9c 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/parse/ColumnDef.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/ColumnDef.java
@@ -21,13 +21,13 @@ import java.sql.SQLException;
 
 import org.apache.phoenix.exception.SQLExceptionCode;
 import org.apache.phoenix.exception.SQLExceptionInfo;
+import org.apache.phoenix.schema.SortOrder;
 import org.apache.phoenix.schema.types.PBinary;
 import org.apache.phoenix.schema.types.PChar;
-import org.apache.phoenix.schema.types.PDecimal;
 import org.apache.phoenix.schema.types.PDataType;
+import org.apache.phoenix.schema.types.PDecimal;
 import org.apache.phoenix.schema.types.PVarbinary;
 import org.apache.phoenix.schema.types.PVarchar;
-import org.apache.phoenix.schema.SortOrder;
 import org.apache.phoenix.util.SchemaUtil;
 
 import com.google.common.base.Preconditions;
@@ -189,4 +189,26 @@ public class ColumnDef {
 	public String getExpression() {
 		return expressionStr;
 	}
+	
+	@Override
+    public String toString() {
+	    StringBuilder buf = new StringBuilder(columnDefName.getColumnNode().toString());
+	    buf.append(' ');
+        buf.append(dataType.getSqlTypeName());
+        if (maxLength != null) {
+            buf.append('(');
+            buf.append(maxLength);
+            if (scale != null) {
+              buf.append(',');
+              buf.append(scale); // has both max length and scale. For ex- decimal(10,2)
+            }       
+            buf.append(')');
+       }
+        if (isArray) {
+            buf.append(' ');
+            buf.append(PDataType.ARRAY_TYPE_SUFFIX);
+            buf.append(' ');
+        }
+	    return buf.toString();
+	}
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/abeaa74a/phoenix-core/src/main/java/org/apache/phoenix/parse/ColumnParseNode.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/ColumnParseNode.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/ColumnParseNode.java
index e7489fd..80c5d0f 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/parse/ColumnParseNode.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/ColumnParseNode.java
@@ -19,7 +19,12 @@ package org.apache.phoenix.parse;
 
 import java.sql.SQLException;
 
+import org.apache.phoenix.compile.ColumnResolver;
 import org.apache.phoenix.query.QueryConstants;
+import org.apache.phoenix.schema.ColumnRef;
+import org.apache.phoenix.schema.PColumn;
+import org.apache.phoenix.schema.PTable;
+import org.apache.phoenix.util.SchemaUtil;
 
 /**
  * Node representing a reference to a column in a SQL expression
@@ -69,11 +74,6 @@ public class ColumnParseNode extends NamedParseNode {
     }
 
     @Override
-    public String toString() {
-        return fullName;
-    }
-
-    @Override
     public int hashCode() {
         return fullName.hashCode();
     }
@@ -90,4 +90,41 @@ public class ColumnParseNode extends NamedParseNode {
     public boolean isTableNameCaseSensitive() {
         return tableName == null ? false : tableName.isTableNameCaseSensitive();
     }
+
+    @Override
+    public void toSQL(ColumnResolver resolver, StringBuilder buf) {
+        // If resolver is not null, then resolve to get fully qualified name
+        String tableName = null;
+        if (resolver == null) {
+            if (this.tableName != null) {
+                tableName = this.tableName.getTableName();
+            }
+        } else {
+            try {
+                ColumnRef ref = resolver.resolveColumn(this.getSchemaName(), this.getTableName(), this.getName());
+                PColumn column = ref.getColumn();
+                if (!SchemaUtil.isPKColumn(column)) {
+                    PTable table = ref.getTable();
+                    String defaultFamilyName = table.getDefaultFamilyName() == null ? QueryConstants.DEFAULT_COLUMN_FAMILY : table.getDefaultFamilyName().getString();
+                    // Translate to the data table column name
+                    String dataFamilyName = column.getFamilyName().getString() ;
+                    tableName = defaultFamilyName.equals(dataFamilyName) ? null : dataFamilyName;
+                }
+                
+            } catch (SQLException e) {
+                throw new RuntimeException(e); // Already resolved, so not possible
+            }
+        }
+        if (tableName != null) {
+            if (isTableNameCaseSensitive()) {
+                buf.append('"');
+                buf.append(tableName);
+                buf.append('"');
+            } else {
+                buf.append(tableName);
+            }
+            buf.append('.');
+        }
+        toSQL(buf);
+    }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/abeaa74a/phoenix-core/src/main/java/org/apache/phoenix/parse/ComparisonParseNode.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/ComparisonParseNode.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/ComparisonParseNode.java
index 6f8e372..51cb833 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/parse/ComparisonParseNode.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/ComparisonParseNode.java
@@ -22,6 +22,8 @@ import java.util.Collections;
 import java.util.List;
 
 import org.apache.hadoop.hbase.filter.CompareFilter;
+import org.apache.phoenix.compile.ColumnResolver;
+import org.apache.phoenix.util.QueryUtil;
 
 /**
  * 
@@ -54,4 +56,12 @@ public abstract class ComparisonParseNode extends BinaryParseNode {
      * Return the inverted operator for the CompareOp
      */
     public abstract CompareFilter.CompareOp getInvertFilterOp();
+    
+    @Override
+    public void toSQL(ColumnResolver resolver, StringBuilder buf) {
+        List<ParseNode> children = getChildren();
+        children.get(0).toSQL(resolver, buf);
+        buf.append(" " + QueryUtil.toSQL(getFilterOp()) + " ");
+        children.get(1).toSQL(resolver, buf);
+    }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/abeaa74a/phoenix-core/src/main/java/org/apache/phoenix/parse/CompoundParseNode.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/CompoundParseNode.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/CompoundParseNode.java
index e0ab22b..fd5d73e 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/parse/CompoundParseNode.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/CompoundParseNode.java
@@ -68,11 +68,6 @@ public abstract class CompoundParseNode extends ParseNode {
     }
 
     @Override
-    public String toString() {
-        return this.getClass().getName() + children.toString();
-    }
-    
-    @Override
 	public int hashCode() {
 		final int prime = 31;
 		int result = 1;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/abeaa74a/phoenix-core/src/main/java/org/apache/phoenix/parse/ConcreteTableNode.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/ConcreteTableNode.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/ConcreteTableNode.java
index 05604d7..640ee7b 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/parse/ConcreteTableNode.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/ConcreteTableNode.java
@@ -38,5 +38,24 @@ public abstract class ConcreteTableNode extends TableNode {
         return name;
     }
 
+    @Override
+    public int hashCode() {
+        final int prime = 31;
+        int result = 1;
+        result = prime * result + ((name == null) ? 0 : name.hashCode());
+        return result;
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+        if (this == obj) return true;
+        if (obj == null) return false;
+        if (getClass() != obj.getClass()) return false;
+        ConcreteTableNode other = (ConcreteTableNode)obj;
+        if (name == null) {
+            if (other.name != null) return false;
+        } else if (!name.equals(other.name)) return false;
+        return true;
+    }
 }
 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/abeaa74a/phoenix-core/src/main/java/org/apache/phoenix/parse/DerivedTableNode.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/DerivedTableNode.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/DerivedTableNode.java
index b86c76d..d1ceb89 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/parse/DerivedTableNode.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/DerivedTableNode.java
@@ -19,6 +19,7 @@ package org.apache.phoenix.parse;
 
 import java.sql.SQLException;
 
+import org.apache.phoenix.compile.ColumnResolver;
 import org.apache.phoenix.util.SchemaUtil;
 
 
@@ -48,4 +49,30 @@ public class DerivedTableNode extends TableNode {
         return visitor.visit(this);
     }
 
+    @Override
+    public void toSQL(ColumnResolver resolver, StringBuilder buf) {
+        buf.append('(');
+        select.toSQL(resolver, buf);
+        buf.append(')');
+    }
+
+    @Override
+    public int hashCode() {
+        final int prime = 31;
+        int result = 1;
+        result = prime * result + ((select == null) ? 0 : select.hashCode());
+        return result;
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+        if (this == obj) return true;
+        if (obj == null) return false;
+        if (getClass() != obj.getClass()) return false;
+        DerivedTableNode other = (DerivedTableNode)obj;
+        if (select == null) {
+            if (other.select != null) return false;
+        } else if (!select.equals(other.select)) return false;
+        return true;
+    }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/abeaa74a/phoenix-core/src/main/java/org/apache/phoenix/parse/DistinctCountParseNode.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/DistinctCountParseNode.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/DistinctCountParseNode.java
index 846da8a..8dc596c 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/parse/DistinctCountParseNode.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/DistinctCountParseNode.java
@@ -20,6 +20,7 @@ package org.apache.phoenix.parse;
 import java.sql.SQLException;
 import java.util.List;
 
+import org.apache.phoenix.compile.ColumnResolver;
 import org.apache.phoenix.compile.StatementContext;
 import org.apache.phoenix.expression.Expression;
 import org.apache.phoenix.expression.function.DistinctCountAggregateFunction;
@@ -41,4 +42,19 @@ public class DistinctCountParseNode extends DelegateConstantToCountParseNode {
             throws SQLException {
         return new DistinctCountAggregateFunction(children, getDelegateFunction(children, context));
     }
+    
+    @Override
+    public void toSQL(ColumnResolver resolver, StringBuilder buf) {
+        buf.append(' ');
+        buf.append("COUNT(DISTINCT ");
+        List<ParseNode> children = getChildren();
+        if (!children.isEmpty()) {
+            for (ParseNode child : children) {
+                child.toSQL(resolver, buf);
+                buf.append(',');
+            }
+            buf.setLength(buf.length()-1);
+        }
+        buf.append(')');
+    }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/abeaa74a/phoenix-core/src/main/java/org/apache/phoenix/parse/DivideParseNode.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/DivideParseNode.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/DivideParseNode.java
index f3ed117..1a2e1f9 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/parse/DivideParseNode.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/DivideParseNode.java
@@ -31,6 +31,13 @@ import java.util.List;
  * @since 0.1
  */
 public class DivideParseNode extends ArithmeticParseNode {
+    public static final String OPERATOR = "/";
+
+    @Override
+    public String getOperator() {
+        return OPERATOR;
+    }
+    
 
     DivideParseNode(List<ParseNode> children) {
         super(children);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/abeaa74a/phoenix-core/src/main/java/org/apache/phoenix/parse/ExistsParseNode.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/ExistsParseNode.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/ExistsParseNode.java
index fde7d76..95d5e20 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/parse/ExistsParseNode.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/ExistsParseNode.java
@@ -21,6 +21,8 @@ import java.sql.SQLException;
 import java.util.Collections;
 import java.util.List;
 
+import org.apache.phoenix.compile.ColumnResolver;
+
 
 
 /**
@@ -72,4 +74,11 @@ public class ExistsParseNode extends UnaryParseNode {
 			return false;
 		return true;
 	}
+
+    @Override
+    public void toSQL(ColumnResolver resolver, StringBuilder buf) {
+        if (negate) buf.append(" NOT");
+        buf.append(" EXISTS ");
+        getChildren().get(0).toSQL(resolver, buf);
+    }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/abeaa74a/phoenix-core/src/main/java/org/apache/phoenix/parse/FamilyWildcardParseNode.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/FamilyWildcardParseNode.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/FamilyWildcardParseNode.java
index 2c939fc..80a08bf 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/parse/FamilyWildcardParseNode.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/FamilyWildcardParseNode.java
@@ -20,6 +20,8 @@ package org.apache.phoenix.parse;
 
 import java.sql.SQLException;
 
+import org.apache.phoenix.compile.ColumnResolver;
+
 /**
  * 
  * Node representing the selection of all columns of a family (cf.*) in the SELECT clause of SQL
@@ -71,5 +73,11 @@ public class FamilyWildcardParseNode extends NamedParseNode {
 			return false;
 		return true;
 	}
+	
+    @Override
+    public void toSQL(ColumnResolver resolver, StringBuilder buf) {
+        toSQL(buf);
+        buf.append(".*");
+    }
 }
 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/abeaa74a/phoenix-core/src/main/java/org/apache/phoenix/parse/FunctionParseNode.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/FunctionParseNode.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/FunctionParseNode.java
index c41fa4f..9764f52 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/parse/FunctionParseNode.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/FunctionParseNode.java
@@ -31,9 +31,7 @@ import java.util.List;
 import java.util.Set;
 
 import org.apache.http.annotation.Immutable;
-
-import com.google.common.collect.ImmutableSet;
-
+import org.apache.phoenix.compile.ColumnResolver;
 import org.apache.phoenix.compile.StatementContext;
 import org.apache.phoenix.expression.Determinism;
 import org.apache.phoenix.expression.Expression;
@@ -41,12 +39,14 @@ import org.apache.phoenix.expression.LiteralExpression;
 import org.apache.phoenix.expression.function.AggregateFunction;
 import org.apache.phoenix.expression.function.FunctionExpression;
 import org.apache.phoenix.schema.ArgumentTypeMismatchException;
+import org.apache.phoenix.schema.ValueRangeExcpetion;
 import org.apache.phoenix.schema.types.PDataType;
 import org.apache.phoenix.schema.types.PDataTypeFactory;
 import org.apache.phoenix.schema.types.PVarchar;
-import org.apache.phoenix.schema.ValueRangeExcpetion;
 import org.apache.phoenix.util.SchemaUtil;
 
+import com.google.common.collect.ImmutableSet;
+
 
 
 /**
@@ -83,18 +83,6 @@ public class FunctionParseNode extends CompoundParseNode {
         return visitor.visitLeave(this, l);
     }
 
-    @Override
-    public String toString() {
-        StringBuilder buf = new StringBuilder(name + "(");
-        for (ParseNode child : getChildren()) {
-            buf.append(child.toString());
-            buf.append(',');
-        }
-        buf.setLength(buf.length()-1);
-        buf.append(')');
-        return buf.toString();
-    }
-
     public boolean isAggregate() {
         return getInfo().isAggregate();
     }
@@ -459,4 +447,20 @@ public class FunctionParseNode extends CompoundParseNode {
 			return false;
 		return true;
 	}
+
+    @Override
+    public void toSQL(ColumnResolver resolver, StringBuilder buf) {
+        buf.append(' ');
+        buf.append(name);
+        buf.append('(');
+        List<ParseNode> children = getChildren();
+        if (!children.isEmpty()) {
+            for (ParseNode child : children) {
+                child.toSQL(resolver, buf);
+                buf.append(',');
+            }
+            buf.setLength(buf.length()-1);
+        }
+        buf.append(')');
+    }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/abeaa74a/phoenix-core/src/main/java/org/apache/phoenix/parse/HintNode.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/HintNode.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/HintNode.java
index 94f9bfb..ce20208 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/parse/HintNode.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/HintNode.java
@@ -21,6 +21,7 @@ import java.util.HashMap;
 import java.util.Map;
 
 import org.apache.phoenix.util.SchemaUtil;
+import org.apache.phoenix.util.StringUtil;
 
 import com.google.common.collect.ImmutableMap;
 
@@ -195,4 +196,39 @@ public class HintNode {
     public boolean hasHint(Hint hint) {
         return hints.containsKey(hint);
     }
+    
+    @Override
+    public String toString() {
+        if (hints.isEmpty()) {
+            return StringUtil.EMPTY_STRING;
+        }
+        StringBuilder buf = new StringBuilder("/*+ ");
+        for (Map.Entry<Hint, String> entry : hints.entrySet()) {
+            buf.append(entry.getKey());
+            buf.append(entry.getValue());
+            buf.append(' ');
+        }
+        buf.append("*/ ");
+        return buf.toString();
+    }
+
+    @Override
+    public int hashCode() {
+        final int prime = 31;
+        int result = 1;
+        result = prime * result + ((hints == null) ? 0 : hints.hashCode());
+        return result;
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+        if (this == obj) return true;
+        if (obj == null) return false;
+        if (getClass() != obj.getClass()) return false;
+        HintNode other = (HintNode)obj;
+        if (hints == null) {
+            if (other.hints != null) return false;
+        } else if (!hints.equals(other.hints)) return false;
+        return true;
+    }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/abeaa74a/phoenix-core/src/main/java/org/apache/phoenix/parse/InListParseNode.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/InListParseNode.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/InListParseNode.java
index fae15f5..b029076 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/parse/InListParseNode.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/InListParseNode.java
@@ -21,6 +21,7 @@ import java.sql.SQLException;
 import java.util.Collections;
 import java.util.List;
 
+import org.apache.phoenix.compile.ColumnResolver;
 import org.apache.phoenix.exception.SQLExceptionCode;
 import org.apache.phoenix.exception.SQLExceptionInfo;
 
@@ -83,4 +84,22 @@ public class InListParseNode extends CompoundParseNode {
 			return false;
 		return true;
 	}
+
+    @Override
+    public void toSQL(ColumnResolver resolver, StringBuilder buf) {
+        List<ParseNode> children = getChildren();
+        children.get(0).toSQL(resolver, buf);
+        buf.append(' ');
+        if (negate) buf.append("NOT ");
+        buf.append("IN");
+        buf.append('(');
+        if (children.size() > 1) {
+            for (int i = 1; i < children.size(); i++) {
+                children.get(i).toSQL(resolver, buf);
+                buf.append(',');
+            }
+            buf.setLength(buf.length()-1);
+        }
+        buf.append(')');
+    }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/abeaa74a/phoenix-core/src/main/java/org/apache/phoenix/parse/InParseNode.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/InParseNode.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/InParseNode.java
index 84984e9..9003fc8 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/parse/InParseNode.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/InParseNode.java
@@ -21,6 +21,8 @@ import java.sql.SQLException;
 import java.util.Collections;
 import java.util.List;
 
+import org.apache.phoenix.compile.ColumnResolver;
+
 
 
 /**
@@ -81,4 +83,13 @@ public class InParseNode extends BinaryParseNode {
 			return false;
 		return true;
 	}
+
+    @Override
+    public void toSQL(ColumnResolver resolver, StringBuilder buf) {
+        getChildren().get(0).toSQL(resolver, buf);
+        if (negate) buf.append(" NOT");
+        buf.append(" IN (");
+        getChildren().get(1).toSQL(resolver, buf);
+        buf.append(')');
+    }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/abeaa74a/phoenix-core/src/main/java/org/apache/phoenix/parse/IsNullParseNode.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/IsNullParseNode.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/IsNullParseNode.java
index 614cfd0..fafa9d1 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/parse/IsNullParseNode.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/IsNullParseNode.java
@@ -21,6 +21,8 @@ import java.sql.SQLException;
 import java.util.Collections;
 import java.util.List;
 
+import org.apache.phoenix.compile.ColumnResolver;
+
 
 
 /**
@@ -72,4 +74,12 @@ public class IsNullParseNode extends UnaryParseNode {
 			return false;
 		return true;
 	}
+
+    @Override
+    public void toSQL(ColumnResolver resolver, StringBuilder buf) {
+        getChildren().get(0).toSQL(resolver, buf);
+        buf.append(" IS");
+        if (negate) buf.append(" NOT");
+        buf.append(" NULL ");
+    }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/abeaa74a/phoenix-core/src/main/java/org/apache/phoenix/parse/JoinTableNode.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/JoinTableNode.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/JoinTableNode.java
index 5dd13f0..d30e4ba 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/parse/JoinTableNode.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/JoinTableNode.java
@@ -19,6 +19,8 @@ package org.apache.phoenix.parse;
 
 import java.sql.SQLException;
 
+import org.apache.phoenix.compile.ColumnResolver;
+
 
 
 /**
@@ -78,5 +80,54 @@ public class JoinTableNode extends TableNode {
     public <T> T accept(TableNodeVisitor<T> visitor) throws SQLException {
         return visitor.visit(this);
     }
+
+    @Override
+    public void toSQL(ColumnResolver resolver, StringBuilder buf) {
+        buf.append(lhs);
+        buf.append(' ');
+        if (onNode == null) {
+            buf.append(',');
+            buf.append(rhs);
+        } else {
+            buf.append(type);
+            buf.append(" JOIN ");
+            buf.append(rhs);
+            buf.append(" ON (");
+            onNode.toSQL(resolver, buf);
+            buf.append(')');
+        }
+    }
+
+    @Override
+    public int hashCode() {
+        final int prime = 31;
+        int result = 1;
+        result = prime * result + ((lhs == null) ? 0 : lhs.hashCode());
+        result = prime * result + ((onNode == null) ? 0 : onNode.hashCode());
+        result = prime * result + ((rhs == null) ? 0 : rhs.hashCode());
+        result = prime * result + (singleValueOnly ? 1231 : 1237);
+        result = prime * result + ((type == null) ? 0 : type.hashCode());
+        return result;
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+        if (this == obj) return true;
+        if (obj == null) return false;
+        if (getClass() != obj.getClass()) return false;
+        JoinTableNode other = (JoinTableNode)obj;
+        if (lhs == null) {
+            if (other.lhs != null) return false;
+        } else if (!lhs.equals(other.lhs)) return false;
+        if (onNode == null) {
+            if (other.onNode != null) return false;
+        } else if (!onNode.equals(other.onNode)) return false;
+        if (rhs == null) {
+            if (other.rhs != null) return false;
+        } else if (!rhs.equals(other.rhs)) return false;
+        if (singleValueOnly != other.singleValueOnly) return false;
+        if (type != other.type) return false;
+        return true;
+    }
 }
 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/abeaa74a/phoenix-core/src/main/java/org/apache/phoenix/parse/LikeParseNode.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/LikeParseNode.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/LikeParseNode.java
index 41d252d..8a510d4 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/parse/LikeParseNode.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/LikeParseNode.java
@@ -21,6 +21,8 @@ import java.sql.SQLException;
 import java.util.Collections;
 import java.util.List;
 
+import org.apache.phoenix.compile.ColumnResolver;
+
 
 
 /**
@@ -85,4 +87,14 @@ public class LikeParseNode extends BinaryParseNode {
 			return false;
 		return true;
 	}
+
+    
+    @Override
+    public void toSQL(ColumnResolver resolver, StringBuilder buf) {
+        List<ParseNode> children = getChildren();
+        children.get(0).toSQL(resolver, buf);
+        if (negate) buf.append(" NOT");
+        buf.append(" " + (likeType == LikeType.CASE_SENSITIVE ? "LIKE" : "ILIKE") + " ");
+        children.get(1).toSQL(resolver, buf);
+    }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/abeaa74a/phoenix-core/src/main/java/org/apache/phoenix/parse/LimitNode.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/LimitNode.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/LimitNode.java
index 1cb77e8..135cf54 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/parse/LimitNode.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/LimitNode.java
@@ -35,4 +35,33 @@ public class LimitNode {
     public ParseNode getLimitParseNode() {
         return bindNode == null ? limitNode : bindNode;
     }
+    
+    @Override
+    public String toString() {
+        return bindNode == null ? limitNode.toString() : bindNode.toString();
+    }
+
+    @Override
+    public int hashCode() {
+        final int prime = 31;
+        int result = 1;
+        result = prime * result + ((bindNode == null) ? 0 : bindNode.hashCode());
+        result = prime * result + ((limitNode == null) ? 0 : limitNode.hashCode());
+        return result;
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+        if (this == obj) return true;
+        if (obj == null) return false;
+        if (getClass() != obj.getClass()) return false;
+        LimitNode other = (LimitNode)obj;
+        if (bindNode == null) {
+            if (other.bindNode != null) return false;
+        } else if (!bindNode.equals(other.bindNode)) return false;
+        if (limitNode == null) {
+            if (other.limitNode != null) return false;
+        } else if (!limitNode.equals(other.limitNode)) return false;
+        return true;
+    }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/abeaa74a/phoenix-core/src/main/java/org/apache/phoenix/parse/LiteralParseNode.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/LiteralParseNode.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/LiteralParseNode.java
index 9e9184f..e0e8c3b 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/parse/LiteralParseNode.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/LiteralParseNode.java
@@ -21,8 +21,9 @@ import java.sql.SQLException;
 import java.util.Collections;
 import java.util.List;
 
+import org.apache.phoenix.compile.ColumnResolver;
 import org.apache.phoenix.schema.types.PDataType;
-import org.apache.phoenix.schema.types.PVarchar;
+import org.apache.phoenix.schema.types.PTimestamp;
 
 /**
  * 
@@ -36,6 +37,8 @@ public class LiteralParseNode extends TerminalParseNode {
     public static final ParseNode NULL = new LiteralParseNode(null);
     public static final ParseNode ZERO = new LiteralParseNode(0);
     public static final ParseNode ONE = new LiteralParseNode(1);
+    public static final ParseNode MINUS_ONE = new LiteralParseNode(-1L);
+    public static final ParseNode TRUE = new LiteralParseNode(true);
     
     private final Object value;
     private final PDataType type;
@@ -76,11 +79,6 @@ public class LiteralParseNode extends TerminalParseNode {
         return type == null ? null : type.toBytes(value);
     }
     
-    @Override
-    public String toString() {
-        return type == PVarchar.INSTANCE ? ("'" + value.toString() + "'") : value == null ? "null" : value.toString();
-    }
-
 	@Override
 	public int hashCode() {
 		final int prime = 31;
@@ -99,6 +97,24 @@ public class LiteralParseNode extends TerminalParseNode {
 		if (getClass() != obj.getClass())
 			return false;
 		LiteralParseNode other = (LiteralParseNode) obj;
+		if (value == other.value) return true;
+		if (type == null) return false;
 		return type.isComparableTo(other.type) && type.compareTo(value, other.value, other.type) == 0;
 	}
+
+    @Override
+    public void toSQL(ColumnResolver resolver, StringBuilder buf) {
+        if (value == null) {
+            buf.append(" null ");
+        } else {
+            // TODO: move into PDataType?
+            if (type.isCoercibleTo(PTimestamp.INSTANCE)) {
+                buf.append(type);
+                buf.append(' ');
+                buf.append(type.toStringLiteral(value, null));
+            } else {
+                buf.append(type.toStringLiteral(value, null));
+            }
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/abeaa74a/phoenix-core/src/main/java/org/apache/phoenix/parse/ModulusParseNode.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/ModulusParseNode.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/ModulusParseNode.java
index 553e13f..15d539d 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/parse/ModulusParseNode.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/ModulusParseNode.java
@@ -31,6 +31,12 @@ import java.util.List;
  * @since 0.1
  */
 public class ModulusParseNode extends ArithmeticParseNode {
+    public static final String OPERATOR = "%";
+
+    @Override
+    public String getOperator() {
+        return OPERATOR;
+    }
 
     ModulusParseNode(List<ParseNode> children) {
         super(children);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/abeaa74a/phoenix-core/src/main/java/org/apache/phoenix/parse/MultiplyParseNode.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/MultiplyParseNode.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/MultiplyParseNode.java
index 17016a4..1fc5436 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/parse/MultiplyParseNode.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/MultiplyParseNode.java
@@ -31,6 +31,12 @@ import java.util.List;
  * @since 0.1
  */
 public class MultiplyParseNode extends ArithmeticParseNode {
+    public static final String OPERATOR = "*";
+
+    @Override
+    public String getOperator() {
+        return OPERATOR;
+    }
 
     MultiplyParseNode(List<ParseNode> children) {
         super(children);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/abeaa74a/phoenix-core/src/main/java/org/apache/phoenix/parse/NamedNode.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/NamedNode.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/NamedNode.java
index 6cfeb60..3f1becc 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/parse/NamedNode.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/NamedNode.java
@@ -59,5 +59,9 @@ public class NamedNode {
         NamedNode other = (NamedNode)obj;
         return name.equals(other.name);
     }
-
+    
+    @Override
+    public String toString() {
+        return (isCaseSensitive ? "\"" : "" ) + name + (isCaseSensitive ? "\"" : "" );
+    }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/abeaa74a/phoenix-core/src/main/java/org/apache/phoenix/parse/NamedParseNode.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/NamedParseNode.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/NamedParseNode.java
index 51da80a..32dfc49 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/parse/NamedParseNode.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/NamedParseNode.java
@@ -18,6 +18,7 @@
 package org.apache.phoenix.parse;
 
 
+
 /**
  * 
  * Abstract node representing named nodes such as binds and column expressions in SQL
@@ -48,11 +49,6 @@ public abstract class NamedParseNode extends TerminalParseNode{
         return namedNode.isCaseSensitive();
     }
     
-    @Override
-    public String toString() {
-        return getName();
-    }
-
 	@Override
 	public int hashCode() {
 		final int prime = 31;
@@ -78,4 +74,15 @@ public abstract class NamedParseNode extends TerminalParseNode{
 			return false;
 		return true;
 	}
+
+    
+    public void toSQL(StringBuilder buf) {
+        if (isCaseSensitive()) {
+            buf.append('"');
+            buf.append(getName());
+            buf.append('"');
+        } else {
+            buf.append(getName());
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/abeaa74a/phoenix-core/src/main/java/org/apache/phoenix/parse/NamedTableNode.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/NamedTableNode.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/NamedTableNode.java
index 9379919..4e0906f 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/parse/NamedTableNode.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/NamedTableNode.java
@@ -21,6 +21,8 @@ import java.sql.SQLException;
 import java.util.Collections;
 import java.util.List;
 
+import org.apache.phoenix.compile.ColumnResolver;
+
 import com.google.common.collect.ImmutableList;
 
 /**
@@ -59,5 +61,41 @@ public class NamedTableNode extends ConcreteTableNode {
     public List<ColumnDef> getDynamicColumns() {
         return dynColumns;
     }
+    
+    @Override
+    public void toSQL(ColumnResolver resolver, StringBuilder buf) {
+        buf.append(this.getName().toString());
+        if (!dynColumns.isEmpty()) {
+            buf.append('(');
+            for (ColumnDef def : dynColumns) {
+                buf.append(def);
+                buf.append(',');
+            }
+            buf.setLength(buf.length()-1);
+            buf.append(')');
+        }
+        if (this.getAlias() != null) buf.append(" " + this.getAlias());
+        buf.append(' ');
+    }
+
+    @Override
+    public int hashCode() {
+        final int prime = 31;
+        int result = super.hashCode();
+        result = prime * result + ((dynColumns == null) ? 0 : dynColumns.hashCode());
+        return result;
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+        if (this == obj) return true;
+        if (!super.equals(obj)) return false;
+        if (getClass() != obj.getClass()) return false;
+        NamedTableNode other = (NamedTableNode)obj;
+        if (dynColumns == null) {
+            if (other.dynColumns != null) return false;
+        } else if (!dynColumns.equals(other.dynColumns)) return false;
+        return true;
+    }
 }
 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/abeaa74a/phoenix-core/src/main/java/org/apache/phoenix/parse/NotParseNode.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/NotParseNode.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/NotParseNode.java
index 9d87404..86ca1cf 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/parse/NotParseNode.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/NotParseNode.java
@@ -21,6 +21,8 @@ import java.sql.SQLException;
 import java.util.Collections;
 import java.util.List;
 
+import org.apache.phoenix.compile.ColumnResolver;
+
 
 
 /**
@@ -44,4 +46,11 @@ public class NotParseNode extends UnaryParseNode {
         }
         return visitor.visitLeave(this, l);
     }
+
+    @Override
+    public void toSQL(ColumnResolver resolver, StringBuilder buf) {
+        List<ParseNode> children = getChildren();
+        buf.append(" NOT ");
+        children.get(0).toSQL(resolver, buf);
+    }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/abeaa74a/phoenix-core/src/main/java/org/apache/phoenix/parse/OrParseNode.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/OrParseNode.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/OrParseNode.java
index 6531533..2a38819 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/parse/OrParseNode.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/OrParseNode.java
@@ -21,6 +21,8 @@ import java.sql.SQLException;
 import java.util.Collections;
 import java.util.List;
 
+import org.apache.phoenix.compile.ColumnResolver;
+
 
 
 /**
@@ -31,6 +33,7 @@ import java.util.List;
  * @since 0.1
  */
 public class OrParseNode extends CompoundParseNode {
+    public static final String NAME = "OR";
 
     OrParseNode(List<ParseNode> children) {
         super(children);
@@ -44,4 +47,16 @@ public class OrParseNode extends CompoundParseNode {
         }
         return visitor.visitLeave(this, l);
     }
+    
+    @Override
+    public void toSQL(ColumnResolver resolver, StringBuilder buf) {
+        buf.append('(');
+        List<ParseNode> children = getChildren();
+        children.get(0).toSQL(resolver, buf);
+        for (int i = 1 ; i < children.size(); i++) {
+            buf.append(" " + NAME + " ");
+            children.get(i).toSQL(resolver, buf);
+        }
+        buf.append(')');
+    }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/abeaa74a/phoenix-core/src/main/java/org/apache/phoenix/parse/OrderByNode.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/OrderByNode.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/OrderByNode.java
index bc24687..2451c4b 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/parse/OrderByNode.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/OrderByNode.java
@@ -17,6 +17,8 @@
  */
 package org.apache.phoenix.parse;
 
+import org.apache.phoenix.compile.ColumnResolver;
+
 
 /**
  * 
@@ -47,9 +49,39 @@ public final class OrderByNode {
     public ParseNode getNode() {
         return child;
     }
-    
+ 
+    @Override
+    public int hashCode() {
+        final int prime = 31;
+        int result = 1;
+        result = prime * result + ((child == null) ? 0 : child.hashCode());
+        result = prime * result + (nullsLast ? 1231 : 1237);
+        result = prime * result + (orderAscending ? 1231 : 1237);
+        return result;
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+        if (this == obj) return true;
+        if (obj == null) return false;
+        if (getClass() != obj.getClass()) return false;
+        OrderByNode other = (OrderByNode)obj;
+        if (child == null) {
+            if (other.child != null) return false;
+        } else if (!child.equals(other.child)) return false;
+        if (nullsLast != other.nullsLast) return false;
+        if (orderAscending != other.orderAscending) return false;
+        return true;
+    }
+
     @Override
     public String toString() {
         return child.toString() + (orderAscending ? " asc" : " desc") + " nulls " + (nullsLast ? "last" : "first");
     }
+
+    public void toSQL(ColumnResolver resolver, StringBuilder buf) {
+        child.toSQL(resolver, buf);
+        if (!orderAscending) buf.append(" DESC");
+        if (nullsLast) buf.append(" NULLS LAST ");
+    }
 }