You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@phoenix.apache.org by nd...@apache.org on 2014/12/18 01:11:58 UTC

[20/22] phoenix git commit: PHOENIX-1514 Break up PDataType Enum

http://git-wip-us.apache.org/repos/asf/phoenix/blob/04ef859b/phoenix-core/src/main/java/org/apache/phoenix/expression/AndOrExpression.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/AndOrExpression.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/AndOrExpression.java
index 89ad02e..ea8c375 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/expression/AndOrExpression.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/AndOrExpression.java
@@ -21,7 +21,8 @@ import java.util.BitSet;
 import java.util.List;
 
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
-import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.schema.types.PBoolean;
+import org.apache.phoenix.schema.types.PDataType;
 import org.apache.phoenix.schema.tuple.Tuple;
 
 
@@ -45,7 +46,7 @@ public abstract class AndOrExpression extends BaseCompoundExpression {
     
     @Override
     public PDataType getDataType() {
-        return PDataType.BOOLEAN;
+        return PBoolean.INSTANCE;
     }
 
     @Override
@@ -70,7 +71,7 @@ public abstract class AndOrExpression extends BaseCompoundExpression {
                 // evaluate versus getValue code path.
                 if (child.evaluate(tuple, ptr)) {
                     // Short circuit if we see our stop value
-                    if (isStopValue((Boolean)PDataType.BOOLEAN.toObject(ptr, child.getDataType()))) {
+                    if (isStopValue((Boolean) PBoolean.INSTANCE.toObject(ptr, child.getDataType()))) {
                         return true;
                     } else if (partialEvalState != null) {
                         partialEvalState.set(i);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/04ef859b/phoenix-core/src/main/java/org/apache/phoenix/expression/ArithmeticExpression.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/ArithmeticExpression.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/ArithmeticExpression.java
index 247e9d7..9212709 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/expression/ArithmeticExpression.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/ArithmeticExpression.java
@@ -19,7 +19,7 @@ package org.apache.phoenix.expression;
 
 import java.util.List;
 
-import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.schema.types.PDataType;
 
 public abstract class ArithmeticExpression extends BaseCompoundExpression {
 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/04ef859b/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 dd23534..9faab96 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
@@ -19,8 +19,8 @@ import java.util.List;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.hadoop.io.WritableUtils;
 import org.apache.phoenix.query.QueryConstants;
-import org.apache.phoenix.schema.PArrayDataType;
-import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.schema.types.PArrayDataType;
+import org.apache.phoenix.schema.types.PDataType;
 import org.apache.phoenix.schema.tuple.Tuple;
 import org.apache.phoenix.util.TrustedByteArrayOutputStream;
 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/04ef859b/phoenix-core/src/main/java/org/apache/phoenix/expression/BaseAddSubtractExpression.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/BaseAddSubtractExpression.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/BaseAddSubtractExpression.java
index da7bd9f..43654b7 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/expression/BaseAddSubtractExpression.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/BaseAddSubtractExpression.java
@@ -19,7 +19,7 @@ package org.apache.phoenix.expression;
 
 import java.util.List;
 
-import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.schema.types.PDataType;
 
 
 abstract public class BaseAddSubtractExpression extends ArithmeticExpression {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/04ef859b/phoenix-core/src/main/java/org/apache/phoenix/expression/BaseExpression.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/BaseExpression.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/BaseExpression.java
index 8993e37..e4f53ae 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/expression/BaseExpression.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/BaseExpression.java
@@ -33,14 +33,15 @@ import org.apache.phoenix.expression.function.FloorDateExpression;
 import org.apache.phoenix.expression.function.FloorDecimalExpression;
 import org.apache.phoenix.expression.function.TimeUnit;
 import org.apache.phoenix.expression.visitor.ExpressionVisitor;
-import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.schema.types.PDecimal;
+import org.apache.phoenix.schema.types.PDataType;
+import org.apache.phoenix.schema.types.PTimestamp;
+import org.apache.phoenix.schema.types.PUnsignedTimestamp;
 import org.apache.phoenix.schema.SortOrder;
 import org.apache.phoenix.schema.TypeMismatchException;
 
 import com.google.common.collect.Lists;
 
-
-
 /**
  * 
  * Base class for Expression hierarchy that provides common
@@ -71,9 +72,9 @@ public abstract class BaseExpression implements Expression {
                 Expression e = rhs;
                 PDataType rhsType = rhs.getDataType();
                 PDataType lhsType = lhs.getDataType();
-                if (rhsType == PDataType.DECIMAL && lhsType != PDataType.DECIMAL) {
+                if (rhsType == PDecimal.INSTANCE && lhsType != PDecimal.INSTANCE) {
                     e = FloorDecimalExpression.create(rhs);
-                } else if ((rhsType == PDataType.TIMESTAMP || rhsType == PDataType.UNSIGNED_TIMESTAMP)  && (lhsType != PDataType.TIMESTAMP && lhsType != PDataType.UNSIGNED_TIMESTAMP)) {
+                } else if ((rhsType == PTimestamp.INSTANCE || rhsType == PUnsignedTimestamp.INSTANCE)  && (lhsType != PTimestamp.INSTANCE && lhsType != PUnsignedTimestamp.INSTANCE)) {
                     e = FloorDateExpression.create(rhs, TimeUnit.MILLISECOND);
                 }
                 e = CoerceExpression.create(e, lhsType, lhs.getSortOrder(), lhs.getMaxLength());
@@ -90,9 +91,9 @@ public abstract class BaseExpression implements Expression {
                 Expression e = rhs;
                 PDataType rhsType = rhs.getDataType();
                 PDataType lhsType = lhs.getDataType();
-                if (rhsType == PDataType.DECIMAL && lhsType != PDataType.DECIMAL) {
+                if (rhsType == PDecimal.INSTANCE && lhsType != PDecimal.INSTANCE) {
                     e = CeilDecimalExpression.create(rhs);
-                } else if ((rhsType == PDataType.TIMESTAMP || rhsType == PDataType.UNSIGNED_TIMESTAMP)  && (lhsType != PDataType.TIMESTAMP && lhsType != PDataType.UNSIGNED_TIMESTAMP)) {
+                } else if ((rhsType == PTimestamp.INSTANCE || rhsType == PUnsignedTimestamp.INSTANCE)  && (lhsType != PTimestamp.INSTANCE && lhsType != PUnsignedTimestamp.INSTANCE)) {
                     e = CeilTimestampExpression.create(rhs);
                 }
                 e = CoerceExpression.create(e, lhsType, lhs.getSortOrder(), lhs.getMaxLength());

http://git-wip-us.apache.org/repos/asf/phoenix/blob/04ef859b/phoenix-core/src/main/java/org/apache/phoenix/expression/CaseExpression.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/CaseExpression.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/CaseExpression.java
index d4784bb..96338cc 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/expression/CaseExpression.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/CaseExpression.java
@@ -29,7 +29,9 @@ import org.apache.hadoop.io.WritableUtils;
 import org.apache.phoenix.exception.SQLExceptionCode;
 import org.apache.phoenix.exception.SQLExceptionInfo;
 import org.apache.phoenix.expression.visitor.ExpressionVisitor;
-import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.schema.types.PDecimal;
+import org.apache.phoenix.schema.types.PBoolean;
+import org.apache.phoenix.schema.types.PDataType;
 import org.apache.phoenix.schema.tuple.Tuple;
 
 
@@ -73,8 +75,8 @@ public class CaseExpression extends BaseCompoundExpression {
         }
         // If we found an "unknown" child type and the return type is a number
         // make the return type be the most general number type of DECIMAL.
-        if (isChildTypeUnknown && returnType != null && returnType.isCoercibleTo(PDataType.DECIMAL)) {
-            returnType = PDataType.DECIMAL;
+        if (isChildTypeUnknown && returnType != null && returnType.isCoercibleTo(PDecimal.INSTANCE)) {
+            returnType = PDecimal.INSTANCE;
         }
         List<Expression> newChildren = children;
         for (int i = 0; i < children.size(); i+=2) {
@@ -156,7 +158,7 @@ public class CaseExpression extends BaseCompoundExpression {
             // If we get null, we have to re-evaluate from that point (special case this in filter, like is null)
             // We may only run this when we're done/have all values
             boolean evaluated = children.get(i+1).evaluate(tuple, ptr);
-            if (evaluated && Boolean.TRUE.equals(PDataType.BOOLEAN.toObject(ptr))) {
+            if (evaluated && Boolean.TRUE.equals(PBoolean.INSTANCE.toObject(ptr))) {
                 if (isPartiallyEvaluating()) {
                     foundIndex = true;
                 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/04ef859b/phoenix-core/src/main/java/org/apache/phoenix/expression/CoerceExpression.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/CoerceExpression.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/CoerceExpression.java
index 99c1934..275ed23 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/expression/CoerceExpression.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/CoerceExpression.java
@@ -26,7 +26,7 @@ import java.util.List;
 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.PDataType;
+import org.apache.phoenix.schema.types.PDataType;
 import org.apache.phoenix.schema.SortOrder;
 import org.apache.phoenix.schema.tuple.Tuple;
 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/04ef859b/phoenix-core/src/main/java/org/apache/phoenix/expression/ColumnExpression.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/ColumnExpression.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/ColumnExpression.java
index eb60591..2c9d378 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/expression/ColumnExpression.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/ColumnExpression.java
@@ -22,7 +22,7 @@ import java.io.DataOutput;
 import java.io.IOException;
 
 import org.apache.hadoop.io.WritableUtils;
-import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.schema.types.PDataType;
 import org.apache.phoenix.schema.PDatum;
 import org.apache.phoenix.schema.SortOrder;
 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/04ef859b/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 a1ed221..38b69b7 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
@@ -31,7 +31,14 @@ import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.hadoop.io.WritableUtils;
 import org.apache.phoenix.expression.function.InlineArrayElemRefExpression;
 import org.apache.phoenix.expression.visitor.ExpressionVisitor;
-import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.schema.types.PDecimal;
+import org.apache.phoenix.schema.types.PBoolean;
+import org.apache.phoenix.schema.types.PChar;
+import org.apache.phoenix.schema.types.PInteger;
+import org.apache.phoenix.schema.types.PDataType;
+import org.apache.phoenix.schema.types.PLong;
+import org.apache.phoenix.schema.types.PUnsignedInt;
+import org.apache.phoenix.schema.types.PUnsignedLong;
 import org.apache.phoenix.schema.SortOrder;
 import org.apache.phoenix.schema.TypeMismatchException;
 import org.apache.phoenix.schema.tuple.Tuple;
@@ -64,7 +71,7 @@ public class ComparisonExpression extends BaseCompoundExpression {
         boolean isLHSNull = ExpressionUtil.isNull(lhs, ptr);
         boolean isRHSNull = ExpressionUtil.isNull(rhs, ptr);
         if (isLHSNull && isRHSNull) { // null == null will end up making the query degenerate
-            andNodes.add(LiteralExpression.newConstant(false, PDataType.BOOLEAN));
+            andNodes.add(LiteralExpression.newConstant(false, PBoolean.INSTANCE));
         } else if (isLHSNull) { // AND rhs IS NULL
             andNodes.add(IsNullExpression.create(rhs, false, ptr));
         } else if (isRHSNull) { // AND lhs IS NULL
@@ -131,7 +138,8 @@ public class ComparisonExpression extends BaseCompoundExpression {
             }
             children = Arrays.asList(lhsExpr, rhsExpr);
         } else if(lhsExprDataType != null && rhsExprDataType != null && !lhsExprDataType.isComparableTo(rhsExprDataType)) {
-            throw TypeMismatchException.newException(lhsExprDataType, rhsExprDataType, toString(op, children));
+            throw TypeMismatchException.newException(lhsExprDataType, rhsExprDataType,
+                toString(op, children));
         }
         Determinism determinism =  lhsExpr.getDeterminism().combine(rhsExpr.getDeterminism());
         
@@ -142,7 +150,7 @@ public class ComparisonExpression extends BaseCompoundExpression {
         if (lhsExpr instanceof LiteralExpression) {
             lhsValue = ((LiteralExpression)lhsExpr).getValue();
             if (lhsValue == null) {
-                return LiteralExpression.newConstant(false, PDataType.BOOLEAN, lhsExpr.getDeterminism());
+                return LiteralExpression.newConstant(false, PBoolean.INSTANCE, lhsExpr.getDeterminism());
             }
         }
         Object rhsValue = null;
@@ -150,7 +158,7 @@ public class ComparisonExpression extends BaseCompoundExpression {
         if (rhsExpr instanceof LiteralExpression) {
             rhsValue = ((LiteralExpression)rhsExpr).getValue();
             if (rhsValue == null) {
-                return LiteralExpression.newConstant(false, PDataType.BOOLEAN, rhsExpr.getDeterminism());
+                return LiteralExpression.newConstant(false, PBoolean.INSTANCE, rhsExpr.getDeterminism());
             }
         }
         if (lhsValue != null && rhsValue != null) {
@@ -170,77 +178,74 @@ public class ComparisonExpression extends BaseCompoundExpression {
                     children = Arrays.asList(children.get(0), LiteralExpression.newConstant(rhsValue, lhsExprDataType, 
                             lhsExpr.getMaxLength(), null, lhsExpr.getSortOrder(), determinism));
                 } else if (op == CompareOp.EQUAL) {
-                    return LiteralExpression.newConstant(false, PDataType.BOOLEAN, Determinism.ALWAYS);
+                    return LiteralExpression.newConstant(false, PBoolean.INSTANCE, Determinism.ALWAYS);
                 } else if (op == CompareOp.NOT_EQUAL) {
-                    return LiteralExpression.newConstant(true, PDataType.BOOLEAN, Determinism.ALWAYS);
+                    return LiteralExpression.newConstant(true, PBoolean.INSTANCE, Determinism.ALWAYS);
                 } else { // TODO: generalize this with PDataType.getMinValue(), PDataTypeType.getMaxValue() methods
-                    switch(rhsExprDataType) {
-                    case DECIMAL:
+                  if (rhsExprDataType == PDecimal.INSTANCE) {
                         /*
                          * We're comparing an int/long to a constant decimal with a fraction part.
                          * We need the types to match in case this is used to form a key. To form the start/stop key,
                          * we need to adjust the decimal by truncating it or taking its ceiling, depending on the comparison
                          * operator, to get a whole number.
                          */
-                        int increment = 0;
-                        switch (op) {
-                        case GREATER_OR_EQUAL: 
-                        case LESS: // get next whole number
-                            increment = 1;
-                        default: // Else, we truncate the value
-                            BigDecimal bd = (BigDecimal)rhsValue;
-                            rhsValue = bd.longValue() + increment;
-                            children = Arrays.asList(lhsExpr, LiteralExpression.newConstant(rhsValue, lhsExprDataType, lhsExpr.getSortOrder(), rhsExpr.getDeterminism()));
-                            break;
-                        }
-                        break;
-                    case LONG:
+                    int increment = 0;
+                    switch (op) {
+                    case GREATER_OR_EQUAL:
+                    case LESS: // get next whole number
+                      increment = 1;
+                    default: // Else, we truncate the value
+                      BigDecimal bd = (BigDecimal)rhsValue;
+                      rhsValue = bd.longValue() + increment;
+                      children = Arrays.asList(lhsExpr, LiteralExpression.newConstant(rhsValue, lhsExprDataType, lhsExpr.getSortOrder(), rhsExpr.getDeterminism()));
+                      break;
+                    }
+                  } else if (rhsExprDataType == PLong.INSTANCE) {
                         /*
                          * We are comparing an int, unsigned_int to a long, or an unsigned_long to a negative long.
                          * int has range of -2147483648 to 2147483647, and unsigned_int has a value range of 0 to 4294967295.
-                         * 
-                         * If lhs is int or unsigned_int, since we already determined that we cannot coerce the rhs 
+                         *
+                         * If lhs is int or unsigned_int, since we already determined that we cannot coerce the rhs
                          * to become the lhs, we know the value on the rhs is greater than lhs if it's positive, or smaller than
                          * lhs if it's negative.
-                         * 
+                         *
                          * If lhs is an unsigned_long, then we know the rhs is definitely a negative long. rhs in this case
                          * will always be bigger than rhs.
                          */
-                        if (lhsExprDataType == PDataType.INTEGER || 
-                        lhsExprDataType == PDataType.UNSIGNED_INT) {
-                            switch (op) {
-                            case LESS:
-                            case LESS_OR_EQUAL:
-                                if ((Long)rhsValue > 0) {
-                                    return LiteralExpression.newConstant(true, PDataType.BOOLEAN, determinism);
-                                } else {
-                                    return LiteralExpression.newConstant(false, PDataType.BOOLEAN, determinism);
-                                }
-                            case GREATER:
-                            case GREATER_OR_EQUAL:
-                                if ((Long)rhsValue > 0) {
-                                    return LiteralExpression.newConstant(false, PDataType.BOOLEAN, determinism);
-                                } else {
-                                    return LiteralExpression.newConstant(true, PDataType.BOOLEAN, determinism);
-                                }
-                            default:
-                                break;
-                            }
-                        } else if (lhsExprDataType == PDataType.UNSIGNED_LONG) {
-                            switch (op) {
-                            case LESS:
-                            case LESS_OR_EQUAL:
-                                return LiteralExpression.newConstant(false, PDataType.BOOLEAN, determinism);
-                            case GREATER:
-                            case GREATER_OR_EQUAL:
-                                return LiteralExpression.newConstant(true, PDataType.BOOLEAN, determinism);
-                            default:
-                                break;
-                            }
+                    if (lhsExprDataType == PInteger.INSTANCE ||
+                        lhsExprDataType == PUnsignedInt.INSTANCE) {
+                      switch (op) {
+                      case LESS:
+                      case LESS_OR_EQUAL:
+                        if ((Long)rhsValue > 0) {
+                          return LiteralExpression.newConstant(true, PBoolean.INSTANCE, determinism);
+                        } else {
+                          return LiteralExpression.newConstant(false, PBoolean.INSTANCE, determinism);
                         }
-                        children = Arrays.asList(lhsExpr, LiteralExpression.newConstant(rhsValue, rhsExprDataType, lhsExpr.getSortOrder(), determinism));
+                      case GREATER:
+                      case GREATER_OR_EQUAL:
+                        if ((Long)rhsValue > 0) {
+                          return LiteralExpression.newConstant(false, PBoolean.INSTANCE, determinism);
+                        } else {
+                          return LiteralExpression.newConstant(true, PBoolean.INSTANCE, determinism);
+                        }
+                      default:
+                        break;
+                      }
+                    } else if (lhsExprDataType == PUnsignedLong.INSTANCE) {
+                      switch (op) {
+                      case LESS:
+                      case LESS_OR_EQUAL:
+                        return LiteralExpression.newConstant(false, PBoolean.INSTANCE, determinism);
+                      case GREATER:
+                      case GREATER_OR_EQUAL:
+                        return LiteralExpression.newConstant(true, PBoolean.INSTANCE, determinism);
+                      default:
                         break;
+                      }
                     }
+                    children = Arrays.asList(lhsExpr, LiteralExpression.newConstant(rhsValue, rhsExprDataType, lhsExpr.getSortOrder(), determinism));
+                  }
                 }
             }
 
@@ -249,9 +254,9 @@ public class ComparisonExpression extends BaseCompoundExpression {
             if (children.get(1).getMaxLength() != null && lhsExpr.getMaxLength() != null && lhsExpr.getMaxLength() < children.get(1).getMaxLength()) {
                 switch (op) {
                 case EQUAL:
-                    return LiteralExpression.newConstant(false, PDataType.BOOLEAN, determinism);
+                    return LiteralExpression.newConstant(false, PBoolean.INSTANCE, determinism);
                 case NOT_EQUAL:
-                    return LiteralExpression.newConstant(true, PDataType.BOOLEAN, determinism);
+                    return LiteralExpression.newConstant(true, PBoolean.INSTANCE, determinism);
                 default:
                     break;
                 }
@@ -291,7 +296,7 @@ public class ComparisonExpression extends BaseCompoundExpression {
 
     @Override
     public PDataType getDataType() {
-        return PDataType.BOOLEAN;
+        return PBoolean.INSTANCE;
     }
 
     @Override
@@ -320,10 +325,10 @@ public class ComparisonExpression extends BaseCompoundExpression {
         int rhsLength = ptr.getLength();
         PDataType rhsDataType = children.get(1).getDataType();
         SortOrder rhsSortOrder = children.get(1).getSortOrder();   
-        if (rhsDataType == PDataType.CHAR) {
+        if (rhsDataType == PChar.INSTANCE) {
             rhsLength = StringUtil.getUnpaddedCharLength(rhsBytes, rhsOffset, rhsLength, rhsSortOrder);
         }
-        if (lhsDataType == PDataType.CHAR) {
+        if (lhsDataType == PChar.INSTANCE) {
             lhsLength = StringUtil.getUnpaddedCharLength(lhsBytes, lhsOffset, lhsLength, lhsSortOrder);
         }
         
@@ -368,4 +373,4 @@ public class ComparisonExpression extends BaseCompoundExpression {
     public String toString() {
         return toString(getFilterOp(), children);
     }    
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/04ef859b/phoenix-core/src/main/java/org/apache/phoenix/expression/DateAddExpression.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/DateAddExpression.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/DateAddExpression.java
index f39e6d1..bc79d15 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/expression/DateAddExpression.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/DateAddExpression.java
@@ -23,8 +23,12 @@ import java.util.List;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 
 import org.apache.phoenix.query.QueryConstants;
+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.PLong;
 import org.apache.phoenix.schema.SortOrder;
-import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.schema.types.PDataType;
 import org.apache.phoenix.schema.tuple.Tuple;
 
 
@@ -52,12 +56,12 @@ public class DateAddExpression extends AddExpression {
             long value;
             PDataType type = children.get(i).getDataType();
             SortOrder sortOrder = children.get(i).getSortOrder();
-            if (type == PDataType.DECIMAL) {
-                BigDecimal bd = (BigDecimal)PDataType.DECIMAL.toObject(ptr, sortOrder);
+            if (type == PDecimal.INSTANCE) {
+                BigDecimal bd = (BigDecimal) PDecimal.INSTANCE.toObject(ptr, sortOrder);
                 value = bd.multiply(BD_MILLIS_IN_DAY).longValue();
-            } else if (type.isCoercibleTo(PDataType.LONG)) {
+            } else if (type.isCoercibleTo(PLong.INSTANCE)) {
                 value = type.getCodec().decodeLong(ptr, sortOrder) * QueryConstants.MILLIS_IN_DAY;
-            } else if (type.isCoercibleTo(PDataType.DOUBLE)) {
+            } else if (type.isCoercibleTo(PDouble.INSTANCE)) {
                 value = (long)(type.getCodec().decodeDouble(ptr, sortOrder) * QueryConstants.MILLIS_IN_DAY);
             } else {
                 value = type.getCodec().decodeLong(ptr, sortOrder);
@@ -72,7 +76,7 @@ public class DateAddExpression extends AddExpression {
 
     @Override
     public final PDataType getDataType() {
-        return PDataType.DATE;
+        return PDate.INSTANCE;
     }
 
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/04ef859b/phoenix-core/src/main/java/org/apache/phoenix/expression/DateSubtractExpression.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/DateSubtractExpression.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/DateSubtractExpression.java
index d13ffee..3f5897c 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/expression/DateSubtractExpression.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/DateSubtractExpression.java
@@ -23,8 +23,12 @@ import java.util.List;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 
 import org.apache.phoenix.query.QueryConstants;
+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.PLong;
 import org.apache.phoenix.schema.SortOrder;
-import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.schema.types.PDataType;
 import org.apache.phoenix.schema.tuple.Tuple;
 
 
@@ -48,12 +52,12 @@ public class DateSubtractExpression extends SubtractExpression {
             long value;
             PDataType type = children.get(i).getDataType();
             SortOrder sortOrder = children.get(i).getSortOrder();
-            if (type == PDataType.DECIMAL) {
-                BigDecimal bd = (BigDecimal)PDataType.DECIMAL.toObject(ptr, sortOrder);
+            if (type == PDecimal.INSTANCE) {
+                BigDecimal bd = (BigDecimal) PDecimal.INSTANCE.toObject(ptr, sortOrder);
                 value = bd.multiply(BD_MILLIS_IN_DAY).longValue();
-            } else if (type.isCoercibleTo(PDataType.LONG)) {
+            } else if (type.isCoercibleTo(PLong.INSTANCE)) {
                 value = type.getCodec().decodeLong(ptr, sortOrder) * QueryConstants.MILLIS_IN_DAY;
-            } else if (type.isCoercibleTo(PDataType.DOUBLE)) {
+            } else if (type.isCoercibleTo(PDouble.INSTANCE)) {
                 value = (long)(type.getCodec().decodeDouble(ptr, sortOrder) * QueryConstants.MILLIS_IN_DAY);
             } else {
                 value = type.getCodec().decodeLong(ptr, sortOrder);
@@ -72,7 +76,7 @@ public class DateSubtractExpression extends SubtractExpression {
 
     @Override
     public final PDataType getDataType() {
-        return PDataType.DATE;
+        return PDate.INSTANCE;
     }
 
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/04ef859b/phoenix-core/src/main/java/org/apache/phoenix/expression/DecimalAddExpression.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/DecimalAddExpression.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/DecimalAddExpression.java
index 3e4e4ed..df1ec4f 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/expression/DecimalAddExpression.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/DecimalAddExpression.java
@@ -22,7 +22,8 @@ import java.util.List;
 
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.phoenix.exception.ValueTypeIncompatibleException;
-import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.schema.types.PDecimal;
+import org.apache.phoenix.schema.types.PDataType;
 import org.apache.phoenix.schema.SortOrder;
 import org.apache.phoenix.schema.tuple.Tuple;
 import org.apache.phoenix.util.NumberUtil;
@@ -61,7 +62,7 @@ public class DecimalAddExpression extends AddExpression {
             
             PDataType childType = childExpr.getDataType();
             SortOrder childSortOrder = childExpr.getSortOrder();
-            BigDecimal bd = (BigDecimal)PDataType.DECIMAL.toObject(ptr, childType, childSortOrder);
+            BigDecimal bd = (BigDecimal) PDecimal.INSTANCE.toObject(ptr, childType, childSortOrder);
             
             if (result == null) {
                 result = bd;
@@ -73,15 +74,15 @@ public class DecimalAddExpression extends AddExpression {
             result = NumberUtil.setDecimalWidthAndScale(result, maxLength, scale);
         }
         if (result == null) {
-            throw new ValueTypeIncompatibleException(PDataType.DECIMAL, maxLength, scale);
+            throw new ValueTypeIncompatibleException(PDecimal.INSTANCE, maxLength, scale);
         }
-        ptr.set(PDataType.DECIMAL.toBytes(result));
+        ptr.set(PDecimal.INSTANCE.toBytes(result));
         return true;
     }
 
     @Override
     public PDataType getDataType() {
-        return PDataType.DECIMAL;
+        return PDecimal.INSTANCE;
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/phoenix/blob/04ef859b/phoenix-core/src/main/java/org/apache/phoenix/expression/DecimalDivideExpression.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/DecimalDivideExpression.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/DecimalDivideExpression.java
index fc37552..81ff3bc 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/expression/DecimalDivideExpression.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/DecimalDivideExpression.java
@@ -22,7 +22,8 @@ import java.util.List;
 
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.phoenix.exception.ValueTypeIncompatibleException;
-import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.schema.types.PDecimal;
+import org.apache.phoenix.schema.types.PDataType;
 import org.apache.phoenix.schema.SortOrder;
 import org.apache.phoenix.schema.tuple.Tuple;
 import org.apache.phoenix.util.NumberUtil;
@@ -51,7 +52,7 @@ public class DecimalDivideExpression extends DivideExpression {
             
             PDataType childType = childExpr.getDataType();
             SortOrder childSortOrder = childExpr.getSortOrder();
-            BigDecimal bd= (BigDecimal)PDataType.DECIMAL.toObject(ptr, childType, childSortOrder);
+            BigDecimal bd= (BigDecimal) PDecimal.INSTANCE.toObject(ptr, childType, childSortOrder);
             
             if (result == null) {
                 result = bd;
@@ -63,14 +64,14 @@ public class DecimalDivideExpression extends DivideExpression {
             result = NumberUtil.setDecimalWidthAndScale(result, getMaxLength(), getScale());
         }
         if (result == null) {
-            throw new ValueTypeIncompatibleException(PDataType.DECIMAL, getMaxLength(), getScale());
+            throw new ValueTypeIncompatibleException(PDecimal.INSTANCE, getMaxLength(), getScale());
         }
-        ptr.set(PDataType.DECIMAL.toBytes(result));
+        ptr.set(PDecimal.INSTANCE.toBytes(result));
         return true;
     }
 
     @Override
     public PDataType getDataType() {
-        return PDataType.DECIMAL;
+        return PDecimal.INSTANCE;
     }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/04ef859b/phoenix-core/src/main/java/org/apache/phoenix/expression/DecimalMultiplyExpression.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/DecimalMultiplyExpression.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/DecimalMultiplyExpression.java
index ea34d36..0a9e3de 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/expression/DecimalMultiplyExpression.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/DecimalMultiplyExpression.java
@@ -22,7 +22,8 @@ import java.util.List;
 
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.phoenix.exception.ValueTypeIncompatibleException;
-import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.schema.types.PDecimal;
+import org.apache.phoenix.schema.types.PDataType;
 import org.apache.phoenix.schema.SortOrder;
 import org.apache.phoenix.schema.tuple.Tuple;
 import org.apache.phoenix.util.NumberUtil;
@@ -51,7 +52,7 @@ public class DecimalMultiplyExpression extends MultiplyExpression {
             
             PDataType childType = children.get(i).getDataType();
             SortOrder childSortOrder = children.get(i).getSortOrder();
-            BigDecimal bd = (BigDecimal)PDataType.DECIMAL.toObject(ptr, childType, childSortOrder);
+            BigDecimal bd = (BigDecimal) PDecimal.INSTANCE.toObject(ptr, childType, childSortOrder);
             
             if (result == null) {
                 result = bd;
@@ -63,14 +64,14 @@ public class DecimalMultiplyExpression extends MultiplyExpression {
             result = NumberUtil.setDecimalWidthAndScale(result, getMaxLength(), getScale());
         }
         if (result == null) {
-            throw new ValueTypeIncompatibleException(PDataType.DECIMAL, getMaxLength(), getScale());
+            throw new ValueTypeIncompatibleException(PDecimal.INSTANCE, getMaxLength(), getScale());
         }
-        ptr.set(PDataType.DECIMAL.toBytes(result));
+        ptr.set(PDecimal.INSTANCE.toBytes(result));
         return true;
     }
 
     @Override
     public PDataType getDataType() {
-        return PDataType.DECIMAL;
+        return PDecimal.INSTANCE;
     }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/04ef859b/phoenix-core/src/main/java/org/apache/phoenix/expression/DecimalSubtractExpression.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/DecimalSubtractExpression.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/DecimalSubtractExpression.java
index 8188a90..c46a70f 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/expression/DecimalSubtractExpression.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/DecimalSubtractExpression.java
@@ -22,7 +22,9 @@ import java.util.List;
 
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.phoenix.exception.ValueTypeIncompatibleException;
-import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.schema.types.PDate;
+import org.apache.phoenix.schema.types.PDecimal;
+import org.apache.phoenix.schema.types.PDataType;
 import org.apache.phoenix.schema.SortOrder;
 import org.apache.phoenix.schema.tuple.Tuple;
 import org.apache.phoenix.util.NumberUtil;
@@ -67,11 +69,11 @@ public class DecimalSubtractExpression extends SubtractExpression {
             }
             
             PDataType childType = childExpr.getDataType();
-            boolean isDate = childType.isCoercibleTo(PDataType.DATE);
+            boolean isDate = childType.isCoercibleTo(PDate.INSTANCE);
             SortOrder childSortOrder = childExpr.getSortOrder();
             BigDecimal bd = isDate ?
                     BigDecimal.valueOf(childType.getCodec().decodeLong(ptr, childSortOrder)) :
-                    (BigDecimal)PDataType.DECIMAL.toObject(ptr, childType, childSortOrder);
+                    (BigDecimal) PDecimal.INSTANCE.toObject(ptr, childType, childSortOrder);
             
             if (result == null) {
                 result = bd;
@@ -90,15 +92,15 @@ public class DecimalSubtractExpression extends SubtractExpression {
             result = NumberUtil.setDecimalWidthAndScale(result, maxLength, scale);
         }
         if (result == null) {
-            throw new ValueTypeIncompatibleException(PDataType.DECIMAL, maxLength, scale);
+            throw new ValueTypeIncompatibleException(PDecimal.INSTANCE, maxLength, scale);
         }
-        ptr.set(PDataType.DECIMAL.toBytes(result));
+        ptr.set(PDecimal.INSTANCE.toBytes(result));
         return true;
     }
 
     @Override
     public PDataType getDataType() {
-        return PDataType.DECIMAL;
+        return PDecimal.INSTANCE;
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/phoenix/blob/04ef859b/phoenix-core/src/main/java/org/apache/phoenix/expression/DelegateExpression.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/DelegateExpression.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/DelegateExpression.java
index 87a0bc0..3ca93dd 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/expression/DelegateExpression.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/DelegateExpression.java
@@ -24,7 +24,7 @@ import java.util.List;
 
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.phoenix.expression.visitor.ExpressionVisitor;
-import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.schema.types.PDataType;
 import org.apache.phoenix.schema.SortOrder;
 import org.apache.phoenix.schema.tuple.Tuple;
 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/04ef859b/phoenix-core/src/main/java/org/apache/phoenix/expression/DivideExpression.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/DivideExpression.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/DivideExpression.java
index 5b5ffd8..d5ff956 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/expression/DivideExpression.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/DivideExpression.java
@@ -20,7 +20,7 @@ package org.apache.phoenix.expression;
 import java.util.List;
 
 import org.apache.phoenix.expression.visitor.ExpressionVisitor;
-import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.schema.types.PDataType;
 
 
 /**

http://git-wip-us.apache.org/repos/asf/phoenix/blob/04ef859b/phoenix-core/src/main/java/org/apache/phoenix/expression/DoubleAddExpression.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/DoubleAddExpression.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/DoubleAddExpression.java
index f02327b..8ca2f05 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/expression/DoubleAddExpression.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/DoubleAddExpression.java
@@ -21,7 +21,8 @@ import java.util.List;
 
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 
-import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.schema.types.PDataType;
+import org.apache.phoenix.schema.types.PDouble;
 import org.apache.phoenix.schema.tuple.Tuple;
 
 public class DoubleAddExpression extends AddExpression {
@@ -62,7 +63,7 @@ public class DoubleAddExpression extends AddExpression {
 
     @Override
     public PDataType getDataType() {
-        return PDataType.DOUBLE;
+        return PDouble.INSTANCE;
     }
 
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/04ef859b/phoenix-core/src/main/java/org/apache/phoenix/expression/DoubleDivideExpression.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/DoubleDivideExpression.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/DoubleDivideExpression.java
index cedeadc..161e060 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/expression/DoubleDivideExpression.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/DoubleDivideExpression.java
@@ -21,7 +21,8 @@ import java.util.List;
 
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 
-import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.schema.types.PDataType;
+import org.apache.phoenix.schema.types.PDouble;
 import org.apache.phoenix.schema.tuple.Tuple;
 
 public class DoubleDivideExpression extends DivideExpression {
@@ -66,7 +67,7 @@ public class DoubleDivideExpression extends DivideExpression {
 
     @Override
     public PDataType getDataType() {
-        return PDataType.DOUBLE;
+        return PDouble.INSTANCE;
     }
 
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/04ef859b/phoenix-core/src/main/java/org/apache/phoenix/expression/DoubleMultiplyExpression.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/DoubleMultiplyExpression.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/DoubleMultiplyExpression.java
index 45a29d8..28a381c 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/expression/DoubleMultiplyExpression.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/DoubleMultiplyExpression.java
@@ -21,7 +21,8 @@ import java.util.List;
 
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 
-import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.schema.types.PDataType;
+import org.apache.phoenix.schema.types.PDouble;
 import org.apache.phoenix.schema.tuple.Tuple;
 
 public class DoubleMultiplyExpression extends MultiplyExpression {
@@ -62,7 +63,7 @@ public class DoubleMultiplyExpression extends MultiplyExpression {
 
     @Override
     public PDataType getDataType() {
-        return PDataType.DOUBLE;
+        return PDouble.INSTANCE;
     }
 
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/04ef859b/phoenix-core/src/main/java/org/apache/phoenix/expression/DoubleSubtractExpression.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/DoubleSubtractExpression.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/DoubleSubtractExpression.java
index 41da7c9..6d2ca3e 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/expression/DoubleSubtractExpression.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/DoubleSubtractExpression.java
@@ -21,7 +21,8 @@ import java.util.List;
 
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 
-import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.schema.types.PDataType;
+import org.apache.phoenix.schema.types.PDouble;
 import org.apache.phoenix.schema.tuple.Tuple;
 
 public class DoubleSubtractExpression extends SubtractExpression {
@@ -66,7 +67,7 @@ public class DoubleSubtractExpression extends SubtractExpression {
 
     @Override
     public PDataType getDataType() {
-        return PDataType.DOUBLE;
+        return PDouble.INSTANCE;
     }
 
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/04ef859b/phoenix-core/src/main/java/org/apache/phoenix/expression/InListExpression.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/InListExpression.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/InListExpression.java
index fdb20ff..685a541 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/expression/InListExpression.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/InListExpression.java
@@ -33,7 +33,8 @@ import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.io.WritableUtils;
 import org.apache.phoenix.expression.visitor.ExpressionVisitor;
 import org.apache.phoenix.hbase.index.util.ImmutableBytesPtr;
-import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.schema.types.PBoolean;
+import org.apache.phoenix.schema.types.PDataType;
 import org.apache.phoenix.schema.SortOrder;
 import org.apache.phoenix.schema.tuple.Tuple;
 import org.apache.phoenix.util.ByteUtil;
@@ -60,7 +61,7 @@ public class InListExpression extends BaseSingleExpression {
         Expression firstChild = children.get(0);
         
         if (firstChild.isStateless() && (!firstChild.evaluate(null, ptr) || ptr.getLength() == 0)) {
-            return LiteralExpression.newConstant(null, PDataType.BOOLEAN, firstChild.getDeterminism());
+            return LiteralExpression.newConstant(null, PBoolean.INSTANCE, firstChild.getDeterminism());
         }
         if (children.size() == 2) {
             return ComparisonExpression.create(isNegate ? CompareOp.NOT_EQUAL : CompareOp.EQUAL, children, ptr);
@@ -86,7 +87,7 @@ public class InListExpression extends BaseSingleExpression {
             throw sqlE;
         }
         if (coercedKeyExpressions.size() == 2 && addedNull) {
-            return LiteralExpression.newConstant(null, PDataType.BOOLEAN, Determinism.ALWAYS);
+            return LiteralExpression.newConstant(null, PBoolean.INSTANCE, Determinism.ALWAYS);
         }
         Expression expression = new InListExpression(coercedKeyExpressions);
         if (isNegate) { 
@@ -178,7 +179,7 @@ public class InListExpression extends BaseSingleExpression {
 
     @Override
     public PDataType getDataType() {
-        return PDataType.BOOLEAN;
+        return PBoolean.INSTANCE;
     }
 
     private int readValue(DataInput input, byte[] valuesBytes, int offset, ImmutableBytesPtr ptr) throws IOException {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/04ef859b/phoenix-core/src/main/java/org/apache/phoenix/expression/IsNullExpression.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/IsNullExpression.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/IsNullExpression.java
index 3fb7a62..b4095e3 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/expression/IsNullExpression.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/IsNullExpression.java
@@ -25,7 +25,8 @@ import java.util.List;
 
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.phoenix.expression.visitor.ExpressionVisitor;
-import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.schema.types.PBoolean;
+import org.apache.phoenix.schema.types.PDataType;
 import org.apache.phoenix.schema.tuple.Tuple;
 import org.apache.phoenix.util.ExpressionUtil;
 
@@ -42,11 +43,11 @@ public class IsNullExpression extends BaseSingleExpression {
 
     public static Expression create(Expression child, boolean negate, ImmutableBytesWritable ptr) throws SQLException {
         if (!child.isNullable()) {
-            return LiteralExpression.newConstant(negate, PDataType.BOOLEAN, child.getDeterminism());
+            return LiteralExpression.newConstant(negate, PBoolean.INSTANCE, child.getDeterminism());
         }
         if (ExpressionUtil.isConstant(child)) {
             boolean evaluated = child.evaluate(null, ptr);
-            return LiteralExpression.newConstant(negate ^ (!evaluated || ptr.getLength() == 0), PDataType.BOOLEAN, child.getDeterminism());
+            return LiteralExpression.newConstant(negate ^ (!evaluated || ptr.getLength() == 0), PBoolean.INSTANCE, child.getDeterminism());
         }
         return new IsNullExpression(child, negate);
     }
@@ -92,7 +93,7 @@ public class IsNullExpression extends BaseSingleExpression {
 
     @Override
     public PDataType getDataType() {
-        return PDataType.BOOLEAN;
+        return PBoolean.INSTANCE;
     }
     
     @Override

http://git-wip-us.apache.org/repos/asf/phoenix/blob/04ef859b/phoenix-core/src/main/java/org/apache/phoenix/expression/LikeExpression.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/LikeExpression.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/LikeExpression.java
index 4bb70bd..7f6323b 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/expression/LikeExpression.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/LikeExpression.java
@@ -26,7 +26,9 @@ import java.util.regex.Pattern;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.phoenix.expression.visitor.ExpressionVisitor;
 import org.apache.phoenix.parse.LikeParseNode.LikeType;
-import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.schema.types.PBoolean;
+import org.apache.phoenix.schema.types.PDataType;
+import org.apache.phoenix.schema.types.PVarchar;
 import org.apache.phoenix.schema.tuple.Tuple;
 import org.apache.phoenix.util.StringUtil;
 import org.slf4j.Logger;
@@ -267,7 +269,7 @@ public class LikeExpression extends BaseCompoundExpression {
                 }
                 return false;
             }
-            String value = (String)PDataType.VARCHAR.toObject(ptr, getPatternExpression().getSortOrder());
+            String value = (String) PVarchar.INSTANCE.toObject(ptr, getPatternExpression().getSortOrder());
             pattern = compilePattern(value);
             if (logger.isDebugEnabled()) {
                 logger.debug("LIKE pattern is expression: " + pattern.pattern());
@@ -284,7 +286,7 @@ public class LikeExpression extends BaseCompoundExpression {
             return true;
         }
 
-        String value = (String)PDataType.VARCHAR.toObject(ptr, getStrExpression().getSortOrder());
+        String value = (String) PVarchar.INSTANCE.toObject(ptr, getStrExpression().getSortOrder());
         boolean matched = pattern.matcher(value).matches();
         ptr.set(matched ? PDataType.TRUE_BYTES : PDataType.FALSE_BYTES);
         if (logger.isDebugEnabled()) {
@@ -306,7 +308,7 @@ public class LikeExpression extends BaseCompoundExpression {
 
     @Override
     public PDataType getDataType() {
-        return PDataType.BOOLEAN;
+        return PBoolean.INSTANCE;
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/phoenix/blob/04ef859b/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 e9daf2d..757ba34 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,8 +25,11 @@ 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.PDataType;
-import org.apache.phoenix.schema.PhoenixArray;
+import org.apache.phoenix.schema.types.PChar;
+import org.apache.phoenix.schema.types.PBoolean;
+import org.apache.phoenix.schema.types.PDataType;
+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;
@@ -55,8 +58,9 @@ public class LiteralExpression extends BaseTerminalExpression {
 	        for (int i = 0; i < PDataType.values().length; i++) {
 	            TYPED_NULL_EXPRESSIONS[i+PDataType.values().length*determinism.ordinal()] = new LiteralExpression(PDataType.values()[i], determinism);
 	        }        
-	        BOOLEAN_EXPRESSIONS[determinism.ordinal()] = new LiteralExpression(Boolean.FALSE, PDataType.BOOLEAN, PDataType.BOOLEAN.toBytes(Boolean.FALSE), determinism);
-	        BOOLEAN_EXPRESSIONS[Determinism.values().length+determinism.ordinal()] = new LiteralExpression(Boolean.TRUE, PDataType.BOOLEAN, PDataType.BOOLEAN.toBytes(Boolean.TRUE), determinism);
+	        BOOLEAN_EXPRESSIONS[determinism.ordinal()] = new LiteralExpression(Boolean.FALSE,
+              PBoolean.INSTANCE, PBoolean.INSTANCE.toBytes(Boolean.FALSE), determinism);
+	        BOOLEAN_EXPRESSIONS[Determinism.values().length+determinism.ordinal()] = new LiteralExpression(Boolean.TRUE, PBoolean.INSTANCE, PBoolean.INSTANCE.toBytes(Boolean.TRUE), determinism);
     	}
     }
     
@@ -111,10 +115,10 @@ public class LiteralExpression extends BaseTerminalExpression {
         if (type.isNull(b)) {
             return getTypedNullLiteralExpression(type, determinism);
         }
-        if (type == PDataType.VARCHAR) {
+        if (type == PVarchar.INSTANCE) {
             String s = (String) value;
             if (s.length() == b.length) { // single byte characters only
-                type = PDataType.CHAR;
+                type = PChar.INSTANCE;
             }
         }
         return new LiteralExpression(value, type, b, determinism);
@@ -161,8 +165,8 @@ public class LiteralExpression extends BaseTerminalExpression {
         }
         value = type.toObject(value, actualType);
         byte[] b = type.toBytes(value, sortOrder);
-        if (type == PDataType.VARCHAR || type == PDataType.CHAR) {
-            if (type == PDataType.CHAR && maxLength != null  && b.length < maxLength) {
+        if (type == PVarchar.INSTANCE || type == PChar.INSTANCE) {
+            if (type == PChar.INSTANCE && maxLength != null  && b.length < maxLength) {
                 b = StringUtil.padChar(b, maxLength);
             } else if (value != null) {
                 maxLength = ((String)value).length();

http://git-wip-us.apache.org/repos/asf/phoenix/blob/04ef859b/phoenix-core/src/main/java/org/apache/phoenix/expression/LongAddExpression.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/LongAddExpression.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/LongAddExpression.java
index 9b295cf..161758a 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/expression/LongAddExpression.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/LongAddExpression.java
@@ -21,7 +21,8 @@ import java.util.List;
 
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 
-import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.schema.types.PDataType;
+import org.apache.phoenix.schema.types.PLong;
 import org.apache.phoenix.schema.tuple.Tuple;
 
 
@@ -46,7 +47,7 @@ public class LongAddExpression extends AddExpression {
             long childvalue = child.getDataType().getCodec().decodeLong(ptr, child.getSortOrder());
             finalResult += childvalue;
         }
-        byte[] resultPtr=new byte[PDataType.LONG.getByteSize()];
+        byte[] resultPtr=new byte[PLong.INSTANCE.getByteSize()];
         getDataType().getCodec().encodeLong(finalResult, resultPtr, 0);
         ptr.set(resultPtr);
         return true;
@@ -54,7 +55,7 @@ public class LongAddExpression extends AddExpression {
 
     @Override
     public final PDataType getDataType() {
-        return PDataType.LONG;
+        return PLong.INSTANCE;
     }
 
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/04ef859b/phoenix-core/src/main/java/org/apache/phoenix/expression/LongDivideExpression.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/LongDivideExpression.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/LongDivideExpression.java
index 1803ca7..599b38e 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/expression/LongDivideExpression.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/LongDivideExpression.java
@@ -21,7 +21,8 @@ import java.util.List;
 
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 
-import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.schema.types.PDataType;
+import org.apache.phoenix.schema.types.PLong;
 import org.apache.phoenix.schema.tuple.Tuple;
 
 
@@ -50,7 +51,7 @@ public class LongDivideExpression extends DivideExpression {
                 finalResult /= childvalue;
             }
         }
-        byte[] resultPtr=new byte[PDataType.LONG.getByteSize()];
+        byte[] resultPtr=new byte[PLong.INSTANCE.getByteSize()];
         getDataType().getCodec().encodeLong(finalResult, resultPtr, 0);
         ptr.set(resultPtr);
         return true;
@@ -58,7 +59,7 @@ public class LongDivideExpression extends DivideExpression {
 
     @Override
     public final PDataType getDataType() {
-        return PDataType.LONG;
+        return PLong.INSTANCE;
     }
 
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/04ef859b/phoenix-core/src/main/java/org/apache/phoenix/expression/LongMultiplyExpression.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/LongMultiplyExpression.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/LongMultiplyExpression.java
index c94b1c1..40d6622 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/expression/LongMultiplyExpression.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/LongMultiplyExpression.java
@@ -21,7 +21,8 @@ import java.util.List;
 
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 
-import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.schema.types.PLong;
+import org.apache.phoenix.schema.types.PDataType;
 import org.apache.phoenix.schema.tuple.Tuple;
 
 
@@ -57,7 +58,7 @@ public class LongMultiplyExpression extends MultiplyExpression {
 
     @Override
     public final PDataType getDataType() {
-        return PDataType.LONG;
+        return PLong.INSTANCE;
     }
 
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/04ef859b/phoenix-core/src/main/java/org/apache/phoenix/expression/LongSubtractExpression.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/LongSubtractExpression.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/LongSubtractExpression.java
index c4c4d47..c7e1ee0 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/expression/LongSubtractExpression.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/LongSubtractExpression.java
@@ -22,7 +22,9 @@ import java.util.List;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 
 import org.apache.phoenix.query.QueryConstants;
-import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.schema.types.PDate;
+import org.apache.phoenix.schema.types.PDataType;
+import org.apache.phoenix.schema.types.PLong;
 import org.apache.phoenix.schema.tuple.Tuple;
 
 
@@ -52,7 +54,7 @@ public class LongSubtractExpression extends SubtractExpression {
                 return false;
             }
             PDataType childType = child.getDataType();
-            boolean isDate = childType.isCoercibleTo(PDataType.DATE);
+            boolean isDate = childType.isCoercibleTo(PDate.INSTANCE);
             long childvalue = childType.getCodec().decodeLong(ptr, child.getSortOrder());
             if (i == 0) {
                 finalResult = childvalue;
@@ -75,7 +77,7 @@ public class LongSubtractExpression extends SubtractExpression {
 
 	@Override
 	public final PDataType getDataType() {
-		return PDataType.LONG;
+		return PLong.INSTANCE;
 	}
 	
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/04ef859b/phoenix-core/src/main/java/org/apache/phoenix/expression/ModulusExpression.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/ModulusExpression.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/ModulusExpression.java
index c8dfe93..bac52fc 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/expression/ModulusExpression.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/ModulusExpression.java
@@ -21,11 +21,8 @@ import java.sql.SQLException;
 import java.util.List;
 
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
-import org.apache.phoenix.expression.Expression;
-import org.apache.phoenix.expression.function.ScalarFunction;
-import org.apache.phoenix.parse.FunctionParseNode.Argument;
-import org.apache.phoenix.parse.FunctionParseNode.BuiltInFunction;
-import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.schema.types.PLong;
+import org.apache.phoenix.schema.types.PDataType;
 import org.apache.phoenix.schema.tuple.Tuple;
 
 
@@ -80,7 +77,7 @@ public class ModulusExpression extends ArithmeticExpression {
         long remainder = dividend % divisor;
         
         // return the result, use encodeLong to avoid extra Long allocation
-        byte[] resultPtr=new byte[PDataType.LONG.getByteSize()];
+        byte[] resultPtr=new byte[PLong.INSTANCE.getByteSize()];
         getDataType().getCodec().encodeLong(remainder, resultPtr, 0);
         ptr.set(resultPtr);
         return true;
@@ -88,7 +85,7 @@ public class ModulusExpression extends ArithmeticExpression {
 
     @Override
     public PDataType getDataType() {
-        return PDataType.LONG;
+        return PLong.INSTANCE;
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/phoenix/blob/04ef859b/phoenix-core/src/main/java/org/apache/phoenix/expression/MultiplyExpression.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/MultiplyExpression.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/MultiplyExpression.java
index 90a4706..fa669e9 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/expression/MultiplyExpression.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/MultiplyExpression.java
@@ -20,7 +20,7 @@ package org.apache.phoenix.expression;
 import java.util.List;
 
 import org.apache.phoenix.expression.visitor.ExpressionVisitor;
-import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.schema.types.PDataType;
 
 
 /**

http://git-wip-us.apache.org/repos/asf/phoenix/blob/04ef859b/phoenix-core/src/main/java/org/apache/phoenix/expression/NotExpression.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/NotExpression.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/NotExpression.java
index 0c17ab8..c2f214e 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/expression/NotExpression.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/NotExpression.java
@@ -22,7 +22,8 @@ import java.util.List;
 
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.phoenix.expression.visitor.ExpressionVisitor;
-import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.schema.types.PBoolean;
+import org.apache.phoenix.schema.types.PDataType;
 import org.apache.phoenix.schema.TypeMismatchException;
 import org.apache.phoenix.schema.tuple.Tuple;
 
@@ -38,14 +39,14 @@ import org.apache.phoenix.schema.tuple.Tuple;
 public class NotExpression extends BaseSingleExpression {
 
     public static Expression create(Expression child, ImmutableBytesWritable ptr) throws SQLException {
-        if (child.getDataType() != PDataType.BOOLEAN) {
-            throw TypeMismatchException.newException(child.getDataType(), PDataType.BOOLEAN, "NOT");
+        if (child.getDataType() != PBoolean.INSTANCE) {
+            throw TypeMismatchException.newException(child.getDataType(), PBoolean.INSTANCE, "NOT");
         }
         if (child.isStateless()) {
             if (!child.evaluate(null, ptr) || ptr.getLength() == 0) {
-                return LiteralExpression.newConstant(null, PDataType.BOOLEAN, child.getDeterminism());
+                return LiteralExpression.newConstant(null, PBoolean.INSTANCE, child.getDeterminism());
             }
-            return LiteralExpression.newConstant(!(Boolean)PDataType.BOOLEAN.toObject(ptr), PDataType.BOOLEAN, child.getDeterminism());
+            return LiteralExpression.newConstant(!(Boolean) PBoolean.INSTANCE.toObject(ptr), PBoolean.INSTANCE, child.getDeterminism());
         }
         return new NotExpression(child);
     }
@@ -66,13 +67,13 @@ public class NotExpression extends BaseSingleExpression {
             return true;
         }
         
-        ptr.set(Boolean.TRUE.equals(PDataType.BOOLEAN.toObject(ptr)) ? PDataType.FALSE_BYTES : PDataType.TRUE_BYTES);
+        ptr.set(Boolean.TRUE.equals(PBoolean.INSTANCE.toObject(ptr)) ? PDataType.FALSE_BYTES : PDataType.TRUE_BYTES);
         return true;
     }
 
     @Override
     public PDataType getDataType() {
-        return PDataType.BOOLEAN;
+        return PBoolean.INSTANCE;
     }
     
     @Override

http://git-wip-us.apache.org/repos/asf/phoenix/blob/04ef859b/phoenix-core/src/main/java/org/apache/phoenix/expression/RowKeyColumnExpression.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/RowKeyColumnExpression.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/RowKeyColumnExpression.java
index 7913ab8..240d013 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/expression/RowKeyColumnExpression.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/RowKeyColumnExpression.java
@@ -23,7 +23,7 @@ import java.io.IOException;
 
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.phoenix.expression.visitor.ExpressionVisitor;
-import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.schema.types.PDataType;
 import org.apache.phoenix.schema.PDatum;
 import org.apache.phoenix.schema.RowKeyValueAccessor;
 import org.apache.phoenix.schema.tuple.Tuple;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/04ef859b/phoenix-core/src/main/java/org/apache/phoenix/expression/RowKeyExpression.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/RowKeyExpression.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/RowKeyExpression.java
index cbd7853..d9f6b90 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/expression/RowKeyExpression.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/RowKeyExpression.java
@@ -20,7 +20,8 @@ package org.apache.phoenix.expression;
 
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 
-import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.schema.types.PDataType;
+import org.apache.phoenix.schema.types.PVarbinary;
 import org.apache.phoenix.schema.tuple.Tuple;
 
 public class RowKeyExpression extends BaseTerminalExpression {
@@ -37,7 +38,7 @@ public class RowKeyExpression extends BaseTerminalExpression {
 
     @Override
     public PDataType getDataType() {
-        return PDataType.VARBINARY;
+        return PVarbinary.INSTANCE;
     }
 
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/04ef859b/phoenix-core/src/main/java/org/apache/phoenix/expression/RowValueConstructorExpression.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/RowValueConstructorExpression.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/RowValueConstructorExpression.java
index 34bd35c..546962e 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/expression/RowValueConstructorExpression.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/RowValueConstructorExpression.java
@@ -33,7 +33,8 @@ import java.util.List;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.phoenix.expression.visitor.ExpressionVisitor;
 import org.apache.phoenix.query.QueryConstants;
-import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.schema.types.PDataType;
+import org.apache.phoenix.schema.types.PVarbinary;
 import org.apache.phoenix.schema.tuple.Tuple;
 import org.apache.phoenix.util.ByteUtil;
 import org.apache.phoenix.util.SchemaUtil;
@@ -98,7 +99,7 @@ public class RowValueConstructorExpression extends BaseCompoundExpression {
     
     @Override
     public PDataType getDataType() {
-        return PDataType.VARBINARY;
+        return PVarbinary.INSTANCE;
     }
     
     @Override

http://git-wip-us.apache.org/repos/asf/phoenix/blob/04ef859b/phoenix-core/src/main/java/org/apache/phoenix/expression/StringConcatExpression.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/StringConcatExpression.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/StringConcatExpression.java
index 942e0bd..b8fcd41 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/expression/StringConcatExpression.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/StringConcatExpression.java
@@ -23,8 +23,9 @@ import java.util.List;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 
 import org.apache.phoenix.expression.visitor.ExpressionVisitor;
+import org.apache.phoenix.schema.types.PVarchar;
 import org.apache.phoenix.schema.SortOrder;
-import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.schema.types.PDataType;
 import org.apache.phoenix.schema.tuple.Tuple;
 import org.apache.phoenix.util.ByteUtil;
 
@@ -76,10 +77,10 @@ public class StringConcatExpression extends BaseCompoundExpression {
             SortOrder sortOrder = children.get(i).getSortOrder();
             // We could potentially not invert the bytes, but we might as well since we're allocating
             // additional space here anyway.
-            if (childType.isCoercibleTo(PDataType.VARCHAR)) {
+            if (childType.isCoercibleTo(PVarchar.INSTANCE)) {
                 result = ByteUtil.concat(result, ByteUtil.concat(sortOrder, ptr));
             } else {
-                result = ByteUtil.concat(result, PDataType.VARCHAR.toBytes(childType.toObject(ptr, sortOrder).toString()));
+                result = ByteUtil.concat(result, PVarchar.INSTANCE.toBytes(childType.toObject(ptr, sortOrder).toString()));
             }
         }
         ptr.set(result);
@@ -88,6 +89,6 @@ public class StringConcatExpression extends BaseCompoundExpression {
 
     @Override
     public PDataType getDataType() {
-        return PDataType.VARCHAR;
+        return PVarchar.INSTANCE;
     }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/04ef859b/phoenix-core/src/main/java/org/apache/phoenix/expression/TimestampAddExpression.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/TimestampAddExpression.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/TimestampAddExpression.java
index 80f3522..892a38c 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/expression/TimestampAddExpression.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/TimestampAddExpression.java
@@ -25,14 +25,18 @@ import java.util.List;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 
 import org.apache.phoenix.query.QueryConstants;
+import org.apache.phoenix.schema.types.PDecimal;
+import org.apache.phoenix.schema.types.PDouble;
+import org.apache.phoenix.schema.types.PTimestamp;
+import org.apache.phoenix.schema.types.PUnsignedTimestamp;
 import org.apache.phoenix.schema.SortOrder;
-import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.schema.types.PDataType;
 import org.apache.phoenix.schema.tuple.Tuple;
 import org.apache.phoenix.util.DateUtil;
 
 /**
  * 
- * Class to encapsulate addition arithmetic for {@link PDataType#TIMESTAMP}.
+ * Class to encapsulate addition arithmetic for {@link org.apache.phoenix.schema.types.PTimestamp}.
  *
  * 
  * @since 2.1.3
@@ -61,11 +65,11 @@ public class TimestampAddExpression extends AddExpression {
             BigDecimal value;
             PDataType type = children.get(i).getDataType();
             SortOrder sortOrder = children.get(i).getSortOrder();
-            if(type == PDataType.TIMESTAMP || type == PDataType.UNSIGNED_TIMESTAMP) {
-                value = (BigDecimal)(PDataType.DECIMAL.toObject(ptr, type, sortOrder));
-            } else if (type.isCoercibleTo(PDataType.DECIMAL)) {
-                value = (((BigDecimal)PDataType.DECIMAL.toObject(ptr, sortOrder)).multiply(QueryConstants.BD_MILLIS_IN_DAY)).setScale(6, RoundingMode.HALF_UP);
-            } else if (type.isCoercibleTo(PDataType.DOUBLE)) {
+            if(type == PTimestamp.INSTANCE || type == PUnsignedTimestamp.INSTANCE) {
+                value = (BigDecimal)(PDecimal.INSTANCE.toObject(ptr, type, sortOrder));
+            } else if (type.isCoercibleTo(PDecimal.INSTANCE)) {
+                value = (((BigDecimal) PDecimal.INSTANCE.toObject(ptr, sortOrder)).multiply(QueryConstants.BD_MILLIS_IN_DAY)).setScale(6, RoundingMode.HALF_UP);
+            } else if (type.isCoercibleTo(PDouble.INSTANCE)) {
                 value = ((BigDecimal.valueOf(type.getCodec().decodeDouble(ptr, sortOrder))).multiply(QueryConstants.BD_MILLIS_IN_DAY)).setScale(6, RoundingMode.HALF_UP);
             } else {
                 value = BigDecimal.valueOf(type.getCodec().decodeLong(ptr, sortOrder));
@@ -74,14 +78,14 @@ public class TimestampAddExpression extends AddExpression {
         }
         Timestamp ts = DateUtil.getTimestamp(finalResult);
         byte[] resultPtr = new byte[getDataType().getByteSize()];
-        PDataType.TIMESTAMP.toBytes(ts, resultPtr, 0);
+        PTimestamp.INSTANCE.toBytes(ts, resultPtr, 0);
         ptr.set(resultPtr);
         return true;
     }
 
     @Override
     public final PDataType getDataType() {
-        return PDataType.TIMESTAMP;
+        return PTimestamp.INSTANCE;
     }
 
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/04ef859b/phoenix-core/src/main/java/org/apache/phoenix/expression/TimestampSubtractExpression.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/TimestampSubtractExpression.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/TimestampSubtractExpression.java
index 0df0c24..5718c94 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/expression/TimestampSubtractExpression.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/TimestampSubtractExpression.java
@@ -24,13 +24,17 @@ import java.util.List;
 
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 
+import org.apache.phoenix.schema.types.PDecimal;
+import org.apache.phoenix.schema.types.PDouble;
+import org.apache.phoenix.schema.types.PTimestamp;
+import org.apache.phoenix.schema.types.PUnsignedTimestamp;
 import org.apache.phoenix.schema.SortOrder;
-import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.schema.types.PDataType;
 import org.apache.phoenix.schema.tuple.Tuple;
 import org.apache.phoenix.util.DateUtil;
 /**
  * 
- * Class to encapsulate subtraction arithmetic for {@link PDataType#TIMESTAMP}.
+ * Class to encapsulate subtraction arithmetic for {@link org.apache.phoenix.schema.types.PTimestamp}.
  *
  * 
  * @since 2.1.3
@@ -58,11 +62,11 @@ public class TimestampSubtractExpression extends SubtractExpression {
             BigDecimal value;
             PDataType type = children.get(i).getDataType();
             SortOrder sortOrder = children.get(i).getSortOrder();
-            if(type == PDataType.TIMESTAMP || type == PDataType.UNSIGNED_TIMESTAMP) {
-                value = (BigDecimal)(PDataType.DECIMAL.toObject(ptr, type, sortOrder));
-            } else if (type.isCoercibleTo(PDataType.DECIMAL)) {
-                value = (((BigDecimal)PDataType.DECIMAL.toObject(ptr, sortOrder)).multiply(BD_MILLIS_IN_DAY)).setScale(6, RoundingMode.HALF_UP);
-            } else if (type.isCoercibleTo(PDataType.DOUBLE)) {
+            if(type == PTimestamp.INSTANCE || type == PUnsignedTimestamp.INSTANCE) {
+                value = (BigDecimal)(PDecimal.INSTANCE.toObject(ptr, type, sortOrder));
+            } else if (type.isCoercibleTo(PDecimal.INSTANCE)) {
+                value = (((BigDecimal) PDecimal.INSTANCE.toObject(ptr, sortOrder)).multiply(BD_MILLIS_IN_DAY)).setScale(6, RoundingMode.HALF_UP);
+            } else if (type.isCoercibleTo(PDouble.INSTANCE)) {
                 value = ((BigDecimal.valueOf(type.getCodec().decodeDouble(ptr, sortOrder))).multiply(BD_MILLIS_IN_DAY)).setScale(6, RoundingMode.HALF_UP);
             } else {
                 value = BigDecimal.valueOf(type.getCodec().decodeLong(ptr, sortOrder));
@@ -75,13 +79,13 @@ public class TimestampSubtractExpression extends SubtractExpression {
         }
         Timestamp ts = DateUtil.getTimestamp(finalResult);
         byte[] resultPtr = new byte[getDataType().getByteSize()];
-        PDataType.TIMESTAMP.toBytes(ts, resultPtr, 0);
+        PTimestamp.INSTANCE.toBytes(ts, resultPtr, 0);
         ptr.set(resultPtr);
         return true;
     }
 
     @Override
     public final PDataType getDataType() {
-        return PDataType.TIMESTAMP;
+        return PTimestamp.INSTANCE;
     }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/04ef859b/phoenix-core/src/main/java/org/apache/phoenix/expression/aggregator/BaseDecimalStddevAggregator.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/aggregator/BaseDecimalStddevAggregator.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/aggregator/BaseDecimalStddevAggregator.java
index 576ce7c..38c3060 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/expression/aggregator/BaseDecimalStddevAggregator.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/aggregator/BaseDecimalStddevAggregator.java
@@ -26,8 +26,9 @@ import org.apache.hadoop.hbase.util.Pair;
 import org.apache.phoenix.expression.ColumnExpression;
 import org.apache.phoenix.expression.Expression;
 import org.apache.phoenix.hbase.index.util.ImmutableBytesPtr;
+import org.apache.phoenix.schema.types.PDecimal;
 import org.apache.phoenix.schema.SortOrder;
-import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.schema.types.PDataType;
 import org.apache.phoenix.schema.tuple.Tuple;
 import org.apache.phoenix.util.BigDecimalUtil;
 import org.apache.phoenix.util.BigDecimalUtil.Operation;
@@ -76,7 +77,7 @@ public abstract class BaseDecimalStddevAggregator extends DistinctValueWithCount
         if (buffer == null) {
             initBuffer();
         }
-        buffer = PDataType.DECIMAL.toBytes(cachedResult);
+        buffer = PDecimal.INSTANCE.toBytes(cachedResult);
         ptr.set(buffer);
         return true;
     }
@@ -87,7 +88,7 @@ public abstract class BaseDecimalStddevAggregator extends DistinctValueWithCount
         BigDecimal m = mean();
         BigDecimal result = BigDecimal.ZERO;
         for (Entry<ImmutableBytesPtr, Integer> entry : valueVsCount.entrySet()) {
-            BigDecimal colValue = (BigDecimal)PDataType.DECIMAL.toObject(entry.getKey());
+            BigDecimal colValue = (BigDecimal) PDecimal.INSTANCE.toObject(entry.getKey());
             BigDecimal delta = colValue.subtract(m);
             result = result.add(delta.multiply(delta).multiply(new BigDecimal(entry.getValue())));
         }
@@ -97,7 +98,7 @@ public abstract class BaseDecimalStddevAggregator extends DistinctValueWithCount
     private BigDecimal mean() {
         BigDecimal sum = BigDecimal.ZERO;
         for (Entry<ImmutableBytesPtr, Integer> entry : valueVsCount.entrySet()) {
-            BigDecimal colValue = (BigDecimal)PDataType.DECIMAL.toObject(entry.getKey());
+            BigDecimal colValue = (BigDecimal) PDecimal.INSTANCE.toObject(entry.getKey());
             sum = sum.add(colValue.multiply(new BigDecimal(entry.getValue())));
         }
         return sum.divide(new BigDecimal(totalCount), PDataType.DEFAULT_MATH_CONTEXT);
@@ -105,6 +106,6 @@ public abstract class BaseDecimalStddevAggregator extends DistinctValueWithCount
 
     @Override
     protected PDataType getResultDataType() {
-        return PDataType.DECIMAL;
+        return PDecimal.INSTANCE;
     }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/04ef859b/phoenix-core/src/main/java/org/apache/phoenix/expression/aggregator/BaseStddevAggregator.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/aggregator/BaseStddevAggregator.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/aggregator/BaseStddevAggregator.java
index a1ed1df..cb9861a 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/expression/aggregator/BaseStddevAggregator.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/aggregator/BaseStddevAggregator.java
@@ -24,8 +24,10 @@ import java.util.Map.Entry;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.phoenix.expression.Expression;
 import org.apache.phoenix.hbase.index.util.ImmutableBytesPtr;
+import org.apache.phoenix.schema.types.PDecimal;
+import org.apache.phoenix.schema.types.PDouble;
 import org.apache.phoenix.schema.SortOrder;
-import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.schema.types.PDataType;
 import org.apache.phoenix.schema.tuple.Tuple;
 
 /**
@@ -52,7 +54,7 @@ public abstract class BaseStddevAggregator extends DistinctValueWithCountClientA
         if (buffer == null) {
             initBuffer();
         }
-        buffer = PDataType.DECIMAL.toBytes(cachedResult);
+        buffer = PDecimal.INSTANCE.toBytes(cachedResult);
         ptr.set(buffer);
         return true;
     }
@@ -63,7 +65,7 @@ public abstract class BaseStddevAggregator extends DistinctValueWithCountClientA
         double m = mean();
         double result = 0.0;
         for (Entry<ImmutableBytesPtr, Integer> entry : valueVsCount.entrySet()) {
-            double colValue = (Double)PDataType.DOUBLE.toObject(entry.getKey(), this.stdDevColExp.getDataType());
+            double colValue = (Double) PDouble.INSTANCE.toObject(entry.getKey(), this.stdDevColExp.getDataType());
             double delta = colValue - m;
             result += (delta * delta) * entry.getValue();
         }
@@ -73,7 +75,7 @@ public abstract class BaseStddevAggregator extends DistinctValueWithCountClientA
     private double mean() {
         double sum = 0.0;
         for (Entry<ImmutableBytesPtr, Integer> entry : valueVsCount.entrySet()) {
-            double colValue = (Double)PDataType.DOUBLE.toObject(entry.getKey(), this.stdDevColExp.getDataType());
+            double colValue = (Double) PDouble.INSTANCE.toObject(entry.getKey(), this.stdDevColExp.getDataType());
             sum += colValue * entry.getValue();
         }
         return sum / totalCount;
@@ -81,6 +83,6 @@ public abstract class BaseStddevAggregator extends DistinctValueWithCountClientA
     
     @Override
     protected PDataType getResultDataType() {
-        return PDataType.DECIMAL;
+        return PDecimal.INSTANCE;
     }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/04ef859b/phoenix-core/src/main/java/org/apache/phoenix/expression/aggregator/CountAggregator.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/aggregator/CountAggregator.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/aggregator/CountAggregator.java
index d815975..bd6725b 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/expression/aggregator/CountAggregator.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/aggregator/CountAggregator.java
@@ -18,7 +18,8 @@
 package org.apache.phoenix.expression.aggregator;
 
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
-import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.schema.types.PDataType;
+import org.apache.phoenix.schema.types.PLong;
 import org.apache.phoenix.schema.SortOrder;
 import org.apache.phoenix.schema.tuple.Tuple;
 import org.apache.phoenix.util.SizedUtil;
@@ -66,7 +67,7 @@ public class CountAggregator extends BaseAggregator {
 
     @Override
     public final PDataType getDataType() {
-        return PDataType.LONG;
+        return PLong.INSTANCE;
     }
 
     @Override