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 17:13:01 UTC

[1/5] phoenix git commit: PHOENIX-688 Add to_time and to_timestamp built-in functions

Repository: phoenix
Updated Branches:
  refs/heads/4.0 e2f7e228b -> 92298f8d5


PHOENIX-688 Add to_time and to_timestamp built-in functions


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

Branch: refs/heads/4.0
Commit: fb7aa1a517efb1074c57a11d1d1d8756ccd6681c
Parents: e2f7e22
Author: James Taylor <jt...@salesforce.com>
Authored: Sat Feb 7 23:54:53 2015 -0800
Committer: James Taylor <jt...@salesforce.com>
Committed: Tue Feb 10 08:10:45 2015 -0800

----------------------------------------------------------------------
 .../phoenix/end2end/ProductMetricsIT.java       |   4 +-
 .../phoenix/end2end/ToDateFunctionIT.java       |  67 +++++-
 .../phoenix/end2end/TruncateFunctionIT.java     |   5 +-
 .../apache/phoenix/end2end/UpsertValuesIT.java  |   6 +-
 .../phoenix/end2end/VariableLengthPKIT.java     |   3 +-
 .../phoenix/mapreduce/CsvBulkLoadToolIT.java    |  10 +-
 .../phoenix/compile/StatementContext.java       |  28 ++-
 .../phoenix/expression/ExpressionType.java      |  10 +-
 .../expression/function/ToDateFunction.java     |  93 ++++++--
 .../expression/function/ToTimeFunction.java     |  63 ++++++
 .../function/ToTimestampFunction.java           |  63 ++++++
 .../apache/phoenix/jdbc/PhoenixConnection.java  |  20 +-
 .../apache/phoenix/parse/ToDateParseNode.java   |   6 +-
 .../apache/phoenix/parse/ToTimeParseNode.java   |  48 +++++
 .../phoenix/parse/ToTimestampParseNode.java     |  48 +++++
 .../org/apache/phoenix/query/QueryServices.java |   3 +
 .../org/apache/phoenix/schema/types/PDate.java  |  12 +-
 .../org/apache/phoenix/schema/types/PTime.java  |  16 +-
 .../apache/phoenix/schema/types/PTimestamp.java |  24 +--
 .../java/org/apache/phoenix/util/DateUtil.java  | 213 +++++++++++++------
 .../phoenix/util/csv/CsvUpsertExecutor.java     |  52 +++--
 .../phoenix/compile/WhereCompilerTest.java      |   8 +-
 .../expression/SortOrderExpressionTest.java     |  34 +--
 .../org/apache/phoenix/util/DateUtilTest.java   |  35 +--
 24 files changed, 656 insertions(+), 215 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/fb7aa1a5/phoenix-core/src/it/java/org/apache/phoenix/end2end/ProductMetricsIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ProductMetricsIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ProductMetricsIT.java
index 975541e..ddc5fab 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ProductMetricsIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ProductMetricsIT.java
@@ -31,8 +31,6 @@ import java.sql.ResultSet;
 import java.sql.SQLException;
 import java.sql.Statement;
 import java.sql.Types;
-import java.text.Format;
-import java.text.ParseException;
 import java.util.Collections;
 import java.util.List;
 import java.util.Properties;
