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:45 UTC

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

http://git-wip-us.apache.org/repos/asf/phoenix/blob/04ef859b/phoenix-core/src/test/java/org/apache/phoenix/expression/RoundFloorCeilExpressionsTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/expression/RoundFloorCeilExpressionsTest.java b/phoenix-core/src/test/java/org/apache/phoenix/expression/RoundFloorCeilExpressionsTest.java
index f8a7391..c9f81fe 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/expression/RoundFloorCeilExpressionsTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/expression/RoundFloorCeilExpressionsTest.java
@@ -17,7 +17,6 @@
  */
 package org.apache.phoenix.expression;
 
-import static org.apache.phoenix.schema.PDataType.DECIMAL;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
@@ -44,98 +43,102 @@ import org.apache.phoenix.expression.function.ScalarFunction;
 import org.apache.phoenix.expression.function.TimeUnit;
 import org.apache.phoenix.hbase.index.util.ImmutableBytesPtr;
 import org.apache.phoenix.query.KeyRange;
+import org.apache.phoenix.schema.types.PDecimal;
 import org.apache.phoenix.schema.IllegalDataException;
-import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.schema.types.PDate;
+import org.apache.phoenix.schema.types.PInteger;
+import org.apache.phoenix.schema.types.PDataType;
+import org.apache.phoenix.schema.types.PVarchar;
 import org.apache.phoenix.util.DateUtil;
 import org.junit.Test;
 
 /**
- * 
+ *
  * Unit tests for {@link RoundDecimalExpression}, {@link FloorDecimalExpression}
  * and {@link CeilDecimalExpression}.
  *
- * 
+ *
  * @since 3.0.0
  */
 public class RoundFloorCeilExpressionsTest {
 
     // Decimal Expression Tests
-    
+
     @Test
     public void testRoundDecimalExpression() throws Exception {
-        LiteralExpression decimalLiteral = LiteralExpression.newConstant(1.23898, PDataType.DECIMAL);
+        LiteralExpression decimalLiteral = LiteralExpression.newConstant(1.23898, PDecimal.INSTANCE);
         Expression roundDecimalExpression = RoundDecimalExpression.create(decimalLiteral, 3);
-        
+
         ImmutableBytesWritable ptr = new ImmutableBytesWritable();
         roundDecimalExpression.evaluate(null, ptr);
         Object result = roundDecimalExpression.getDataType().toObject(ptr);
-        
+
         assertTrue(result instanceof BigDecimal);
         BigDecimal resultDecimal = (BigDecimal)result;
         assertEquals(BigDecimal.valueOf(1.239), resultDecimal);
     }
-    
+
     @Test
     public void testRoundDecimalExpressionNoop() throws Exception {
-        LiteralExpression decimalLiteral = LiteralExpression.newConstant(5, PDataType.INTEGER);
+        LiteralExpression decimalLiteral = LiteralExpression.newConstant(5, PInteger.INSTANCE);
         Expression roundDecimalExpression = RoundDecimalExpression.create(decimalLiteral, 3);
-        
+
         assertEquals(roundDecimalExpression, decimalLiteral);
     }
-    
+
     @Test
     public void testFloorDecimalExpression() throws Exception {
-        LiteralExpression decimalLiteral = LiteralExpression.newConstant(1.23898, PDataType.DECIMAL);
+        LiteralExpression decimalLiteral = LiteralExpression.newConstant(1.23898, PDecimal.INSTANCE);
         Expression floorDecimalExpression = FloorDecimalExpression.create(decimalLiteral, 3);
-        
+
         ImmutableBytesWritable ptr = new ImmutableBytesWritable();
         floorDecimalExpression.evaluate(null, ptr);
         Object result = floorDecimalExpression.getDataType().toObject(ptr);
-        
+
         assertTrue(result instanceof BigDecimal);
         BigDecimal resultDecimal = (BigDecimal)result;
         assertEquals(BigDecimal.valueOf(1.238), resultDecimal);
     }
-    
+
     @Test
     public void testFloorDecimalExpressionNoop() throws Exception {
-        LiteralExpression decimalLiteral = LiteralExpression.newConstant(5, PDataType.INTEGER);
+        LiteralExpression decimalLiteral = LiteralExpression.newConstant(5, PInteger.INSTANCE);
         Expression floorDecimalExpression = FloorDecimalExpression.create(decimalLiteral, 3);
-        
+
         assertEquals(floorDecimalExpression, decimalLiteral);
     }
-    
+
     @Test
     public void testCeilDecimalExpression() throws Exception {
-        LiteralExpression decimalLiteral = LiteralExpression.newConstant(1.23898, PDataType.DECIMAL);
+        LiteralExpression decimalLiteral = LiteralExpression.newConstant(1.23898, PDecimal.INSTANCE);
         Expression ceilDecimalExpression = CeilDecimalExpression.create(decimalLiteral, 3);
-        
+
         ImmutableBytesWritable ptr = new ImmutableBytesWritable();
         ceilDecimalExpression.evaluate(null, ptr);
         Object result = ceilDecimalExpression.getDataType().toObject(ptr);
-        
+
         assertTrue(result instanceof BigDecimal);
         BigDecimal resultDecimal = (BigDecimal)result;
         assertEquals(BigDecimal.valueOf(1.239), resultDecimal);
     }
-    
+
     @Test
     public void testCeilDecimalExpressionNoop() throws Exception {
-        LiteralExpression decimalLiteral = LiteralExpression.newConstant(5, PDataType.INTEGER);
+        LiteralExpression decimalLiteral = LiteralExpression.newConstant(5, PInteger.INSTANCE);
         Expression ceilDecimalExpression = CeilDecimalExpression.create(decimalLiteral, 3);
-        
+
         assertEquals(ceilDecimalExpression, decimalLiteral);
     }
-    
+
     @Test
     public void testRoundDecimalExpressionScaleParamValidation() throws Exception {
-        LiteralExpression decimalLiteral = LiteralExpression.newConstant(1.23898, PDataType.DECIMAL);
-        LiteralExpression scale = LiteralExpression.newConstant("3", PDataType.VARCHAR);
-        
+        LiteralExpression decimalLiteral = LiteralExpression.newConstant(1.23898, PDecimal.INSTANCE);
+        LiteralExpression scale = LiteralExpression.newConstant("3", PVarchar.INSTANCE);
+
         List<Expression> childExpressions = new ArrayList<Expression>(2);
         childExpressions.add(decimalLiteral);
         childExpressions.add(scale);
-        
+
         try {
             RoundDecimalExpression.create(childExpressions);
             fail("Evaluation should have failed because only an INTEGER is allowed for second param in a RoundDecimalExpression");
@@ -143,47 +146,47 @@ public class RoundFloorCeilExpressionsTest {
 
         }
     }
-    
+
     // KeyRange explicit simple / sanity tests
-    
+
     @Test
     public void testRoundDecimalExpressionKeyRangeSimple() throws Exception {
         ScalarFunction roundDecimalExpression = (ScalarFunction)RoundDecimalExpression.create(DUMMY_DECIMAL, 3);
-        
-        byte[] upperBound = DECIMAL.toBytes(new BigDecimal("1.2385"));
-        byte[] lowerBound = DECIMAL.toBytes(new BigDecimal("1.2375"));
+
+        byte[] upperBound = PDecimal.INSTANCE.toBytes(new BigDecimal("1.2385"));
+        byte[] lowerBound = PDecimal.INSTANCE.toBytes(new BigDecimal("1.2375"));
         KeyRange expectedKeyRange = KeyRange.getKeyRange(lowerBound, upperBound);
-        
+
         KeyPart keyPart = roundDecimalExpression.newKeyPart(null);
-        assertEquals(expectedKeyRange, keyPart.getKeyRange(CompareOp.EQUAL, LiteralExpression.newConstant(new BigDecimal("1.238"), DECIMAL)));
+        assertEquals(expectedKeyRange, keyPart.getKeyRange(CompareOp.EQUAL, LiteralExpression.newConstant(new BigDecimal("1.238"), PDecimal.INSTANCE)));
     }
-    
+
     @Test
     public void testFloorDecimalExpressionKeyRangeSimple() throws Exception {
         ScalarFunction floorDecimalExpression = (ScalarFunction)FloorDecimalExpression.create(DUMMY_DECIMAL, 3);
-        
-        byte[] upperBound = DECIMAL.toBytes(new BigDecimal("1.239"));
-        byte[] lowerBound = DECIMAL.toBytes(new BigDecimal("1.238"));
+
+        byte[] upperBound = PDecimal.INSTANCE.toBytes(new BigDecimal("1.239"));
+        byte[] lowerBound = PDecimal.INSTANCE.toBytes(new BigDecimal("1.238"));
         KeyRange expectedKeyRange = KeyRange.getKeyRange(lowerBound, true, upperBound, false);
-        
+
         KeyPart keyPart = floorDecimalExpression.newKeyPart(null);
-        assertEquals(expectedKeyRange, keyPart.getKeyRange(CompareOp.EQUAL, LiteralExpression.newConstant(new BigDecimal("1.238"), DECIMAL)));
+        assertEquals(expectedKeyRange, keyPart.getKeyRange(CompareOp.EQUAL, LiteralExpression.newConstant(new BigDecimal("1.238"), PDecimal.INSTANCE)));
     }
-    
+
     @Test
     public void testCeilDecimalExpressionKeyRangeSimple() throws Exception {
         ScalarFunction ceilDecimalExpression = (ScalarFunction)CeilDecimalExpression.create(DUMMY_DECIMAL, 3);
-        
-        byte[] upperBound = DECIMAL.toBytes(new BigDecimal("1.238"));
-        byte[] lowerBound = DECIMAL.toBytes(new BigDecimal("1.237"));
+
+        byte[] upperBound = PDecimal.INSTANCE.toBytes(new BigDecimal("1.238"));
+        byte[] lowerBound = PDecimal.INSTANCE.toBytes(new BigDecimal("1.237"));
         KeyRange expectedKeyRange = KeyRange.getKeyRange(lowerBound, false, upperBound, true);
-        
+
         KeyPart keyPart = ceilDecimalExpression.newKeyPart(null);
-        assertEquals(expectedKeyRange, keyPart.getKeyRange(CompareOp.EQUAL, LiteralExpression.newConstant(new BigDecimal("1.238"), DECIMAL)));
+        assertEquals(expectedKeyRange, keyPart.getKeyRange(CompareOp.EQUAL, LiteralExpression.newConstant(new BigDecimal("1.238"), PDecimal.INSTANCE)));
     }
-    
+
     // KeyRange complex / generated tests
-    
+
     @Test
     public void testRoundDecimalExpressionKeyRangeCoverage() throws Exception {
         for(int scale : SCALES) {
@@ -192,7 +195,7 @@ public class RoundFloorCeilExpressionsTest {
             verifyKeyPart(RoundingType.ROUND, scale, keyPart);
         }
     }
-    
+
     @Test
     public void testFloorDecimalExpressionKeyRangeCoverage() throws Exception {
         for(int scale : SCALES) {
@@ -201,7 +204,7 @@ public class RoundFloorCeilExpressionsTest {
             verifyKeyPart(RoundingType.FLOOR, scale, keyPart);
         }
     }
-    
+
     @Test
     public void testCeilDecimalExpressionKeyRangeCoverage() throws Exception {
         for(int scale : SCALES) {
@@ -210,31 +213,31 @@ public class RoundFloorCeilExpressionsTest {
             verifyKeyPart(RoundingType.CEIL, scale, keyPart);
         }
     }
-    
+
     /**
      * Represents the three different types of rounding expression and produces
      * expressions of their type when given a Decimal key and scale.
      */
     private static enum RoundingType {
-        ROUND("ROUND"), 
-        FLOOR("FLOOR"), 
+        ROUND("ROUND"),
+        FLOOR("FLOOR"),
         CEIL("CEIL");
-        
+
         public final String name;
-        
+
         RoundingType(String name) {
             this.name = name;
         }
-        
+
         /**
          * Returns a rounding expression of this type that will round the given decimal key at the
-         * given scale. 
+         * given scale.
          * @param key  the byte key for the Decimal to round
          * @param scale  the scale to round the decimal to
          * @return  the expression containing the above parameters
          */
         public Expression getExpression(byte[] key, int scale) throws SQLException {
-            LiteralExpression decimalLiteral = LiteralExpression.newConstant(DECIMAL.toObject(key), DECIMAL);
+            LiteralExpression decimalLiteral = LiteralExpression.newConstant(PDecimal.INSTANCE.toObject(key), PDecimal.INSTANCE);
             switch(this) {
                 case ROUND:
                     return RoundDecimalExpression.create(decimalLiteral, scale);
@@ -247,7 +250,7 @@ public class RoundFloorCeilExpressionsTest {
             }
         }
     }
-    
+
     /**
      * Represents a possible relational operator used in rounding expression where clauses.
      * Includes information not kept by CompareFilter.CompareOp, including a string symbol
@@ -259,15 +262,15 @@ public class RoundFloorCeilExpressionsTest {
         GREATER_OR_EQUAL(CompareOp.GREATER_OR_EQUAL, ">="),
         LESS(CompareOp.LESS, "<"),
         LESS_OR_EQUAL(CompareOp.LESS_OR_EQUAL, "<=");
-        
+
         public final CompareOp compareOp;
         public final String symbol;
-        
+
         Relation(CompareOp compareOp, String symbol) {
             this.compareOp = compareOp;
             this.symbol = symbol;
         }
-        
+
         public <E extends Comparable<? super E>> boolean compare(E lhs, E rhs) {
             int comparison = lhs.compareTo(rhs);
             switch(this) {
@@ -286,7 +289,7 @@ public class RoundFloorCeilExpressionsTest {
             }
         }
     }
-    
+
     /**
      * Produces a string error message containing the given information, formatted like a where
      * clause. <br>
@@ -297,53 +300,53 @@ public class RoundFloorCeilExpressionsTest {
      * @param relation
      * @param rhs
      * @param range
-     * @return 
+     * @return
      */
     private static String getMessage(RoundingType exprType, int scale, Relation relation, BigDecimal rhs, KeyRange range) {
         String where = exprType.name + "(?, " + scale + ") " + relation.symbol + " " + rhs;
         return "'where " + where + "' (produced range: " + formatDecimalKeyRange(range) + " )";
     }
-    
+
     /**
      * Interpreting the KeyRange as a range of decimal, produces a nicely formatted string
-     * representation. 
+     * representation.
      * @param range  the KeyRange to format
      * @return  the string representation, e.g. [2.45, 2.55)
      */
     private static String formatDecimalKeyRange(KeyRange range) {
-        return (range.isLowerInclusive() ? "[" : "(") 
-            + (range.lowerUnbound() ? "*" : DECIMAL.toObject(range.getLowerRange())) 
-            + ", " 
-            + (range.upperUnbound() ? "*" : DECIMAL.toObject(range.getUpperRange())) 
+        return (range.isLowerInclusive() ? "[" : "(")
+            + (range.lowerUnbound() ? "*" : PDecimal.INSTANCE.toObject(range.getLowerRange()))
+            + ", "
+            + (range.upperUnbound() ? "*" : PDecimal.INSTANCE.toObject(range.getUpperRange()))
             + (range.isUpperInclusive() ? "]" : ")");
     }
-    
+
     // create methods need a dummy expression that is not coercible to to a long
     // value doesn't matter because we only use those expressions to produce a keypart
     private static final LiteralExpression DUMMY_DECIMAL = LiteralExpression.newConstant(new BigDecimal("2.5"));
-    
+
     private static final List<BigDecimal> DECIMALS = Collections.unmodifiableList(
         Arrays.asList(
             BigDecimal.valueOf(Long.MIN_VALUE * 17L - 13L, 9),
             BigDecimal.valueOf(Long.MIN_VALUE, 8),
-            new BigDecimal("-200300"), 
-            new BigDecimal("-8.44"), 
-            new BigDecimal("-2.00"), 
-            new BigDecimal("-0.6"), 
-            new BigDecimal("-0.00032"), 
-            BigDecimal.ZERO, 
-            BigDecimal.ONE, 
-            new BigDecimal("0.00000984"), 
-            new BigDecimal("0.74"), 
-            new BigDecimal("2.00"), 
-            new BigDecimal("7.09"), 
+            new BigDecimal("-200300"),
+            new BigDecimal("-8.44"),
+            new BigDecimal("-2.00"),
+            new BigDecimal("-0.6"),
+            new BigDecimal("-0.00032"),
+            BigDecimal.ZERO,
+            BigDecimal.ONE,
+            new BigDecimal("0.00000984"),
+            new BigDecimal("0.74"),
+            new BigDecimal("2.00"),
+            new BigDecimal("7.09"),
             new BigDecimal("84900800"),
             BigDecimal.valueOf(Long.MAX_VALUE, 8),
             BigDecimal.valueOf(Long.MAX_VALUE * 31L + 17L, 7)
         ));
-    
+
     private static final List<Integer> SCALES = Collections.unmodifiableList(Arrays.asList(0, 1, 2, 3, 8));
-    
+
     /**
      * Checks that a given KeyPart produces the right key ranges for each relational operator and
      * a variety of right-hand-side decimals.
@@ -353,19 +356,19 @@ public class RoundFloorCeilExpressionsTest {
      */
     private void verifyKeyPart(RoundingType exprType, int scale, KeyPart keyPart) throws SQLException {
         for(BigDecimal rhsDecimal : DECIMALS) {
-            LiteralExpression rhsExpression = LiteralExpression.newConstant(rhsDecimal, DECIMAL);
+            LiteralExpression rhsExpression = LiteralExpression.newConstant(rhsDecimal, PDecimal.INSTANCE);
             for(Relation relation : Relation.values()) {
                 KeyRange keyRange = keyPart.getKeyRange(relation.compareOp, rhsExpression);
                 verifyKeyRange(exprType, scale, relation, rhsDecimal, keyRange);
             }
         }
     }
-    
+
     /**
      * Checks that a given KeyRange's boundaries match with the given rounding expression type,
      * rounding scale, relational operator, and right hand side decimal.
      * Does so by checking the decimal values immediately on either side of the KeyRange border and
-     * verifying that they either match or do not match the "where clause" formed by the 
+     * verifying that they either match or do not match the "where clause" formed by the
      * rounding type, scale, relation, and rhs decimal. If a relation should produce an unbounded
      * upper or lower range, verifies that that end of the range is unbounded. Finally, if the
      * range is empty, verifies that the rhs decimal required more precision than could be
@@ -379,19 +382,19 @@ public class RoundFloorCeilExpressionsTest {
     private void verifyKeyRange(RoundingType exprType, int scale, Relation relation, BigDecimal rhs, KeyRange range) throws SQLException {
         // dump of values for debugging
         final String dump = getMessage(exprType, scale, relation, rhs, range);
-        
+
         ImmutableBytesPtr rhsPtr = new ImmutableBytesPtr();
-        LiteralExpression.newConstant(rhs, DECIMAL).evaluate(null, rhsPtr);
-        
+        LiteralExpression.newConstant(rhs, PDecimal.INSTANCE).evaluate(null, rhsPtr);
+
         ImmutableBytesPtr lhsPtr = new ImmutableBytesPtr();
-        
+
         // we should only get an empty range if we can verify that precision makes a match impossible
         if(range == KeyRange.EMPTY_RANGE) {
             assertTrue("should only get empty key range for unmatchable rhs precision (" + dump + ")", rhs.scale() > scale);
             assertEquals("should only get empty key range for equals checks (" + dump + ")", Relation.EQUAL, relation);
             return;
         }
-        
+
         // if it should have an upper bound
         if(relation != Relation.GREATER && relation != Relation.GREATER_OR_EQUAL) {
             // figure out what the upper bound is
@@ -405,20 +408,20 @@ public class RoundFloorCeilExpressionsTest {
                 highestHighIncluded = prevDecimalKey(range.getUpperRange());
                 lowestHighExcluded = range.getUpperRange();
             }
-            
+
             // check on either side of the boundary to validate that it is in fact the boundary
             exprType.getExpression(highestHighIncluded, scale).evaluate(null, lhsPtr);
-            assertTrue("incorrectly excluding " + DECIMAL.toObject(highestHighIncluded) 
+            assertTrue("incorrectly excluding " + PDecimal.INSTANCE.toObject(highestHighIncluded)
                 + " in upper bound for " + dump, relation.compare(lhsPtr, rhsPtr));
             exprType.getExpression(lowestHighExcluded, scale).evaluate(null, lhsPtr);
-            assertFalse("incorrectly including " + DECIMAL.toObject(lowestHighExcluded) 
+            assertFalse("incorrectly including " + PDecimal.INSTANCE.toObject(lowestHighExcluded)
                 + " in upper bound for " + dump, relation.compare(lhsPtr, rhsPtr));
         }
         else {
             // otherwise verify that it does not have an upper bound
             assertTrue("should not have a upper bound for " + dump, range.upperUnbound());
         }
-        
+
         // if it should have a lower bound
         if(relation != Relation.LESS && relation != Relation.LESS_OR_EQUAL) {
             // figure out what the lower bound is
@@ -432,13 +435,13 @@ public class RoundFloorCeilExpressionsTest {
                 lowestLowIncluded = nextDecimalKey(range.getLowerRange());
                 highestLowExcluded = range.getLowerRange();
             }
-            
+
             // check on either side of the boundary to validate that it is in fact the boundary
             exprType.getExpression(lowestLowIncluded, scale).evaluate(null, lhsPtr);
-            assertTrue("incorrectly excluding " + DECIMAL.toObject(lowestLowIncluded) 
+            assertTrue("incorrectly excluding " + PDecimal.INSTANCE.toObject(lowestLowIncluded)
                 + " in lower bound for " + dump, relation.compare(lhsPtr, rhsPtr));
             exprType.getExpression(highestLowExcluded, scale).evaluate(null, lhsPtr);
-            assertFalse("incorrectly including " + DECIMAL.toObject(highestLowExcluded) 
+            assertFalse("incorrectly including " + PDecimal.INSTANCE.toObject(highestLowExcluded)
                 + " in lower bound for " + dump, relation.compare(lhsPtr, rhsPtr));
         }
         else {
@@ -446,7 +449,7 @@ public class RoundFloorCeilExpressionsTest {
             assertTrue("should not have a lower bound for " + dump, range.lowerUnbound());
         }
     }
-    
+
     /**
      * Produces the previous Decimal key relative to the given key. The new key will differ from
      * the old key in as small a unit as possible while still maintaining accurate serialization.
@@ -454,11 +457,11 @@ public class RoundFloorCeilExpressionsTest {
      * @return  bytes for the new Decimal key, a single unit previous to the old one
      */
     private static byte[] prevDecimalKey(byte[] key) {
-        BigDecimal decimal = (BigDecimal) DECIMAL.toObject(key);
+        BigDecimal decimal = (BigDecimal) PDecimal.INSTANCE.toObject(key);
         BigDecimal prev = decimal.subtract(getSmallestUnit(decimal));
-        return DECIMAL.toBytes(prev);
+        return PDecimal.INSTANCE.toBytes(prev);
     }
-    
+
     /**
      * Produces the next Decimal key relative to the given key. The new key will differ from the
      * old key in as small a unit as possible while still maintaining accurate serialization.
@@ -466,14 +469,14 @@ public class RoundFloorCeilExpressionsTest {
      * @return  bytes for the new Decimal key, a single unit next from the old one
      */
     private static byte[] nextDecimalKey(byte[] key) {
-        BigDecimal decimal = (BigDecimal) DECIMAL.toObject(key);
+        BigDecimal decimal = (BigDecimal) PDecimal.INSTANCE.toObject(key);
         BigDecimal next = decimal.add(getSmallestUnit(decimal));
-        return DECIMAL.toBytes(next);
+        return PDecimal.INSTANCE.toBytes(next);
     }
-    
+
     /**
      * Produces the smallest unit of difference possible for the given decimal that will still
-     * be serialized accurately. For example, if the MAXIMUM_RELIABLE_PRECISION were 4, then 
+     * be serialized accurately. For example, if the MAXIMUM_RELIABLE_PRECISION were 4, then
      * getSmallestUnit(2.3) would produce 0.001, as 2.301 could be serialized accurately but
      * 2.3001 could not.
      * @param decimal  the decimal to find the smallest unit in relation to
@@ -493,7 +496,7 @@ public class RoundFloorCeilExpressionsTest {
     
     @Test
     public void testRoundDateExpression() throws Exception {
-        LiteralExpression dateLiteral = LiteralExpression.newConstant(DateUtil.parseDate("2012-01-01 14:25:28"), PDataType.DATE);
+        LiteralExpression dateLiteral = LiteralExpression.newConstant(DateUtil.parseDate("2012-01-01 14:25:28"), PDate.INSTANCE);
         Expression roundDateExpression = RoundDateExpression.create(dateLiteral, TimeUnit.DAY);
         
         ImmutableBytesWritable ptr = new ImmutableBytesWritable();
@@ -507,7 +510,7 @@ public class RoundFloorCeilExpressionsTest {
     
     @Test
     public void testRoundDateExpressionWithMultiplier() throws Exception {
-        Expression dateLiteral = LiteralExpression.newConstant(DateUtil.parseDate("2012-01-01 14:25:28"), PDataType.DATE);
+        Expression dateLiteral = LiteralExpression.newConstant(DateUtil.parseDate("2012-01-01 14:25:28"), PDate.INSTANCE);
         Expression roundDateExpression = RoundDateExpression.create(dateLiteral, TimeUnit.MINUTE, 10);
         
         ImmutableBytesWritable ptr = new ImmutableBytesWritable();
@@ -521,7 +524,7 @@ public class RoundFloorCeilExpressionsTest {
     
     @Test
     public void testFloorDateExpression() throws Exception {
-        LiteralExpression dateLiteral = LiteralExpression.newConstant(DateUtil.parseDate("2012-01-01 14:25:28"), PDataType.DATE);
+        LiteralExpression dateLiteral = LiteralExpression.newConstant(DateUtil.parseDate("2012-01-01 14:25:28"), PDate.INSTANCE);
         Expression floorDateExpression = FloorDateExpression.create(dateLiteral, TimeUnit.DAY);
         
         ImmutableBytesWritable ptr = new ImmutableBytesWritable();
@@ -535,7 +538,7 @@ public class RoundFloorCeilExpressionsTest {
     
     @Test
     public void testFloorDateExpressionWithMultiplier() throws Exception {
-        Expression dateLiteral = LiteralExpression.newConstant(DateUtil.parseDate("2012-01-01 14:25:28"), PDataType.DATE);
+        Expression dateLiteral = LiteralExpression.newConstant(DateUtil.parseDate("2012-01-01 14:25:28"), PDate.INSTANCE);
         Expression floorDateExpression = FloorDateExpression.create(dateLiteral, TimeUnit.SECOND, 10);
         
         ImmutableBytesWritable ptr = new ImmutableBytesWritable();
@@ -549,7 +552,7 @@ public class RoundFloorCeilExpressionsTest {
     
     @Test
     public void testCeilDateExpression() throws Exception {
-        LiteralExpression dateLiteral = LiteralExpression.newConstant(DateUtil.parseDate("2012-01-01 14:25:28"), PDataType.DATE);
+        LiteralExpression dateLiteral = LiteralExpression.newConstant(DateUtil.parseDate("2012-01-01 14:25:28"), PDate.INSTANCE);
         Expression ceilDateExpression = CeilDateExpression.create(dateLiteral, TimeUnit.DAY);
         
         ImmutableBytesWritable ptr = new ImmutableBytesWritable();
@@ -563,7 +566,7 @@ public class RoundFloorCeilExpressionsTest {
     
     @Test
     public void testCeilDateExpressionWithMultiplier() throws Exception {
-        Expression dateLiteral = LiteralExpression.newConstant(DateUtil.parseDate("2012-01-01 14:25:28"), PDataType.DATE);
+        Expression dateLiteral = LiteralExpression.newConstant(DateUtil.parseDate("2012-01-01 14:25:28"), PDate.INSTANCE);
         Expression ceilDateExpression = CeilDateExpression.create(dateLiteral, TimeUnit.SECOND, 10);
         
         ImmutableBytesWritable ptr = new ImmutableBytesWritable();
@@ -580,7 +583,7 @@ public class RoundFloorCeilExpressionsTest {
      */
     @Test
     public void testRoundDateExpressionValidation_1() throws Exception {
-        LiteralExpression dateLiteral = LiteralExpression.newConstant(DateUtil.parseDate("2012-01-01 14:25:28"), PDataType.DATE);
+        LiteralExpression dateLiteral = LiteralExpression.newConstant(DateUtil.parseDate("2012-01-01 14:25:28"), PDate.INSTANCE);
         
         List<Expression> childExpressions = new ArrayList<Expression>(1);
         childExpressions.add(dateLiteral);
@@ -598,8 +601,8 @@ public class RoundFloorCeilExpressionsTest {
      */
     @Test
     public void testRoundDateExpressionValidation_2() throws Exception {
-        LiteralExpression dateLiteral = LiteralExpression.newConstant(DateUtil.parseDate("2012-01-01 14:25:28"), PDataType.DATE);
-        LiteralExpression timeUnitLiteral = LiteralExpression.newConstant("millis", PDataType.VARCHAR);
+        LiteralExpression dateLiteral = LiteralExpression.newConstant(DateUtil.parseDate("2012-01-01 14:25:28"), PDate.INSTANCE);
+        LiteralExpression timeUnitLiteral = LiteralExpression.newConstant("millis", PVarchar.INSTANCE);
         
         List<Expression> childExpressions = new ArrayList<Expression>(1);
         childExpressions.add(dateLiteral);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/04ef859b/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 d7f4172..464153d 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,6 +30,19 @@ 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;
@@ -50,7 +63,7 @@ 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.PDataType;
+import org.apache.phoenix.schema.types.PDataType;
 import org.apache.phoenix.util.DateUtil;
 
 /**
@@ -60,31 +73,31 @@ public class SortOrderExpressionTest {
     
     @Test
     public void substr() throws Exception {
-        List<Expression> args = Lists.newArrayList(getInvertedLiteral("blah", PDataType.CHAR), getLiteral(3), getLiteral(2));
+        List<Expression> args = Lists.newArrayList(getInvertedLiteral("blah", PChar.INSTANCE), getLiteral(3), getLiteral(2));
         evaluateAndAssertResult(new SubstrFunction(args), "ah");
     }
     
     @Test
     public void regexpSubstr() throws Exception {
-        List<Expression> args = Lists.newArrayList(getInvertedLiteral("blah", PDataType.CHAR), getLiteral("l.h"), getLiteral(2));
+        List<Expression> args = Lists.newArrayList(getInvertedLiteral("blah", PChar.INSTANCE), getLiteral("l.h"), getLiteral(2));
         evaluateAndAssertResult(new RegexpSubstrFunction(args), "lah");
     }
     
     @Test
     public void regexpReplace() throws Exception {
-        List<Expression> args = Lists.newArrayList(getInvertedLiteral("blah", PDataType.CHAR), getLiteral("l.h"), getLiteral("foo"));
+        List<Expression> args = Lists.newArrayList(getInvertedLiteral("blah", PChar.INSTANCE), getLiteral("l.h"), getLiteral("foo"));
         evaluateAndAssertResult(new RegexpReplaceFunction(args), "bfoo");
     }
     
     @Test
     public void ltrim() throws Exception {
-        List<Expression> args = Lists.newArrayList(getInvertedLiteral("   blah", PDataType.CHAR));
+        List<Expression> args = Lists.newArrayList(getInvertedLiteral("   blah", PChar.INSTANCE));
         evaluateAndAssertResult(new LTrimFunction(args), "blah");
     }
     
     @Test
     public void substrLtrim() throws Exception {
-        List<Expression> ltrimArgs = Lists.newArrayList(getInvertedLiteral("   blah", PDataType.CHAR));
+        List<Expression> ltrimArgs = Lists.newArrayList(getInvertedLiteral("   blah", PChar.INSTANCE));
         Expression ltrim = new LTrimFunction(ltrimArgs);
         List<Expression> substrArgs = Lists.newArrayList(ltrim, getLiteral(3), getLiteral(2));
         evaluateAndAssertResult(new SubstrFunction(substrArgs), "ah");
@@ -92,157 +105,157 @@ public class SortOrderExpressionTest {
     
     @Test
     public void rtrim() throws Exception {
-        List<Expression> args = Lists.newArrayList(getInvertedLiteral("blah    ", PDataType.CHAR));
+        List<Expression> args = Lists.newArrayList(getInvertedLiteral("blah    ", PChar.INSTANCE));
         evaluateAndAssertResult(new RTrimFunction(args), "blah");
     }
     
     @Test
     public void lower() throws Exception {
-        List<Expression> args = Lists.newArrayList(getInvertedLiteral("BLAH", PDataType.CHAR));
+        List<Expression> args = Lists.newArrayList(getInvertedLiteral("BLAH", PChar.INSTANCE));
         evaluateAndAssertResult(new LowerFunction(args), "blah");        
     }
     
     @Test
     public void upper() throws Exception {
-        List<Expression> args = Lists.newArrayList(getInvertedLiteral("blah", PDataType.CHAR));
+        List<Expression> args = Lists.newArrayList(getInvertedLiteral("blah", PChar.INSTANCE));
         evaluateAndAssertResult(new UpperFunction(args), "BLAH");        
     }
     
     @Test
     public void length() throws Exception {
-        List<Expression> args = Lists.newArrayList(getInvertedLiteral("blah", PDataType.CHAR));
+        List<Expression> args = Lists.newArrayList(getInvertedLiteral("blah", PChar.INSTANCE));
         evaluateAndAssertResult(new LengthFunction(args), 4);
     }
     
     @Test
     public void round() throws Exception {
-        List<Expression> args = Lists.newArrayList(getInvertedLiteral(date(12, 11, 2001), PDataType.DATE), getLiteral("hour"), getLiteral(1));
+        List<Expression> args = Lists.newArrayList(getInvertedLiteral(date(12, 11, 2001), PDate.INSTANCE), getLiteral("hour"), getLiteral(1));
         evaluateAndAssertResult(RoundDateExpression.create(args), date(12, 11, 2001));
     }
     
     @Test
     public void sqlTypeName() throws Exception {
-        List<Expression> args = Lists.newArrayList(getInvertedLiteral(12, PDataType.INTEGER));
+        List<Expression> args = Lists.newArrayList(getInvertedLiteral(12, PInteger.INSTANCE));
         evaluateAndAssertResult(new SqlTypeNameFunction(args), "VARCHAR");        
     }
     
     @Test
     public void toChar() throws Exception {
-        List<Expression> args = Lists.newArrayList(getInvertedLiteral(date(12, 11, 2001), PDataType.DATE));
+        List<Expression> args = Lists.newArrayList(getInvertedLiteral(date(12, 11, 2001), PDate.INSTANCE));
         evaluateAndAssertResult(new ToCharFunction(args, FunctionArgumentType.TEMPORAL, "", DateUtil.getDateFormatter("MM/dd/yy hh:mm a")), "12/11/01 12:00 AM");
     }
     
     @Test
     public void toDate() throws Exception {
-        List<Expression> args = Lists.newArrayList(getInvertedLiteral("2001-11-30 00:00:00:0", PDataType.VARCHAR));
+        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));
     }
     
     @Test
     public void toNumber() throws Exception {
-        List<Expression> args = Lists.newArrayList(getInvertedLiteral("10", PDataType.VARCHAR));
+        List<Expression> args = Lists.newArrayList(getInvertedLiteral("10", PVarchar.INSTANCE));
         evaluateAndAssertResult(new ToNumberFunction(args, FunctionArgumentType.CHAR, "", null), new BigDecimal(BigInteger.valueOf(1), -1));
     }
     
     @Test
     public void trim() throws Exception {
-        List<Expression> args = Lists.newArrayList(getInvertedLiteral("   blah    ", PDataType.CHAR));
+        List<Expression> args = Lists.newArrayList(getInvertedLiteral("   blah    ", PChar.INSTANCE));
         evaluateAndAssertResult(new TrimFunction(args), "blah");
     }
     
     @Test
     public void lpad() throws Exception {
-        List<Expression> args = Lists.newArrayList(getInvertedLiteral("ABCD", PDataType.CHAR), getLiteral(7), getLiteral("12"));
+        List<Expression> args = Lists.newArrayList(getInvertedLiteral("ABCD", PChar.INSTANCE), getLiteral(7), getLiteral("12"));
         evaluateAndAssertResult(new LpadFunction(args), "121ABCD");
     }
     
     @Test
     public void add() throws Exception {
-        List<Expression> args = Lists.newArrayList(getInvertedLiteral(10, PDataType.INTEGER), getLiteral(2));
+        List<Expression> args = Lists.newArrayList(getInvertedLiteral(10, PInteger.INSTANCE), getLiteral(2));
         evaluateAndAssertResult(new DecimalAddExpression(args), BigDecimal.valueOf(12));
         
-        args = Lists.newArrayList(getInvertedLiteral(10, PDataType.INTEGER), getLiteral(2));
+        args = Lists.newArrayList(getInvertedLiteral(10, PInteger.INSTANCE), getLiteral(2));
         evaluateAndAssertResult(new LongAddExpression(args), 12l);
         
-        args = Lists.newArrayList(getInvertedLiteral(10.0, PDataType.FLOAT), getLiteral(2));
+        args = Lists.newArrayList(getInvertedLiteral(10.0, PFloat.INSTANCE), getLiteral(2));
         evaluateAndAssertResult(new DoubleAddExpression(args), 12.0);
         
-        args = Lists.newArrayList(getInvertedLiteral(10.0, PDataType.UNSIGNED_FLOAT), getLiteral(2));
+        args = Lists.newArrayList(getInvertedLiteral(10.0, PUnsignedFloat.INSTANCE), getLiteral(2));
         evaluateAndAssertResult(new DoubleAddExpression(args), 12.0);
         
-        args = Lists.newArrayList(getInvertedLiteral(10.0, PDataType.UNSIGNED_DOUBLE), getLiteral(2));
+        args = Lists.newArrayList(getInvertedLiteral(10.0, PUnsignedDouble.INSTANCE), getLiteral(2));
         evaluateAndAssertResult(new DoubleAddExpression(args), 12.0);
         
-        args = Lists.newArrayList(getInvertedLiteral(10.0, PDataType.DOUBLE), getLiteral(2));
+        args = Lists.newArrayList(getInvertedLiteral(10.0, PDouble.INSTANCE), getLiteral(2));
         evaluateAndAssertResult(new DoubleAddExpression(args), 12.0);
     }
 
     @Test
     public void subtract() throws Exception {
-        List<Expression> args = Lists.newArrayList(getInvertedLiteral(10, PDataType.INTEGER), getLiteral(2));
+        List<Expression> args = Lists.newArrayList(getInvertedLiteral(10, PInteger.INSTANCE), getLiteral(2));
         evaluateAndAssertResult(new DecimalSubtractExpression(args), BigDecimal.valueOf(8));
         
-        args = Lists.newArrayList(getInvertedLiteral(10, PDataType.INTEGER), getLiteral(2));
+        args = Lists.newArrayList(getInvertedLiteral(10, PInteger.INSTANCE), getLiteral(2));
         evaluateAndAssertResult(new LongSubtractExpression(args), 8l);
         
-        args = Lists.newArrayList(getInvertedLiteral(10.0, PDataType.FLOAT), getLiteral(2));
+        args = Lists.newArrayList(getInvertedLiteral(10.0, PFloat.INSTANCE), getLiteral(2));
         evaluateAndAssertResult(new DoubleSubtractExpression(args), 8.0);
         
-        args = Lists.newArrayList(getInvertedLiteral(10.0, PDataType.UNSIGNED_FLOAT), getLiteral(2));
+        args = Lists.newArrayList(getInvertedLiteral(10.0, PUnsignedFloat.INSTANCE), getLiteral(2));
         evaluateAndAssertResult(new DoubleSubtractExpression(args), 8.0);
         
-        args = Lists.newArrayList(getInvertedLiteral(10.0, PDataType.UNSIGNED_DOUBLE), getLiteral(2));
+        args = Lists.newArrayList(getInvertedLiteral(10.0, PUnsignedDouble.INSTANCE), getLiteral(2));
         evaluateAndAssertResult(new DoubleSubtractExpression(args), 8.0);
         
-        args = Lists.newArrayList(getInvertedLiteral(10.0, PDataType.DOUBLE), getLiteral(2));
+        args = Lists.newArrayList(getInvertedLiteral(10.0, PDouble.INSTANCE), getLiteral(2));
         evaluateAndAssertResult(new DoubleSubtractExpression(args), 8.0);
     }
     
     @Test
     public void divide() throws Exception {
-        List<Expression> args = Lists.newArrayList(getInvertedLiteral(10, PDataType.INTEGER), getLiteral(2));
+        List<Expression> args = Lists.newArrayList(getInvertedLiteral(10, PInteger.INSTANCE), getLiteral(2));
         evaluateAndAssertResult(new DecimalDivideExpression(args), BigDecimal.valueOf(5));
         
-        args = Lists.newArrayList(getInvertedLiteral(10, PDataType.INTEGER), getLiteral(2));
+        args = Lists.newArrayList(getInvertedLiteral(10, PInteger.INSTANCE), getLiteral(2));
         evaluateAndAssertResult(new LongDivideExpression(args), 5l);
         
-        args = Lists.newArrayList(getInvertedLiteral(10.0, PDataType.FLOAT), getLiteral(2));
+        args = Lists.newArrayList(getInvertedLiteral(10.0, PFloat.INSTANCE), getLiteral(2));
         evaluateAndAssertResult(new DoubleDivideExpression(args), 5.0);
         
-        args = Lists.newArrayList(getInvertedLiteral(10.0, PDataType.UNSIGNED_FLOAT), getLiteral(2));
+        args = Lists.newArrayList(getInvertedLiteral(10.0, PUnsignedFloat.INSTANCE), getLiteral(2));
         evaluateAndAssertResult(new DoubleDivideExpression(args), 5.0);
         
-        args = Lists.newArrayList(getInvertedLiteral(10.0, PDataType.UNSIGNED_DOUBLE), getLiteral(2));
+        args = Lists.newArrayList(getInvertedLiteral(10.0, PUnsignedDouble.INSTANCE), getLiteral(2));
         evaluateAndAssertResult(new DoubleDivideExpression(args), 5.0);
         
-        args = Lists.newArrayList(getInvertedLiteral(10.0, PDataType.DOUBLE), getLiteral(2));
+        args = Lists.newArrayList(getInvertedLiteral(10.0, PDouble.INSTANCE), getLiteral(2));
         evaluateAndAssertResult(new DoubleDivideExpression(args), 5.0);
     }
     
     @Test
     public void multiply() throws Exception {
-        List<Expression> args = Lists.newArrayList(getInvertedLiteral(10, PDataType.INTEGER), getLiteral(2));
+        List<Expression> args = Lists.newArrayList(getInvertedLiteral(10, PInteger.INSTANCE), getLiteral(2));
         evaluateAndAssertResult(new DecimalMultiplyExpression(args), new BigDecimal(BigInteger.valueOf(2), -1));
         
-        args = Lists.newArrayList(getInvertedLiteral(10, PDataType.INTEGER), getLiteral(2));
+        args = Lists.newArrayList(getInvertedLiteral(10, PInteger.INSTANCE), getLiteral(2));
         evaluateAndAssertResult(new LongMultiplyExpression(args), 20l);
         
-        args = Lists.newArrayList(getInvertedLiteral(10.0, PDataType.FLOAT), getLiteral(2));
+        args = Lists.newArrayList(getInvertedLiteral(10.0, PFloat.INSTANCE), getLiteral(2));
         evaluateAndAssertResult(new DoubleMultiplyExpression(args), 20.0);
         
-        args = Lists.newArrayList(getInvertedLiteral(10.0, PDataType.UNSIGNED_FLOAT), getLiteral(2));
+        args = Lists.newArrayList(getInvertedLiteral(10.0, PUnsignedFloat.INSTANCE), getLiteral(2));
         evaluateAndAssertResult(new DoubleMultiplyExpression(args), 20.0);
         
-        args = Lists.newArrayList(getInvertedLiteral(10.0, PDataType.UNSIGNED_DOUBLE), getLiteral(2));
+        args = Lists.newArrayList(getInvertedLiteral(10.0, PUnsignedDouble.INSTANCE), getLiteral(2));
         evaluateAndAssertResult(new DoubleMultiplyExpression(args), 20.0);
         
-        args = Lists.newArrayList(getInvertedLiteral(10.0, PDataType.DOUBLE), getLiteral(2));
+        args = Lists.newArrayList(getInvertedLiteral(10.0, PDouble.INSTANCE), getLiteral(2));
         evaluateAndAssertResult(new DoubleMultiplyExpression(args), 20.0);
     }
         
     @Test
     public void compareNumbers() throws Exception {
-        PDataType[] numberDataTypes = new PDataType[]{PDataType.INTEGER, PDataType.LONG, PDataType.DECIMAL, PDataType.UNSIGNED_INT, PDataType.UNSIGNED_LONG};
+        PDataType[] numberDataTypes = new PDataType[]{ PInteger.INSTANCE, PLong.INSTANCE, PDecimal.INSTANCE, PUnsignedInt.INSTANCE, PUnsignedLong.INSTANCE};
         for (PDataType lhsDataType : numberDataTypes) {
             for (PDataType rhsDataType : numberDataTypes) {
                 runCompareTest(CompareOp.GREATER, true, 10, lhsDataType, 2, rhsDataType);
@@ -252,7 +265,7 @@ public class SortOrderExpressionTest {
     
     @Test
     public void compareCharacters() throws Exception {
-        PDataType[] textDataTypes = new PDataType[]{PDataType.CHAR, PDataType.VARCHAR};
+        PDataType[] textDataTypes = new PDataType[]{ PChar.INSTANCE, PVarchar.INSTANCE};
         for (PDataType lhsDataType : textDataTypes) {
             for (PDataType rhsDataType : textDataTypes) {
                 runCompareTest(CompareOp.GREATER, true, "xxx", lhsDataType, "bbb", rhsDataType);
@@ -262,15 +275,15 @@ public class SortOrderExpressionTest {
     
     @Test
     public void compareBooleans() throws Exception {
-        runCompareTest(CompareOp.GREATER, true, true, PDataType.BOOLEAN, false, PDataType.BOOLEAN);        
+        runCompareTest(CompareOp.GREATER, true, true, PBoolean.INSTANCE, false, PBoolean.INSTANCE);
     }
     
     @Test
     public void stringConcat() throws Exception {
-        List<Expression> args = Lists.newArrayList(getInvertedLiteral("blah", PDataType.VARCHAR), getInvertedLiteral("foo", PDataType.VARCHAR)); 
+        List<Expression> args = Lists.newArrayList(getInvertedLiteral("blah", PVarchar.INSTANCE), getInvertedLiteral("foo", PVarchar.INSTANCE));
         evaluateAndAssertResult(new StringConcatExpression(args), "blahfoo");
         
-        args = Lists.newArrayList(getInvertedLiteral("blah", PDataType.VARCHAR), getInvertedLiteral(10, PDataType.INTEGER)); 
+        args = Lists.newArrayList(getInvertedLiteral("blah", PVarchar.INSTANCE), getInvertedLiteral(10, PInteger.INSTANCE));
         evaluateAndAssertResult(new StringConcatExpression(args), "blah10");        
     }
     

http://git-wip-us.apache.org/repos/asf/phoenix/blob/04ef859b/phoenix-core/src/test/java/org/apache/phoenix/expression/function/ExternalSqlTypeIdFunctionTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/expression/function/ExternalSqlTypeIdFunctionTest.java b/phoenix-core/src/test/java/org/apache/phoenix/expression/function/ExternalSqlTypeIdFunctionTest.java
index bc32414..7f9cc9b 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/expression/function/ExternalSqlTypeIdFunctionTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/expression/function/ExternalSqlTypeIdFunctionTest.java
@@ -21,7 +21,8 @@ import com.google.common.collect.Lists;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.phoenix.expression.Expression;
 import org.apache.phoenix.expression.LiteralExpression;
-import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.schema.types.PInteger;
+import org.apache.phoenix.schema.types.PIntegerArray;
 import org.junit.Test;
 
 import java.sql.SQLException;
@@ -35,7 +36,7 @@ public class ExternalSqlTypeIdFunctionTest {
     @Test
     public void testEvaluate() throws SQLException {
         Expression inputArg = LiteralExpression.newConstant(
-                PDataType.INTEGER.getSqlType(), PDataType.INTEGER);
+                PInteger.INSTANCE.getSqlType(), PInteger.INSTANCE);
 
         Object returnValue = executeFunction(inputArg);
 
@@ -45,7 +46,7 @@ public class ExternalSqlTypeIdFunctionTest {
     @Test
     public void testEvaluateArrayType() throws SQLException {
         Expression inputArg = LiteralExpression.newConstant(
-                PDataType.INTEGER_ARRAY.getSqlType(), PDataType.INTEGER);
+                PIntegerArray.INSTANCE.getSqlType(), PInteger.INSTANCE);
 
         Object returnValue = executeFunction(inputArg);
 
@@ -59,7 +60,7 @@ public class ExternalSqlTypeIdFunctionTest {
         ImmutableBytesWritable ptr = new ImmutableBytesWritable();
         assertTrue(externalIdFunction.evaluate(null, ptr));
 
-        return PDataType.INTEGER.toObject(ptr.get(), ptr.getOffset(), ptr.getLength(),
-                PDataType.INTEGER, inputArg.getSortOrder());
+        return PInteger.INSTANCE.toObject(ptr.get(), ptr.getOffset(), ptr.getLength(),
+            PInteger.INSTANCE, inputArg.getSortOrder());
     }
 }