@@ -87,7 +85,7 @@ public class ProductMetricsIT extends BaseClientManagedTimeIT {
     }
     
     private static Date toDate(String dateString) {
-        return DateUtil.parseDateTime(dateString);
+        return DateUtil.parseDate(dateString);
     }
     
     private static void initTable(byte[][] splits, long ts) throws Exception {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/fb7aa1a5/phoenix-core/src/it/java/org/apache/phoenix/end2end/ToDateFunctionIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ToDateFunctionIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ToDateFunctionIT.java
index 984e21b..bda4ea5 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ToDateFunctionIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ToDateFunctionIT.java
@@ -18,24 +18,25 @@
 
 package org.apache.phoenix.end2end;
 
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
 import java.sql.Connection;
 import java.sql.Date;
 import java.sql.DriverManager;
 import java.sql.ResultSet;
 import java.sql.SQLException;
 import java.sql.Statement;
+import java.sql.Time;
+import java.sql.Timestamp;
 import java.util.Properties;
 
 import org.apache.phoenix.query.QueryServices;
-import org.apache.phoenix.util.DateUtil;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
 
 public class ToDateFunctionIT extends BaseHBaseManagedTimeIT {
 
@@ -53,18 +54,26 @@ public class ToDateFunctionIT extends BaseHBaseManagedTimeIT {
         conn.close();
     }
 
-    private static Date callToDateFunction(Connection conn, String invocation) throws SQLException {
+    private static java.util.Date callToDateFunction(Connection conn, String invocation) throws SQLException {
         Statement stmt = conn.createStatement();
-        ResultSet rs = stmt.executeQuery(String.format("SELECT %s FROM SYSTEM.CATALOG", invocation));
+        ResultSet rs = stmt.executeQuery(String.format("SELECT %s FROM SYSTEM.CATALOG LIMIT 1", invocation));
         assertTrue(rs.next());
-        Date returnValue = rs.getDate(1);
+        java.util.Date returnValue = (java.util.Date)rs.getObject(1);
         rs.close();
         stmt.close();
         return returnValue;
     }
 
     private Date callToDateFunction(String invocation) throws SQLException {
-        return callToDateFunction(conn, invocation);
+        return (Date)callToDateFunction(conn, invocation);
+    }
+
+    private Time callToTimeFunction(String invocation) throws SQLException {
+        return (Time)callToDateFunction(conn, invocation);
+    }
+
+    private Timestamp callToTimestampFunction(String invocation) throws SQLException {
+        return (Timestamp)callToDateFunction(conn, invocation);
     }
 
     @Test
@@ -87,6 +96,44 @@ public class ToDateFunctionIT extends BaseHBaseManagedTimeIT {
     }
 
     @Test
+    public void testToTime_Default() throws SQLException {
+        // Default time zone is GMT, so this is timestamp 0
+        assertEquals(0L, callToTimeFunction("TO_TIME('1970-01-01 00:00:00')").getTime());
+        assertEquals(0L, callToTimeFunction("TO_TIME('1970-01-01 00:00:00.000')").getTime());
+        assertEquals(0L, callToTimeFunction("TO_TIME('1970-01-01')").getTime());
+        assertEquals(0L, callToTimeFunction("TO_TIME('1970/01/01','yyyy/MM/dd')").getTime());
+
+        // Test other ISO 8601 Date Compliant Formats to verify they can be parsed
+        try {
+            callToTimeFunction("TO_TIME('2015-01-27T16:17:57+00:00')");
+            callToTimeFunction("TO_TIME('2015-01-27T16:17:57Z')");
+            callToTimeFunction("TO_TIME('2015-W05')");
+            callToTimeFunction("TO_TIME('2015-W05-2')");
+        } catch (Exception ex) {
+            fail("TO_TIME Parse ISO8601 Time Failed due to:" + ex);
+        }
+    }
+
+    @Test
+    public void testToTimestamp_Default() throws SQLException {
+        // Default time zone is GMT, so this is timestamp 0
+        assertEquals(0L, callToTimestampFunction("TO_TIMESTAMP('1970-01-01 00:00:00')").getTime());
+        assertEquals(0L, callToTimestampFunction("TO_TIMESTAMP('1970-01-01 00:00:00.000')").getTime());
+        assertEquals(0L, callToTimestampFunction("TO_TIMESTAMP('1970-01-01')").getTime());
+        assertEquals(0L, callToTimestampFunction("TO_TIMESTAMP('1970/01/01','yyyy/MM/dd')").getTime());
+
+        // Test other ISO 8601 Date Compliant Formats to verify they can be parsed
+        try {
+            callToTimestampFunction("TO_TIMESTAMP('2015-01-27T16:17:57+00:00')");
+            callToTimestampFunction("TO_TIMESTAMP('2015-01-27T16:17:57Z')");
+            callToTimestampFunction("TO_TIMESTAMP('2015-W05')");
+            callToTimestampFunction("TO_TIMESTAMP('2015-W05-2')");
+        } catch (Exception ex) {
+            fail("TO_TIMESTAMP Parse ISO8601 Time Failed due to:" + ex);
+        }
+    }
+
+    @Test
     public void testToDate_CustomDateFormat() throws SQLException {
         // A date without time component is at midnight
         assertEquals(0L, callToDateFunction("TO_DATE('1970-01-01', 'yyyy-MM-dd')").getTime());
@@ -115,7 +162,7 @@ public class ToDateFunctionIT extends BaseHBaseManagedTimeIT {
 
         assertEquals(
                 -ONE_HOUR_IN_MILLIS,
-                callToDateFunction(customTimeZoneConn, "TO_DATE('1970-01-01 00:00:00')").getTime());
+                callToDateFunction(customTimeZoneConn, "TO_DATE('1970-01-01 00:00:00.000')").getTime());
     }
 
     @Test

http://git-wip-us.apache.org/repos/asf/phoenix/blob/fb7aa1a5/phoenix-core/src/it/java/org/apache/phoenix/end2end/TruncateFunctionIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/TruncateFunctionIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/TruncateFunctionIT.java
index 59c499d..9e8f2c0 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/TruncateFunctionIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/TruncateFunctionIT.java
@@ -32,7 +32,6 @@ import java.sql.DriverManager;
 import java.sql.PreparedStatement;
 import java.sql.ResultSet;
 import java.sql.Timestamp;
-import java.text.Format;
 import java.text.ParseException;
 import java.util.Properties;
 
@@ -47,11 +46,11 @@ public class TruncateFunctionIT extends BaseClientManagedTimeIT {
     private static final String DS3 = "1970-01-30 01:30:24.353";
     
     private static Date toDate(String s) throws ParseException {
-        return DateUtil.parseDateTime(s);
+        return DateUtil.parseDate(s);
     }
     
     private static Timestamp toTimestamp(String s) throws ParseException {
-        return new Timestamp((DateUtil.parseDateTime(s)).getTime());
+        return DateUtil.parseTimestamp(s);
     }
     
     @Test

http://git-wip-us.apache.org/repos/asf/phoenix/blob/fb7aa1a5/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpsertValuesIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpsertValuesIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpsertValuesIT.java
index b44fbff..8e07af5 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpsertValuesIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpsertValuesIT.java
@@ -33,8 +33,8 @@ import java.sql.PreparedStatement;
 import java.sql.ResultSet;
 import java.sql.SQLException;
 import java.sql.Statement;
-import java.sql.Timestamp;
 import java.sql.Time;
+import java.sql.Timestamp;
 import java.util.Properties;
 
 import org.apache.phoenix.exception.SQLExceptionCode;
@@ -156,7 +156,7 @@ public class UpsertValuesIT extends BaseClientManagedTimeIT {
         ResultSet rs = conn.createStatement().executeQuery("select k,to_char(date) from UpsertDateTest");
         assertTrue(rs.next());
         assertEquals("a", rs.getString(1));
-        assertEquals("2013-06-08 00:00:00", rs.getString(2));
+        assertEquals("2013-06-08 00:00:00.000", rs.getString(2));
     }
 
     @Test
@@ -548,7 +548,7 @@ public class UpsertValuesIT extends BaseClientManagedTimeIT {
     }
     
     private static Date toDate(String dateString) {
-        return DateUtil.parseDateTime(dateString);
+        return DateUtil.parseDate(dateString);
     }
     
     @Test

http://git-wip-us.apache.org/repos/asf/phoenix/blob/fb7aa1a5/phoenix-core/src/it/java/org/apache/phoenix/end2end/VariableLengthPKIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/VariableLengthPKIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/VariableLengthPKIT.java
index 0d9aeb2..417d147 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/VariableLengthPKIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/VariableLengthPKIT.java
@@ -38,7 +38,6 @@ import java.sql.SQLException;
 import java.sql.Statement;
 import java.sql.Types;
 import java.text.Format;
-import java.text.ParseException;
 import java.util.Properties;
 
 import org.apache.phoenix.exception.SQLExceptionCode;
@@ -55,7 +54,7 @@ public class VariableLengthPKIT extends BaseClientManagedTimeIT {
     private static final Date D1 = toDate(DS1);
 
     private static Date toDate(String dateString) {
-        return DateUtil.parseDateTime(dateString);
+        return DateUtil.parseDate(dateString);
     }
 
     protected static void initGroupByRowKeyColumns(long ts) throws Exception {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/fb7aa1a5/phoenix-core/src/it/java/org/apache/phoenix/mapreduce/CsvBulkLoadToolIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/mapreduce/CsvBulkLoadToolIT.java b/phoenix-core/src/it/java/org/apache/phoenix/mapreduce/CsvBulkLoadToolIT.java
index 00968ae..392395d 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/mapreduce/CsvBulkLoadToolIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/mapreduce/CsvBulkLoadToolIT.java
@@ -44,12 +44,6 @@ import org.junit.BeforeClass;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
-import static org.apache.phoenix.query.BaseTest.setUpConfigForMiniCluster;
-import static org.junit.Assert.assertArrayEquals;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-
 @Category(NeedsOwnMiniClusterTest.class)
 public class CsvBulkLoadToolIT {
 
@@ -119,11 +113,11 @@ public class CsvBulkLoadToolIT {
         assertTrue(rs.next());
         assertEquals(1, rs.getInt(1));
         assertEquals("Name 1", rs.getString(2));
-        assertEquals(DateUtil.parseDateTime("1970-01-01"), rs.getDate(3));
+        assertEquals(DateUtil.parseDate("1970-01-01"), rs.getDate(3));
         assertTrue(rs.next());
         assertEquals(2, rs.getInt(1));
         assertEquals("Name 2", rs.getString(2));
-        assertEquals(DateUtil.parseDateTime("1970-01-02"), rs.getDate(3));
+        assertEquals(DateUtil.parseDate("1970-01-02"), rs.getDate(3));
         assertFalse(rs.next());
 
         rs.close();

http://git-wip-us.apache.org/repos/asf/phoenix/blob/fb7aa1a5/phoenix-core/src/main/java/org/apache/phoenix/compile/StatementContext.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/StatementContext.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/StatementContext.java
index f48f613..d726488 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/StatementContext.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/StatementContext.java
@@ -60,8 +60,12 @@ public class StatementContext {
     private final ExpressionManager expressions;
     private final AggregationManager aggregates;
     private final String dateFormat;
-    private final TimeZone dateFormatTimeZone;
     private final Format dateFormatter;
+    private final String timeFormat;
+    private final Format timeFormatter;
+    private final String timestampFormat;
+    private final Format timestampFormatter;
+    private final TimeZone dateFormatTimeZone;
     private final String numberFormat;
     private final ImmutableBytesWritable tempPtr;
     private final PhoenixStatement statement;
@@ -99,9 +103,13 @@ public class StatementContext {
         this.expressions = new ExpressionManager();
         PhoenixConnection connection = statement.getConnection();
         this.dateFormat = connection.getQueryServices().getProps().get(QueryServices.DATE_FORMAT_ATTRIB, DateUtil.DEFAULT_DATE_FORMAT);
+        this.dateFormatter = DateUtil.getDateFormatter(dateFormat);
+        this.timeFormat = connection.getQueryServices().getProps().get(QueryServices.TIME_FORMAT_ATTRIB, DateUtil.DEFAULT_TIME_FORMAT);
+        this.timeFormatter = DateUtil.getTimeFormatter(timeFormat);
+        this.timestampFormat = connection.getQueryServices().getProps().get(QueryServices.TIMESTAMP_FORMAT_ATTRIB, DateUtil.DEFAULT_TIMESTAMP_FORMAT);
+        this.timestampFormatter = DateUtil.getTimestampFormatter(timestampFormat);
         this.dateFormatTimeZone = TimeZone.getTimeZone(
                 connection.getQueryServices().getProps().get(QueryServices.DATE_FORMAT_TIMEZONE_ATTRIB, DateUtil.DEFAULT_TIME_ZONE_ID));
-        this.dateFormatter = DateUtil.getDateFormatter(dateFormat);
         this.numberFormat = connection.getQueryServices().getProps().get(QueryServices.NUMBER_FORMAT_ATTRIB, NumberUtil.DEFAULT_NUMBER_FORMAT);
         this.tempPtr = new ImmutableBytesWritable();
         this.currentTable = resolver != null && !resolver.getTables().isEmpty() ? resolver.getTables().get(0) : null;
@@ -151,6 +159,22 @@ public class StatementContext {
         return dateFormatter;
     }
 
+    public String getTimeFormat() {
+        return timeFormat;
+    }
+
+    public Format getTimeFormatter() {
+        return timeFormatter;
+    }
+
+    public String getTimestampFormat() {
+        return timestampFormat;
+    }
+
+    public Format getTimestampFormatter() {
+        return timestampFormatter;
+    }
+
     public String getNumberFormat() {
         return numberFormat;
     }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/fb7aa1a5/phoenix-core/src/main/java/org/apache/phoenix/expression/ExpressionType.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/ExpressionType.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/ExpressionType.java
index c76d93a..cd03ac8 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/expression/ExpressionType.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/ExpressionType.java
@@ -21,6 +21,7 @@ import java.util.Map;
 
 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.ArrayIndexFunction;
 import org.apache.phoenix.expression.function.ArrayLengthFunction;
 import org.apache.phoenix.expression.function.CeilDateExpression;
@@ -40,7 +41,6 @@ import org.apache.phoenix.expression.function.FloorDateExpression;
 import org.apache.phoenix.expression.function.FloorDecimalExpression;
 import org.apache.phoenix.expression.function.FloorFunction;
 import org.apache.phoenix.expression.function.IndexStateNameFunction;
-import org.apache.phoenix.expression.function.ArrayElemRefExpression;
 import org.apache.phoenix.expression.function.InvertFunction;
 import org.apache.phoenix.expression.function.LTrimFunction;
 import org.apache.phoenix.expression.function.LastValueFunction;
@@ -76,6 +76,8 @@ import org.apache.phoenix.expression.function.TimezoneOffsetFunction;
 import org.apache.phoenix.expression.function.ToCharFunction;
 import org.apache.phoenix.expression.function.ToDateFunction;
 import org.apache.phoenix.expression.function.ToNumberFunction;
+import org.apache.phoenix.expression.function.ToTimeFunction;
+import org.apache.phoenix.expression.function.ToTimestampFunction;
 import org.apache.phoenix.expression.function.TrimFunction;
 import org.apache.phoenix.expression.function.TruncFunction;
 import org.apache.phoenix.expression.function.UpperFunction;
@@ -186,7 +188,11 @@ public enum ExpressionType {
     ModulusExpression(ModulusExpression.class),
     DistinctValueAggregateFunction(DistinctValueAggregateFunction.class),
     RegexpSplitFunctiond(RegexpSplitFunction.class),
-    RandomFunction(RandomFunction.class);
+    RandomFunction(RandomFunction.class),
+    ToTimeFunction(ToTimeFunction.class),
+    ToTimestampFunction(ToTimestampFunction.class),
+    ;
+    
     ExpressionType(Class<? extends Expression> clazz) {
         this.clazz = clazz;
     }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/fb7aa1a5/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ToDateFunction.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ToDateFunction.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ToDateFunction.java
index 73ca3ed..01b0dfd 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ToDateFunction.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ToDateFunction.java
@@ -17,24 +17,23 @@
  */
 package org.apache.phoenix.expression.function;
 
-import java.io.*;
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
 import java.sql.SQLException;
-import java.text.Format;
-import java.text.ParseException;
 import java.util.List;
 
-import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.hadoop.io.WritableUtils;
-
 import org.apache.phoenix.expression.Expression;
-import org.apache.phoenix.parse.*;
+import org.apache.phoenix.expression.LiteralExpression;
 import org.apache.phoenix.parse.FunctionParseNode.Argument;
 import org.apache.phoenix.parse.FunctionParseNode.BuiltInFunction;
-import org.apache.phoenix.schema.types.PDate;
+import org.apache.phoenix.parse.ToDateParseNode;
+import org.apache.phoenix.schema.tuple.Tuple;
 import org.apache.phoenix.schema.types.PDataType;
+import org.apache.phoenix.schema.types.PDate;
 import org.apache.phoenix.schema.types.PVarchar;
-import org.apache.phoenix.schema.tuple.Tuple;
 import org.apache.phoenix.util.DateUtil;
 
 
@@ -46,7 +45,6 @@ import org.apache.phoenix.util.DateUtil;
  * valid (constant) timezone id, or the string "local". The third argument is also optional, and
  * it defaults to GMT.
  *
- * @since 0.1
  */
 @BuiltInFunction(name=ToDateFunction.NAME, nodeClass=ToDateParseNode.class,
         args={@Argument(allowedTypes={PVarchar.class}),
@@ -56,33 +54,47 @@ public class ToDateFunction extends ScalarFunction {
     public static final String NAME = "TO_DATE";
     private DateUtil.DateTimeParser dateParser;
     private String dateFormat;
+    private String timeZoneId;
 
     public ToDateFunction() {
     }
 
-    public ToDateFunction(List<Expression> children, String dateFormat, DateUtil.DateTimeParser dateParser) throws SQLException {
-        super(children.subList(0, 1));
+    public ToDateFunction(List<Expression> children, String dateFormat, String timeZoneId) throws SQLException {
+        super(children);
+        init(dateFormat, timeZoneId);
+    }
+    
+    private void init(String dateFormat, String timeZoneId) {
         this.dateFormat = dateFormat;
-        this.dateParser = dateParser;
+        this.dateParser = DateUtil.getDateTimeParser(dateFormat, getDataType(), timeZoneId);
+        // Store resolved timeZoneId, as if it's LOCAL, we don't want the
+        // server to evaluate using the local time zone. Instead, we want
+        // to use the client local time zone.
+        this.timeZoneId = this.dateParser.getTimeZone().getID();
     }
 
     @Override
     public int hashCode() {
         final int prime = 31;
-        int result = 1;
-        result = prime * result + dateFormat.hashCode();
-        result = prime * result + getExpression().hashCode();
+        int result = super.hashCode();
+        result = prime * result + ((dateFormat == null) ? 0 : dateFormat.hashCode());
+        result = prime * result + ((timeZoneId == null) ? 0 : timeZoneId.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;
         ToDateFunction other = (ToDateFunction)obj;
-        if (!getExpression().equals(other.getExpression())) return false;
-        if (!dateFormat.equals(other.dateFormat)) return false;
+        // Only compare first child, as the other two are potentially resolved on the fly.
+        if (!this.getChildren().get(0).equals(other.getChildren().get(0))) return false;
+        if (dateFormat == null) {
+            if (other.dateFormat != null) return false;
+        } else if (!dateFormat.equals(other.dateFormat)) return false;
+        if (timeZoneId == null) {
+            if (other.timeZoneId != null) return false;
+        } else if (!timeZoneId.equals(other.timeZoneId)) return false;
         return true;
     }
 
@@ -94,8 +106,10 @@ public class ToDateFunction extends ScalarFunction {
         }
         PDataType type = expression.getDataType();
         String dateStr = (String)type.toObject(ptr, expression.getSortOrder());
-        Object value = dateParser.parseDateTime(dateStr);
-        byte[] byteValue = getDataType().toBytes(value);
+        long epochTime = dateParser.parseDateTime(dateStr);
+        PDataType returnType = getDataType();
+        byte[] byteValue = new byte[returnType.getByteSize()];
+        returnType.getCodec().encodeLong(epochTime, byteValue, 0);
         ptr.set(byteValue);
         return true;
      }
@@ -110,17 +124,50 @@ public class ToDateFunction extends ScalarFunction {
         return getExpression().isNullable();
     }
 
+    private String getTimeZoneIdArg() {
+        return children.size() < 3 ? null : (String) ((LiteralExpression) children.get(2)).getValue();
+    }
+    
+    private String getDateFormatArg() {
+        return children.size() < 2 ? null : (String) ((LiteralExpression) children.get(1)).getValue();
+    }
+    
     @Override
     public void readFields(DataInput input) throws IOException {
         super.readFields(input);
-        dateFormat = WritableUtils.readString(input);
-        dateParser = DateUtil.getDateParser(dateFormat);
+        String timeZoneId;
+        String dateFormat = WritableUtils.readString(input);  
+        if (dateFormat.length() != 0) { // pre 4.3
+            timeZoneId = DateUtil.DEFAULT_TIME_ZONE_ID;         
+        } else {
+            int nChildren = children.size();
+            if (nChildren == 1) {
+                dateFormat = WritableUtils.readString(input); 
+                timeZoneId =  WritableUtils.readString(input);
+            } else if (nChildren == 2 || DateUtil.LOCAL_TIME_ZONE_ID.equalsIgnoreCase(getTimeZoneIdArg())) {
+                dateFormat = getDateFormatArg();
+                timeZoneId =  WritableUtils.readString(input);
+            } else {
+                dateFormat = getDateFormatArg();
+                timeZoneId =  getTimeZoneIdArg();
+            }
+        }
+        init(dateFormat, timeZoneId);
     }
 
     @Override
     public void write(DataOutput output) throws IOException {
         super.write(output);
-        WritableUtils.writeString(output, dateFormat);
+        WritableUtils.writeString(output, ""); // For b/w compat
+        int nChildren = children.size();
+        // If dateFormat and/or timeZoneId are supplied as children, don't write them again,
+        // except if using LOCAL, in which case we want to write the resolved/actual time zone.
+        if (nChildren == 1) {
+            WritableUtils.writeString(output, dateFormat);
+            WritableUtils.writeString(output, timeZoneId);
+        } else if (nChildren == 2 || DateUtil.LOCAL_TIME_ZONE_ID.equalsIgnoreCase(getTimeZoneIdArg())) {
+            WritableUtils.writeString(output, timeZoneId);
+        }
     }
 
     private Expression getExpression() {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/fb7aa1a5/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ToTimeFunction.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ToTimeFunction.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ToTimeFunction.java
new file mode 100644
index 0000000..3a26584
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ToTimeFunction.java
@@ -0,0 +1,63 @@
+/*
+ * 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.expression.function;
+
+import java.sql.SQLException;
+import java.util.List;
+
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.parse.FunctionParseNode.Argument;
+import org.apache.phoenix.parse.FunctionParseNode.BuiltInFunction;
+import org.apache.phoenix.parse.ToTimeParseNode;
+import org.apache.phoenix.schema.types.PDataType;
+import org.apache.phoenix.schema.types.PTime;
+import org.apache.phoenix.schema.types.PVarchar;
+
+/**
+*
+* Implementation of the {@code TO_TIME(<string>,[<format-string>,[<timezone-string>]])} built-in function.
+* The second argument is optional and defaults to the phoenix.query.dateFormat value
+* from the HBase config. If present it must be a constant string. The third argument is either a
+* valid (constant) timezone id, or the string "LOCAL". The third argument is also optional, and
+* it defaults to GMT.
+*
+*/
+@BuiltInFunction(name=ToTimeFunction.NAME, nodeClass=ToTimeParseNode.class,
+       args={@Argument(allowedTypes={PVarchar.class}),
+               @Argument(allowedTypes={PVarchar.class},isConstant=true,defaultValue="null"),
+               @Argument(allowedTypes={PVarchar.class}, isConstant=true, defaultValue = "null") } )
+public class ToTimeFunction extends ToDateFunction {
+    public static final String NAME = "TO_TIME";
+
+    public ToTimeFunction() {
+    }
+
+    public ToTimeFunction(List<Expression> children, String dateFormat, String timeZoneId) throws SQLException {
+        super(children, dateFormat, timeZoneId);
+    }
+
+    @Override
+    public PDataType getDataType() {
+        return PTime.INSTANCE;
+    }
+
+    @Override
+    public String getName() {
+        return NAME;
+    }
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/fb7aa1a5/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ToTimestampFunction.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ToTimestampFunction.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ToTimestampFunction.java
new file mode 100644
index 0000000..17643a2
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ToTimestampFunction.java
@@ -0,0 +1,63 @@
+/*
+ * 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.expression.function;
+
+import java.sql.SQLException;
+import java.util.List;
+
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.parse.FunctionParseNode.Argument;
+import org.apache.phoenix.parse.FunctionParseNode.BuiltInFunction;
+import org.apache.phoenix.parse.ToTimestampParseNode;
+import org.apache.phoenix.schema.types.PDataType;
+import org.apache.phoenix.schema.types.PTimestamp;
+import org.apache.phoenix.schema.types.PVarchar;
+
+/**
+*
+* Implementation of the {@code TO_TIMESTAMP(<string>,[<format-string>,[<timezone-string>]])} built-in function.
+* The second argument is optional and defaults to the phoenix.query.timestampFormat value
+* from the HBase config. If present it must be a constant string. The third argument is either a
+* valid (constant) timezone id, or the string "local". The third argument is also optional, and
+* it defaults to GMT.
+*
+*/
+@BuiltInFunction(name=ToTimestampFunction.NAME, nodeClass=ToTimestampParseNode.class,
+       args={@Argument(allowedTypes={PVarchar.class}),
+               @Argument(allowedTypes={PVarchar.class},isConstant=true,defaultValue="null"),
+               @Argument(allowedTypes={PVarchar.class}, isConstant=true, defaultValue = "null") } )
+public class ToTimestampFunction extends ToDateFunction {
+    public static final String NAME = "TO_TIMESTAMP";
+
+    public ToTimestampFunction() {
+    }
+
+    public ToTimestampFunction(List<Expression> children, String dateFormat, String timeZoneId) throws SQLException {
+        super(children, dateFormat, timeZoneId);
+    }
+
+    @Override
+    public PDataType getDataType() {
+        return PTimestamp.INSTANCE;
+    }
+
+    @Override
+    public String getName() {
+        return NAME;
+    }
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/fb7aa1a5/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixConnection.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixConnection.java b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixConnection.java
index b778a57..de9e323 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixConnection.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixConnection.java
@@ -128,6 +128,8 @@ public class PhoenixConnection implements Connection, org.apache.phoenix.jdbc.Jd
     private PMetaData metaData;
     private final PName tenantId;
     private final String datePattern;
+    private final String timePattern;
+    private final String timestampPattern;
     
     private boolean isClosed = false;
     private Sampler<?> sampler;
@@ -204,15 +206,19 @@ public class PhoenixConnection implements Connection, org.apache.phoenix.jdbc.Jd
         this.tenantId = tenantId;
         this.mutateBatchSize = JDBCUtil.getMutateBatchSize(url, this.info, this.services.getProps());
         datePattern = this.services.getProps().get(QueryServices.DATE_FORMAT_ATTRIB, DateUtil.DEFAULT_DATE_FORMAT);
+        timePattern = this.services.getProps().get(QueryServices.TIME_FORMAT_ATTRIB, DateUtil.DEFAULT_TIME_FORMAT);
+        timestampPattern = this.services.getProps().get(QueryServices.TIMESTAMP_FORMAT_ATTRIB, DateUtil.DEFAULT_TIMESTAMP_FORMAT);
         String numberPattern = this.services.getProps().get(QueryServices.NUMBER_FORMAT_ATTRIB, NumberUtil.DEFAULT_NUMBER_FORMAT);
         int maxSize = this.services.getProps().getInt(QueryServices.MAX_MUTATION_SIZE_ATTRIB,QueryServicesOptions.DEFAULT_MAX_MUTATION_SIZE);
-        Format dateTimeFormat = DateUtil.getDateFormatter(datePattern);
-        formatters.put(PDate.INSTANCE, dateTimeFormat);
-        formatters.put(PTime.INSTANCE, dateTimeFormat);
-        formatters.put(PTimestamp.INSTANCE, dateTimeFormat);
-        formatters.put(PUnsignedDate.INSTANCE, dateTimeFormat);
-        formatters.put(PUnsignedTime.INSTANCE, dateTimeFormat);
-        formatters.put(PUnsignedTimestamp.INSTANCE, dateTimeFormat);
+        Format dateFormat = DateUtil.getDateFormatter(datePattern);
+        Format timeFormat = DateUtil.getDateFormatter(timePattern);
+        Format timestampFormat = DateUtil.getDateFormatter(timestampPattern);
+        formatters.put(PDate.INSTANCE, dateFormat);
+        formatters.put(PTime.INSTANCE, timeFormat);
+        formatters.put(PTimestamp.INSTANCE, timestampFormat);
+        formatters.put(PUnsignedDate.INSTANCE, dateFormat);
+        formatters.put(PUnsignedTime.INSTANCE, timeFormat);
+        formatters.put(PUnsignedTimestamp.INSTANCE, timestampFormat);
         formatters.put(PDecimal.INSTANCE, FunctionArgumentType.NUMERIC.getFormatter(numberPattern));
         // We do not limit the metaData on a connection less than the global one,
         // as there's not much that will be cached here.

http://git-wip-us.apache.org/repos/asf/phoenix/blob/fb7aa1a5/phoenix-core/src/main/java/org/apache/phoenix/parse/ToDateParseNode.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/ToDateParseNode.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/ToDateParseNode.java
index 6140dbc..fd4d16a 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/parse/ToDateParseNode.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/ToDateParseNode.java
@@ -18,16 +18,13 @@
 package org.apache.phoenix.parse;
 
 import java.sql.SQLException;
-import java.text.Format;
 import java.util.List;
-import java.util.TimeZone;
 
 import org.apache.phoenix.compile.StatementContext;
 import org.apache.phoenix.expression.Expression;
 import org.apache.phoenix.expression.LiteralExpression;
 import org.apache.phoenix.expression.function.FunctionExpression;
 import org.apache.phoenix.expression.function.ToDateFunction;
-import org.apache.phoenix.util.DateUtil;
 
 
 public class ToDateParseNode extends FunctionParseNode {
@@ -46,7 +43,6 @@ public class ToDateParseNode extends FunctionParseNode {
         if (timeZoneId == null) {
             timeZoneId = context.getDateFormatTimeZone().getID();
         }
-        DateUtil.DateTimeParser dateParser = DateUtil.getDateParser(dateFormat, timeZoneId);
-        return new ToDateFunction(children, dateFormat, dateParser);
+        return new ToDateFunction(children, dateFormat, timeZoneId);
     }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/fb7aa1a5/phoenix-core/src/main/java/org/apache/phoenix/parse/ToTimeParseNode.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/ToTimeParseNode.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/ToTimeParseNode.java
new file mode 100644
index 0000000..5f0a72d
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/ToTimeParseNode.java
@@ -0,0 +1,48 @@
+/*
+ * 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.sql.SQLException;
+import java.util.List;
+
+import org.apache.phoenix.compile.StatementContext;
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.expression.LiteralExpression;
+import org.apache.phoenix.expression.function.FunctionExpression;
+import org.apache.phoenix.expression.function.ToTimeFunction;
+
+
+public class ToTimeParseNode extends FunctionParseNode { 
+
+    public ToTimeParseNode(String name, List<ParseNode> children, BuiltInFunctionInfo info) {
+        super(name, children, info);
+    }
+
+    @Override
+    public FunctionExpression create(List<Expression> children, StatementContext context) throws SQLException {
+        String dateFormat = (String) ((LiteralExpression) children.get(1)).getValue();
+        String timeZoneId = (String) ((LiteralExpression) children.get(2)).getValue();
+        if (dateFormat == null) {
+            dateFormat = context.getTimeFormat();
+        }
+        if (timeZoneId == null) {
+            timeZoneId = context.getDateFormatTimeZone().getID();
+        }
+        return new ToTimeFunction(children, dateFormat, timeZoneId);
+    }
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/fb7aa1a5/phoenix-core/src/main/java/org/apache/phoenix/parse/ToTimestampParseNode.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/ToTimestampParseNode.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/ToTimestampParseNode.java
new file mode 100644
index 0000000..2a3f5ec
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/ToTimestampParseNode.java
@@ -0,0 +1,48 @@
+/*
+ * 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.sql.SQLException;
+import java.util.List;
+
+import org.apache.phoenix.compile.StatementContext;
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.expression.LiteralExpression;
+import org.apache.phoenix.expression.function.FunctionExpression;
+import org.apache.phoenix.expression.function.ToTimestampFunction;
+
+
+public class ToTimestampParseNode extends FunctionParseNode { 
+
+    public ToTimestampParseNode(String name, List<ParseNode> children, BuiltInFunctionInfo info) {
+        super(name, children, info);
+    }
+
+    @Override
+    public FunctionExpression create(List<Expression> children, StatementContext context) throws SQLException {
+        String dateFormat = (String) ((LiteralExpression) children.get(1)).getValue();
+        String timeZoneId = (String) ((LiteralExpression) children.get(2)).getValue();
+        if (dateFormat == null) {
+            dateFormat = context.getTimestampFormat();
+        }
+        if (timeZoneId == null) {
+            timeZoneId = context.getDateFormatTimeZone().getID();
+        }
+        return new ToTimestampFunction(children, dateFormat, timeZoneId);
+    }
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/fb7aa1a5/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServices.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServices.java b/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServices.java
index d21695d..e20d5ee 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServices.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServices.java
@@ -71,6 +71,9 @@ public interface QueryServices extends SQLCloseable {
     public static final String MAX_SERVER_CACHE_SIZE_ATTRIB = "phoenix.query.maxServerCacheBytes";
     public static final String DATE_FORMAT_TIMEZONE_ATTRIB = "phoenix.query.dateFormatTimeZone";
     public static final String DATE_FORMAT_ATTRIB = "phoenix.query.dateFormat";
+    public static final String TIME_FORMAT_ATTRIB = "phoenix.query.timeFormat";
+    public static final String TIMESTAMP_FORMAT_ATTRIB = "phoenix.query.timestampFormat";
+
     public static final String NUMBER_FORMAT_ATTRIB = "phoenix.query.numberFormat";
     public static final String CALL_QUEUE_ROUND_ROBIN_ATTRIB = "ipc.server.callqueue.roundrobin";
     public static final String SCAN_CACHE_SIZE_ATTRIB = "hbase.client.scanner.caching";

http://git-wip-us.apache.org/repos/asf/phoenix/blob/fb7aa1a5/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PDate.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PDate.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PDate.java
index 13a828f..bbd0a35 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PDate.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PDate.java
@@ -17,16 +17,16 @@
  */
 package org.apache.phoenix.schema.types;
 
-import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.phoenix.schema.SortOrder;
-import org.apache.phoenix.util.DateUtil;
-
 import java.math.BigDecimal;
 import java.sql.Date;
 import java.sql.Types;
 import java.text.Format;
 
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.phoenix.schema.SortOrder;
+import org.apache.phoenix.util.DateUtil;
+
 public class PDate extends PDataType<Date> {
 
   public static final PDate INSTANCE = new PDate();
@@ -71,7 +71,7 @@ public class PDate extends PDataType<Date> {
     } else if (actualType == PDecimal.INSTANCE) {
       return new Date(((BigDecimal) object).longValueExact());
     } else if (actualType == PVarchar.INSTANCE) {
-      return DateUtil.parseDateTime((String) object);
+      return DateUtil.parseDate((String) object);
     }
     return throwConstraintViolationException(actualType, this);
   }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/fb7aa1a5/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PTime.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PTime.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PTime.java
index d824885..81cbaff 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PTime.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PTime.java
@@ -17,15 +17,15 @@
  */
 package org.apache.phoenix.schema.types;
 
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.phoenix.schema.SortOrder;
-import org.apache.phoenix.util.DateUtil;
-
 import java.math.BigDecimal;
 import java.sql.Time;
 import java.sql.Types;
 import java.text.Format;
 
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.phoenix.schema.SortOrder;
+import org.apache.phoenix.util.DateUtil;
+
 public class PTime extends PDataType<Time> {
 
   public static final PTime INSTANCE = new PTime();
@@ -78,7 +78,7 @@ public class PTime extends PDataType<Time> {
     } else if (actualType == PDecimal.INSTANCE) {
       return new java.sql.Time(((BigDecimal) object).longValueExact());
     } else if (actualType == PVarchar.INSTANCE) {
-      return DateUtil.parseDateTime((String) object);
+      return DateUtil.parseTime((String) object);
     }
     return throwConstraintViolationException(actualType, this);
   }
@@ -128,8 +128,10 @@ public class PTime extends PDataType<Time> {
 
   @Override
   public String toStringLiteral(byte[] b, int offset, int length, Format formatter) {
-    // TODO: different default formatter for TIME?
-    return PDate.INSTANCE.toStringLiteral(b, offset, length, formatter);
+      if (formatter == null) {
+          formatter = DateUtil.DEFAULT_TIME_FORMATTER;
+        }
+        return "'" + super.toStringLiteral(b, offset, length, formatter) + "'";
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/phoenix/blob/fb7aa1a5/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PTimestamp.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PTimestamp.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PTimestamp.java
index 4bdcb86..8182e33 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PTimestamp.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PTimestamp.java
@@ -17,17 +17,17 @@
  */
 package org.apache.phoenix.schema.types;
 
+import java.math.BigDecimal;
+import java.sql.Timestamp;
+import java.sql.Types;
+import java.text.Format;
+
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.phoenix.query.QueryConstants;
 import org.apache.phoenix.schema.SortOrder;
 import org.apache.phoenix.util.DateUtil;
 
-import java.math.BigDecimal;
-import java.sql.Timestamp;
-import java.sql.Types;
-import java.text.Format;
-
 public class PTimestamp extends PDataType<Timestamp> {
 
   public static final PTimestamp INSTANCE = new PTimestamp();
@@ -84,7 +84,7 @@ public class PTimestamp extends PDataType<Timestamp> {
               .intValue();
       return DateUtil.getTimestamp(ms, nanos);
     } else if (actualType == PVarchar.INSTANCE) {
-      return new Timestamp(DateUtil.parseDateTime((String) object).getTime());
+      return DateUtil.parseTimestamp((String) object);
     }
     return throwConstraintViolationException(actualType, this);
   }
@@ -181,15 +181,13 @@ public class PTimestamp extends PDataType<Timestamp> {
 
   @Override
   public String toStringLiteral(byte[] b, int offset, int length, Format formatter) {
-    java.sql.Timestamp value = (java.sql.Timestamp) toObject(b, offset, length);
-    if (formatter == null || formatter == DateUtil.DEFAULT_DATE_FORMATTER) {
-      // If default formatter has not been overridden,
-      // use one that displays milliseconds.
-      formatter = DateUtil.DEFAULT_MS_DATE_FORMATTER;
-    }
-    return "'" + super.toStringLiteral(b, offset, length, formatter) + "." + value.getNanos() + "'";
+      if (formatter == null) {
+          formatter = DateUtil.DEFAULT_TIMESTAMP_FORMATTER;
+        }
+        return "'" + super.toStringLiteral(b, offset, length, formatter) + "'";
   }
 
+
   @Override
   public int getNanos(ImmutableBytesWritable ptr, SortOrder sortOrder) {
     int nanos = PUnsignedInt.INSTANCE.getCodec()

http://git-wip-us.apache.org/repos/asf/phoenix/blob/fb7aa1a5/phoenix-core/src/main/java/org/apache/phoenix/util/DateUtil.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/util/DateUtil.java b/phoenix-core/src/main/java/org/apache/phoenix/util/DateUtil.java
index fbc74ba..0f4b54a 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/util/DateUtil.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/DateUtil.java
@@ -21,85 +21,122 @@ import java.math.BigDecimal;
 import java.sql.Date;
 import java.sql.Time;
 import java.sql.Timestamp;
+import java.sql.Types;
 import java.text.Format;
 import java.text.ParseException;
 import java.text.SimpleDateFormat;
+import java.util.List;
 import java.util.TimeZone;
 
 import org.apache.commons.lang.time.FastDateFormat;
 import org.apache.phoenix.query.QueryConstants;
 import org.apache.phoenix.schema.IllegalDataException;
+import org.apache.phoenix.schema.types.PDataType;
+import org.joda.time.DateTimeZone;
 import org.joda.time.chrono.ISOChronology;
 import org.joda.time.format.DateTimeFormatter;
 import org.joda.time.format.DateTimeFormatterBuilder;
 import org.joda.time.format.ISODateTimeFormat;
 
+import com.google.common.collect.Lists;
 
-@SuppressWarnings("serial")
+
+@SuppressWarnings({ "serial", "deprecation" })
 public class DateUtil {
     public static final String DEFAULT_TIME_ZONE_ID = "GMT";
+    public static final String LOCAL_TIME_ZONE_ID = "LOCAL";
     private static final TimeZone DEFAULT_TIME_ZONE = TimeZone.getTimeZone(DEFAULT_TIME_ZONE_ID);
-    public static final String DEFAULT_DATE_FORMAT = "yyyy-MM-dd HH:mm:ss"; // This is the format the app sets in NLS settings for every connection.
-    public static final Format DEFAULT_DATE_FORMATTER = FastDateFormat.getInstance(
-            DEFAULT_DATE_FORMAT, TimeZone.getTimeZone(DEFAULT_TIME_ZONE_ID));
-
+    
     public static final String DEFAULT_MS_DATE_FORMAT = "yyyy-MM-dd HH:mm:ss.SSS";
     public static final Format DEFAULT_MS_DATE_FORMATTER = FastDateFormat.getInstance(
             DEFAULT_MS_DATE_FORMAT, TimeZone.getTimeZone(DEFAULT_TIME_ZONE_ID));
 
-    private static final DateTimeFormatter ISO_DATE_TIME_PARSER = new DateTimeFormatterBuilder()
-            .append(ISODateTimeFormat.dateParser())
-            .appendOptional(new DateTimeFormatterBuilder()
-                    .appendLiteral(' ').toParser())
-            .appendOptional(new DateTimeFormatterBuilder()
-                    .append(ISODateTimeFormat.timeParser()).toParser())
-            .toFormatter()
-            .withZoneUTC()
-            .withChronology(ISOChronology.getInstanceUTC());
+    public static final String DEFAULT_DATE_FORMAT = DEFAULT_MS_DATE_FORMAT;
+    public static final Format DEFAULT_DATE_FORMATTER = DEFAULT_MS_DATE_FORMATTER;
 
-    private DateUtil() {
-    }
+    public static final String DEFAULT_TIME_FORMAT = DEFAULT_MS_DATE_FORMAT;
+    public static final Format DEFAULT_TIME_FORMATTER = DEFAULT_MS_DATE_FORMATTER;
 
-    public static DateTimeParser getDateParser(String pattern, TimeZone timeZone) {
-        if(DateUtil.DEFAULT_DATE_FORMAT.equals(pattern) &&
-                timeZone.getID().equalsIgnoreCase(DateUtil.DEFAULT_TIME_ZONE_ID)) {
-            return ISODateFormatParser.getInstance();
-        } else {
-            return new SimpleDateFormatParser(pattern, timeZone);
-        }
+    public static final String DEFAULT_TIMESTAMP_FORMAT = DEFAULT_MS_DATE_FORMAT;
+    public static final Format DEFAULT_TIMESTAMP_FORMATTER = DEFAULT_MS_DATE_FORMATTER;
+
+    private static final DateTimeFormatter ISO_DATE_TIME_FORMATTER = new DateTimeFormatterBuilder()
+        .append(ISODateTimeFormat.dateParser())
+        .appendOptional(new DateTimeFormatterBuilder()
+                .appendLiteral(' ').toParser())
+        .appendOptional(new DateTimeFormatterBuilder()
+                .append(ISODateTimeFormat.timeParser()).toParser())
+        .toFormatter().withChronology(ISOChronology.getInstanceUTC());
+    
+    private DateUtil() {
     }
 
-    public static DateTimeParser getDateParser(String pattern, String timeZoneId) {
-        if(timeZoneId == null) {
-            timeZoneId = DateUtil.DEFAULT_TIME_ZONE_ID;
-        }
+    private static TimeZone getTimeZone(String timeZoneId) {
         TimeZone parserTimeZone;
-        if ("LOCAL".equalsIgnoreCase(timeZoneId)) {
+        if (timeZoneId == null) {
+            parserTimeZone = DateUtil.DEFAULT_TIME_ZONE;
+        } else if (LOCAL_TIME_ZONE_ID.equalsIgnoreCase(timeZoneId)) {
             parserTimeZone = TimeZone.getDefault();
         } else {
             parserTimeZone = TimeZone.getTimeZone(timeZoneId);
         }
-        return getDateParser(pattern, parserTimeZone);
+        return parserTimeZone;
     }
-
-    public static DateTimeParser getDateParser(String pattern) {
-        return getDateParser(pattern, DEFAULT_TIME_ZONE);
-    }
-
-    public static DateTimeParser getTimeParser(String pattern, TimeZone timeZone) {
-        return getDateParser(pattern, timeZone);
+    
+    private static String[] defaultPattern;
+    static {
+        int maxOrdinal = Integer.MIN_VALUE;
+        List<PDataType> timeDataTypes = Lists.newArrayListWithExpectedSize(6);
+        for (PDataType type : PDataType.values()) {
+            if (java.util.Date.class.isAssignableFrom(type.getJavaClass())) {
+                timeDataTypes.add(type);
+                if (type.ordinal() > maxOrdinal) {
+                    maxOrdinal = type.ordinal();
+                }
+            }
+        }
+        defaultPattern = new String[maxOrdinal+1];
+        for (PDataType type : timeDataTypes) {
+            switch (type.getResultSetSqlType()) {
+            case Types.TIMESTAMP:
+                defaultPattern[type.ordinal()] = DateUtil.DEFAULT_TIMESTAMP_FORMAT;
+                break;
+            case Types.TIME:
+                defaultPattern[type.ordinal()] = DateUtil.DEFAULT_TIME_FORMAT;
+                break;
+            case Types.DATE:
+                defaultPattern[type.ordinal()] = DateUtil.DEFAULT_DATE_FORMAT;
+                break;
+            }
+        }
     }
-
-    public static DateTimeParser getTimeParser(String pattern) {
-        return getTimeParser(pattern, DEFAULT_TIME_ZONE);
+    
+    private static String getDefaultFormat(PDataType type) {
+        int ordinal = type.ordinal();
+        if (ordinal >= 0 || ordinal < defaultPattern.length) {
+            String format = defaultPattern[ordinal];
+            if (format != null) {
+                return format;
+            }
+        }
+        throw new IllegalArgumentException("Expected a date/time type, but got " + type);
     }
 
-    public static DateTimeParser getTimestampParser(String pattern, TimeZone timeZone) {
-        return getDateParser(pattern, timeZone);
+    public static DateTimeParser getDateTimeParser(String pattern, PDataType pDataType, String timeZoneId) {
+        TimeZone timeZone = getTimeZone(timeZoneId);
+        String defaultPattern = getDefaultFormat(pDataType);
+        if (pattern == null || pattern.length() == 0) {
+            pattern = defaultPattern;
+        }
+        if(defaultPattern.equals(pattern)) {
+            return ISODateFormatParserFactory.getParser(timeZone);
+        } else {
+            return new SimpleDateFormatParser(pattern, timeZone);
+        }
     }
 
-    public static DateTimeParser getTimestampParser(String pattern) {
-        return getTimestampParser(pattern, DEFAULT_TIME_ZONE);
+    public static DateTimeParser getDateTimeParser(String pattern, PDataType pDataType) {
+        return getDateTimeParser(pattern, pDataType, null);
     }
 
     public static Format getDateFormatter(String pattern) {
@@ -108,20 +145,32 @@ public class DateUtil {
                 : FastDateFormat.getInstance(pattern, DateUtil.DEFAULT_TIME_ZONE);
     }
 
-    public static Date parseDateTime(String dateTimeValue) {
+    public static Format getTimeFormatter(String pattern) {
+        return DateUtil.DEFAULT_TIME_FORMAT.equals(pattern)
+                ? DateUtil.DEFAULT_TIME_FORMATTER
+                : FastDateFormat.getInstance(pattern, DateUtil.DEFAULT_TIME_ZONE);
+    }
+
+    public static Format getTimestampFormatter(String pattern) {
+        return DateUtil.DEFAULT_TIMESTAMP_FORMAT.equals(pattern)
+                ? DateUtil.DEFAULT_TIMESTAMP_FORMATTER
+                : FastDateFormat.getInstance(pattern, DateUtil.DEFAULT_TIME_ZONE);
+    }
+
+    private static long parseDateTime(String dateTimeValue) {
         return ISODateFormatParser.getInstance().parseDateTime(dateTimeValue);
     }
 
     public static Date parseDate(String dateValue) {
-        return parseDateTime(dateValue);
+        return new Date(parseDateTime(dateValue));
     }
 
     public static Time parseTime(String timeValue) {
-        return new Time(parseDateTime(timeValue).getTime());
+        return new Time(parseDateTime(timeValue));
     }
 
     public static Timestamp parseTimestamp(String timestampValue) {
-        return new Timestamp(parseDateTime(timestampValue).getTime());
+        return new Timestamp(parseDateTime(timestampValue));
     }
 
     /**
@@ -145,7 +194,8 @@ public class DateUtil {
     }
 
     public static interface DateTimeParser {
-        public Date parseDateTime(String dateTimeString) throws IllegalDataException;
+        public long parseDateTime(String dateTimeString) throws IllegalDataException;
+        public TimeZone getTimeZone();
     }
 
     /**
@@ -168,41 +218,76 @@ public class DateUtil {
         }
 
         @Override
-        public Date parseDateTime(String dateTimeString) throws IllegalDataException {
+        public long parseDateTime(String dateTimeString) throws IllegalDataException {
             try {
                 java.util.Date date =parser.parse(dateTimeString);
-                return new java.sql.Date(date.getTime());
+                return date.getTime();
             } catch (ParseException e) {
-                throw new IllegalDataException("to_date('" + dateTimeString + "') did not match expected date format of '" + datePattern + "'.");
+                throw new IllegalDataException("Unable to parse date/time '" + dateTimeString + "' using format string of '" + datePattern + "'.");
             }
         }
+
+        @Override
+        public TimeZone getTimeZone() {
+            return parser.getTimeZone();
+        }
     }
 
+    private static class ISODateFormatParserFactory {
+        private ISODateFormatParserFactory() {}
+        
+        public static DateTimeParser getParser(final TimeZone timeZone) {
+            // If timeZone matches default, get singleton DateTimeParser
+            if (timeZone.equals(DEFAULT_TIME_ZONE)) {
+                return ISODateFormatParser.getInstance();
+            }
+            // Otherwise, create new DateTimeParser
+            return new DateTimeParser() {
+                private final DateTimeFormatter formatter = ISO_DATE_TIME_FORMATTER
+                        .withZone(DateTimeZone.forTimeZone(timeZone));
+
+                @Override
+                public long parseDateTime(String dateTimeString) throws IllegalDataException {
+                    try {
+                        return formatter.parseDateTime(dateTimeString).getMillis();
+                    } catch(IllegalArgumentException ex) {
+                        throw new IllegalDataException(ex);
+                    }
+                }
+
+                @Override
+                public TimeZone getTimeZone() {
+                    return timeZone;
+                }
+            };
+        }
+    }
     /**
      * This class is our default DateTime string parser
      */
     private static class ISODateFormatParser implements DateTimeParser {
-        private static ISODateFormatParser inst = null;
-        private static Object lock = new Object();
-        private ISODateFormatParser() {}
+        private static final ISODateFormatParser INSTANCE = new ISODateFormatParser();
 
         public static ISODateFormatParser getInstance() {
-            if(inst != null) return inst;
-
-            synchronized (lock) {
-                if (inst == null) {
-                    inst = new ISODateFormatParser();
-                }
-            }
-            return inst;
+            return INSTANCE;
         }
 
-        public Date parseDateTime(String dateTimeString) throws IllegalDataException {
+        private final DateTimeFormatter formatter = ISO_DATE_TIME_FORMATTER.withZoneUTC();
+
+        private ISODateFormatParser() {}
+
+        @Override
+        public long parseDateTime(String dateTimeString) throws IllegalDataException {
             try {
-                return new Date(ISO_DATE_TIME_PARSER.parseDateTime(dateTimeString).getMillis());
+                return formatter.parseDateTime(dateTimeString).getMillis();
             } catch(IllegalArgumentException ex) {
                 throw new IllegalDataException(ex);
             }
         }
+
+        @Override
+        public TimeZone getTimeZone() {
+            return formatter.getZone().toTimeZone();
+        }
     }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/fb7aa1a5/phoenix-core/src/main/java/org/apache/phoenix/util/csv/CsvUpsertExecutor.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/util/csv/CsvUpsertExecutor.java b/phoenix-core/src/main/java/org/apache/phoenix/util/csv/CsvUpsertExecutor.java
index 731a13f..b5f6f9f 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/util/csv/CsvUpsertExecutor.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/csv/CsvUpsertExecutor.java
@@ -17,16 +17,22 @@
  */
 package org.apache.phoenix.util.csv;
 
-import com.google.common.base.Function;
-import com.google.common.collect.Lists;
+import java.io.Closeable;
+import java.io.IOException;
+import java.sql.Connection;
+import java.sql.PreparedStatement;
+import java.sql.SQLException;
+import java.sql.Types;
+import java.util.List;
+import java.util.Properties;
+
+import javax.annotation.Nullable;
+
 import org.apache.commons.csv.CSVRecord;
-import org.apache.phoenix.expression.LiteralExpression;
 import org.apache.phoenix.jdbc.PhoenixConnection;
 import org.apache.phoenix.query.QueryServices;
 import org.apache.phoenix.query.QueryServicesOptions;
 import org.apache.phoenix.schema.types.PDataType;
-import org.apache.phoenix.schema.types.PDate;
-import org.apache.phoenix.schema.types.PTime;
 import org.apache.phoenix.schema.types.PTimestamp;
 import org.apache.phoenix.util.ColumnInfo;
 import org.apache.phoenix.util.DateUtil;
@@ -34,16 +40,8 @@ import org.apache.phoenix.util.QueryUtil;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import javax.annotation.Nullable;
-import java.io.Closeable;
-import java.io.IOException;
-import java.sql.Connection;
-import java.sql.Date;
-import java.sql.PreparedStatement;
-import java.sql.SQLException;
-import java.util.List;
-import java.util.Properties;
-import java.util.TimeZone;
+import com.google.common.base.Function;
+import com.google.common.collect.Lists;
 
 /**
  * Executes upsert statements on a provided {@code PreparedStatement} based on incoming CSV records, notifying a
@@ -205,12 +203,23 @@ public class CsvUpsertExecutor implements Closeable {
                 throw new RuntimeException(e);
             }
             this.dataType = dataType;
-            if(dataType.equals(PDate.INSTANCE) || dataType.equals(PTime.INSTANCE) || dataType.equals(PTimestamp.INSTANCE)) {
-                String dateFormat = props.getProperty(QueryServices.DATE_FORMAT_ATTRIB,
-                        QueryServicesOptions.DEFAULT_DATE_FORMAT);
+            if(dataType.isCoercibleTo(PTimestamp.INSTANCE)) {
+                // TODO: move to DateUtil
+                String dateFormat;
+                int dateSqlType = dataType.getResultSetSqlType();
+                if (dateSqlType == Types.DATE) {
+                    dateFormat = props.getProperty(QueryServices.DATE_FORMAT_ATTRIB,
+                            DateUtil.DEFAULT_DATE_FORMAT);
+                } else if (dateSqlType == Types.TIME) {
+                    dateFormat = props.getProperty(QueryServices.TIME_FORMAT_ATTRIB,
+                            DateUtil.DEFAULT_TIME_FORMAT);
+                } else {
+                    dateFormat = props.getProperty(QueryServices.TIMESTAMP_FORMAT_ATTRIB,
+                            DateUtil.DEFAULT_TIMESTAMP_FORMAT);                    
+                }
                 String timeZoneId = props.getProperty(QueryServices.DATE_FORMAT_TIMEZONE_ATTRIB,
                         QueryServicesOptions.DEFAULT_DATE_FORMAT_TIMEZONE);
-                this.dateTimeParser = DateUtil.getDateParser(dateFormat, timeZoneId);
+                this.dateTimeParser = DateUtil.getDateTimeParser(dateFormat, dataType, timeZoneId);
             } else {
                 this.dateTimeParser = null;
             }
@@ -220,7 +229,10 @@ public class CsvUpsertExecutor implements Closeable {
         @Override
         public Object apply(@Nullable String input) {
             if(dateTimeParser != null) {
-                return dateTimeParser.parseDateTime(input);
+                long epochTime = dateTimeParser.parseDateTime(input);
+                byte[] byteValue = new byte[dataType.getByteSize()];
+                dataType.getCodec().encodeLong(epochTime, byteValue, 0);
+                return dataType.toObject(byteValue);
             }
             return dataType.toObject(input);
         }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/fb7aa1a5/phoenix-core/src/test/java/org/apache/phoenix/compile/WhereCompilerTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/compile/WhereCompilerTest.java b/phoenix-core/src/test/java/org/apache/phoenix/compile/WhereCompilerTest.java
index b064f07..ff4ad8b 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/compile/WhereCompilerTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/compile/WhereCompilerTest.java
@@ -37,9 +37,9 @@ import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
 import java.math.BigDecimal;
+import java.sql.Date;
 import java.sql.DriverManager;
 import java.sql.SQLException;
-import java.text.Format;
 import java.util.Arrays;
 import java.util.Collections;
 import java.util.List;
@@ -60,11 +60,11 @@ import org.apache.phoenix.query.BaseConnectionlessQueryTest;
 import org.apache.phoenix.query.KeyRange;
 import org.apache.phoenix.query.QueryConstants;
 import org.apache.phoenix.query.QueryServicesOptions;
+import org.apache.phoenix.schema.RowKeyValueAccessor;
+import org.apache.phoenix.schema.SaltingUtil;
 import org.apache.phoenix.schema.types.PChar;
 import org.apache.phoenix.schema.types.PLong;
 import org.apache.phoenix.schema.types.PVarchar;
-import org.apache.phoenix.schema.RowKeyValueAccessor;
-import org.apache.phoenix.schema.SaltingUtil;
 import org.apache.phoenix.util.ByteUtil;
 import org.apache.phoenix.util.DateUtil;
 import org.apache.phoenix.util.NumberUtil;
@@ -277,7 +277,7 @@ public class WhereCompilerTest extends BaseConnectionlessQueryTest {
         Scan scan = plan.getContext().getScan();
         Filter filter = scan.getFilter();
 
-        Object date = DateUtil.parseDateTime(dateStr);
+        Date date = DateUtil.parseDate(dateStr);
 
         assertEquals(
             singleKVFilter(constantComparison(

http://git-wip-us.apache.org/repos/asf/phoenix/blob/fb7aa1a5/phoenix-core/src/test/java/org/apache/phoenix/expression/SortOrderExpressionTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/expression/SortOrderExpressionTest.java b/phoenix-core/src/test/java/org/apache/phoenix/expression/SortOrderExpressionTest.java
index f75bb3e..8fb1a6c 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/expression/SortOrderExpressionTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/expression/SortOrderExpressionTest.java
@@ -30,22 +30,6 @@ import java.util.TimeZone;
 
 import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
-import org.apache.phoenix.schema.types.PChar;
-import org.apache.phoenix.schema.types.PDecimal;
-import org.apache.phoenix.schema.types.PBoolean;
-import org.apache.phoenix.schema.types.PDate;
-import org.apache.phoenix.schema.types.PDouble;
-import org.apache.phoenix.schema.types.PFloat;
-import org.apache.phoenix.schema.types.PInteger;
-import org.apache.phoenix.schema.types.PLong;
-import org.apache.phoenix.schema.types.PUnsignedDouble;
-import org.apache.phoenix.schema.types.PUnsignedFloat;
-import org.apache.phoenix.schema.types.PUnsignedInt;
-import org.apache.phoenix.schema.types.PUnsignedLong;
-import org.apache.phoenix.schema.types.PVarchar;
-import org.junit.Test;
-
-import com.google.common.collect.Lists;
 import org.apache.phoenix.expression.function.FunctionArgumentType;
 import org.apache.phoenix.expression.function.LTrimFunction;
 import org.apache.phoenix.expression.function.LengthFunction;
@@ -63,8 +47,24 @@ import org.apache.phoenix.expression.function.ToNumberFunction;
 import org.apache.phoenix.expression.function.TrimFunction;
 import org.apache.phoenix.expression.function.UpperFunction;
 import org.apache.phoenix.schema.SortOrder;
+import org.apache.phoenix.schema.types.PBoolean;
+import org.apache.phoenix.schema.types.PChar;
 import org.apache.phoenix.schema.types.PDataType;
+import org.apache.phoenix.schema.types.PDate;
+import org.apache.phoenix.schema.types.PDecimal;
+import org.apache.phoenix.schema.types.PDouble;
+import org.apache.phoenix.schema.types.PFloat;
+import org.apache.phoenix.schema.types.PInteger;
+import org.apache.phoenix.schema.types.PLong;
+import org.apache.phoenix.schema.types.PUnsignedDouble;
+import org.apache.phoenix.schema.types.PUnsignedFloat;
+import org.apache.phoenix.schema.types.PUnsignedInt;
+import org.apache.phoenix.schema.types.PUnsignedLong;
+import org.apache.phoenix.schema.types.PVarchar;
 import org.apache.phoenix.util.DateUtil;
+import org.junit.Test;
+
+import com.google.common.collect.Lists;
 
 /**
  * @since 1.2
@@ -148,7 +148,7 @@ public class SortOrderExpressionTest {
     @Test
     public void toDate() throws Exception {
         List<Expression> args = Lists.newArrayList(getInvertedLiteral("2001-11-30 00:00:00:0", PVarchar.INSTANCE));
-        evaluateAndAssertResult(new ToDateFunction(args, null, DateUtil.getDateParser("yyyy-MM-dd HH:mm:ss:S")), date(11, 30, 2001));
+        evaluateAndAssertResult(new ToDateFunction(args, "yyyy-MM-dd HH:mm:ss:S",DateUtil.DEFAULT_TIME_ZONE_ID), date(11, 30, 2001));
     }
     
     @Test

http://git-wip-us.apache.org/repos/asf/phoenix/blob/fb7aa1a5/phoenix-core/src/test/java/org/apache/phoenix/util/DateUtilTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/util/DateUtilTest.java b/phoenix-core/src/test/java/org/apache/phoenix/util/DateUtilTest.java
index 702e556..ec0bc01 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/util/DateUtilTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/util/DateUtilTest.java
@@ -28,6 +28,9 @@ import java.text.ParseException;
 import java.util.TimeZone;
 
 import org.apache.phoenix.schema.IllegalDataException;
+import org.apache.phoenix.schema.types.PDate;
+import org.apache.phoenix.schema.types.PTime;
+import org.apache.phoenix.schema.types.PTimestamp;
 import org.junit.Test;
 
 /**
@@ -68,64 +71,64 @@ public class DateUtilTest {
 
     @Test
     public void testGetDateParser_DefaultTimeZone() throws ParseException {
-        Date date = DateUtil.getDateParser("yyyy-MM-dd").parseDateTime("1970-01-01");
+        Date date = new Date(DateUtil.getDateTimeParser("yyyy-MM-dd", PDate.INSTANCE).parseDateTime("1970-01-01"));
         assertEquals(0, date.getTime());
     }
 
     @Test
     public void testGetDateParser_CustomTimeZone() throws ParseException {
-        Date date = DateUtil.getDateParser(
-                "yyyy-MM-dd", TimeZone.getTimeZone("GMT+1")).parseDateTime("1970-01-01");
+        Date date = new Date(DateUtil.getDateTimeParser(
+                "yyyy-MM-dd", PDate.INSTANCE, TimeZone.getTimeZone("GMT+1").getID()).parseDateTime("1970-01-01"));
         assertEquals(-ONE_HOUR_IN_MILLIS, date.getTime());
     }
 
     @Test
     public void testGetDateParser_LocalTimeZone() throws ParseException {
-        Date date = DateUtil.getDateParser(
-                "yyyy-MM-dd", TimeZone.getDefault()).parseDateTime("1970-01-01");
+        Date date = new Date(DateUtil.getDateTimeParser(
+                "yyyy-MM-dd", PDate.INSTANCE, TimeZone.getDefault().getID()).parseDateTime("1970-01-01"));
         assertEquals(Date.valueOf("1970-01-01"), date);
     }
 
     @Test
     public void testGetTimestampParser_DefaultTimeZone() throws ParseException {
-        Timestamp ts = new Timestamp(DateUtil.getTimestampParser("yyyy-MM-dd HH:mm:ss")
-                .parseDateTime("1970-01-01 00:00:00").getTime());
+        Timestamp ts = new Timestamp(DateUtil.getDateTimeParser("yyyy-MM-dd HH:mm:ss", PTimestamp.INSTANCE)
+                .parseDateTime("1970-01-01 00:00:00"));
         assertEquals(0, ts.getTime());
     }
 
     @Test
     public void testGetTimestampParser_CustomTimeZone() throws ParseException {
-        Timestamp ts = new Timestamp(DateUtil.getTimestampParser("yyyy-MM-dd HH:mm:ss", TimeZone.getTimeZone("GMT+1"))
-                .parseDateTime("1970-01-01 00:00:00").getTime());
+        Timestamp ts = new Timestamp(DateUtil.getDateTimeParser("yyyy-MM-dd HH:mm:ss", PTimestamp.INSTANCE, TimeZone.getTimeZone("GMT+1").getID())
+                .parseDateTime("1970-01-01 00:00:00"));
         assertEquals(-ONE_HOUR_IN_MILLIS, ts.getTime());
     }
 
     @Test
     public void testGetTimestampParser_LocalTimeZone() throws ParseException {
-        Timestamp ts = new Timestamp(DateUtil.getTimestampParser(
+        Timestamp ts = new Timestamp(DateUtil.getDateTimeParser(
                 "yyyy-MM-dd HH:mm:ss",
-                TimeZone.getDefault()).parseDateTime("1970-01-01 00:00:00").getTime());
+                PTimestamp.INSTANCE, TimeZone.getDefault().getID()).parseDateTime("1970-01-01 00:00:00"));
         assertEquals(Timestamp.valueOf("1970-01-01 00:00:00"), ts);
     }
 
     @Test
     public void testGetTimeParser_DefaultTimeZone() throws ParseException {
-        Time time = new Time(DateUtil.getTimeParser("HH:mm:ss").parseDateTime("00:00:00").getTime());
+        Time time = new Time(DateUtil.getDateTimeParser("HH:mm:ss", PTime.INSTANCE).parseDateTime("00:00:00"));
         assertEquals(0, time.getTime());
     }
 
     @Test
     public void testGetTimeParser_CustomTimeZone() throws ParseException {
-        Time time = new Time(DateUtil.getTimeParser(
+        Time time = new Time(DateUtil.getDateTimeParser(
                 "HH:mm:ss",
-                TimeZone.getTimeZone("GMT+1")).parseDateTime("00:00:00").getTime());
+                PTime.INSTANCE, TimeZone.getTimeZone("GMT+1").getID()).parseDateTime("00:00:00"));
         assertEquals(-ONE_HOUR_IN_MILLIS, time.getTime());
     }
 
     @Test
     public void testGetTimeParser_LocalTimeZone() throws ParseException {
-        Time time = new Time(DateUtil.getTimeParser(
-                "HH:mm:ss", TimeZone.getDefault()).parseDateTime("00:00:00").getTime());
+        Time time = new Time(DateUtil.getDateTimeParser(
+                "HH:mm:ss", PTime.INSTANCE, TimeZone.getDefault().getID()).parseDateTime("00:00:00"));
         assertEquals(Time.valueOf("00:00:00"), time);
     }
 


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

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/phoenix/blob/92298f8d/phoenix-core/src/test/java/org/apache/phoenix/compile/WhereCompilerTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/compile/WhereCompilerTest.java b/phoenix-core/src/test/java/org/apache/phoenix/compile/WhereCompilerTest.java
index ff4ad8b..28942ac 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/compile/WhereCompilerTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/compile/WhereCompilerTest.java
@@ -79,12 +79,18 @@ import com.google.common.collect.ImmutableList;
 
 public class WhereCompilerTest extends BaseConnectionlessQueryTest {
 
+    private PhoenixPreparedStatement newPreparedStatement(PhoenixConnection pconn, String query) throws SQLException {
+        PhoenixPreparedStatement pstmt = new PhoenixPreparedStatement(pconn, query);
+        assertRoundtrip(query);
+        return pstmt;
+    }
+    
     @Test
     public void testSingleEqualFilter() throws SQLException {
         String tenantId = "000000000000001";
         String query = "select * from atable where organization_id='" + tenantId + "' and a_integer=0";
         PhoenixConnection pconn = DriverManager.getConnection(getUrl(), PropertiesUtil.deepCopy(TEST_PROPERTIES)).unwrap(PhoenixConnection.class);
-        PhoenixPreparedStatement pstmt = new PhoenixPreparedStatement(pconn, query);
+        PhoenixPreparedStatement pstmt = newPreparedStatement(pconn, query);
         QueryPlan plan = pstmt.optimizeQuery();
         Scan scan = plan.getContext().getScan();
         Filter filter = scan.getFilter();
@@ -101,7 +107,7 @@ public class WhereCompilerTest extends BaseConnectionlessQueryTest {
         PhoenixConnection pconn = DriverManager.getConnection(getUrl(), PropertiesUtil.deepCopy(TEST_PROPERTIES)).unwrap(PhoenixConnection.class);
         pconn.createStatement().execute("CREATE TABLE t (k bigint not null primary key, v varchar) SALT_BUCKETS=20");
         String query = "select * from t where k=" + 1;
-        PhoenixPreparedStatement pstmt = new PhoenixPreparedStatement(pconn, query);
+        PhoenixPreparedStatement pstmt = newPreparedStatement(pconn, query);
         QueryPlan plan = pstmt.optimizeQuery();
         Scan scan = plan.getContext().getScan();
         Filter filter = scan.getFilter();
@@ -122,7 +128,7 @@ public class WhereCompilerTest extends BaseConnectionlessQueryTest {
         PhoenixConnection pconn = DriverManager.getConnection(getUrl(), PropertiesUtil.deepCopy(TEST_PROPERTIES)).unwrap(PhoenixConnection.class);
         pconn.createStatement().execute("CREATE TABLE t (k varchar primary key, v varchar) SALT_BUCKETS=20");
         String query = "select * from t where k='a'";
-        PhoenixPreparedStatement pstmt = new PhoenixPreparedStatement(pconn, query);
+        PhoenixPreparedStatement pstmt = newPreparedStatement(pconn, query);
         QueryPlan plan = pstmt.optimizeQuery();
         Scan scan = plan.getContext().getScan();
         Filter filter = scan.getFilter();
@@ -143,7 +149,7 @@ public class WhereCompilerTest extends BaseConnectionlessQueryTest {
         PhoenixConnection pconn = DriverManager.getConnection(getUrl(), PropertiesUtil.deepCopy(TEST_PROPERTIES)).unwrap(PhoenixConnection.class);
         pconn.createStatement().execute("CREATE TABLE t (k bigint not null primary key, v varchar) SALT_BUCKETS=20");
         String query = "select * from t where k in (1,3)";
-        PhoenixPreparedStatement pstmt = new PhoenixPreparedStatement(pconn, query);
+        PhoenixPreparedStatement pstmt = newPreparedStatement(pconn, query);
         QueryPlan plan = pstmt.optimizeQuery();
         Scan scan = plan.getContext().getScan();
         Filter filter = scan.getFilter();
@@ -195,7 +201,7 @@ public class WhereCompilerTest extends BaseConnectionlessQueryTest {
         String tenantId = "000000000000001";
         String query = "select * from atable where organization_id='" + tenantId + "' and a_string=b_string";
         PhoenixConnection pconn = DriverManager.getConnection(getUrl(), PropertiesUtil.deepCopy(TEST_PROPERTIES)).unwrap(PhoenixConnection.class);
-        PhoenixPreparedStatement pstmt = new PhoenixPreparedStatement(pconn, query);
+        PhoenixPreparedStatement pstmt = newPreparedStatement(pconn, query);
         QueryPlan plan = pstmt.optimizeQuery();
         Scan scan = plan.getContext().getScan();
         Filter filter = scan.getFilter();
@@ -212,7 +218,7 @@ public class WhereCompilerTest extends BaseConnectionlessQueryTest {
         String tenantId = "000000000000001";
         String query = "select * from atable where organization_id='" + tenantId + "' and substr(entity_id,null) = 'foo'";
         PhoenixConnection pconn = DriverManager.getConnection(getUrl(), PropertiesUtil.deepCopy(TEST_PROPERTIES)).unwrap(PhoenixConnection.class);
-        PhoenixPreparedStatement pstmt = new PhoenixPreparedStatement(pconn, query);
+        PhoenixPreparedStatement pstmt = newPreparedStatement(pconn, query);
         QueryPlan plan = pstmt.optimizeQuery();
         Scan scan = plan.getContext().getScan();
         Filter filter = scan.getFilter();
@@ -229,7 +235,7 @@ public class WhereCompilerTest extends BaseConnectionlessQueryTest {
         List<Object> binds = Arrays.<Object>asList(tenantId);
         
         PhoenixConnection pconn = DriverManager.getConnection(getUrl(), PropertiesUtil.deepCopy(TEST_PROPERTIES)).unwrap(PhoenixConnection.class);
-        PhoenixPreparedStatement pstmt = new PhoenixPreparedStatement(pconn, query);
+        PhoenixPreparedStatement pstmt = newPreparedStatement(pconn, query);
         bindParams(pstmt, binds);
         QueryPlan plan = pstmt.optimizeQuery();
         Scan scan = plan.getContext().getScan();
@@ -253,7 +259,7 @@ public class WhereCompilerTest extends BaseConnectionlessQueryTest {
         String tenantId = "000000000000001";
         String query = "select * from atable where organization_id='" + tenantId + "' and 0 >= a_integer";
         PhoenixConnection pconn = DriverManager.getConnection(getUrl(), PropertiesUtil.deepCopy(TEST_PROPERTIES)).unwrap(PhoenixConnection.class);
-        PhoenixPreparedStatement pstmt = new PhoenixPreparedStatement(pconn, query);
+        PhoenixPreparedStatement pstmt = newPreparedStatement(pconn, query);
         QueryPlan plan = pstmt.optimizeQuery();
         Scan scan = plan.getContext().getScan();
         
@@ -272,7 +278,7 @@ public class WhereCompilerTest extends BaseConnectionlessQueryTest {
         String dateStr = "2012-01-01 12:00:00";
         String query = "select * from atable where organization_id='" + tenantId + "' and a_date >= to_date('" + dateStr + "')";
         PhoenixConnection pconn = DriverManager.getConnection(getUrl(), PropertiesUtil.deepCopy(TEST_PROPERTIES)).unwrap(PhoenixConnection.class);
-        PhoenixPreparedStatement pstmt = new PhoenixPreparedStatement(pconn, query);
+        PhoenixPreparedStatement pstmt = newPreparedStatement(pconn, query);
         QueryPlan plan = pstmt.optimizeQuery();
         Scan scan = plan.getContext().getScan();
         Filter filter = scan.getFilter();
@@ -291,7 +297,7 @@ public class WhereCompilerTest extends BaseConnectionlessQueryTest {
         String tenantId = "000000000000001";
         String query = "select * from atable where organization_id='" + tenantId + "' and x_decimal >= " + toNumberClause;
         PhoenixConnection pconn = DriverManager.getConnection(getUrl(), PropertiesUtil.deepCopy(TEST_PROPERTIES)).unwrap(PhoenixConnection.class);
-        PhoenixPreparedStatement pstmt = new PhoenixPreparedStatement(pconn, query);
+        PhoenixPreparedStatement pstmt = newPreparedStatement(pconn, query);
         QueryPlan plan = pstmt.optimizeQuery();
         Scan scan = plan.getContext().getScan();
         Filter filter = scan.getFilter();
@@ -348,7 +354,7 @@ public class WhereCompilerTest extends BaseConnectionlessQueryTest {
         String query = "select * from atable where substr(entity_id,1,3)=?";
         List<Object> binds = Arrays.<Object>asList(keyPrefix);
         PhoenixConnection pconn = DriverManager.getConnection(getUrl(), PropertiesUtil.deepCopy(TEST_PROPERTIES)).unwrap(PhoenixConnection.class);
-        PhoenixPreparedStatement pstmt = new PhoenixPreparedStatement(pconn, query);
+        PhoenixPreparedStatement pstmt = newPreparedStatement(pconn, query);
         bindParams(pstmt, binds);
         QueryPlan plan = pstmt.optimizeQuery();
         Scan scan = plan.getContext().getScan();
@@ -374,7 +380,7 @@ public class WhereCompilerTest extends BaseConnectionlessQueryTest {
         String query = "select * from atable where entity_id=?";
         List<Object> binds = Arrays.<Object>asList(keyPrefix);
         PhoenixConnection pconn = DriverManager.getConnection(getUrl(), PropertiesUtil.deepCopy(TEST_PROPERTIES)).unwrap(PhoenixConnection.class);
-        PhoenixPreparedStatement pstmt = new PhoenixPreparedStatement(pconn, query);
+        PhoenixPreparedStatement pstmt = newPreparedStatement(pconn, query);
         bindParams(pstmt, binds);
         QueryPlan plan = pstmt.optimizeQuery();
         Scan scan = plan.getContext().getScan();
@@ -390,7 +396,7 @@ public class WhereCompilerTest extends BaseConnectionlessQueryTest {
         String query = "select * from atable where organization_id=? AND entity_id=?";
         List<Object> binds = Arrays.<Object>asList(tenantId,keyPrefix);
         PhoenixConnection pconn = DriverManager.getConnection(getUrl(), PropertiesUtil.deepCopy(TEST_PROPERTIES)).unwrap(PhoenixConnection.class);
-        PhoenixPreparedStatement pstmt = new PhoenixPreparedStatement(pconn, query);
+        PhoenixPreparedStatement pstmt = newPreparedStatement(pconn, query);
         bindParams(pstmt, binds);
         QueryPlan plan = pstmt.optimizeQuery();
         Scan scan = plan.getContext().getScan();
@@ -405,7 +411,7 @@ public class WhereCompilerTest extends BaseConnectionlessQueryTest {
         String query = "select * from atable where substr(entity_id,1,3)=?";
         List<Object> binds = Arrays.<Object>asList(keyPrefix);
         PhoenixConnection pconn = DriverManager.getConnection(getUrl(), PropertiesUtil.deepCopy(TEST_PROPERTIES)).unwrap(PhoenixConnection.class);
-        PhoenixPreparedStatement pstmt = new PhoenixPreparedStatement(pconn, query);
+        PhoenixPreparedStatement pstmt = newPreparedStatement(pconn, query);
         bindParams(pstmt, binds);
         QueryPlan plan = pstmt.optimizeQuery();
         // Degenerate b/c "foobar" is more than 3 characters
@@ -420,7 +426,7 @@ public class WhereCompilerTest extends BaseConnectionlessQueryTest {
         String query = "select * from atable where a_string=?";
         List<Object> binds = Arrays.<Object>asList(aString);
         PhoenixConnection pconn = DriverManager.getConnection(getUrl(), PropertiesUtil.deepCopy(TEST_PROPERTIES)).unwrap(PhoenixConnection.class);
-        PhoenixPreparedStatement pstmt = new PhoenixPreparedStatement(pconn, query);
+        PhoenixPreparedStatement pstmt = newPreparedStatement(pconn, query);
         bindParams(pstmt, binds);
         QueryPlan plan = pstmt.optimizeQuery();
         // Degenerate b/c a_string length is 100
@@ -435,7 +441,7 @@ public class WhereCompilerTest extends BaseConnectionlessQueryTest {
         String query = "select * from atable where organization_id=? and (substr(entity_id,1,3)=? or a_integer=?)";
         List<Object> binds = Arrays.<Object>asList(tenantId, keyPrefix, aInt);
         PhoenixConnection pconn = DriverManager.getConnection(getUrl(), PropertiesUtil.deepCopy(TEST_PROPERTIES)).unwrap(PhoenixConnection.class);
-        PhoenixPreparedStatement pstmt = new PhoenixPreparedStatement(pconn, query);
+        PhoenixPreparedStatement pstmt = newPreparedStatement(pconn, query);
         bindParams(pstmt, binds);
         QueryPlan plan = pstmt.optimizeQuery();
         Scan scan = plan.getContext().getScan();
@@ -464,7 +470,7 @@ public class WhereCompilerTest extends BaseConnectionlessQueryTest {
         String tenantId = "000000000000001";
         String query = "select * from atable where organization_id='" + tenantId + "' and a_integer > 'foo'";
         PhoenixConnection pconn = DriverManager.getConnection(getUrl(), PropertiesUtil.deepCopy(TEST_PROPERTIES)).unwrap(PhoenixConnection.class);
-        PhoenixPreparedStatement pstmt = new PhoenixPreparedStatement(pconn, query);
+        PhoenixPreparedStatement pstmt = newPreparedStatement(pconn, query);
 
         try {
             pstmt.optimizeQuery();
@@ -479,7 +485,7 @@ public class WhereCompilerTest extends BaseConnectionlessQueryTest {
         String tenantId = "000000000000001";
         String query = "select * from atable where organization_id='" + tenantId + "' and a_integer=0 and 2=3";
         PhoenixConnection pconn = DriverManager.getConnection(getUrl(), PropertiesUtil.deepCopy(TEST_PROPERTIES)).unwrap(PhoenixConnection.class);
-        PhoenixPreparedStatement pstmt = new PhoenixPreparedStatement(pconn, query);
+        PhoenixPreparedStatement pstmt = newPreparedStatement(pconn, query);
         QueryPlan plan = pstmt.optimizeQuery();
         assertDegenerate(plan.getContext());
     }
@@ -489,7 +495,7 @@ public class WhereCompilerTest extends BaseConnectionlessQueryTest {
         String tenantId = "000000000000001";
         String query = "select * from atable where organization_id='" + tenantId + "' and 2=3";
         PhoenixConnection pconn = DriverManager.getConnection(getUrl(), PropertiesUtil.deepCopy(TEST_PROPERTIES)).unwrap(PhoenixConnection.class);
-        PhoenixPreparedStatement pstmt = new PhoenixPreparedStatement(pconn, query);
+        PhoenixPreparedStatement pstmt = newPreparedStatement(pconn, query);
         QueryPlan plan = pstmt.optimizeQuery();
         assertDegenerate(plan.getContext());
     }
@@ -499,7 +505,7 @@ public class WhereCompilerTest extends BaseConnectionlessQueryTest {
         String tenantId = "000000000000001";
         String query = "select * from atable where organization_id='" + tenantId + "' and 2<=2";
         PhoenixConnection pconn = DriverManager.getConnection(getUrl(), PropertiesUtil.deepCopy(TEST_PROPERTIES)).unwrap(PhoenixConnection.class);
-        PhoenixPreparedStatement pstmt = new PhoenixPreparedStatement(pconn, query);
+        PhoenixPreparedStatement pstmt = newPreparedStatement(pconn, query);
         QueryPlan plan = pstmt.optimizeQuery();
         Scan scan = plan.getContext().getScan();
         assertNull(scan.getFilter());
@@ -514,7 +520,7 @@ public class WhereCompilerTest extends BaseConnectionlessQueryTest {
         String tenantId = "000000000000001";
         String query = "select * from atable where organization_id='" + tenantId + "' and a_integer=0 and 2<3";
         PhoenixConnection pconn = DriverManager.getConnection(getUrl(), PropertiesUtil.deepCopy(TEST_PROPERTIES)).unwrap(PhoenixConnection.class);
-        PhoenixPreparedStatement pstmt = new PhoenixPreparedStatement(pconn, query);
+        PhoenixPreparedStatement pstmt = newPreparedStatement(pconn, query);
         QueryPlan plan = pstmt.optimizeQuery();
         Scan scan = plan.getContext().getScan();
         Filter filter = scan.getFilter();
@@ -536,7 +542,7 @@ public class WhereCompilerTest extends BaseConnectionlessQueryTest {
         String tenantId = "000000000000001";
         String query = "select * from atable where organization_id='" + tenantId + "' and (a_integer=0 or 3!=3)";
         PhoenixConnection pconn = DriverManager.getConnection(getUrl(), PropertiesUtil.deepCopy(TEST_PROPERTIES)).unwrap(PhoenixConnection.class);
-        PhoenixPreparedStatement pstmt = new PhoenixPreparedStatement(pconn, query);
+        PhoenixPreparedStatement pstmt = newPreparedStatement(pconn, query);
         QueryPlan plan = pstmt.optimizeQuery();
         Scan scan = plan.getContext().getScan();
         Filter filter = scan.getFilter();
@@ -557,7 +563,7 @@ public class WhereCompilerTest extends BaseConnectionlessQueryTest {
         String tenantId = "000000000000001";
         String query = "select * from atable where organization_id='" + tenantId + "' and (a_integer=0 or 3>2)";
         PhoenixConnection pconn = DriverManager.getConnection(getUrl(), PropertiesUtil.deepCopy(TEST_PROPERTIES)).unwrap(PhoenixConnection.class);
-        PhoenixPreparedStatement pstmt = new PhoenixPreparedStatement(pconn, query);
+        PhoenixPreparedStatement pstmt = newPreparedStatement(pconn, query);
         QueryPlan plan = pstmt.optimizeQuery();
         Scan scan = plan.getContext().getScan();
         Filter filter = scan.getFilter();
@@ -573,7 +579,7 @@ public class WhereCompilerTest extends BaseConnectionlessQueryTest {
         String tenantId = "000000000000001";
         String query = "select * from atable where organization_id='" + tenantId + "' and a_string IN ('a','b')";
         PhoenixConnection pconn = DriverManager.getConnection(getUrl(), PropertiesUtil.deepCopy(TEST_PROPERTIES)).unwrap(PhoenixConnection.class);
-        PhoenixPreparedStatement pstmt = new PhoenixPreparedStatement(pconn, query);
+        PhoenixPreparedStatement pstmt = newPreparedStatement(pconn, query);
         QueryPlan plan = pstmt.optimizeQuery();
         Scan scan = plan.getContext().getScan();
         byte[] startRow = PVarchar.INSTANCE.toBytes(tenantId);
@@ -598,7 +604,7 @@ public class WhereCompilerTest extends BaseConnectionlessQueryTest {
         String query = String.format("select * from %s where organization_id IN ('%s','%s','%s')",
                 ATABLE_NAME, tenantId1, tenantId3, tenantId2);
         PhoenixConnection pconn = DriverManager.getConnection(getUrl(), PropertiesUtil.deepCopy(TEST_PROPERTIES)).unwrap(PhoenixConnection.class);
-        PhoenixPreparedStatement pstmt = new PhoenixPreparedStatement(pconn, query);
+        PhoenixPreparedStatement pstmt = newPreparedStatement(pconn, query);
         QueryPlan plan = pstmt.optimizeQuery();
         Scan scan = plan.getContext().getScan();
         byte[] startRow = PVarchar.INSTANCE.toBytes(tenantId1);
@@ -625,7 +631,7 @@ public class WhereCompilerTest extends BaseConnectionlessQueryTest {
         String query = String.format("select * from %s where organization_id='%s' OR organization_id='%s' OR organization_id='%s'",
                 ATABLE_NAME, tenantId1, tenantId3, tenantId2);
         PhoenixConnection pconn = DriverManager.getConnection(getUrl(), PropertiesUtil.deepCopy(TEST_PROPERTIES)).unwrap(PhoenixConnection.class);
-        PhoenixPreparedStatement pstmt = new PhoenixPreparedStatement(pconn, query);
+        PhoenixPreparedStatement pstmt = newPreparedStatement(pconn, query);
         QueryPlan plan = pstmt.optimizeQuery();
         Scan scan = plan.getContext().getScan();
 
@@ -653,7 +659,7 @@ public class WhereCompilerTest extends BaseConnectionlessQueryTest {
         String query = String.format("select * from %s where organization_id='%s' AND entity_id IN ('%s','%s')",
                 ATABLE_NAME, tenantId, entityId1, entityId2);
         PhoenixConnection pconn = DriverManager.getConnection(getUrl(), PropertiesUtil.deepCopy(TEST_PROPERTIES)).unwrap(PhoenixConnection.class);
-        PhoenixPreparedStatement pstmt = new PhoenixPreparedStatement(pconn, query);
+        PhoenixPreparedStatement pstmt = newPreparedStatement(pconn, query);
         QueryPlan plan = pstmt.optimizeQuery();
         Scan scan = plan.getContext().getScan();
         byte[] startRow = PVarchar.INSTANCE.toBytes(tenantId + entityId1);
@@ -683,7 +689,7 @@ public class WhereCompilerTest extends BaseConnectionlessQueryTest {
         String query = String.format("select * from %s where organization_id IN ('%s','%s','%s') AND entity_id>='%s' AND entity_id<='%s'",
                 ATABLE_NAME, tenantId1, tenantId3, tenantId2, entityId1, entityId2);
         PhoenixConnection pconn = DriverManager.getConnection(getUrl(), PropertiesUtil.deepCopy(TEST_PROPERTIES)).unwrap(PhoenixConnection.class);
-        PhoenixPreparedStatement pstmt = new PhoenixPreparedStatement(pconn, query);
+        PhoenixPreparedStatement pstmt = newPreparedStatement(pconn, query);
         QueryPlan plan = pstmt.optimizeQuery();
         Scan scan = plan.getContext().getScan();
         Filter filter = scan.getFilter();
@@ -712,7 +718,7 @@ public class WhereCompilerTest extends BaseConnectionlessQueryTest {
         String query = String.format("select * from %s where organization_id IN ('%s','%s','%s') AND entity_id='%s'",
                 ATABLE_NAME, tenantId1, tenantId3, tenantId2, entityId);
         PhoenixConnection pconn = DriverManager.getConnection(getUrl(), PropertiesUtil.deepCopy(TEST_PROPERTIES)).unwrap(PhoenixConnection.class);
-        PhoenixPreparedStatement pstmt = new PhoenixPreparedStatement(pconn, query);
+        PhoenixPreparedStatement pstmt = newPreparedStatement(pconn, query);
         QueryPlan plan = pstmt.optimizeQuery();
         Scan scan = plan.getContext().getScan();
         Filter filter = scan.getFilter();
@@ -735,7 +741,7 @@ public class WhereCompilerTest extends BaseConnectionlessQueryTest {
         String query = String.format("select * from %s where organization_id IN ('%s','%s','%s') AND entity_id='%s'",
                 ATABLE_NAME, tenantId1, tenantId3, tenantId2, entityId);
         PhoenixConnection pconn = DriverManager.getConnection(getUrl(), PropertiesUtil.deepCopy(TEST_PROPERTIES)).unwrap(PhoenixConnection.class);
-        PhoenixPreparedStatement pstmt = new PhoenixPreparedStatement(pconn, query);
+        PhoenixPreparedStatement pstmt = newPreparedStatement(pconn, query);
         QueryPlan plan = pstmt.optimizeQuery();
         Scan scan = plan.getContext().getScan();
         byte[] startRow = ByteUtil.concat(PVarchar.INSTANCE.toBytes(tenantId1), PVarchar.INSTANCE.toBytes(entityId));
@@ -765,7 +771,7 @@ public class WhereCompilerTest extends BaseConnectionlessQueryTest {
         String query = String.format("select * from %s where organization_id IN ('%s','%s') AND entity_id IN ('%s', '%s')",
                 ATABLE_NAME, tenantId1, tenantId2, entityId1, entityId2);
         PhoenixConnection pconn = DriverManager.getConnection(getUrl(), PropertiesUtil.deepCopy(TEST_PROPERTIES)).unwrap(PhoenixConnection.class);
-        PhoenixPreparedStatement pstmt = new PhoenixPreparedStatement(pconn, query);
+        PhoenixPreparedStatement pstmt = newPreparedStatement(pconn, query);
         QueryPlan plan = pstmt.optimizeQuery();
         Scan scan = plan.getContext().getScan();
 
@@ -789,7 +795,7 @@ public class WhereCompilerTest extends BaseConnectionlessQueryTest {
         String query = String.format("select * from %s where organization_id > '%s' AND organization_id < '%s'",
                 ATABLE_NAME, tenantId1, tenantId2);
         PhoenixConnection pconn = DriverManager.getConnection(getUrl(), PropertiesUtil.deepCopy(TEST_PROPERTIES)).unwrap(PhoenixConnection.class);
-        PhoenixPreparedStatement pstmt = new PhoenixPreparedStatement(pconn, query);
+        PhoenixPreparedStatement pstmt = newPreparedStatement(pconn, query);
         QueryPlan plan = pstmt.optimizeQuery();
         Scan scan = plan.getContext().getScan();
 
@@ -810,7 +816,7 @@ public class WhereCompilerTest extends BaseConnectionlessQueryTest {
         String query = String.format("select * from %s where organization_id IN ('%s','%s','%s') AND entity_id IN ('%s', '%s')",
                 ATABLE_NAME, tenantId1, tenantId3, tenantId2, entityId1, entityId2);
         PhoenixConnection pconn = DriverManager.getConnection(getUrl(), PropertiesUtil.deepCopy(TEST_PROPERTIES)).unwrap(PhoenixConnection.class);
-        PhoenixPreparedStatement pstmt = new PhoenixPreparedStatement(pconn, query);
+        PhoenixPreparedStatement pstmt = newPreparedStatement(pconn, query);
         QueryPlan plan = pstmt.optimizeQuery();
         Scan scan = plan.getContext().getScan();
         byte[] startRow = ByteUtil.concat(PVarchar.INSTANCE.toBytes(tenantId1), PVarchar.INSTANCE.toBytes(entityId1));
@@ -825,7 +831,7 @@ public class WhereCompilerTest extends BaseConnectionlessQueryTest {
         String tenantId = "000000000000001";
         String query = "select * from atable where organization_id='" + tenantId + "' and a_integer between 0 and 10";
         PhoenixConnection pconn = DriverManager.getConnection(getUrl(), PropertiesUtil.deepCopy(TEST_PROPERTIES)).unwrap(PhoenixConnection.class);
-        PhoenixPreparedStatement pstmt = new PhoenixPreparedStatement(pconn, query);
+        PhoenixPreparedStatement pstmt = newPreparedStatement(pconn, query);
         QueryPlan plan = pstmt.optimizeQuery();
         Scan scan = plan.getContext().getScan();
         Filter filter = scan.getFilter();
@@ -847,7 +853,7 @@ public class WhereCompilerTest extends BaseConnectionlessQueryTest {
         String tenantId = "000000000000001";
         String query = "select * from atable where organization_id='" + tenantId + "' and a_integer not between 0 and 10";
         PhoenixConnection pconn = DriverManager.getConnection(getUrl(), PropertiesUtil.deepCopy(TEST_PROPERTIES)).unwrap(PhoenixConnection.class);
-        PhoenixPreparedStatement pstmt = new PhoenixPreparedStatement(pconn, query);
+        PhoenixPreparedStatement pstmt = newPreparedStatement(pconn, query);
         QueryPlan plan = pstmt.optimizeQuery();
         Scan scan = plan.getContext().getScan();
         Filter filter = scan.getFilter();
@@ -875,7 +881,7 @@ public class WhereCompilerTest extends BaseConnectionlessQueryTest {
         
         String query = "select * from tenant_filter_test where a_integer=0 and a_string='foo'";
         PhoenixConnection pconn = DriverManager.getConnection(url, PropertiesUtil.deepCopy(TEST_PROPERTIES)).unwrap(PhoenixConnection.class);
-        PhoenixPreparedStatement pstmt = new PhoenixPreparedStatement(pconn, query);
+        PhoenixPreparedStatement pstmt = newPreparedStatement(pconn, query);
         QueryPlan plan = pstmt.optimizeQuery();
         Scan scan = plan.getContext().getScan();
         Filter filter = scan.getFilter();
@@ -907,7 +913,7 @@ public class WhereCompilerTest extends BaseConnectionlessQueryTest {
         
         String query = "select * from tenant_filter_test where a_integer=0 and a_string='foo'";
         PhoenixConnection pconn = DriverManager.getConnection(getUrl(tenantId), PropertiesUtil.deepCopy(TEST_PROPERTIES)).unwrap(PhoenixConnection.class);
-        PhoenixPreparedStatement pstmt = new PhoenixPreparedStatement(pconn, query);
+        PhoenixPreparedStatement pstmt = newPreparedStatement(pconn, query);
         QueryPlan plan = pstmt.optimizeQuery();
         Scan scan = plan.getContext().getScan();
         Filter filter = scan.getFilter();
@@ -934,7 +940,7 @@ public class WhereCompilerTest extends BaseConnectionlessQueryTest {
     public void testScanCaching_Default() throws SQLException {
         String query = "select * from atable where a_integer=0";
         PhoenixConnection pconn = DriverManager.getConnection(getUrl(), PropertiesUtil.deepCopy(TEST_PROPERTIES)).unwrap(PhoenixConnection.class);
-        PhoenixPreparedStatement pstmt = new PhoenixPreparedStatement(pconn, query);
+        PhoenixPreparedStatement pstmt = newPreparedStatement(pconn, query);
         QueryPlan plan = pstmt.optimizeQuery();
         Scan scan = plan.getContext().getScan();
         assertEquals(QueryServicesOptions.DEFAULT_SCAN_CACHE_SIZE, pstmt.getFetchSize());
@@ -945,7 +951,7 @@ public class WhereCompilerTest extends BaseConnectionlessQueryTest {
     public void testScanCaching_CustomFetchSizeOnStatement() throws SQLException {
         String query = "select * from atable where a_integer=0";
         PhoenixConnection pconn = DriverManager.getConnection(getUrl(), PropertiesUtil.deepCopy(TEST_PROPERTIES)).unwrap(PhoenixConnection.class);
-        PhoenixPreparedStatement pstmt = new PhoenixPreparedStatement(pconn, query);
+        PhoenixPreparedStatement pstmt = newPreparedStatement(pconn, query);
         final int FETCH_SIZE = 25;
         pstmt.setFetchSize(FETCH_SIZE);
         QueryPlan plan = pstmt.optimizeQuery();

http://git-wip-us.apache.org/repos/asf/phoenix/blob/92298f8d/phoenix-core/src/test/java/org/apache/phoenix/compile/WhereOptimizerTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/compile/WhereOptimizerTest.java b/phoenix-core/src/test/java/org/apache/phoenix/compile/WhereOptimizerTest.java
index ad8e5f5..ddbacb7 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/compile/WhereOptimizerTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/compile/WhereOptimizerTest.java
@@ -54,8 +54,8 @@ import org.apache.phoenix.jdbc.PhoenixPreparedStatement;
 import org.apache.phoenix.query.BaseConnectionlessQueryTest;
 import org.apache.phoenix.query.KeyRange;
 import org.apache.phoenix.query.QueryConstants;
-import org.apache.phoenix.schema.types.PChar;
 import org.apache.phoenix.schema.ColumnNotFoundException;
+import org.apache.phoenix.schema.types.PChar;
 import org.apache.phoenix.schema.types.PDate;
 import org.apache.phoenix.schema.types.PUnsignedLong;
 import org.apache.phoenix.schema.types.PVarchar;
@@ -86,6 +86,7 @@ public class WhereOptimizerTest extends BaseConnectionlessQueryTest {
     private static StatementContext compileStatement(String query, List<Object> binds, Integer limit) throws SQLException {
         PhoenixConnection pconn = DriverManager.getConnection(getUrl(), PropertiesUtil.deepCopy(TEST_PROPERTIES)).unwrap(PhoenixConnection.class);
         PhoenixPreparedStatement pstmt = new PhoenixPreparedStatement(pconn, query);
+        assertRoundtrip(query);
         TestUtil.bindParams(pstmt, binds);
         QueryPlan plan = pstmt.compileQuery();
         assertEquals(limit, plan.getLimit());

http://git-wip-us.apache.org/repos/asf/phoenix/blob/92298f8d/phoenix-core/src/test/java/org/apache/phoenix/parse/QueryParserTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/parse/QueryParserTest.java b/phoenix-core/src/test/java/org/apache/phoenix/parse/QueryParserTest.java
index 866365a..bf599ae 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/parse/QueryParserTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/parse/QueryParserTest.java
@@ -22,6 +22,7 @@ import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
+import java.io.IOException;
 import java.io.StringReader;
 import java.sql.SQLException;
 import java.sql.SQLFeatureNotSupportedException;
@@ -29,24 +30,43 @@ import java.util.List;
 
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.phoenix.exception.SQLExceptionCode;
+import org.apache.phoenix.jdbc.PhoenixStatement.Operation;
 import org.apache.phoenix.schema.SortOrder;
 import org.junit.Test;
 
 
 public class QueryParserTest {
+    private void parseQuery(String sql) throws IOException, SQLException {
+        SQLParser parser = new SQLParser(new StringReader(sql));
+        BindableStatement stmt = null;
+        stmt = parser.parseStatement();
+        if (stmt.getOperation() != Operation.QUERY) {
+            return;
+        }
+        String newSQL = stmt.toString();
+        SQLParser newParser = new SQLParser(new StringReader(newSQL));
+        BindableStatement newStmt = null;
+        try {
+            newStmt = newParser.parseStatement();
+        } catch (SQLException e) {
+            fail("Unable to parse new:\n" + newSQL);
+        }
+        assertEquals("Expected equality:\n" + sql + "\n" + newSQL, stmt, newStmt);
+    }
+    
     @Test
     public void testParsePreQuery0() throws Exception {
-        SQLParser parser = new SQLParser(new StringReader(
+        String sql = ((
             "select a from b\n" +
             "where ((ind.name = 'X')" +
             "and rownum <= (1000 + 1000))\n"
             ));
-        parser.parseStatement();
+        parseQuery(sql);
     }
 
     @Test
     public void testParsePreQuery1() throws Exception {
-        SQLParser parser = new SQLParser(new StringReader(
+        String sql = ((
             "select /*gatherSlowStats*/ count(1) from core.search_name_lookup ind\n" +
             "where( (ind.name = 'X'\n" +
             "and rownum <= 1 + 2)\n" +
@@ -54,12 +74,12 @@ public class QueryParserTest {
             "and (ind.key_prefix = '00T')\n" +
             "and (ind.name_type = 't'))"
             ));
-        parser.parseStatement();
+        parseQuery(sql);
     }
 
     @Test
     public void testParsePreQuery2() throws Exception {
-        SQLParser parser = new SQLParser(new StringReader(
+        String sql = ((
             "select /*gatherSlowStats*/ count(1) from core.custom_index_value ind\n" + 
             "where (ind.string_value in ('a', 'b', 'c', 'd'))\n" + 
             "and rownum <= ( 3 + 1 )\n" + 
@@ -68,12 +88,12 @@ public class QueryParserTest {
             "and (ind.deleted = '0')\n" + 
             "and (ind.index_num = 1)"
             ));
-        parser.parseStatement();
+        parseQuery(sql);
     }
 
     @Test
     public void testParsePreQuery3() throws Exception {
-        SQLParser parser = new SQLParser(new StringReader(
+        String sql = ((
             "select /*gatherSlowStats*/ count(1) from core.custom_index_value ind\n" + 
             "where (ind.number_value > 3)\n" + 
             "and rownum <= 1000\n" + 
@@ -82,54 +102,54 @@ public class QueryParserTest {
             "and (ind.deleted = '0'))\n" + 
             "and (ind.index_num = 2)"
             ));
-        parser.parseStatement();
+        parseQuery(sql);
     }
 
     @Test
     public void testParsePreQuery4() throws Exception {
-        SQLParser parser = new SQLParser(new StringReader(
+        String sql = ((
             "select /*+ index(t iecustom_entity_data_created) */ /*gatherSlowStats*/ count(1) from core.custom_entity_data t\n" + 
             "where (t.created_date > to_date('01/01/2001'))\n" + 
             "and rownum <= 4500\n" + 
             "and (t.organization_id = '000000000000000')\n" + 
             "and (t.key_prefix = '001')"
             ));
-        parser.parseStatement();
+        parseQuery(sql);
     }
 
     @Test
     public void testCountDistinctQuery() throws Exception {
-        SQLParser parser = new SQLParser(new StringReader(
+        String sql = ((
                 "select count(distinct foo) from core.custom_entity_data t\n"
                         + "where (t.created_date > to_date('01/01/2001'))\n"
                         + "and (t.organization_id = '000000000000000')\n"
                         + "and (t.key_prefix = '001')\n" + "limit 4500"));
-        parser.parseStatement();
+        parseQuery(sql);
     }
 
     @Test
     public void testIsNullQuery() throws Exception {
-        SQLParser parser = new SQLParser(new StringReader(
+        String sql = ((
             "select count(foo) from core.custom_entity_data t\n" + 
             "where (t.created_date is null)\n" + 
             "and (t.organization_id is not null)\n"
             ));
-        parser.parseStatement();
+        parseQuery(sql);
     }
 
     @Test
     public void testAsInColumnAlias() throws Exception {
-        SQLParser parser = new SQLParser(new StringReader(
+        String sql = ((
             "select count(foo) AS c from core.custom_entity_data t\n" + 
             "where (t.created_date is null)\n" + 
             "and (t.organization_id is not null)\n"
             ));
-        parser.parseStatement();
+        parseQuery(sql);
     }
 
     @Test
     public void testParseJoin1() throws Exception {
-        SQLParser parser = new SQLParser(new StringReader(
+        String sql = ((
             "select /*SOQL*/ \"Id\"\n" + 
             "from (select /*+ ordered index(cft) */\n" + 
             "cft.val188 \"Marketing_Offer_Code__c\",\n" + 
@@ -143,12 +163,12 @@ public class QueryParserTest {
             "and (t.account_id != '000000000000000'))\n" + 
             "where (\"Marketing_Offer_Code__c\" = 'FSCR')"
             ));
-        parser.parseStatement();
+        parseQuery(sql);
     }
 
     @Test
     public void testParseJoin2() throws Exception {
-        SQLParser parser = new SQLParser(new StringReader(
+        String sql = ((
             "select /*rptacctlist 00O40000002C3of*/ \"00N40000001M8VK\",\n" + 
             "\"00N40000001M8VK.ID\",\n" + 
             "\"00N30000000r0K2\",\n" + 
@@ -175,12 +195,12 @@ public class QueryParserTest {
             "AND (\"00N40000001M8VK\" is null or \"00N40000001M8VK\" in ('BRIAN IRWIN', 'BRIAN MILLER', 'COLLEEN HORNYAK', 'ERNIE ZAVORAL JR', 'JAMIE TRIMBUR', 'JOE ANTESBERGER', 'MICHAEL HYTLA', 'NATHAN DELSIGNORE', 'SANJAY GANDHI', 'TOM BASHIOUM'))\n" + 
             "AND (\"LAST_UPDATE\" >= to_date('2009-08-01 07:00:00'))"
             ));
-        parser.parseStatement();
+        parseQuery(sql);
     }
     
     @Test
     public void testNegative1() throws Exception {
-        SQLParser parser = new SQLParser(new StringReader(
+        String sql = ((
             "select /*gatherSlowStats*/ count(1) core.search_name_lookup ind\n" +
             "where (ind.name = 'X')\n" +
             "and rownum <= 2000\n" +
@@ -189,7 +209,7 @@ public class QueryParserTest {
             "and (ind.name_type = 't')"
             ));
         try {
-            parser.parseStatement();
+            parseQuery(sql);
             fail();
         } catch (SQLException e) {
             assertEquals(SQLExceptionCode.MISMATCHED_TOKEN.getErrorCode(), e.getErrorCode());
@@ -198,7 +218,7 @@ public class QueryParserTest {
 
     @Test
     public void testNegative2() throws Exception {
-        SQLParser parser = new SQLParser(new StringReader(
+        String sql = ((
             "seelect /*gatherSlowStats*/ count(1) from core.search_name_lookup ind\n" +
             "where (ind.name = 'X')\n" +
             "and rownum <= 2000\n" +
@@ -207,7 +227,7 @@ public class QueryParserTest {
             "and (ind.name_type = 't')"
             ));
         try {
-            parser.parseStatement();
+            parseQuery(sql);
             fail();
         } catch (SQLException e) {
             assertTrue(e.getMessage(), e.getMessage().contains("ERROR 601 (42P00): Syntax error. Encountered \"seelect\" at line 1, column 1."));
@@ -216,7 +236,7 @@ public class QueryParserTest {
 
     @Test
     public void testNegative3() throws Exception {
-        SQLParser parser = new SQLParser(new StringReader(
+        String sql = ((
             "select /*gatherSlowStats*/ count(1) from core.search_name_lookup ind\n" +
             "where (ind.name = 'X')\n" +
             "and rownum <= 2000\n" +
@@ -225,7 +245,7 @@ public class QueryParserTest {
             "and (ind.name_type = 't'))"
             ));
         try {
-            parser.parseStatement();
+            parseQuery(sql);
             fail();
         } catch (SQLException e) {
             assertTrue(e.getMessage(), e.getMessage().contains("ERROR 603 (42P00): Syntax error. Unexpected input. Expecting \"EOF\", got \")\" at line 6, column 26."));
@@ -234,7 +254,7 @@ public class QueryParserTest {
 
     @Test
     public void testNegativeCountDistinct() throws Exception {
-        SQLParser parser = new SQLParser(new StringReader(
+        String sql = ((
             "select /*gatherSlowStats*/ max( distinct 1) from core.search_name_lookup ind\n" +
             "where (ind.name = 'X')\n" +
             "and rownum <= 2000\n" +
@@ -243,7 +263,7 @@ public class QueryParserTest {
             "and (ind.name_type = 't')"
             ));
         try {
-            parser.parseStatement();
+            parseQuery(sql);
             fail();
         } catch (SQLFeatureNotSupportedException e) {
             // expected
@@ -252,7 +272,7 @@ public class QueryParserTest {
 
     @Test
     public void testNegativeCountStar() throws Exception {
-        SQLParser parser = new SQLParser(new StringReader(
+        String sql = ((
             "select /*gatherSlowStats*/ max(*) from core.search_name_lookup ind\n" +
             "where (ind.name = 'X')\n" +
             "and rownum <= 2000\n" +
@@ -261,7 +281,7 @@ public class QueryParserTest {
             "and (ind.name_type = 't')"
             ));
         try {
-            parser.parseStatement();
+            parseQuery(sql);
             fail();
         } catch (SQLException e) {
             assertTrue(e.getMessage(), e.getMessage().contains("ERROR 601 (42P00): Syntax error. Encountered \"*\" at line 1, column 32."));
@@ -270,7 +290,7 @@ public class QueryParserTest {
 
     @Test
     public void testUnknownFunction() throws Exception {
-        SQLParser parser = new SQLParser(new StringReader(
+        String sql = ((
             "select /*gatherSlowStats*/ bogus_function(ind.key_prefix) from core.search_name_lookup ind\n" +
             "where (ind.name = 'X')\n" +
             "and rownum <= 2000\n" +
@@ -279,7 +299,7 @@ public class QueryParserTest {
             "and (ind.name_type = 't')"
             ));
         try {
-            parser.parseStatement();
+            parseQuery(sql);
             fail();
         } catch (SQLException e) {
             assertEquals(SQLExceptionCode.UNKNOWN_FUNCTION.getErrorCode(), e.getErrorCode());
@@ -288,12 +308,12 @@ public class QueryParserTest {
 
     @Test
     public void testNegativeNonBooleanWhere() throws Exception {
-        SQLParser parser = new SQLParser(new StringReader(
+        String sql = ((
             "select /*gatherSlowStats*/ max( distinct 1) from core.search_name_lookup ind\n" +
             "where 1"
             ));
         try {
-            parser.parseStatement();
+            parseQuery(sql);
             fail();
         } catch (SQLFeatureNotSupportedException e) {
             // expected
@@ -302,48 +322,48 @@ public class QueryParserTest {
     
     @Test
     public void testCommentQuery() throws Exception {
-        SQLParser parser = new SQLParser(new StringReader(
+        String sql = ((
             "select a from b -- here we come\n" +
             "where ((ind.name = 'X') // to save the day\n" +
             "and rownum /* won't run */ <= (1000 + 1000))\n"
             ));
-        parser.parseStatement();
+        parseQuery(sql);
     }
 
     @Test
     public void testQuoteEscapeQuery() throws Exception {
-        SQLParser parser = new SQLParser(new StringReader(
+        String sql = ((
             "select a from b\n" +
             "where ind.name = 'X''Y'\n"
             ));
-        parser.parseStatement();
+        parseQuery(sql);
     }
 
     @Test
     public void testSubtractionInSelect() throws Exception {
-        SQLParser parser = new SQLParser(new StringReader(
+        String sql = ((
             "select a, 3-1-2, -4- -1-1 from b\n" +
             "where d = c - 1\n"
             ));
-        parser.parseStatement();
+        parseQuery(sql);
     }
 
     @Test
     public void testParsingStatementWithMispellToken() throws Exception {
         try {
-            SQLParser parser = new SQLParser(new StringReader(
+            String sql = ((
                     "selects a from b\n" +
                     "where e = d\n"));
-            parser.parseStatement();
+            parseQuery(sql);
             fail("Should have caught exception.");
         } catch (SQLException e) {
             assertTrue(e.getMessage(), e.getMessage().contains("ERROR 601 (42P00): Syntax error. Encountered \"selects\" at line 1, column 1."));
         }
         try {
-            SQLParser parser = new SQLParser(new StringReader(
+            String sql = ((
                     "select a froms b\n" +
                     "where e = d\n"));
-            parser.parseStatement();
+            parseQuery(sql);
             fail("Should have caught exception.");
         } catch (SQLException e) {
             assertTrue(e.getMessage(), e.getMessage().contains("ERROR 602 (42P00): Syntax error. Missing \"FROM\" at line 1, column 16."));
@@ -353,19 +373,19 @@ public class QueryParserTest {
     @Test
     public void testParsingStatementWithExtraToken() throws Exception {
         try {
-            SQLParser parser = new SQLParser(new StringReader(
+            String sql = ((
                     "select a,, from b\n" +
                     "where e = d\n"));
-            parser.parseStatement();
+            parseQuery(sql);
             fail("Should have caught exception.");
         } catch (SQLException e) {
             assertTrue(e.getMessage(), e.getMessage().contains("ERROR 601 (42P00): Syntax error. Encountered \",\" at line 1, column 10."));
         }
         try {
-            SQLParser parser = new SQLParser(new StringReader(
+            String sql = ((
                     "select a from from b\n" +
                     "where e = d\n"));
-            parser.parseStatement();
+            parseQuery(sql);
             fail("Should have caught exception.");
         } catch (SQLException e) {
             assertTrue(e.getMessage(), e.getMessage().contains("ERROR 601 (42P00): Syntax error. Encountered \"from\" at line 1, column 15."));
@@ -375,10 +395,10 @@ public class QueryParserTest {
     @Test
     public void testParsingStatementWithMissingToken() throws Exception {
         try {
-            SQLParser parser = new SQLParser(new StringReader(
+            String sql = ((
                     "select a b\n" +
                     "where e = d\n"));
-            parser.parseStatement();
+            parseQuery(sql);
             fail("Should have caught exception.");
         } catch (SQLException e) {
             assertEquals(SQLExceptionCode.MISMATCHED_TOKEN.getErrorCode(), e.getErrorCode());
@@ -389,7 +409,7 @@ public class QueryParserTest {
     public void testParseCreateTableInlinePrimaryKeyWithOrder() throws Exception {
     	for (String order : new String[]{"asc", "desc"}) {
             String s = "create table core.entity_history_archive (id char(15) primary key ${o})".replace("${o}", order);
-    		CreateTableStatement stmt = (CreateTableStatement)new SQLParser(new StringReader(s)).parseStatement();
+    		CreateTableStatement stmt = (CreateTableStatement)new SQLParser((s)).parseStatement();
     		List<ColumnDef> columnDefs = stmt.getColumnDefs();
     		assertEquals(1, columnDefs.size());
     		assertEquals(SortOrder.fromDDLValue(order), columnDefs.iterator().next().getSortOrder()); 
@@ -401,7 +421,7 @@ public class QueryParserTest {
     	for (String order : new String[]{"asc", "desc"}) {
     		String stmt = "create table core.entity_history_archive (id varchar(20) ${o})".replace("${o}", order);
     		try {
-    			new SQLParser(new StringReader(stmt)).parseStatement();
+    			new SQLParser((stmt)).parseStatement();
     			fail("Expected parse exception to be thrown");
     		} catch (SQLException e) {
     			String errorMsg = "ERROR 603 (42P00): Syntax error. Unexpected input. Expecting \"RPAREN\", got \"${o}\"".replace("${o}", order);
@@ -414,7 +434,7 @@ public class QueryParserTest {
     public void testParseCreateTablePrimaryKeyConstraintWithOrder() throws Exception {
     	for (String order : new String[]{"asc", "desc"}) {
     		String s = "create table core.entity_history_archive (id CHAR(15), name VARCHAR(150), constraint pk primary key (id ${o}, name ${o}))".replace("${o}", order);
-    		CreateTableStatement stmt = (CreateTableStatement)new SQLParser(new StringReader(s)).parseStatement();
+    		CreateTableStatement stmt = (CreateTableStatement)new SQLParser((s)).parseStatement();
     		PrimaryKeyConstraint pkConstraint = stmt.getPrimaryKeyConstraint();
     		List<Pair<ColumnName,SortOrder>> columns = pkConstraint.getColumnNames();
     		assertEquals(2, columns.size());
@@ -429,7 +449,7 @@ public class QueryParserTest {
         for (String leadingComma : new String[]{",", ""}) {
             String s = "create table core.entity_history_archive (id CHAR(15), name VARCHAR(150)${o} constraint pk primary key (id))".replace("${o}", leadingComma);
 
-            CreateTableStatement stmt = (CreateTableStatement)new SQLParser(new StringReader(s)).parseStatement();
+            CreateTableStatement stmt = (CreateTableStatement)new SQLParser((s)).parseStatement();
 
             assertEquals(2, stmt.getColumnDefs().size());
             assertNotNull(stmt.getPrimaryKeyConstraint());
@@ -438,11 +458,11 @@ public class QueryParserTest {
 
     @Test
     public void testInvalidTrailingCommaOnCreateTable() throws Exception {
-        SQLParser parser = new SQLParser(
-                new StringReader(
+        String sql = (
+                (
                         "create table foo (c1 varchar primary key, c2 varchar,)"));
         try {
-            parser.parseStatement();
+            parseQuery(sql);
             fail();
         } catch (SQLException e) {
             assertEquals(SQLExceptionCode.MISMATCHED_TOKEN.getErrorCode(), e.getErrorCode());
@@ -451,42 +471,42 @@ public class QueryParserTest {
 
     @Test
 	public void testCreateSequence() throws Exception {
-		SQLParser parser = new SQLParser(new StringReader(
+		String sql = ((
 				"create sequence foo.bar\n" + 
 						"start with 0\n"	+ 
 						"increment by 1\n"));
-		parser.parseStatement();
+		parseQuery(sql);
 	}
 	
 	@Test
 	public void testNextValueForSelect() throws Exception {
-		SQLParser parser = new SQLParser(new StringReader(
+		String sql = ((
 				"select next value for foo.bar \n" + 
 						"from core.custom_entity_data\n"));						
-		parser.parseStatement();
+		parseQuery(sql);
 	}
 	
 	@Test
     public void testNextValueForWhere() throws Exception {
-        SQLParser parser = new SQLParser(new StringReader(
+        String sql = ((
                 "upsert into core.custom_entity_data\n" + 
                         "select next value for foo.bar from core.custom_entity_data\n"));                    
-        parser.parseStatement();
+        parseQuery(sql);
     }
 	
     public void testBadCharDef() throws Exception {
         try {
-            SQLParser parser = new SQLParser("CREATE TABLE IF NOT EXISTS testBadVarcharDef" + 
+            String sql = ("CREATE TABLE IF NOT EXISTS testBadVarcharDef" + 
                     "  (pk VARCHAR NOT NULL PRIMARY KEY, col CHAR(0))");
-            parser.parseStatement();
+            parseQuery(sql);
             fail("Should have caught bad char definition.");
         } catch (SQLException e) {
             assertTrue(e.getMessage(), e.getMessage().contains("ERROR 208 (22003): CHAR or VARCHAR must have a positive length. columnName=COL"));
         }
         try {
-            SQLParser parser = new SQLParser("CREATE TABLE IF NOT EXISTS testBadVarcharDef" + 
+            String sql = ("CREATE TABLE IF NOT EXISTS testBadVarcharDef" + 
                     "  (pk VARCHAR NOT NULL PRIMARY KEY, col CHAR)");
-            parser.parseStatement();
+            parseQuery(sql);
             fail("Should have caught bad char definition.");
         } catch (SQLException e) {
             assertTrue(e.getMessage(), e.getMessage().contains("ERROR 207 (22003): Missing length for CHAR. columnName=COL"));
@@ -496,9 +516,9 @@ public class QueryParserTest {
     @Test
     public void testBadVarcharDef() throws Exception {
         try {
-            SQLParser parser = new SQLParser("CREATE TABLE IF NOT EXISTS testBadVarcharDef" + 
+            String sql = ("CREATE TABLE IF NOT EXISTS testBadVarcharDef" + 
                     "  (pk VARCHAR NOT NULL PRIMARY KEY, col VARCHAR(0))");
-            parser.parseStatement();
+            parseQuery(sql);
             fail("Should have caught bad varchar definition.");
         } catch (SQLException e) {
             assertTrue(e.getMessage(), e.getMessage().contains("ERROR 208 (22003): CHAR or VARCHAR must have a positive length. columnName=COL"));
@@ -508,17 +528,17 @@ public class QueryParserTest {
     @Test
     public void testBadDecimalDef() throws Exception {
         try {
-            SQLParser parser = new SQLParser("CREATE TABLE IF NOT EXISTS testBadDecimalDef" + 
+            String sql = ("CREATE TABLE IF NOT EXISTS testBadDecimalDef" + 
                     "  (pk VARCHAR NOT NULL PRIMARY KEY, col DECIMAL(0, 5))");
-            parser.parseStatement();
+            parseQuery(sql);
             fail("Should have caught bad decimal definition.");
         } catch (SQLException e) {
             assertTrue(e.getMessage(), e.getMessage().contains("ERROR 209 (22003): Decimal precision outside of range. Should be within 1 and 38. columnName=COL"));
         }
         try {
-            SQLParser parser = new SQLParser("CREATE TABLE IF NOT EXISTS testBadDecimalDef" + 
+            String sql = ("CREATE TABLE IF NOT EXISTS testBadDecimalDef" + 
                     "  (pk VARCHAR NOT NULL PRIMARY KEY, col DECIMAL(40, 5))");
-            parser.parseStatement();
+            parseQuery(sql);
             fail("Should have caught bad decimal definition.");
         } catch (SQLException e) {
             assertTrue(e.getMessage(), e.getMessage().contains("ERROR 209 (22003): Decimal precision outside of range. Should be within 1 and 38. columnName=COL"));
@@ -528,17 +548,17 @@ public class QueryParserTest {
     @Test
     public void testBadBinaryDef() throws Exception {
         try {
-            SQLParser parser = new SQLParser("CREATE TABLE IF NOT EXISTS testBadBinaryDef" + 
+            String sql = ("CREATE TABLE IF NOT EXISTS testBadBinaryDef" + 
                     "  (pk VARCHAR NOT NULL PRIMARY KEY, col BINARY(0))");
-            parser.parseStatement();
+            parseQuery(sql);
             fail("Should have caught bad binary definition.");
         } catch (SQLException e) {
             assertTrue(e.getMessage(), e.getMessage().contains("ERROR 211 (22003): BINARY must have a positive length. columnName=COL"));
         }
         try {
-            SQLParser parser = new SQLParser("CREATE TABLE IF NOT EXISTS testBadVarcharDef" + 
+            String sql = ("CREATE TABLE IF NOT EXISTS testBadVarcharDef" + 
                     "  (pk VARCHAR NOT NULL PRIMARY KEY, col BINARY)");
-            parser.parseStatement();
+            parseQuery(sql);
             fail("Should have caught bad char definition.");
         } catch (SQLException e) {
             assertTrue(e.getMessage(), e.getMessage().contains("ERROR 210 (22003): Missing length for BINARY. columnName=COL"));
@@ -547,67 +567,67 @@ public class QueryParserTest {
 
     @Test
     public void testPercentileQuery1() throws Exception {
-        SQLParser parser = new SQLParser(
-                new StringReader(
+        String sql = (
+                (
                         "select PERCENTILE_CONT(0.9) WITHIN GROUP (ORDER BY salary DESC) from core.custom_index_value ind"));
-        parser.parseStatement();
+        parseQuery(sql);
     }
 
     @Test
     public void testPercentileQuery2() throws Exception {
-        SQLParser parser = new SQLParser(
-                new StringReader(
+        String sql = (
+                (
                         "select PERCENTILE_CONT(0.5) WITHIN GROUP (ORDER BY mark ASC) from core.custom_index_value ind"));
-        parser.parseStatement();
+        parseQuery(sql);
     }
     
     @Test
     public void testRowValueConstructorQuery() throws Exception {
-        SQLParser parser = new SQLParser(
-                new StringReader(
+        String sql = (
+                (
                         "select a_integer FROM aTable where (x_integer, y_integer) > (3, 4)"));
-        parser.parseStatement();
+        parseQuery(sql);
     }
 
     @Test
     public void testSingleTopLevelNot() throws Exception {
-        SQLParser parser = new SQLParser(
-                new StringReader(
+        String sql = (
+                (
                         "select * from t where not c = 5"));
-        parser.parseStatement();
+        parseQuery(sql);
     }
 
     @Test
     public void testTopLevelNot() throws Exception {
-        SQLParser parser = new SQLParser(
-                new StringReader(
+        String sql = (
+                (
                         "select * from t where not c"));
-        parser.parseStatement();
+        parseQuery(sql);
     }
 
     @Test
     public void testRVCInList() throws Exception {
-        SQLParser parser = new SQLParser(
-                new StringReader(
+        String sql = (
+                (
                         "select * from t where k in ( (1,2), (3,4) )"));
-        parser.parseStatement();
+        parseQuery(sql);
     }
 
     @Test
     public void testInList() throws Exception {
-        SQLParser parser = new SQLParser(
-                new StringReader(
+        String sql = (
+                (
                         "select * from t where k in ( 1,2 )"));
-        parser.parseStatement();
+        parseQuery(sql);
     }
 
     @Test
     public void testInvalidSelectStar() throws Exception {
-        SQLParser parser = new SQLParser(
-                new StringReader(
+        String sql = (
+                (
                         "select *,k from t where k in ( 1,2 )"));
         try {
-            parser.parseStatement();
+            parseQuery(sql);
             fail();
         } catch (SQLException e) {
             assertEquals(SQLExceptionCode.MISMATCHED_TOKEN.getErrorCode(), e.getErrorCode());
@@ -616,11 +636,11 @@ public class QueryParserTest {
 
     @Test
     public void testInvalidUpsertSelectHint() throws Exception {
-        SQLParser parser = new SQLParser(
-                new StringReader(
+        String sql = (
+                (
                         "upsert into t select /*+ NO_INDEX */ k from t where k in ( 1,2 )"));
         try {
-            parser.parseStatement();
+            parseQuery(sql);
             fail();
         } catch (SQLException e) {
             assertEquals(SQLExceptionCode.PARSER_ERROR.getErrorCode(), e.getErrorCode());
@@ -629,11 +649,11 @@ public class QueryParserTest {
 
     @Test
     public void testTableNameStartsWithUnderscore() throws Exception {
-        SQLParser parser = new SQLParser(
-                new StringReader(
+        String sql = (
+                (
                         "select* from _t where k in ( 1,2 )"));
         try {
-            parser.parseStatement();
+            parseQuery(sql);
             fail();
         } catch (SQLException e) {
             assertEquals(SQLExceptionCode.PARSER_ERROR.getErrorCode(), e.getErrorCode());
@@ -642,16 +662,16 @@ public class QueryParserTest {
 
     @Test
     public void testValidUpsertSelectHint() throws Exception {
-        SQLParser parser = new SQLParser(
-                new StringReader(
+        String sql = (
+                (
                         "upsert /*+ NO_INDEX */ into t select k from t where k in ( 1,2 )"));
-            parser.parseStatement();
+            parseQuery(sql);
     }
 
     @Test
     public void testHavingWithNot() throws Exception {
-        SQLParser parser = new SQLParser(
-                new StringReader(
+        String sql = (
+                (
                         "select\n" + 
                         "\"WEB_STAT_ALIAS\".\"DOMAIN\" as \"c0\"\n" + 
                         "from \"WEB_STAT\" \"WEB_STAT_ALIAS\"\n" + 
@@ -668,73 +688,73 @@ public class QueryParserTest {
                         ")\n" + 
                         "order by CASE WHEN \"WEB_STAT_ALIAS\".\"DOMAIN\" IS NULL THEN 1 ELSE 0 END,\n" + 
                         "\"WEB_STAT_ALIAS\".\"DOMAIN\" ASC"));
-        parser.parseStatement();
+        parseQuery(sql);
     }
 
     @Test
     public void testToDateInList() throws Exception {
-        SQLParser parser = new SQLParser(
-                new StringReader("select * from date_test where d in (to_date('2013-11-04 09:12:00'))"));
-        parser.parseStatement();
+        String sql = (
+                ("select * from date_test where d in (to_date('2013-11-04 09:12:00'))"));
+        parseQuery(sql);
     }
     
     @Test
     public void testDateLiteral() throws Exception {
-        SQLParser parser = new SQLParser(
-                new StringReader(
+        String sql = (
+                (
                         "select * from t where d = DATE '2013-11-04 09:12:00'"));
-        parser.parseStatement();
+        parseQuery(sql);
     }
 
     @Test
     public void testTimeLiteral() throws Exception {
-        SQLParser parser = new SQLParser(
-                new StringReader(
+        String sql = (
+                (
                         "select * from t where d = TIME '2013-11-04 09:12:00'"));
-        parser.parseStatement();
+        parseQuery(sql);
     }
 
 
     @Test
     public void testTimestampLiteral() throws Exception {
-        SQLParser parser = new SQLParser(
-                new StringReader(
+        String sql = (
+                (
                         "select * from t where d = TIMESTAMP '2013-11-04 09:12:00'"));
-        parser.parseStatement();
+        parseQuery(sql);
     }
     
     @Test
     public void testUnsignedDateLiteral() throws Exception {
-        SQLParser parser = new SQLParser(
-                new StringReader(
+        String sql = (
+                (
                         "select * from t where d = UNSIGNED_DATE '2013-11-04 09:12:00'"));
-        parser.parseStatement();
+        parseQuery(sql);
     }
 
     @Test
     public void testUnsignedTimeLiteral() throws Exception {
-        SQLParser parser = new SQLParser(
-                new StringReader(
+        String sql = (
+                (
                         "select * from t where d = UNSIGNED_TIME '2013-11-04 09:12:00'"));
-        parser.parseStatement();
+        parseQuery(sql);
     }
 
 
     @Test
     public void testUnsignedTimestampLiteral() throws Exception {
-        SQLParser parser = new SQLParser(
-                new StringReader(
+        String sql = (
+                (
                         "select * from t where d = UNSIGNED_TIMESTAMP '2013-11-04 09:12:00'"));
-        parser.parseStatement();
+        parseQuery(sql);
     }
     
     @Test
     public void testUnknownLiteral() throws Exception {
-        SQLParser parser = new SQLParser(
-                new StringReader(
+        String sql = (
+                (
                         "select * from t where d = FOO '2013-11-04 09:12:00'"));
         try {
-            parser.parseStatement();
+            parseQuery(sql);
             fail();
         } catch (SQLException e) {
             assertEquals(SQLExceptionCode.ILLEGAL_DATA.getErrorCode(), e.getErrorCode());
@@ -743,14 +763,32 @@ public class QueryParserTest {
     
     @Test
     public void testUnsupportedLiteral() throws Exception {
-        SQLParser parser = new SQLParser(
-                new StringReader(
+        String sql = (
+                (
                         "select * from t where d = DECIMAL '2013-11-04 09:12:00'"));
         try {
-            parser.parseStatement();
+            parseQuery(sql);
             fail();
         } catch (SQLException e) {
             assertEquals(SQLExceptionCode.TYPE_MISMATCH.getErrorCode(), e.getErrorCode());
         }
     }
+    
+    @Test
+    public void testAnyElementExpression1() throws Exception {
+        String sql = "select * from t where 'a' = ANY(a)";
+        parseQuery(sql);
+    }
+
+    @Test
+    public void testAnyElementExpression2() throws Exception {
+        String sql = "select * from t where 'a' <= ANY(a-b+1)";
+        parseQuery(sql);
+    }
+
+    @Test
+    public void testAllElementExpression() throws Exception {
+        String sql = "select * from t where 'a' <= ALL(a-b+1)";
+        parseQuery(sql);
+    }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/92298f8d/phoenix-core/src/test/java/org/apache/phoenix/query/BaseConnectionlessQueryTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/query/BaseConnectionlessQueryTest.java b/phoenix-core/src/test/java/org/apache/phoenix/query/BaseConnectionlessQueryTest.java
index 8f17a7c..abaaeb5 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/query/BaseConnectionlessQueryTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/query/BaseConnectionlessQueryTest.java
@@ -32,17 +32,23 @@ import static org.apache.phoenix.util.TestUtil.PTSDB3_NAME;
 import static org.apache.phoenix.util.TestUtil.PTSDB_NAME;
 import static org.apache.phoenix.util.TestUtil.TABLE_WITH_ARRAY;
 import static org.apache.phoenix.util.TestUtil.TEST_PROPERTIES;
+import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
 
 import java.sql.DriverManager;
+import java.sql.SQLException;
 import java.util.Properties;
 
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.phoenix.expression.Expression;
 import org.apache.phoenix.jdbc.PhoenixConnection;
 import org.apache.phoenix.jdbc.PhoenixEmbeddedDriver;
+import org.apache.phoenix.jdbc.PhoenixStatement.Operation;
 import org.apache.phoenix.jdbc.PhoenixTestDriver;
+import org.apache.phoenix.parse.BindableStatement;
+import org.apache.phoenix.parse.SQLParser;
 import org.apache.phoenix.schema.ColumnRef;
 import org.apache.phoenix.schema.PTable;
 import org.apache.phoenix.schema.PTableKey;
@@ -146,4 +152,21 @@ public class BaseConnectionlessQueryTest extends BaseTest {
         }
     }
 
+    protected static void assertRoundtrip(String sql) throws SQLException {
+        SQLParser parser = new SQLParser(sql);
+        BindableStatement stmt = null;
+        stmt = parser.parseStatement();
+        if (stmt.getOperation() != Operation.QUERY) {
+            return;
+        }
+        String newSQL = stmt.toString();
+        SQLParser newParser = new SQLParser(newSQL);
+        BindableStatement newStmt = null;
+        try {
+            newStmt = newParser.parseStatement();
+        } catch (SQLException e) {
+            fail("Unable to parse new:\n" + newSQL);
+        }
+        assertEquals("Expected equality:\n" + sql + "\n" + newSQL, stmt, newStmt);
+    }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/92298f8d/phoenix-core/src/test/java/org/apache/phoenix/schema/types/PDataTypeTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/schema/types/PDataTypeTest.java b/phoenix-core/src/test/java/org/apache/phoenix/schema/types/PDataTypeTest.java
index 321567c..90730bc 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/schema/types/PDataTypeTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/schema/types/PDataTypeTest.java
@@ -1746,4 +1746,11 @@ public class PDataTypeTest {
              coercibleToMap.toString());
     }
     
+    @Test
+    public void testIntVersusLong() {
+        long l = -1L;
+        int i = -1;
+        assertTrue(PLong.INSTANCE.compareTo(l, i, PInteger.INSTANCE)==0);
+        assertTrue(PInteger.INSTANCE.compareTo(i, l, PLong.INSTANCE)==0);
+    }
 }


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

Posted by ja...@apache.org.
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/92298f8d
Tree: http://git-wip-us.apache.org/repos/asf/phoenix/tree/92298f8d
Diff: http://git-wip-us.apache.org/repos/asf/phoenix/diff/92298f8d

Branch: refs/heads/4.0
Commit: 92298f8d5d646ae3bbdd19767136b0fa3921af59
Parents: 8982d8d
Author: James Taylor <jt...@salesforce.com>
Authored: Mon Feb 9 16:36:34 2015 -0800
Committer: James Taylor <jt...@salesforce.com>
Committed: Tue Feb 10 08:12:38 2015 -0800

----------------------------------------------------------------------
 .../apache/phoenix/end2end/DerivedTableIT.java  |   6 +-
 .../org/apache/phoenix/end2end/HashJoinIT.java  |  30 +-
 .../phoenix/end2end/HashJoinLocalIndexIT.java   |   2 +-
 .../org/apache/phoenix/end2end/SubqueryIT.java  |  16 +-
 .../end2end/SubqueryUsingSortMergeJoinIT.java   |  24 +-
 .../java/org/apache/phoenix/end2end/ViewIT.java |  29 +-
 .../end2end/index/BaseMutableIndexIT.java       |   6 +-
 .../index/GlobalIndexOptimizationIT.java        |   2 +-
 .../phoenix/end2end/index/LocalIndexIT.java     |   4 +-
 .../phoenix/compile/CreateTableCompiler.java    |   4 +-
 .../phoenix/compile/ExpressionCompiler.java     |  22 +-
 .../phoenix/compile/IndexStatementRewriter.java |   5 +-
 .../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 +
 80 files changed, 1405 insertions(+), 419 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/92298f8d/phoenix-core/src/it/java/org/apache/phoenix/end2end/DerivedTableIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/DerivedTableIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/DerivedTableIT.java
index 562a98e..187025a 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/DerivedTableIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/DerivedTableIT.java
@@ -91,15 +91,15 @@ public class DerivedTableIT extends BaseClientManagedTimeIT {
                 "CREATE INDEX ATABLE_DERIVED_IDX ON aTable (a_byte) INCLUDE (A_STRING, B_STRING)" 
                 }, {
                 "CLIENT PARALLEL 1-WAY FULL SCAN OVER ATABLE_DERIVED_IDX\n" +
-                "    SERVER AGGREGATE INTO DISTINCT ROWS BY [A_STRING, B_STRING]\n" +
+                "    SERVER AGGREGATE INTO DISTINCT ROWS BY [\"A_STRING\", \"B_STRING\"]\n" +
                 "CLIENT MERGE SORT\n" +
-                "CLIENT SORTED BY [B_STRING]\n" +
+                "CLIENT SORTED BY [\"B_STRING\"]\n" +
                 "CLIENT SORTED BY [A]\n" +
                 "CLIENT AGGREGATE INTO DISTINCT ROWS BY [A]\n" +
                 "CLIENT SORTED BY [A DESC]",
                 
                 "CLIENT PARALLEL 1-WAY FULL SCAN OVER ATABLE_DERIVED_IDX\n" +
-                "    SERVER AGGREGATE INTO DISTINCT ROWS BY [A_STRING, B_STRING]\n" +
+                "    SERVER AGGREGATE INTO DISTINCT ROWS BY [\"A_STRING\", \"B_STRING\"]\n" +
                 "CLIENT MERGE SORT\n" +
                 "CLIENT AGGREGATE INTO DISTINCT ROWS BY [A]\n" +
                 "CLIENT DISTINCT ON [COLLECTDISTINCT(B)]"}});

http://git-wip-us.apache.org/repos/asf/phoenix/blob/92298f8d/phoenix-core/src/it/java/org/apache/phoenix/end2end/HashJoinIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/HashJoinIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/HashJoinIT.java
index 76eab22..5d2f522 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/HashJoinIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/HashJoinIT.java
@@ -478,9 +478,9 @@ public class HashJoinIT extends BaseHBaseManagedTimeIT {
                  *     GROUP BY i.name ORDER BY i.name
                  */     
                 "CLIENT PARALLEL 1-WAY FULL SCAN OVER " + JOIN_ORDER_TABLE_DISPLAY_NAME + "\n" +
-                "    SERVER AGGREGATE INTO DISTINCT ROWS BY [I.0:NAME]\n" +
+                "    SERVER AGGREGATE INTO DISTINCT ROWS BY [\"I.0:NAME\"]\n" +
                 "CLIENT MERGE SORT\n" +
-                "CLIENT SORTED BY [I.0:NAME]\n" +
+                "CLIENT SORTED BY [\"I.0:NAME\"]\n" +
                 "    PARALLEL LEFT-JOIN TABLE 0\n" +
                 "        CLIENT PARALLEL 1-WAY FULL SCAN OVER " + JOIN_SCHEMA + ".idx_item\n" +
                 "            SERVER FILTER BY FIRST KEY ONLY",
@@ -518,7 +518,7 @@ public class HashJoinIT extends BaseHBaseManagedTimeIT {
                  */
                 "CLIENT PARALLEL 1-WAY FULL SCAN OVER " + JOIN_SCHEMA + ".idx_item\n" +
                 "    SERVER FILTER BY FIRST KEY ONLY\n" +
-                "    SERVER AGGREGATE INTO ORDERED DISTINCT ROWS BY [I.0:NAME]\n" +
+                "    SERVER AGGREGATE INTO ORDERED DISTINCT ROWS BY [\"I.0:NAME\"]\n" +
                 "CLIENT MERGE SORT\n" +
                 "    PARALLEL LEFT-JOIN TABLE 0\n" +
                 "        CLIENT PARALLEL 1-WAY FULL SCAN OVER " + JOIN_ORDER_TABLE_DISPLAY_NAME,
@@ -601,7 +601,7 @@ public class HashJoinIT extends BaseHBaseManagedTimeIT {
                  */
                 "CLIENT PARALLEL 1-WAY FULL SCAN OVER " + JOIN_SCHEMA + ".idx_item\n" +
                 "    SERVER FILTER BY FIRST KEY ONLY\n" +
-                "    SERVER SORTED BY [I1.0:NAME, I2.0:NAME]\n" +
+                "    SERVER SORTED BY [\"I1.0:NAME\", \"I2.0:NAME\"]\n" +
                 "CLIENT MERGE SORT\n" +
                 "    PARALLEL INNER-JOIN TABLE 0\n" +
                 "        CLIENT PARALLEL 1-WAY FULL SCAN OVER " + JOIN_SCHEMA + ".idx_item",
@@ -651,14 +651,14 @@ public class HashJoinIT extends BaseHBaseManagedTimeIT {
                  *     ORDER BY c.customer_id, i.name
                  */
                 "CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + JOIN_CUSTOMER_TABLE_DISPLAY_NAME + " [*] - ['0000000005']\n" +
-                "    SERVER SORTED BY [\"C.customer_id\", I.0:NAME]\n" +
+                "    SERVER SORTED BY [\"C.customer_id\", \"I.0:NAME\"]\n" +
                 "CLIENT MERGE SORT\n" +
                 "    PARALLEL INNER-JOIN TABLE 0\n" +
                 "        CLIENT PARALLEL 1-WAY FULL SCAN OVER " + JOIN_ORDER_TABLE_DISPLAY_NAME + "\n" +
                 "            SERVER FILTER BY \"order_id\" != '000000000000003'\n" +
                 "            PARALLEL INNER-JOIN TABLE 0\n" +
                 "                CLIENT PARALLEL 1-WAY FULL SCAN OVER " + JOIN_SCHEMA + ".idx_item\n" +
-                "                    SERVER FILTER BY NAME != 'T3'\n" +
+                "                    SERVER FILTER BY \"NAME\" != 'T3'\n" +
                 "                    PARALLEL LEFT-JOIN TABLE 0\n" +
                 "                        CLIENT PARALLEL 1-WAY FULL SCAN OVER " + JOIN_SUPPLIER_TABLE_DISPLAY_NAME + "\n" +
                 "    DYNAMIC SERVER FILTER BY \"customer_id\" IN (\"O.customer_id\")",
@@ -750,7 +750,7 @@ public class HashJoinIT extends BaseHBaseManagedTimeIT {
                 "            SERVER FILTER BY \"order_id\" != '000000000000003'\n" +
                 "            PARALLEL INNER-JOIN TABLE 0\n" +
                 "                CLIENT PARALLEL 1-WAY FULL SCAN OVER " + JOIN_SCHEMA + ".idx_item\n" +
-                "                    SERVER FILTER BY NAME != 'T3'\n" +
+                "                    SERVER FILTER BY \"NAME\" != 'T3'\n" +
                 "                    PARALLEL LEFT-JOIN TABLE 0\n" +
                 "                        CLIENT PARALLEL 1-WAY FULL SCAN OVER " + JOIN_SUPPLIER_TABLE_DISPLAY_NAME,
                 /*
@@ -845,9 +845,9 @@ public class HashJoinIT extends BaseHBaseManagedTimeIT {
                  *     GROUP BY i.name ORDER BY i.name
                  */     
                 "CLIENT PARALLEL 1-WAY FULL SCAN OVER " + JOIN_ORDER_TABLE_DISPLAY_NAME + "\n" +
-                "    SERVER AGGREGATE INTO DISTINCT ROWS BY [I.0:NAME]\n" +
+                "    SERVER AGGREGATE INTO DISTINCT ROWS BY [\"I.0:NAME\"]\n" +
                 "CLIENT MERGE SORT\n" +
-                "CLIENT SORTED BY [I.0:NAME]\n" +
+                "CLIENT SORTED BY [\"I.0:NAME\"]\n" +
                 "    PARALLEL LEFT-JOIN TABLE 0\n" +
                 "        CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + MetaDataUtil.LOCAL_INDEX_TABLE_PREFIX + "" + JOIN_ITEM_TABLE_DISPLAY_NAME +" [-32768]\n" +
                 "            SERVER FILTER BY FIRST KEY ONLY\n" +
@@ -887,9 +887,9 @@ public class HashJoinIT extends BaseHBaseManagedTimeIT {
                  */
                 "CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + MetaDataUtil.LOCAL_INDEX_TABLE_PREFIX + "" + JOIN_ITEM_TABLE_DISPLAY_NAME+" [-32768]\n" +
                 "    SERVER FILTER BY FIRST KEY ONLY\n" +
-                "    SERVER AGGREGATE INTO DISTINCT ROWS BY [I.0:NAME]\n" +
+                "    SERVER AGGREGATE INTO DISTINCT ROWS BY [\"I.0:NAME\"]\n" +
                 "CLIENT MERGE SORT\n" +
-                "CLIENT SORTED BY [I.0:NAME]\n" +
+                "CLIENT SORTED BY [\"I.0:NAME\"]\n" +
                 "    PARALLEL LEFT-JOIN TABLE 0\n" +
                 "        CLIENT PARALLEL 1-WAY FULL SCAN OVER " + JOIN_ORDER_TABLE_DISPLAY_NAME,
                 /*
@@ -979,7 +979,7 @@ public class HashJoinIT extends BaseHBaseManagedTimeIT {
                  */
                 "CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + MetaDataUtil.LOCAL_INDEX_TABLE_PREFIX +""+ JOIN_ITEM_TABLE_DISPLAY_NAME +" [-32768]\n"  +
                 "    SERVER FILTER BY FIRST KEY ONLY\n" +
-                "    SERVER SORTED BY [I1.0:NAME, I2.0:NAME]\n" +
+                "    SERVER SORTED BY [\"I1.0:NAME\", \"I2.0:NAME\"]\n" +
                 "CLIENT MERGE SORT\n" +
                 "    PARALLEL INNER-JOIN TABLE 0\n" +
                 "        CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + MetaDataUtil.LOCAL_INDEX_TABLE_PREFIX +""+ JOIN_ITEM_TABLE_DISPLAY_NAME +" [-32768]\n" +
@@ -1035,14 +1035,14 @@ public class HashJoinIT extends BaseHBaseManagedTimeIT {
                  *     ORDER BY c.customer_id, i.name
                  */
                 "CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + JOIN_CUSTOMER_TABLE_DISPLAY_NAME + " [*] - ['0000000005']\n" +
-                "    SERVER SORTED BY [\"C.customer_id\", I.0:NAME]\n"+
+                "    SERVER SORTED BY [\"C.customer_id\", \"I.0:NAME\"]\n"+
                 "CLIENT MERGE SORT\n" +
                 "    PARALLEL INNER-JOIN TABLE 0\n" +
                 "        CLIENT PARALLEL 1-WAY FULL SCAN OVER " + JOIN_ORDER_TABLE_DISPLAY_NAME + "\n" +
                 "            SERVER FILTER BY \"order_id\" != '000000000000003'\n" +
                 "            PARALLEL INNER-JOIN TABLE 0\n" +
                 "                CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + MetaDataUtil.LOCAL_INDEX_TABLE_PREFIX +""+ JOIN_ITEM_TABLE_DISPLAY_NAME +" [-32768]\n" +
-                "                    SERVER FILTER BY NAME != 'T3'\n" +
+                "                    SERVER FILTER BY \"NAME\" != 'T3'\n" +
                 "                CLIENT MERGE SORT\n" +
                 "                    PARALLEL LEFT-JOIN TABLE 0\n" +
                 "                        CLIENT PARALLEL 1-WAY FULL SCAN OVER " + JOIN_SUPPLIER_TABLE_DISPLAY_NAME + "\n" +
@@ -1137,7 +1137,7 @@ public class HashJoinIT extends BaseHBaseManagedTimeIT {
                 "            SERVER FILTER BY \"order_id\" != '000000000000003'\n" +
                 "            PARALLEL INNER-JOIN TABLE 0\n" +
                 "                CLIENT PARALLEL 1-WAY RANGE SCAN OVER " +  MetaDataUtil.LOCAL_INDEX_TABLE_PREFIX +""+JOIN_ITEM_TABLE_DISPLAY_NAME + " [-32768]\n" +
-                "                    SERVER FILTER BY NAME != 'T3'\n" +
+                "                    SERVER FILTER BY \"NAME\" != 'T3'\n" +
                 "                CLIENT MERGE SORT\n" +      
                 "                    PARALLEL LEFT-JOIN TABLE 0\n" +
                 "                        CLIENT PARALLEL 1-WAY FULL SCAN OVER " + JOIN_SUPPLIER_TABLE_DISPLAY_NAME,

http://git-wip-us.apache.org/repos/asf/phoenix/blob/92298f8d/phoenix-core/src/it/java/org/apache/phoenix/end2end/HashJoinLocalIndexIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/HashJoinLocalIndexIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/HashJoinLocalIndexIT.java
index 45e80c6..2d0cc72 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/HashJoinLocalIndexIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/HashJoinLocalIndexIT.java
@@ -109,7 +109,7 @@ public class HashJoinLocalIndexIT extends BaseHBaseManagedTimeIT {
                 
                 "CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + MetaDataUtil.LOCAL_INDEX_TABLE_PREFIX + JOIN_SUPPLIER_TABLE_DISPLAY_NAME + " [-32768,'S1']\n" +
                 "    SERVER FILTER BY FIRST KEY ONLY\n" + 
-                "    SERVER AGGREGATE INTO DISTINCT ROWS BY [S.0:PHONE]\n" +
+                "    SERVER AGGREGATE INTO DISTINCT ROWS BY [\"S.0:PHONE\"]\n" +
                 "CLIENT MERGE SORT\n" +
                 "    PARALLEL INNER-JOIN TABLE 0\n" +
                 "        CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + MetaDataUtil.LOCAL_INDEX_TABLE_PREFIX + JOIN_ITEM_TABLE_DISPLAY_NAME + " [-32768,*] - [-32768,'T6']\n" +

http://git-wip-us.apache.org/repos/asf/phoenix/blob/92298f8d/phoenix-core/src/it/java/org/apache/phoenix/end2end/SubqueryIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SubqueryIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SubqueryIT.java
index 2d11c5c..7bc97e7 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SubqueryIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SubqueryIT.java
@@ -186,7 +186,7 @@ public class SubqueryIT extends BaseHBaseManagedTimeIT {
                 
                 "CLIENT PARALLEL 1-WAY FULL SCAN OVER " + JOIN_SCHEMA + ".idx_supplier\n" +
                 "    SERVER FILTER BY FIRST KEY ONLY\n" + 
-                "    SERVER SORTED BY [I.0:NAME]\n" +
+                "    SERVER SORTED BY [\"I.0:NAME\"]\n" +
                 "CLIENT MERGE SORT\n" +
                 "    PARALLEL LEFT-JOIN TABLE 0\n" +
                 "        CLIENT PARALLEL 1-WAY FULL SCAN OVER " + JOIN_SCHEMA + ".idx_item\n" +
@@ -200,7 +200,7 @@ public class SubqueryIT extends BaseHBaseManagedTimeIT {
                 "    PARALLEL LEFT-JOIN TABLE 0\n" +
                 "        CLIENT PARALLEL 1-WAY FULL SCAN OVER " + JOIN_SCHEMA + ".idx_item\n" +
                 "            SERVER FILTER BY FIRST KEY ONLY\n" +
-                "            SERVER AGGREGATE INTO ORDERED DISTINCT ROWS BY \\[NAME, \"item_id\"\\]\n" +
+                "            SERVER AGGREGATE INTO ORDERED DISTINCT ROWS BY \\[\"NAME\", \"item_id\"\\]\n" +
                 "        CLIENT MERGE SORT\n" +
                 "            PARALLEL ANTI-JOIN TABLE 0 \\(SKIP MERGE\\)\n" +
                 "                CLIENT PARALLEL 1-WAY FULL SCAN OVER " + JOIN_ORDER_TABLE_DISPLAY_NAME + "\n" +
@@ -209,7 +209,7 @@ public class SubqueryIT extends BaseHBaseManagedTimeIT {
                 "    PARALLEL LEFT-JOIN TABLE 1\n" +
                 "        CLIENT PARALLEL 1-WAY FULL SCAN OVER " + JOIN_SCHEMA + ".idx_item\n" +
                 "            SERVER FILTER BY FIRST KEY ONLY\n" +
-                "            SERVER AGGREGATE INTO ORDERED DISTINCT ROWS BY \\[NAME, \"item_id\"\\]\n" +
+                "            SERVER AGGREGATE INTO ORDERED DISTINCT ROWS BY \\[\"NAME\", \"item_id\"\\]\n" +
                 "        CLIENT MERGE SORT\n" +
                 "            PARALLEL SEMI-JOIN TABLE 0 \\(SKIP MERGE\\)\n" +
                 "                CLIENT PARALLEL 1-WAY FULL SCAN OVER " + JOIN_ORDER_TABLE_DISPLAY_NAME + "\n" +
@@ -237,7 +237,7 @@ public class SubqueryIT extends BaseHBaseManagedTimeIT {
                 "                CLIENT PARALLEL 1-WAY FULL SCAN OVER " + JOIN_ORDER_TABLE_DISPLAY_NAME + "\n" +
                 "                    SERVER AGGREGATE INTO DISTINCT ROWS BY \\[\"item_id\"\\]\n" +
                 "                CLIENT MERGE SORT\n" +
-                "            AFTER-JOIN SERVER FILTER BY \\(I.0:NAME = 'T2' OR O.QUANTITY > \\$\\d+.\\$\\d+\\)"
+                "            AFTER-JOIN SERVER FILTER BY \\(\"I.0:NAME\" = 'T2' OR O.QUANTITY > \\$\\d+.\\$\\d+\\)"
                 }});
         testCases.add(new String[][] {
                 {
@@ -259,7 +259,7 @@ public class SubqueryIT extends BaseHBaseManagedTimeIT {
                             
                 "CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + MetaDataUtil.LOCAL_INDEX_TABLE_PREFIX + JOIN_SUPPLIER_TABLE_DISPLAY_NAME + " [-32768]\n" +
                 "    SERVER FILTER BY FIRST KEY ONLY\n" + 
-                "    SERVER SORTED BY [I.0:NAME]\n" +
+                "    SERVER SORTED BY [\"I.0:NAME\"]\n" +
                 "CLIENT MERGE SORT\n" +
                 "    PARALLEL LEFT-JOIN TABLE 0\n" +
                 "        CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + MetaDataUtil.LOCAL_INDEX_TABLE_PREFIX + JOIN_ITEM_TABLE_DISPLAY_NAME + " [-32768]\n" +
@@ -274,7 +274,7 @@ public class SubqueryIT extends BaseHBaseManagedTimeIT {
                 "    PARALLEL LEFT-JOIN TABLE 0\n" +
                 "        CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + MetaDataUtil.LOCAL_INDEX_TABLE_PREFIX + JOIN_ITEM_TABLE_DISPLAY_NAME + " \\[-32768\\]\n" +
                 "            SERVER FILTER BY FIRST KEY ONLY\n" +
-                "            SERVER AGGREGATE INTO ORDERED DISTINCT ROWS BY \\[NAME, \"item_id\"\\]\n" +
+                "            SERVER AGGREGATE INTO ORDERED DISTINCT ROWS BY \\[\"NAME\", \"item_id\"\\]\n" +
                 "        CLIENT MERGE SORT\n" +
                 "            PARALLEL ANTI-JOIN TABLE 0 \\(SKIP MERGE\\)\n" +
                 "                CLIENT PARALLEL 1-WAY FULL SCAN OVER " + JOIN_ORDER_TABLE_DISPLAY_NAME + "\n" +
@@ -283,7 +283,7 @@ public class SubqueryIT extends BaseHBaseManagedTimeIT {
                 "    PARALLEL LEFT-JOIN TABLE 1\n" +
                 "        CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + MetaDataUtil.LOCAL_INDEX_TABLE_PREFIX + JOIN_ITEM_TABLE_DISPLAY_NAME + " \\[-32768\\]\n" +
                 "            SERVER FILTER BY FIRST KEY ONLY\n" +
-                "            SERVER AGGREGATE INTO ORDERED DISTINCT ROWS BY \\[NAME, \"item_id\"\\]\n" +
+                "            SERVER AGGREGATE INTO ORDERED DISTINCT ROWS BY \\[\"NAME\", \"item_id\"\\]\n" +
                 "        CLIENT MERGE SORT\n" +
                 "            PARALLEL SEMI-JOIN TABLE 0 \\(SKIP MERGE\\)\n" +
                 "                CLIENT PARALLEL 1-WAY FULL SCAN OVER " + JOIN_ORDER_TABLE_DISPLAY_NAME + "\n" +
@@ -315,7 +315,7 @@ public class SubqueryIT extends BaseHBaseManagedTimeIT {
                 "                    SERVER AGGREGATE INTO DISTINCT ROWS BY \\[\"item_id\"\\]\n" +
                 "                CLIENT MERGE SORT\n" +
                 "            DYNAMIC SERVER FILTER BY \"item_id\" IN \\(\"O.item_id\"\\)\n" +
-                "            AFTER-JOIN SERVER FILTER BY \\(I.0:NAME = 'T2' OR O.QUANTITY > \\$\\d+.\\$\\d+\\)\n" +
+                "            AFTER-JOIN SERVER FILTER BY \\(\"I.0:NAME\" = 'T2' OR O.QUANTITY > \\$\\d+.\\$\\d+\\)\n" +
                 "    DYNAMIC SERVER FILTER BY \"customer_id\" IN \\(\\$\\d+.\\$\\d+\\)"
                 }});
         return testCases;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/92298f8d/phoenix-core/src/it/java/org/apache/phoenix/end2end/SubqueryUsingSortMergeJoinIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SubqueryUsingSortMergeJoinIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SubqueryUsingSortMergeJoinIT.java
index 7457e02..f931bae 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SubqueryUsingSortMergeJoinIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SubqueryUsingSortMergeJoinIT.java
@@ -188,7 +188,7 @@ public class SubqueryUsingSortMergeJoinIT extends BaseHBaseManagedTimeIT {
                 "        SERVER AGGREGATE INTO DISTINCT ROWS BY [\"item_id\"]\n" +
                 "    CLIENT MERGE SORT\n" +
                 "    CLIENT SORTED BY [\"item_id\"]\n" +
-                "CLIENT SORTED BY [I.0:NAME]",
+                "CLIENT SORTED BY [\"I.0:NAME\"]",
 
                 "SORT-MERGE-JOIN \\(LEFT\\) TABLES\n" +
                 "    SORT-MERGE-JOIN \\(LEFT\\) TABLES\n" +
@@ -197,9 +197,9 @@ public class SubqueryUsingSortMergeJoinIT extends BaseHBaseManagedTimeIT {
                 "    AND\n" +
                 "        CLIENT PARALLEL 1-WAY FULL SCAN OVER " + JOIN_SCHEMA + ".idx_item\n" +
                 "            SERVER FILTER BY FIRST KEY ONLY\n" +
-                "            SERVER AGGREGATE INTO ORDERED DISTINCT ROWS BY \\[NAME, \"item_id\"\\]\n" +
+                "            SERVER AGGREGATE INTO ORDERED DISTINCT ROWS BY \\[\"NAME\", \"item_id\"\\]\n" +
                 "        CLIENT MERGE SORT\n" +
-                "        CLIENT SORTED BY \\[\"item_id\", NAME\\]\n" +
+                "        CLIENT SORTED BY \\[\"item_id\", \"NAME\"\\]\n" +
                 "            PARALLEL ANTI-JOIN TABLE 0 \\(SKIP MERGE\\)\n" +
                 "                CLIENT PARALLEL 1-WAY FULL SCAN OVER " + JOIN_ORDER_TABLE_DISPLAY_NAME + "\n" +
                 "                    SERVER AGGREGATE INTO DISTINCT ROWS BY \\[\"item_id\"\\]\n" +
@@ -208,9 +208,9 @@ public class SubqueryUsingSortMergeJoinIT extends BaseHBaseManagedTimeIT {
                 "AND\n" +
                 "    CLIENT PARALLEL 1-WAY FULL SCAN OVER " + JOIN_SCHEMA + ".idx_item\n" +
                 "        SERVER FILTER BY FIRST KEY ONLY\n" +
-                "        SERVER AGGREGATE INTO ORDERED DISTINCT ROWS BY \\[NAME, \"item_id\"\\]\n" +
+                "        SERVER AGGREGATE INTO ORDERED DISTINCT ROWS BY \\[\"NAME\", \"item_id\"\\]\n" +
                 "    CLIENT MERGE SORT\n" +
-                "    CLIENT SORTED BY \\[\"item_id\", NAME\\]\n" +
+                "    CLIENT SORTED BY \\[\"item_id\", \"NAME\"\\]\n" +
                 "        PARALLEL SEMI-JOIN TABLE 0 \\(SKIP MERGE\\)\n" +
                 "            CLIENT PARALLEL 1-WAY FULL SCAN OVER " + JOIN_ORDER_TABLE_DISPLAY_NAME + "\n" +
                 "                SERVER AGGREGATE INTO DISTINCT ROWS BY \\[\"item_id\"\\]\n" +
@@ -234,7 +234,7 @@ public class SubqueryUsingSortMergeJoinIT extends BaseHBaseManagedTimeIT {
                 "            CLIENT PARALLEL 1-WAY FULL SCAN OVER " + JOIN_ORDER_TABLE_DISPLAY_NAME + "\n" +
                 "                SERVER AGGREGATE INTO DISTINCT ROWS BY \\[\"item_id\"\\]\n" +
                 "            CLIENT MERGE SORT\n" +
-                "        AFTER-JOIN SERVER FILTER BY \\(I.0:NAME = 'T2' OR O.QUANTITY > \\$\\d+.\\$\\d+\\)",
+                "        AFTER-JOIN SERVER FILTER BY \\(\"I.0:NAME\" = 'T2' OR O.QUANTITY > \\$\\d+.\\$\\d+\\)",
                 }});
         testCases.add(new String[][] {
                 {
@@ -258,7 +258,7 @@ public class SubqueryUsingSortMergeJoinIT extends BaseHBaseManagedTimeIT {
                 "        SERVER AGGREGATE INTO DISTINCT ROWS BY [\"item_id\"]\n" +
                 "    CLIENT MERGE SORT\n" +
                 "    CLIENT SORTED BY [\"item_id\"]\n" +
-                "CLIENT SORTED BY [I.0:NAME]",
+                "CLIENT SORTED BY [\"I.0:NAME\"]",
 
                 "SORT-MERGE-JOIN \\(LEFT\\) TABLES\n" +
                 "    SORT-MERGE-JOIN \\(LEFT\\) TABLES\n" +
@@ -267,9 +267,9 @@ public class SubqueryUsingSortMergeJoinIT extends BaseHBaseManagedTimeIT {
                 "    AND\n" +
                 "        CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + MetaDataUtil.LOCAL_INDEX_TABLE_PREFIX + JOIN_ITEM_TABLE_DISPLAY_NAME + " \\[-32768\\]\n" +
                 "            SERVER FILTER BY FIRST KEY ONLY\n" +
-                "            SERVER AGGREGATE INTO ORDERED DISTINCT ROWS BY \\[NAME, \"item_id\"\\]\n" +
+                "            SERVER AGGREGATE INTO ORDERED DISTINCT ROWS BY \\[\"NAME\", \"item_id\"\\]\n" +
                 "        CLIENT MERGE SORT\n" +
-                "        CLIENT SORTED BY \\[\"item_id\", NAME\\]\n" +
+                "        CLIENT SORTED BY \\[\"item_id\", \"NAME\"\\]\n" +
                 "            PARALLEL ANTI-JOIN TABLE 0 \\(SKIP MERGE\\)\n" +
                 "                CLIENT PARALLEL 1-WAY FULL SCAN OVER " + JOIN_ORDER_TABLE_DISPLAY_NAME + "\n" +
                 "                    SERVER AGGREGATE INTO DISTINCT ROWS BY \\[\"item_id\"\\]\n" +
@@ -278,9 +278,9 @@ public class SubqueryUsingSortMergeJoinIT extends BaseHBaseManagedTimeIT {
                 "AND\n" +
                 "    CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + MetaDataUtil.LOCAL_INDEX_TABLE_PREFIX + JOIN_ITEM_TABLE_DISPLAY_NAME + " \\[-32768\\]\n" +
                 "        SERVER FILTER BY FIRST KEY ONLY\n" +
-                "        SERVER AGGREGATE INTO ORDERED DISTINCT ROWS BY \\[NAME, \"item_id\"\\]\n" +
+                "        SERVER AGGREGATE INTO ORDERED DISTINCT ROWS BY \\[\"NAME\", \"item_id\"\\]\n" +
                 "    CLIENT MERGE SORT\n" +
-                "    CLIENT SORTED BY \\[\"item_id\", NAME\\]\n" +
+                "    CLIENT SORTED BY \\[\"item_id\", \"NAME\"\\]\n" +
                 "        PARALLEL SEMI-JOIN TABLE 0 \\(SKIP MERGE\\)\n" +
                 "            CLIENT PARALLEL 1-WAY FULL SCAN OVER " + JOIN_ORDER_TABLE_DISPLAY_NAME + "\n" +
                 "                SERVER AGGREGATE INTO DISTINCT ROWS BY \\[\"item_id\"\\]\n" +
@@ -306,7 +306,7 @@ public class SubqueryUsingSortMergeJoinIT extends BaseHBaseManagedTimeIT {
                 "                SERVER AGGREGATE INTO DISTINCT ROWS BY \\[\"item_id\"\\]\n" +
                 "            CLIENT MERGE SORT\n" +
                 "        DYNAMIC SERVER FILTER BY \"item_id\" IN \\(\"O.item_id\"\\)\n" +
-                "        AFTER-JOIN SERVER FILTER BY \\(I.0:NAME = 'T2' OR O.QUANTITY > \\$\\d+.\\$\\d+\\)",
+                "        AFTER-JOIN SERVER FILTER BY \\(\"I.0:NAME\" = 'T2' OR O.QUANTITY > \\$\\d+.\\$\\d+\\)",
                 }});
         return testCases;
     }    

http://git-wip-us.apache.org/repos/asf/phoenix/blob/92298f8d/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..003db4c 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)";
@@ -424,6 +451,6 @@ public class ViewIT extends BaseViewIT {
         String queryPlan = QueryUtil.getExplainPlan(rs);
         assertEquals(
                 "CLIENT PARALLEL 1-WAY SKIP SCAN ON 4 KEYS OVER I1 [1,100] - [2,109]\n" + 
-                "    SERVER FILTER BY (S2 = 'bas' AND \"S1\" = 'foo')", queryPlan);
+                "    SERVER FILTER BY (\"S2\" = 'bas' AND \"S1\" = 'foo')", queryPlan);
     }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/92298f8d/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/BaseMutableIndexIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/BaseMutableIndexIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/BaseMutableIndexIT.java
index db1f6fb..b2f8630 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/BaseMutableIndexIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/BaseMutableIndexIT.java
@@ -314,7 +314,7 @@ public abstract class BaseMutableIndexIT extends BaseHBaseManagedTimeIT {
                 query = "SELECT b.* from " + TestUtil.DEFAULT_DATA_TABLE_FULL_NAME + " where int_col1 = 4";
                 rs = conn.createStatement().executeQuery("EXPLAIN " + query);
                 assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER _LOCAL_IDX_" + TestUtil.DEFAULT_DATA_TABLE_FULL_NAME +" [-32768]\n" +
-                		"    SERVER FILTER BY TO_INTEGER(INT_COL1) = 4\nCLIENT MERGE SORT", QueryUtil.getExplainPlan(rs));
+                		"    SERVER FILTER BY TO_INTEGER(\"INT_COL1\") = 4\nCLIENT MERGE SORT", QueryUtil.getExplainPlan(rs));
                 rs = conn.createStatement().executeQuery(query);
                 assertTrue(rs.next());
                 assertEquals("varchar_b", rs.getString(1));
@@ -391,11 +391,11 @@ public abstract class BaseMutableIndexIT extends BaseHBaseManagedTimeIT {
         rs = conn.createStatement().executeQuery("EXPLAIN " + query);
         if(localIndex){
             assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER _LOCAL_IDX_" +TestUtil.DEFAULT_DATA_TABLE_FULL_NAME + " [-32768,~'1']\n" + 
-                    "    SERVER SORTED BY [V1]\n" + 
+                    "    SERVER SORTED BY [\"V1\"]\n" + 
                     "CLIENT MERGE SORT", QueryUtil.getExplainPlan(rs));
         } else {
             assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER " +TestUtil.DEFAULT_INDEX_TABLE_FULL_NAME + " [~'1']\n" + 
-                    "    SERVER SORTED BY [V1]\n" + 
+                    "    SERVER SORTED BY [\"V1\"]\n" + 
                     "CLIENT MERGE SORT", QueryUtil.getExplainPlan(rs));
         }
 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/92298f8d/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/GlobalIndexOptimizationIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/GlobalIndexOptimizationIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/GlobalIndexOptimizationIT.java
index e54e6a2..e4ba2c6 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/GlobalIndexOptimizationIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/GlobalIndexOptimizationIT.java
@@ -322,7 +322,7 @@ public class GlobalIndexOptimizationIT extends BaseHBaseManagedTimeIT {
                     "    SERVER FILTER BY V1 = 'a'\n" +
                     "    SKIP-SCAN-JOIN TABLE 0\n" +
                     "        CLIENT PARALLEL 1-WAY SKIP SCAN ON 2 KEYS OVER _IDX_T \\[-32768,1\\] - \\[-32768,2\\]\n" +
-                    "            SERVER FILTER BY FIRST KEY ONLY AND K2 IN \\(3,4\\)\n" +
+                    "            SERVER FILTER BY FIRST KEY ONLY AND \"K2\" IN \\(3,4\\)\n" +
                     "    DYNAMIC SERVER FILTER BY \\(\"T_ID\", \"K1\", \"K2\"\\) IN \\(\\(\\$\\d+.\\$\\d+, \\$\\d+.\\$\\d+, \\$\\d+.\\$\\d+\\)\\)";
             assertTrue("Expected:\n" + expected + "\ndid not match\n" + actual, Pattern.matches(expected,actual));
             

http://git-wip-us.apache.org/repos/asf/phoenix/blob/92298f8d/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/LocalIndexIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/LocalIndexIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/LocalIndexIT.java
index 6ff0475..4080730 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/LocalIndexIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/LocalIndexIT.java
@@ -471,7 +471,7 @@ public class LocalIndexIT extends BaseHBaseManagedTimeIT {
                 "CLIENT PARALLEL " + numRegions + "-WAY RANGE SCAN OVER "
                         + MetaDataUtil.getLocalIndexTableName(TestUtil.DEFAULT_DATA_TABLE_NAME)+" [-32768,*] - [-32768,'z']\n"
                         + "    SERVER FILTER BY FIRST KEY ONLY\n"
-                        + "    SERVER AGGREGATE INTO DISTINCT ROWS BY [V1, T_ID, K3]\n" + "CLIENT MERGE SORT",
+                        + "    SERVER AGGREGATE INTO DISTINCT ROWS BY [\"V1\", \"T_ID\", K3]\n" + "CLIENT MERGE SORT",
                 QueryUtil.getExplainPlan(rs));
             
             rs = conn1.createStatement().executeQuery(query);
@@ -499,7 +499,7 @@ public class LocalIndexIT extends BaseHBaseManagedTimeIT {
                 "CLIENT PARALLEL " + numRegions + "-WAY RANGE SCAN OVER "
                         + MetaDataUtil.getLocalIndexTableName(TestUtil.DEFAULT_DATA_TABLE_NAME)+" [-32768,*] - [-32768,'z']\n"
                         + "    SERVER FILTER BY FIRST KEY ONLY\n"
-                        + "    SERVER AGGREGATE INTO ORDERED DISTINCT ROWS BY [V1]\nCLIENT MERGE SORT",
+                        + "    SERVER AGGREGATE INTO ORDERED DISTINCT ROWS BY [\"V1\"]\nCLIENT MERGE SORT",
                 QueryUtil.getExplainPlan(rs));
             
             PhoenixStatement stmt = conn1.createStatement().unwrap(PhoenixStatement.class);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/92298f8d/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/92298f8d/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/92298f8d/phoenix-core/src/main/java/org/apache/phoenix/compile/IndexStatementRewriter.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/IndexStatementRewriter.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/IndexStatementRewriter.java
index 4c66dd7..6f2cbfa 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/IndexStatementRewriter.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/IndexStatementRewriter.java
@@ -34,8 +34,8 @@ import org.apache.phoenix.parse.TableWildcardParseNode;
 import org.apache.phoenix.parse.WildcardParseNode;
 import org.apache.phoenix.schema.ColumnRef;
 import org.apache.phoenix.schema.PColumn;
-import org.apache.phoenix.schema.types.PDataType;
 import org.apache.phoenix.schema.TableRef;
+import org.apache.phoenix.schema.types.PDataType;
 import org.apache.phoenix.util.IndexUtil;
 
 public class IndexStatementRewriter extends ParseNodeRewriter {
@@ -107,8 +107,7 @@ public class IndexStatementRewriter extends ParseNodeRewriter {
             return node;
 
         String indexColName = IndexUtil.getIndexColumnName(dataCol);
-        // FIXME: why isn't this always case sensitive?
-        ParseNode indexColNode = new ColumnParseNode(tName, node.isCaseSensitive() ? '"' + indexColName + '"' : indexColName, node.getAlias());
+        ParseNode indexColNode = new ColumnParseNode(tName, '"' + indexColName + '"', node.getAlias());
         PDataType indexColType = IndexUtil.getIndexColumnDataType(dataCol);
         PDataType dataColType = dataColRef.getColumn().getDataType();
 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/92298f8d/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/92298f8d/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/92298f8d/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/92298f8d/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/92298f8d/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/92298f8d/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/92298f8d/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/92298f8d/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/92298f8d/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/92298f8d/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/92298f8d/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/92298f8d/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/92298f8d/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/92298f8d/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/92298f8d/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/92298f8d/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/92298f8d/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/92298f8d/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/92298f8d/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/92298f8d/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/92298f8d/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/92298f8d/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;
+    }
 }
 


[2/5] phoenix git commit: PHOENIX-653 Support ANSI-standard date literals from SQL 2003

Posted by ja...@apache.org.
PHOENIX-653 Support ANSI-standard date literals from SQL 2003


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

Branch: refs/heads/4.0
Commit: 8982d8dc52aadb868d6e50bb647348a7c949c59e
Parents: fb7aa1a
Author: James Taylor <jt...@salesforce.com>
Authored: Sun Feb 8 20:46:46 2015 -0800
Committer: James Taylor <jt...@salesforce.com>
Committed: Tue Feb 10 08:11:04 2015 -0800

----------------------------------------------------------------------
 phoenix-core/src/main/antlr3/PhoenixSQL.g       | 17 ++++-
 .../expression/ArrayConstructorExpression.java  | 12 ++++
 .../phoenix/expression/LiteralExpression.java   | 17 +++--
 .../apache/phoenix/parse/ParseNodeFactory.java  | 14 ++++
 .../phoenix/schema/types/PArrayDataType.java    | 17 +++++
 .../apache/phoenix/schema/types/PBinary.java    | 15 ++--
 .../org/apache/phoenix/schema/types/PChar.java  | 14 ++--
 .../apache/phoenix/schema/types/PDataType.java  | 12 ++--
 .../org/apache/phoenix/schema/types/PDate.java  | 14 ++--
 .../apache/phoenix/schema/types/PDecimal.java   | 19 +++--
 .../org/apache/phoenix/schema/types/PTime.java  |  4 +-
 .../apache/phoenix/schema/types/PTimestamp.java |  4 +-
 .../phoenix/schema/types/PUnsignedDate.java     | 12 ++--
 .../phoenix/schema/types/PUnsignedTime.java     | 11 ++-
 .../schema/types/PUnsignedTimestamp.java        | 11 ++-
 .../apache/phoenix/schema/types/PVarbinary.java | 25 ++++---
 .../apache/phoenix/schema/types/PVarchar.java   | 15 ++--
 .../compile/StatementHintsCompilationTest.java  |  2 +-
 .../apache/phoenix/parse/QueryParserTest.java   | 76 ++++++++++++++++++++
 .../org/apache/phoenix/query/QueryPlanTest.java |  4 +-
 20 files changed, 242 insertions(+), 73 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/8982d8dc/phoenix-core/src/main/antlr3/PhoenixSQL.g
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/antlr3/PhoenixSQL.g b/phoenix-core/src/main/antlr3/PhoenixSQL.g
index fad5fb3..cda93fe 100644
--- a/phoenix-core/src/main/antlr3/PhoenixSQL.g
+++ b/phoenix-core/src/main/antlr3/PhoenixSQL.g
@@ -152,6 +152,12 @@ import org.apache.phoenix.schema.PTableType;
 import org.apache.phoenix.schema.PTable.IndexType;
 import org.apache.phoenix.schema.stats.StatisticsCollectionScope;
 import org.apache.phoenix.schema.types.PDataType;
+import org.apache.phoenix.schema.types.PDate;
+import org.apache.phoenix.schema.types.PTime;
+import org.apache.phoenix.schema.types.PTimestamp;
+import org.apache.phoenix.schema.types.PUnsignedDate;
+import org.apache.phoenix.schema.types.PUnsignedTime;
+import org.apache.phoenix.schema.types.PUnsignedTimestamp;
 import org.apache.phoenix.util.SchemaUtil;
 import org.apache.phoenix.parse.LikeParseNode.LikeType;
 }
@@ -864,7 +870,9 @@ literal_or_bind returns [ParseNode ret]
 
 // Get a string, integer, double, date, boolean, or NULL value.
 literal returns [LiteralParseNode ret]
-    :   t=STRING_LITERAL { ret = factory.literal(t.getText()); }
+    :   t=STRING_LITERAL {
+            ret = factory.literal(t.getText()); 
+        }
     |   l=int_literal { ret = l; }
     |   l=long_literal { ret = l; }
     |   l=double_literal { ret = l; }
@@ -878,6 +886,13 @@ literal returns [LiteralParseNode ret]
     |   NULL {ret = factory.literal(null);}
     |   TRUE {ret = factory.literal(Boolean.TRUE);} 
     |   FALSE {ret = factory.literal(Boolean.FALSE);}
+    |   dt=identifier t=STRING_LITERAL { 
+            try {
+                ret = factory.literal(t.getText(), dt);
+            } catch (SQLException e) {
+                throw new RuntimeException(e);
+            }
+        }
     ;
     
 int_literal returns [LiteralParseNode ret]

http://git-wip-us.apache.org/repos/asf/phoenix/blob/8982d8dc/phoenix-core/src/main/java/org/apache/phoenix/expression/ArrayConstructorExpression.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/ArrayConstructorExpression.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/ArrayConstructorExpression.java
index 9b0ee8f..15cd14c 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/expression/ArrayConstructorExpression.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/ArrayConstructorExpression.java
@@ -166,4 +166,16 @@ public class ArrayConstructorExpression extends BaseCompoundExpression {
         }
         return t;
     }
+    
+    @Override
+    public String toString() {
+        StringBuilder buf = new StringBuilder(PArrayDataType.ARRAY_TYPE_SUFFIX + "[");
+        if (children.size()==0)
+            return buf.append("]").toString();
+        for (int i = 0; i < children.size() - 1; i++) {
+            buf.append(children.get(i) + ",");
+        }
+        buf.append(children.get(children.size()-1) + "]");
+        return buf.toString();
+    }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/phoenix/blob/8982d8dc/phoenix-core/src/main/java/org/apache/phoenix/expression/LiteralExpression.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/LiteralExpression.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/LiteralExpression.java
index e2bdc82..26c076c 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/expression/LiteralExpression.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/LiteralExpression.java
@@ -25,17 +25,17 @@ import java.sql.SQLException;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.hadoop.io.WritableUtils;
 import org.apache.phoenix.expression.visitor.ExpressionVisitor;
-import org.apache.phoenix.schema.types.PChar;
+import org.apache.phoenix.schema.SortOrder;
+import org.apache.phoenix.schema.TypeMismatchException;
+import org.apache.phoenix.schema.tuple.Tuple;
 import org.apache.phoenix.schema.types.PBoolean;
+import org.apache.phoenix.schema.types.PChar;
 import org.apache.phoenix.schema.types.PDataType;
 import org.apache.phoenix.schema.types.PDate;
 import org.apache.phoenix.schema.types.PTime;
 import org.apache.phoenix.schema.types.PTimestamp;
 import org.apache.phoenix.schema.types.PVarchar;
 import org.apache.phoenix.schema.types.PhoenixArray;
-import org.apache.phoenix.schema.SortOrder;
-import org.apache.phoenix.schema.TypeMismatchException;
-import org.apache.phoenix.schema.tuple.Tuple;
 import org.apache.phoenix.util.ByteUtil;
 import org.apache.phoenix.util.StringUtil;
 
@@ -220,7 +220,14 @@ public class LiteralExpression extends BaseTerminalExpression {
     
     @Override
     public String toString() {
-        return value == null ? "null" : type.toStringLiteral(byteValue, null);
+        if (value == null) {
+            return "null";
+        }
+        // TODO: move into PDataType?
+        if (type.isCoercibleTo(PTimestamp.INSTANCE)) {
+            return type + " " + type.toStringLiteral(value, null);
+        }
+        return type.toStringLiteral(value, null);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/phoenix/blob/8982d8dc/phoenix-core/src/main/java/org/apache/phoenix/parse/ParseNodeFactory.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/ParseNodeFactory.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/ParseNodeFactory.java
index 57507b8..c92dbb6 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/parse/ParseNodeFactory.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/ParseNodeFactory.java
@@ -48,6 +48,7 @@ import org.apache.phoenix.schema.SortOrder;
 import org.apache.phoenix.schema.TypeMismatchException;
 import org.apache.phoenix.schema.stats.StatisticsCollectionScope;
 import org.apache.phoenix.schema.types.PDataType;
+import org.apache.phoenix.schema.types.PTimestamp;
 import org.apache.phoenix.util.SchemaUtil;
 
 import com.google.common.collect.ListMultimap;
@@ -467,6 +468,19 @@ public class ParseNodeFactory {
             value = expectedType.toObject(value, actualType);
         }
         return new LiteralParseNode(value);
+        /*
+        Object typedValue = expectedType.toObject(value.toString());
+        return new LiteralParseNode(typedValue);
+        */
+    }
+
+    public LiteralParseNode literal(String value, String sqlTypeName) throws SQLException {
+        PDataType expectedType = sqlTypeName == null ? null : PDataType.fromSqlTypeName(SchemaUtil.normalizeIdentifier(sqlTypeName));
+        if (expectedType == null || !expectedType.isCoercibleTo(PTimestamp.INSTANCE)) {
+            throw TypeMismatchException.newException(expectedType, PTimestamp.INSTANCE);
+        }
+        Object typedValue = expectedType.toObject(value);
+        return new LiteralParseNode(typedValue);
     }
 
     public LiteralParseNode coerce(LiteralParseNode literalNode, PDataType expectedType) throws SQLException {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/8982d8dc/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PArrayDataType.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PArrayDataType.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PArrayDataType.java
index 30fab95..c183b7a 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PArrayDataType.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PArrayDataType.java
@@ -20,6 +20,7 @@ package org.apache.phoenix.schema.types;
 import java.io.DataOutputStream;
 import java.io.IOException;
 import java.nio.ByteBuffer;
+import java.text.Format;
 
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -653,4 +654,20 @@ public abstract class PArrayDataType<T> extends PDataType<T> {
         return instantiatePhoenixArray(baseType, array);
     }
 
+    @Override
+    public String toStringLiteral(Object o, Format formatter) {
+        StringBuilder buf = new StringBuilder(PArrayDataType.ARRAY_TYPE_SUFFIX + "[");
+        PhoenixArray array = (PhoenixArray)o;
+        PDataType baseType = PDataType.arrayBaseType(this);
+        int len = array.getDimensions();
+        if (len != 0)  {
+            for (int i = 0; i < len; i++) {
+                buf.append(baseType.toStringLiteral(array.getElement(i), null));
+                buf.append(',');
+            }
+            buf.setLength(buf.length()-1);
+        }
+        buf.append(']');
+        return buf.toString();
+    }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/8982d8dc/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PBinary.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PBinary.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PBinary.java
index d188387..69d3796 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PBinary.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PBinary.java
@@ -17,15 +17,15 @@
  */
 package org.apache.phoenix.schema.types;
 
+import java.sql.Types;
+import java.text.Format;
+
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.hadoop.hbase.util.Base64;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.phoenix.exception.ValueTypeIncompatibleException;
 import org.apache.phoenix.schema.SortOrder;
 
-import java.sql.Types;
-import java.text.Format;
-
 public class PBinary extends PDataType<byte[]> {
 
   public static final PBinary INSTANCE = new PBinary();
@@ -176,13 +176,18 @@ public class PBinary extends PDataType<byte[]> {
 
   @Override
   public String toStringLiteral(byte[] b, int offset, int length, Format formatter) {
-    if (formatter == null && b.length == 1) {
-      return Integer.toString(0xFF & b[0]);
+    if (length == 1) {
+      return Integer.toString(0xFF & b[offset]);
     }
     return PVarbinary.INSTANCE.toStringLiteral(b, offset, length, formatter);
   }
 
   @Override
+  public String toStringLiteral(Object o, Format formatter) {
+    return toStringLiteral((byte[])o, 0, ((byte[]) o).length, formatter);
+  }
+
+  @Override
   public Object getSampleValue(Integer maxLength, Integer arrayLength) {
     return PVarbinary.INSTANCE.getSampleValue(maxLength, arrayLength);
   }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/8982d8dc/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PChar.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PChar.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PChar.java
index 3100f89..aaee1ba 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PChar.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PChar.java
@@ -17,16 +17,17 @@
  */
 package org.apache.phoenix.schema.types;
 
-import com.google.common.base.Strings;
+import java.sql.Types;
+import java.text.Format;
+import java.util.Arrays;
+
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.phoenix.exception.ValueTypeIncompatibleException;
 import org.apache.phoenix.schema.SortOrder;
 import org.apache.phoenix.util.StringUtil;
 
-import java.sql.Types;
-import java.text.Format;
-import java.util.Arrays;
+import com.google.common.base.Strings;
 
 /**
  * Fixed length single byte characters
@@ -197,6 +198,11 @@ public class PChar extends PDataType<String> {
     }
 
     @Override
+    public String toStringLiteral(Object o, Format formatter) {
+      return PVarchar.INSTANCE.toStringLiteral(o, formatter);
+    }
+
+    @Override
     public Object getSampleValue(Integer maxLength, Integer arrayLength) {
       return PVarchar.INSTANCE.getSampleValue(maxLength, arrayLength);
     }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/8982d8dc/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PDataType.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PDataType.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PDataType.java
index 85e5711..8f46a3b 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PDataType.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PDataType.java
@@ -1091,10 +1091,14 @@ public abstract class PDataType<T> implements DataType<T>, Comparable<PDataType<
 
   public String toStringLiteral(byte[] b, int offset, int length, Format formatter) {
     Object o = toObject(b, offset, length);
-    if (formatter != null) {
-      return formatter.format(o);
-    }
-    return o.toString();
+    return toStringLiteral(o, formatter);
+  }
+  
+  public String toStringLiteral(Object o, Format formatter) {
+      if (formatter != null) {
+          return formatter.format(o);
+        }
+        return o.toString();
   }
 
   private static final PhoenixArrayFactory DEFAULT_ARRAY_FACTORY = new PhoenixArrayFactory() {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/8982d8dc/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PDate.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PDate.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PDate.java
index bbd0a35..fa070d3 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PDate.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PDate.java
@@ -147,13 +147,13 @@ public class PDate extends PDataType<Date> {
   }
 
   @Override
-  public String toStringLiteral(byte[] b, int offset, int length, Format formatter) {
-    if (formatter == null || formatter == DateUtil.DEFAULT_DATE_FORMATTER) {
-      // If default formatter has not been overridden,
-      // use one that displays milliseconds.
-      formatter = DateUtil.DEFAULT_MS_DATE_FORMATTER;
-    }
-    return "'" + super.toStringLiteral(b, offset, length, formatter) + "'";
+  public String toStringLiteral(Object o, Format formatter) {
+      if (formatter == null) {
+          // If default formatter has not been overridden,
+          // use one that displays milliseconds.
+          formatter = DateUtil.DEFAULT_DATE_FORMATTER;
+        }
+        return "'" + super.toStringLiteral(o, formatter) + "'";
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/phoenix/blob/8982d8dc/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PDecimal.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PDecimal.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PDecimal.java
index 6b2dc84..e90491b 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PDecimal.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PDecimal.java
@@ -17,7 +17,11 @@
  */
 package org.apache.phoenix.schema.types;
 
-import com.google.common.base.Preconditions;
+import java.math.BigDecimal;
+import java.sql.Timestamp;
+import java.sql.Types;
+import java.text.Format;
+
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.phoenix.query.QueryConstants;
@@ -25,10 +29,7 @@ import org.apache.phoenix.schema.SortOrder;
 import org.apache.phoenix.util.ByteUtil;
 import org.apache.phoenix.util.NumberUtil;
 
-import java.math.BigDecimal;
-import java.sql.Timestamp;
-import java.sql.Types;
-import java.text.Format;
+import com.google.common.base.Preconditions;
 
 public class PDecimal extends PDataType<BigDecimal> {
 
@@ -390,6 +391,14 @@ public class PDecimal extends PDataType<BigDecimal> {
   }
 
   @Override
+  public String toStringLiteral(Object o, Format formatter) {
+      if (formatter == null) {
+          return ((BigDecimal)o).toPlainString();
+        }
+        return super.toStringLiteral(o, formatter);
+  }
+
+  @Override
   public Object getSampleValue(Integer maxLength, Integer arrayLength) {
     return new BigDecimal((Long) PLong.INSTANCE.getSampleValue(maxLength, arrayLength));
   }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/8982d8dc/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PTime.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PTime.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PTime.java
index 81cbaff..0cfb0e8 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PTime.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PTime.java
@@ -127,11 +127,11 @@ public class PTime extends PDataType<Time> {
   }
 
   @Override
-  public String toStringLiteral(byte[] b, int offset, int length, Format formatter) {
+  public String toStringLiteral(Object o, Format formatter) {
       if (formatter == null) {
           formatter = DateUtil.DEFAULT_TIME_FORMATTER;
         }
-        return "'" + super.toStringLiteral(b, offset, length, formatter) + "'";
+        return "'" + super.toStringLiteral(o, formatter) + "'";
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/phoenix/blob/8982d8dc/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PTimestamp.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PTimestamp.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PTimestamp.java
index 8182e33..9a82cc0 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PTimestamp.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PTimestamp.java
@@ -180,11 +180,11 @@ public class PTimestamp extends PDataType<Timestamp> {
   }
 
   @Override
-  public String toStringLiteral(byte[] b, int offset, int length, Format formatter) {
+  public String toStringLiteral(Object o, Format formatter) {
       if (formatter == null) {
           formatter = DateUtil.DEFAULT_TIMESTAMP_FORMATTER;
         }
-        return "'" + super.toStringLiteral(b, offset, length, formatter) + "'";
+        return "'" + super.toStringLiteral(o, formatter) + "'";
   }
 
 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/8982d8dc/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PUnsignedDate.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PUnsignedDate.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PUnsignedDate.java
index 8b63fbb..a6b1bc3 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PUnsignedDate.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PUnsignedDate.java
@@ -17,14 +17,14 @@
  */
 package org.apache.phoenix.schema.types;
 
+import java.sql.Date;
+import java.sql.Types;
+import java.text.Format;
+
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.phoenix.schema.SortOrder;
 import org.apache.phoenix.util.DateUtil;
 
-import java.sql.Types;
-import java.sql.Date;
-import java.text.Format;
-
 public class PUnsignedDate extends PDataType<Date> {
 
   public static final PUnsignedDate INSTANCE = new PUnsignedDate();
@@ -109,14 +109,14 @@ public class PUnsignedDate extends PDataType<Date> {
   }
 
   @Override
-  public String toStringLiteral(byte[] b, int offset, int length, Format formatter) {
+  public String toStringLiteral(Object o, Format formatter) {
     // Can't delegate, as the super.toStringLiteral calls this.toBytes
     if (formatter == null || formatter == DateUtil.DEFAULT_DATE_FORMATTER) {
       // If default formatter has not been overridden,
       // use one that displays milliseconds.
       formatter = DateUtil.DEFAULT_MS_DATE_FORMATTER;
     }
-    return "'" + super.toStringLiteral(b, offset, length, formatter) + "'";
+    return "'" + super.toStringLiteral(o, formatter) + "'";
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/phoenix/blob/8982d8dc/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PUnsignedTime.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PUnsignedTime.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PUnsignedTime.java
index f738f44..4173be1 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PUnsignedTime.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PUnsignedTime.java
@@ -17,13 +17,13 @@
  */
 package org.apache.phoenix.schema.types;
 
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.phoenix.schema.SortOrder;
-
 import java.sql.Time;
 import java.sql.Types;
 import java.text.Format;
 
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.phoenix.schema.SortOrder;
+
 public class PUnsignedTime extends PDataType<Time> {
 
   public static final PUnsignedTime INSTANCE = new PUnsignedTime();
@@ -103,6 +103,11 @@ public class PUnsignedTime extends PDataType<Time> {
   }
 
   @Override
+  public String toStringLiteral(Object o, Format formatter) {
+    return PUnsignedDate.INSTANCE.toStringLiteral(o, formatter);
+  }
+
+  @Override
   public int getResultSetSqlType() {
     return Types.TIME;
   }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/8982d8dc/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PUnsignedTimestamp.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PUnsignedTimestamp.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PUnsignedTimestamp.java
index c13de56..450408f 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PUnsignedTimestamp.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PUnsignedTimestamp.java
@@ -114,14 +114,11 @@ public class PUnsignedTimestamp extends PDataType<Timestamp> {
   }
 
   @Override
-  public String toStringLiteral(byte[] b, int offset, int length, Format formatter) {
-    java.sql.Timestamp value = (java.sql.Timestamp) toObject(b, offset, length);
-    if (formatter == null || formatter == DateUtil.DEFAULT_DATE_FORMATTER) {
-      // If default formatter has not been overridden,
-      // use one that displays milliseconds.
-      formatter = DateUtil.DEFAULT_MS_DATE_FORMATTER;
+  public String toStringLiteral(Object o, Format formatter) {
+    if (formatter == null) {
+      formatter = DateUtil.DEFAULT_TIMESTAMP_FORMATTER;
     }
-    return "'" + super.toStringLiteral(b, offset, length, formatter) + "." + value.getNanos() + "'";
+    return "'" + super.toStringLiteral(o, formatter) + "'";
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/phoenix/blob/8982d8dc/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PVarbinary.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PVarbinary.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PVarbinary.java
index 6ba4dc4..bb1d4c6 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PVarbinary.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PVarbinary.java
@@ -17,15 +17,15 @@
  */
 package org.apache.phoenix.schema.types;
 
+import java.sql.Types;
+import java.text.Format;
+
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.hadoop.hbase.util.Base64;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.phoenix.schema.SortOrder;
 import org.apache.phoenix.util.ByteUtil;
 
-import java.sql.Types;
-import java.text.Format;
-
 public class PVarbinary extends PDataType<byte[]> {
 
   public static final PVarbinary INSTANCE = new PVarbinary();
@@ -148,20 +148,25 @@ public class PVarbinary extends PDataType<byte[]> {
 
   @Override
   public String toStringLiteral(byte[] b, int o, int length, Format formatter) {
-    if (formatter != null) {
-      return formatter.format(b);
-    }
     StringBuilder buf = new StringBuilder();
     buf.append('[');
-    for (int i = 0; i < b.length; i++) {
-      buf.append(0xFF & b[i]);
-      buf.append(',');
+    if (length > 0) {
+        for (int i = o; i < length; i++) {
+          buf.append(0xFF & b[i]);
+          buf.append(',');
+        }
+        buf.setLength(buf.length()-1);
     }
-    buf.setCharAt(buf.length() - 1, ']');
+    buf.append(']');
     return buf.toString();
   }
 
   @Override
+  public String toStringLiteral(Object o, Format formatter) {
+      return toStringLiteral((byte[])o, 0, ((byte[]) o).length, formatter);
+  }
+  
+  @Override
   public Object getSampleValue(Integer maxLength, Integer arrayLength) {
     int length = maxLength != null && maxLength > 0 ? maxLength : 1;
     byte[] b = new byte[length];

http://git-wip-us.apache.org/repos/asf/phoenix/blob/8982d8dc/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PVarchar.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PVarchar.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PVarchar.java
index 9ecfb4e..6956942 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PVarchar.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PVarchar.java
@@ -17,14 +17,15 @@
  */
 package org.apache.phoenix.schema.types;
 
-import com.google.common.base.Preconditions;
+import java.sql.Types;
+import java.text.Format;
+
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.phoenix.schema.SortOrder;
 import org.apache.phoenix.util.ByteUtil;
 
-import java.sql.Types;
-import java.text.Format;
+import com.google.common.base.Preconditions;
 
 public class PVarchar extends PDataType<String> {
 
@@ -137,15 +138,11 @@ public class PVarchar extends PDataType<String> {
   }
 
   @Override
-  public String toStringLiteral(byte[] b, int offset, int length, Format formatter) {
-    while (b[length - 1] == 0) {
-      length--;
-    }
+  public String toStringLiteral(Object o, Format formatter) {
     if (formatter != null) {
-      Object o = toObject(b, offset, length);
       return "'" + formatter.format(o) + "'";
     }
-    return "'" + Bytes.toStringBinary(b, offset, length) + "'";
+    return "'" + o + "'";
   }
 
   private char[] sampleChars = new char[1];

http://git-wip-us.apache.org/repos/asf/phoenix/blob/8982d8dc/phoenix-core/src/test/java/org/apache/phoenix/compile/StatementHintsCompilationTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/compile/StatementHintsCompilationTest.java b/phoenix-core/src/test/java/org/apache/phoenix/compile/StatementHintsCompilationTest.java
index 13e2860..7f8adfa 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/compile/StatementHintsCompilationTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/compile/StatementHintsCompilationTest.java
@@ -99,7 +99,7 @@ public class StatementHintsCompilationTest extends BaseConnectionlessQueryTest {
         conn.createStatement().execute("create table eh (organization_id char(15) not null,parent_id char(15) not null, created_date date not null, entity_history_id char(15) not null constraint pk primary key (organization_id, parent_id, created_date, entity_history_id))");
         ResultSet rs = conn.createStatement().executeQuery("explain select /*+ RANGE_SCAN */ ORGANIZATION_ID, PARENT_ID, CREATED_DATE, ENTITY_HISTORY_ID from eh where ORGANIZATION_ID='111111111111111' and SUBSTR(PARENT_ID, 1, 3) = 'foo' and CREATED_DATE >= TO_DATE ('2012-11-01 00:00:00') and CREATED_DATE < TO_DATE ('2012-11-30 00:00:00') order by ORGANIZATION_ID, PARENT_ID, CREATED_DATE DESC, ENTITY_HISTORY_ID limit 100");
         assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER EH ['111111111111111','foo            ','2012-11-01 00:00:00.000'] - ['111111111111111','fop            ','2012-11-30 00:00:00.000']\n" + 
-                "    SERVER FILTER BY FIRST KEY ONLY AND (CREATED_DATE >= '2012-11-01 00:00:00.000' AND CREATED_DATE < '2012-11-30 00:00:00.000')\n" + 
+                "    SERVER FILTER BY FIRST KEY ONLY AND (CREATED_DATE >= DATE '2012-11-01 00:00:00.000' AND CREATED_DATE < DATE '2012-11-30 00:00:00.000')\n" + 
                 "    SERVER TOP 100 ROWS SORTED BY [ORGANIZATION_ID, PARENT_ID, CREATED_DATE DESC, ENTITY_HISTORY_ID]\n" + 
                 "CLIENT MERGE SORT",QueryUtil.getExplainPlan(rs));
     }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/8982d8dc/phoenix-core/src/test/java/org/apache/phoenix/parse/QueryParserTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/parse/QueryParserTest.java b/phoenix-core/src/test/java/org/apache/phoenix/parse/QueryParserTest.java
index 201172b..866365a 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/parse/QueryParserTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/parse/QueryParserTest.java
@@ -677,4 +677,80 @@ public class QueryParserTest {
                 new StringReader("select * from date_test where d in (to_date('2013-11-04 09:12:00'))"));
         parser.parseStatement();
     }
+    
+    @Test
+    public void testDateLiteral() throws Exception {
+        SQLParser parser = new SQLParser(
+                new StringReader(
+                        "select * from t where d = DATE '2013-11-04 09:12:00'"));
+        parser.parseStatement();
+    }
+
+    @Test
+    public void testTimeLiteral() throws Exception {
+        SQLParser parser = new SQLParser(
+                new StringReader(
+                        "select * from t where d = TIME '2013-11-04 09:12:00'"));
+        parser.parseStatement();
+    }
+
+
+    @Test
+    public void testTimestampLiteral() throws Exception {
+        SQLParser parser = new SQLParser(
+                new StringReader(
+                        "select * from t where d = TIMESTAMP '2013-11-04 09:12:00'"));
+        parser.parseStatement();
+    }
+    
+    @Test
+    public void testUnsignedDateLiteral() throws Exception {
+        SQLParser parser = new SQLParser(
+                new StringReader(
+                        "select * from t where d = UNSIGNED_DATE '2013-11-04 09:12:00'"));
+        parser.parseStatement();
+    }
+
+    @Test
+    public void testUnsignedTimeLiteral() throws Exception {
+        SQLParser parser = new SQLParser(
+                new StringReader(
+                        "select * from t where d = UNSIGNED_TIME '2013-11-04 09:12:00'"));
+        parser.parseStatement();
+    }
+
+
+    @Test
+    public void testUnsignedTimestampLiteral() throws Exception {
+        SQLParser parser = new SQLParser(
+                new StringReader(
+                        "select * from t where d = UNSIGNED_TIMESTAMP '2013-11-04 09:12:00'"));
+        parser.parseStatement();
+    }
+    
+    @Test
+    public void testUnknownLiteral() throws Exception {
+        SQLParser parser = new SQLParser(
+                new StringReader(
+                        "select * from t where d = FOO '2013-11-04 09:12:00'"));
+        try {
+            parser.parseStatement();
+            fail();
+        } catch (SQLException e) {
+            assertEquals(SQLExceptionCode.ILLEGAL_DATA.getErrorCode(), e.getErrorCode());
+        }
+    }
+    
+    @Test
+    public void testUnsupportedLiteral() throws Exception {
+        SQLParser parser = new SQLParser(
+                new StringReader(
+                        "select * from t where d = DECIMAL '2013-11-04 09:12:00'"));
+        try {
+            parser.parseStatement();
+            fail();
+        } catch (SQLException e) {
+            assertEquals(SQLExceptionCode.TYPE_MISMATCH.getErrorCode(), e.getErrorCode());
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/8982d8dc/phoenix-core/src/test/java/org/apache/phoenix/query/QueryPlanTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/query/QueryPlanTest.java b/phoenix-core/src/test/java/org/apache/phoenix/query/QueryPlanTest.java
index 6139aa5..7ad3e25 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/query/QueryPlanTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/query/QueryPlanTest.java
@@ -52,12 +52,12 @@ public class QueryPlanTest extends BaseConnectionlessQueryTest {
 
                 "SELECT host FROM PTSDB WHERE inst IS NULL AND host IS NOT NULL AND date >= to_date('2013-01-01')",
                 "CLIENT PARALLEL 1-WAY RANGE SCAN OVER PTSDB [null,not null]\n" + 
-                "    SERVER FILTER BY FIRST KEY ONLY AND DATE >= '2013-01-01 00:00:00.000'",
+                "    SERVER FILTER BY FIRST KEY ONLY AND DATE >= DATE '2013-01-01 00:00:00.000'",
 
                 // Since inst IS NOT NULL is unbounded, we won't continue optimizing
                 "SELECT host FROM PTSDB WHERE inst IS NOT NULL AND host IS NULL AND date >= to_date('2013-01-01')",
                 "CLIENT PARALLEL 1-WAY RANGE SCAN OVER PTSDB [not null]\n" + 
-                "    SERVER FILTER BY FIRST KEY ONLY AND (HOST IS NULL AND DATE >= '2013-01-01 00:00:00.000')",
+                "    SERVER FILTER BY FIRST KEY ONLY AND (HOST IS NULL AND DATE >= DATE '2013-01-01 00:00:00.000')",
 
                 "SELECT a_string,b_string FROM atable WHERE organization_id = '000000000000001' AND entity_id = '000000000000002' AND x_integer = 2 AND a_integer < 5 ",
                 "CLIENT PARALLEL 1-WAY POINT LOOKUP ON 1 KEY OVER ATABLE\n" + 


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

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/phoenix/blob/92298f8d/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/92298f8d/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/92298f8d/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/92298f8d/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/92298f8d/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/92298f8d/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/92298f8d/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/92298f8d/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/92298f8d/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/92298f8d/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/92298f8d/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/92298f8d/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/92298f8d/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/92298f8d/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/92298f8d/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/92298f8d/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/92298f8d/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/92298f8d/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/92298f8d/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/92298f8d/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/92298f8d/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/92298f8d/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 ");
+    }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/92298f8d/phoenix-core/src/main/java/org/apache/phoenix/parse/OuterJoinParseNode.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/OuterJoinParseNode.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/OuterJoinParseNode.java
deleted file mode 100644
index 97f636b..0000000
--- a/phoenix-core/src/main/java/org/apache/phoenix/parse/OuterJoinParseNode.java
+++ /dev/null
@@ -1,47 +0,0 @@
-/*
- * 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.sql.SQLException;
-import java.util.Collections;
-import java.util.List;
-
-
-
-/**
- * 
- * Node representing an outer join qualifier (+) in SQL
- * TODO: remove Oracle specific syntax
- *
- * 
- * @since 0.1
- */
-public class OuterJoinParseNode extends UnaryParseNode{
-    OuterJoinParseNode(ParseNode node) {
-        super(node);
-    }
-
-    @Override
-    public <T> T accept(ParseNodeVisitor<T> visitor) throws SQLException {
-        List<T> l = Collections.emptyList();
-        if (visitor.visitEnter(this)) {
-            l = acceptChildren(visitor);
-        }
-        return visitor.visitLeave(this, l);
-    }
-}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/92298f8d/phoenix-core/src/main/java/org/apache/phoenix/parse/ParseNode.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/ParseNode.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/ParseNode.java
index 2ee8a83..b32674e 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/parse/ParseNode.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/ParseNode.java
@@ -20,6 +20,8 @@ package org.apache.phoenix.parse;
 import java.sql.SQLException;
 import java.util.List;
 
+import org.apache.phoenix.compile.ColumnResolver;
+
 
 
 
@@ -47,4 +49,13 @@ public abstract class ParseNode {
     public String getAlias() {
         return null;
     }
+    
+    @Override
+    public final String toString() {
+        StringBuilder buf = new StringBuilder();
+        toSQL(null, buf);
+        return buf.toString();
+    }
+    
+    public abstract void toSQL(ColumnResolver resolver, StringBuilder buf);
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/92298f8d/phoenix-core/src/main/java/org/apache/phoenix/parse/ParseNodeFactory.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/ParseNodeFactory.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/ParseNodeFactory.java
index c92dbb6..ddfaa03 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/parse/ParseNodeFactory.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/ParseNodeFactory.java
@@ -19,7 +19,6 @@ package org.apache.phoenix.parse;
 
 import java.lang.reflect.Constructor;
 import java.sql.SQLException;
-import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
 import java.util.List;
@@ -52,6 +51,7 @@ import org.apache.phoenix.schema.types.PTimestamp;
 import org.apache.phoenix.util.SchemaUtil;
 
 import com.google.common.collect.ListMultimap;
+import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
 
 /**
@@ -391,12 +391,22 @@ public class ParseNodeFactory {
     public FunctionParseNode function(String name, List<ParseNode> valueNodes,
             List<ParseNode> columnNodes, boolean isAscending) {
 
-        List<ParseNode> children = new ArrayList<ParseNode>();
-        children.addAll(columnNodes);
-        children.add(new LiteralParseNode(Boolean.valueOf(isAscending)));
-        children.addAll(valueNodes);
+        List<ParseNode> args = Lists.newArrayListWithExpectedSize(columnNodes.size() + valueNodes.size() + 1);
+        args.addAll(columnNodes);
+        args.add(new LiteralParseNode(Boolean.valueOf(isAscending)));
+        args.addAll(valueNodes);
 
-        return function(name, children);
+        BuiltInFunctionInfo info = getInfo(name, args);
+        Constructor<? extends FunctionParseNode> ctor = info.getNodeCtor();
+        if (ctor == null) {
+            return new AggregateFunctionWithinGroupParseNode(name, args, info);
+        } else {
+            try {
+                return ctor.newInstance(name, args, info);
+            } catch (Exception e) {
+                throw new RuntimeException(e);
+            }
+        }
     }
 
     public HintNode hint(String hint) {
@@ -561,8 +571,12 @@ public class ParseNodeFactory {
     	return new ArrayConstructorNode(upsertStmtArray);
     }
 
-    public MultiplyParseNode negate(ParseNode child) {
-        return new MultiplyParseNode(Arrays.asList(child,this.literal(-1)));
+    public ParseNode negate(ParseNode child) {
+        // Prevents reparsing of -1 from becoming 1*-1 and 1*1*-1 with each re-parsing
+        if (LiteralParseNode.ONE.equals(child)) {
+            return LiteralParseNode.MINUS_ONE;
+        }
+        return new MultiplyParseNode(Arrays.asList(child,LiteralParseNode.MINUS_ONE));
     }
 
     public NotEqualParseNode notEqual(ParseNode lhs, ParseNode rhs) {
@@ -588,10 +602,6 @@ public class ParseNodeFactory {
     }
 
 
-    public OuterJoinParseNode outer(ParseNode node) {
-        return new OuterJoinParseNode(node);
-    }
-
     public SelectStatement select(TableNode from, HintNode hint, boolean isDistinct, List<AliasedNode> select, ParseNode where,
             List<ParseNode> groupBy, ParseNode having, List<OrderByNode> orderBy, LimitNode limit, int bindCount, boolean isAggregate, boolean hasSequence) {
 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/92298f8d/phoenix-core/src/main/java/org/apache/phoenix/parse/RowValueConstructorParseNode.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/RowValueConstructorParseNode.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/RowValueConstructorParseNode.java
index 87038c7..3d6d7f1 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/parse/RowValueConstructorParseNode.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/RowValueConstructorParseNode.java
@@ -21,6 +21,8 @@ import java.sql.SQLException;
 import java.util.Collections;
 import java.util.List;
 
+import org.apache.phoenix.compile.ColumnResolver;
+
 /**
  * 
  * Node representing a row value constructor in SQL.  
@@ -43,4 +45,18 @@ public class RowValueConstructorParseNode extends CompoundParseNode {
         return visitor.visitLeave(this, l);
     }
 
+    @Override
+    public void toSQL(ColumnResolver resolver, StringBuilder buf) {
+        List<ParseNode> children = getChildren();
+        buf.append(' ');
+        buf.append('(');
+        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/92298f8d/phoenix-core/src/main/java/org/apache/phoenix/parse/SelectStatement.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/SelectStatement.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/SelectStatement.java
index 961846b..71cabd6 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/parse/SelectStatement.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/SelectStatement.java
@@ -21,6 +21,7 @@ import java.util.Arrays;
 import java.util.Collections;
 import java.util.List;
 
+import org.apache.phoenix.compile.ColumnResolver;
 import org.apache.phoenix.expression.function.CountAggregateFunction;
 import org.apache.phoenix.jdbc.PhoenixStatement.Operation;
 import org.apache.phoenix.parse.FunctionParseNode.BuiltInFunction;
@@ -93,6 +94,104 @@ public class SelectStatement implements FilterableStatement {
     private final boolean isAggregate;
     private final boolean hasSequence;
     
+    @Override
+    public final String toString() {
+        StringBuilder buf = new StringBuilder();
+        toSQL(null,buf);
+        return buf.toString();
+    }
+
+    public void toSQL(ColumnResolver resolver, StringBuilder buf) {
+        buf.append("SELECT ");
+        if (hint != null) buf.append(hint);
+        if (isDistinct) buf.append("DISTINCT ");
+        for (AliasedNode selectNode : select) {
+            selectNode.toSQL(resolver, buf);
+            buf.append(',');
+        }
+        buf.setLength(buf.length()-1);
+        buf.append(" FROM ");
+        fromTable.toSQL(resolver, buf);
+        if (where != null) {
+            buf.append(" WHERE ");
+            where.toSQL(resolver, buf);
+        }
+        if (!groupBy.isEmpty()) {
+            buf.append(" GROUP BY ");
+            for (ParseNode node : groupBy) {
+                node.toSQL(resolver, buf);
+                buf.append(',');
+            }
+            buf.setLength(buf.length()-1);
+        }
+        if (having != null) {
+            buf.append(" HAVING ");
+            having.toSQL(resolver, buf);            
+        }
+        if (!orderBy.isEmpty()) {
+            buf.append(" ORDER BY ");
+            for (OrderByNode node : orderBy) {
+                node.toSQL(resolver, buf);
+                buf.append(',');
+            }
+            buf.setLength(buf.length()-1);
+        }
+        if (limit != null) {
+            buf.append(" LIMIT " + limit.toString());
+        }
+    }    
+
+    
+    @Override
+    public int hashCode() {
+        final int prime = 31;
+        int result = 1;
+        result = prime * result + ((fromTable == null) ? 0 : fromTable.hashCode());
+        result = prime * result + ((groupBy == null) ? 0 : groupBy.hashCode());
+        result = prime * result + ((having == null) ? 0 : having.hashCode());
+        result = prime * result + ((hint == null) ? 0 : hint.hashCode());
+        result = prime * result + (isDistinct ? 1231 : 1237);
+        result = prime * result + ((limit == null) ? 0 : limit.hashCode());
+        result = prime * result + ((orderBy == null) ? 0 : orderBy.hashCode());
+        result = prime * result + ((select == null) ? 0 : select.hashCode());
+        result = prime * result + ((where == null) ? 0 : where.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;
+        SelectStatement other = (SelectStatement)obj;
+        if (fromTable == null) {
+            if (other.fromTable != null) return false;
+        } else if (!fromTable.equals(other.fromTable)) return false;
+        if (groupBy == null) {
+            if (other.groupBy != null) return false;
+        } else if (!groupBy.equals(other.groupBy)) return false;
+        if (having == null) {
+            if (other.having != null) return false;
+        } else if (!having.equals(other.having)) return false;
+        if (hint == null) {
+            if (other.hint != null) return false;
+        } else if (!hint.equals(other.hint)) return false;
+        if (isDistinct != other.isDistinct) return false;
+        if (limit == null) {
+            if (other.limit != null) return false;
+        } else if (!limit.equals(other.limit)) return false;
+        if (orderBy == null) {
+            if (other.orderBy != null) return false;
+        } else if (!orderBy.equals(other.orderBy)) return false;
+        if (select == null) {
+            if (other.select != null) return false;
+        } else if (!select.equals(other.select)) return false;
+        if (where == null) {
+            if (other.where != null) return false;
+        } else if (!where.equals(other.where)) return false;
+        return true;
+    }
+
     // Count constant expressions
     private static int countConstants(List<ParseNode> nodes) {
         int count = 0;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/92298f8d/phoenix-core/src/main/java/org/apache/phoenix/parse/SequenceValueParseNode.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/SequenceValueParseNode.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/SequenceValueParseNode.java
index 260584f..a5d60fe 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/parse/SequenceValueParseNode.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/SequenceValueParseNode.java
@@ -20,6 +20,8 @@ package org.apache.phoenix.parse;
 
 import java.sql.SQLException;
 
+import org.apache.phoenix.compile.ColumnResolver;
+
 
 public class SequenceValueParseNode extends TerminalParseNode {
     public enum Op {
@@ -89,4 +91,12 @@ public class SequenceValueParseNode extends TerminalParseNode {
 			return false;
 		return true;
 	}
+
+    @Override
+    public void toSQL(ColumnResolver resolver, StringBuilder buf) {
+        buf.append(' ');
+        buf.append(op.getName());
+        buf.append(" VALUE FOR ");
+        buf.append(tableName);
+    }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/phoenix/blob/92298f8d/phoenix-core/src/main/java/org/apache/phoenix/parse/StringConcatParseNode.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/StringConcatParseNode.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/StringConcatParseNode.java
index 3fd27de..5eba979 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/parse/StringConcatParseNode.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/StringConcatParseNode.java
@@ -21,6 +21,8 @@ import java.sql.SQLException;
 import java.util.Collections;
 import java.util.List;
 
+import org.apache.phoenix.compile.ColumnResolver;
+
 
 
 
@@ -46,4 +48,16 @@ public class StringConcatParseNode 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(" || ");
+            children.get(i).toSQL(resolver, buf);
+        }
+        buf.append(')');
+    }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/92298f8d/phoenix-core/src/main/java/org/apache/phoenix/parse/SubqueryParseNode.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/SubqueryParseNode.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/SubqueryParseNode.java
index b7bcb64..d73958e 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/parse/SubqueryParseNode.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/SubqueryParseNode.java
@@ -19,6 +19,8 @@ package org.apache.phoenix.parse;
 
 import java.sql.SQLException;
 
+import org.apache.phoenix.compile.ColumnResolver;
+
 
 
 /**
@@ -78,4 +80,10 @@ public class SubqueryParseNode extends TerminalParseNode {
 		return true;
 	}
     
+    @Override
+    public void toSQL(ColumnResolver resolver, StringBuilder buf) {
+        buf.append('(');
+        select.toSQL(resolver, buf);
+        buf.append(')');
+    }    
 }

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

http://git-wip-us.apache.org/repos/asf/phoenix/blob/92298f8d/phoenix-core/src/main/java/org/apache/phoenix/parse/TableName.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/TableName.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/TableName.java
index 654e899..61bfa6b 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/parse/TableName.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/TableName.java
@@ -60,7 +60,9 @@ public class TableName {
     
     @Override
     public String toString() {
-        return (schemaName == null ? "" : schemaName + QueryConstants.NAME_SEPARATOR)  + tableName;
+        return (schemaName == null ? "" : ((isSchemaNameCaseSensitive ? "\"" : "") + schemaName
+                + (isSchemaNameCaseSensitive ? "\"" : "") + QueryConstants.NAME_SEPARATOR))
+                + ((isTableNameCaseSensitive ? "\"" : "") + tableName + (isTableNameCaseSensitive ? "\"" : ""));
     }
     
 	@Override

http://git-wip-us.apache.org/repos/asf/phoenix/blob/92298f8d/phoenix-core/src/main/java/org/apache/phoenix/parse/TableNode.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/TableNode.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/TableNode.java
index 7ab8d0c..7c37234 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/parse/TableNode.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/TableNode.java
@@ -19,6 +19,8 @@ package org.apache.phoenix.parse;
 
 import java.sql.SQLException;
 
+import org.apache.phoenix.compile.ColumnResolver;
+
 
 
 /**
@@ -39,6 +41,14 @@ public abstract class TableNode {
         return alias;
     }
 
+    @Override
+    public final String toString() {
+        StringBuilder buf = new StringBuilder();
+        toSQL(null,buf);
+        return buf.toString();
+    }
+
     public abstract <T> T accept(TableNodeVisitor<T> visitor) throws SQLException;
+    public abstract void toSQL(ColumnResolver resolver, StringBuilder buf);
 }
 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/92298f8d/phoenix-core/src/main/java/org/apache/phoenix/parse/TableWildcardParseNode.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/TableWildcardParseNode.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/TableWildcardParseNode.java
index 7292347..7c7f416 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/parse/TableWildcardParseNode.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/TableWildcardParseNode.java
@@ -19,6 +19,8 @@ package org.apache.phoenix.parse;
 
 import java.sql.SQLException;
 
+import org.apache.phoenix.compile.ColumnResolver;
+
 public class TableWildcardParseNode extends NamedParseNode {
     private final TableName tableName;
     private final boolean isRewrite;
@@ -75,5 +77,10 @@ public class TableWildcardParseNode extends NamedParseNode {
 		return true;
 	}
 
+    @Override
+    public void toSQL(ColumnResolver resolver, StringBuilder buf) {
+        toSQL(buf);
+        buf.append(".*");
+    }
 }
 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/92298f8d/phoenix-core/src/main/java/org/apache/phoenix/parse/WildcardParseNode.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/WildcardParseNode.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/WildcardParseNode.java
index fdfb64f..9922c3f 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/parse/WildcardParseNode.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/WildcardParseNode.java
@@ -19,6 +19,8 @@ package org.apache.phoenix.parse;
 
 import java.sql.SQLException;
 
+import org.apache.phoenix.compile.ColumnResolver;
+
 
 
 /**
@@ -44,11 +46,6 @@ public class WildcardParseNode extends TerminalParseNode {
         return visitor.visit(this);
     }
 
-    @Override
-    public String toString() {
-        return NAME;
-    }
-
     public boolean isRewrite() {
         return isRewrite;
     }
@@ -73,6 +70,13 @@ public class WildcardParseNode extends TerminalParseNode {
 		if (isRewrite != other.isRewrite)
 			return false;
 		return true;
-	}    
+	}
+
+    @Override
+    public void toSQL(ColumnResolver resolver, StringBuilder buf) {
+        buf.append(' ');
+        buf.append(NAME);
+        buf.append(' ');
+    }    
     
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/92298f8d/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java
index fceb724..2722cb6 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java
@@ -1002,6 +1002,12 @@ public class MetaDataClient {
                     }
                     unusedPkColumns.remove(expression);
                     
+                    // Go through parse node to get string as otherwise we
+                    // can lose information during compilation
+                    StringBuilder buf = new StringBuilder();
+                    parseNode.toSQL(resolver, buf);
+                    String expressionStr = buf.toString();
+                    
                     ColumnName colName = null;
                     ColumnRef colRef = expressionIndexCompiler.getColumnRef();
 					if (colRef!=null) { 
@@ -1013,13 +1019,13 @@ public class MetaDataClient {
 					else { 
 						// if this is an expression
 					    // TODO column names cannot have double quotes, remove this once this PHOENIX-1621 is fixed
-						String name = expression.toString().replaceAll("\"", "'");
+						String name = expressionStr.replaceAll("\"", "'");
                         colName = ColumnName.caseSensitiveColumnName(IndexUtil.getIndexColumnName(null, name));
 					}
 					indexedColumnNames.add(colName);
                 	PDataType dataType = IndexUtil.getIndexColumnDataType(expression.isNullable(), expression.getDataType());
                     allPkColumns.add(new Pair<ColumnName, SortOrder>(colName, pair.getSecond()));
-                    columnDefs.add(FACTORY.columnDef(colName, dataType.getSqlTypeName(), expression.isNullable(), expression.getMaxLength(), expression.getScale(), false, pair.getSecond(), expression.toString()));
+                    columnDefs.add(FACTORY.columnDef(colName, dataType.getSqlTypeName(), expression.isNullable(), expression.getMaxLength(), expression.getScale(), false, pair.getSecond(), expressionStr));
                 }
 
                 // Next all the PK columns from the data table that aren't indexed

http://git-wip-us.apache.org/repos/asf/phoenix/blob/92298f8d/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PDate.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PDate.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PDate.java
index fa070d3..b926afb 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PDate.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PDate.java
@@ -26,6 +26,7 @@ import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.phoenix.schema.SortOrder;
 import org.apache.phoenix.util.DateUtil;
+import org.apache.phoenix.util.StringUtil;
 
 public class PDate extends PDataType<Date> {
 
@@ -150,10 +151,10 @@ public class PDate extends PDataType<Date> {
   public String toStringLiteral(Object o, Format formatter) {
       if (formatter == null) {
           // If default formatter has not been overridden,
-          // use one that displays milliseconds.
+          // use default one.
           formatter = DateUtil.DEFAULT_DATE_FORMATTER;
         }
-        return "'" + super.toStringLiteral(o, formatter) + "'";
+        return "'" + StringUtil.escapeStringConstant(super.toStringLiteral(o, formatter)) + "'";
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/phoenix/blob/92298f8d/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PVarchar.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PVarchar.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PVarchar.java
index 6956942..9883e12 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PVarchar.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PVarchar.java
@@ -24,6 +24,7 @@ import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.phoenix.schema.SortOrder;
 import org.apache.phoenix.util.ByteUtil;
+import org.apache.phoenix.util.StringUtil;
 
 import com.google.common.base.Preconditions;
 
@@ -142,7 +143,7 @@ public class PVarchar extends PDataType<String> {
     if (formatter != null) {
       return "'" + formatter.format(o) + "'";
     }
-    return "'" + o + "'";
+    return "'" + StringUtil.escapeStringConstant(o.toString()) + "'";
   }
 
   private char[] sampleChars = new char[1];

http://git-wip-us.apache.org/repos/asf/phoenix/blob/92298f8d/phoenix-core/src/main/java/org/apache/phoenix/util/IndexUtil.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/util/IndexUtil.java b/phoenix-core/src/main/java/org/apache/phoenix/util/IndexUtil.java
index 72dac8d..b72d3db 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/util/IndexUtil.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/IndexUtil.java
@@ -451,8 +451,11 @@ public class IndexUtil {
         };
         ColumnResolver indexResolver = FromCompiler.getResolver(indexTableRef);
         StatementContext context = new StatementContext(statement, indexResolver);
-        Expression whereClause = WhereCompiler.compile(context, whereNode);
-        return QueryUtil.getViewStatement(index.getSchemaName().getString(), index.getTableName().getString(), whereClause);
+        // Compile to ensure validity
+        WhereCompiler.compile(context, whereNode);
+        StringBuilder buf = new StringBuilder();
+        whereNode.toSQL(indexResolver, buf);
+        return QueryUtil.getViewStatement(index.getSchemaName().getString(), index.getTableName().getString(), buf.toString());
     }
     
     public static void wrapResultUsingOffset(final ObserverContext<RegionCoprocessorEnvironment> c,

http://git-wip-us.apache.org/repos/asf/phoenix/blob/92298f8d/phoenix-core/src/main/java/org/apache/phoenix/util/QueryUtil.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/util/QueryUtil.java b/phoenix-core/src/main/java/org/apache/phoenix/util/QueryUtil.java
index b91fddc..a2953f2 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/util/QueryUtil.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/QueryUtil.java
@@ -34,9 +34,9 @@ import javax.annotation.Nullable;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
 import org.apache.hadoop.hbase.util.Addressing;
 import org.apache.hadoop.hbase.zookeeper.ZKConfig;
-import org.apache.phoenix.expression.Expression;
 import org.apache.phoenix.iterate.ResultIterator;
 import org.apache.phoenix.jdbc.PhoenixDriver;
 import org.apache.phoenix.parse.WildcardParseNode;
@@ -76,6 +76,19 @@ public final class QueryUtil {
     private static final String SELECT = "SELECT";
     private static final String FROM = "FROM";
     private static final String WHERE = "WHERE";
+    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()] = "<=";
+    }
+
+    public static String toSQL(CompareOp op) {
+        return CompareOpString[op.ordinal()];
+    }
     
     /**
      * Private constructor
@@ -263,11 +276,11 @@ public final class QueryUtil {
         return getUrl(server, port);
     }
     
-    public static String getViewStatement(String schemaName, String tableName, Expression whereClause) {
+    public static String getViewStatement(String schemaName, String tableName, String where) {
         // Only form we currently support for VIEWs: SELECT * FROM t WHERE ...
         return SELECT + " " + WildcardParseNode.NAME + " " + FROM + " " +
                 (schemaName == null || schemaName.length() == 0 ? "" : ("\"" + schemaName + "\".")) +
                 ("\"" + tableName + "\" ") +
-                (WHERE + " " + whereClause.toString());
+                (WHERE + " " + where);
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/phoenix/blob/92298f8d/phoenix-core/src/main/java/org/apache/phoenix/util/StringUtil.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/util/StringUtil.java b/phoenix-core/src/main/java/org/apache/phoenix/util/StringUtil.java
index d65af15..a83098a 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/util/StringUtil.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/StringUtil.java
@@ -19,6 +19,7 @@ package org.apache.phoenix.util;
 
 import java.util.Arrays;
 
+import org.apache.commons.lang.StringEscapeUtils;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.phoenix.exception.UndecodableByteException;
 import org.apache.phoenix.schema.SortOrder;
@@ -325,5 +326,9 @@ public class StringUtil {
         if (toIndex > length) {
             throw new ArrayIndexOutOfBoundsException(toIndex);
         }
+    }
+
+    public static String escapeStringConstant(String pattern) {
+        return StringEscapeUtils.escapeSql(pattern); // Need to escape double quotes
     }   
 }