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

[4/6] Replace generic ColumnModifier enum with more specific SortOrder enum. Ensure that we always pass through either SortOrder.ASC or SortOrder.DESC but never null. https://issues.apache.org/jira/browse/PHOENIX-32 Reviewed by jamestaylor.

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/ae131f56/phoenix-core/src/main/java/org/apache/phoenix/expression/function/RegexpReplaceFunction.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/RegexpReplaceFunction.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/RegexpReplaceFunction.java
index 8a62b4a..3d07e4a 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/RegexpReplaceFunction.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/RegexpReplaceFunction.java
@@ -85,7 +85,7 @@ public class RegexpReplaceFunction extends ScalarFunction {
         if (!sourceStrExpression.evaluate(tuple, ptr)) {
             return false;
         }
-        String sourceStr = (String)PDataType.VARCHAR.toObject(ptr, sourceStrExpression.getColumnModifier());
+        String sourceStr = (String)PDataType.VARCHAR.toObject(ptr, sourceStrExpression.getSortOrder());
         if (sourceStr == null) {
             return false;
         }
@@ -95,7 +95,7 @@ public class RegexpReplaceFunction extends ScalarFunction {
             if (!replaceStrExpression.evaluate(tuple, ptr)) {
                 return false;
             }
-            replaceStr = (String)PDataType.VARCHAR.toObject(ptr, replaceStrExpression.getColumnModifier());
+            replaceStr = (String)PDataType.VARCHAR.toObject(ptr, replaceStrExpression.getSortOrder());
         } else {
             replaceStr = "";
         }

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/ae131f56/phoenix-core/src/main/java/org/apache/phoenix/expression/function/RegexpSubstrFunction.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/RegexpSubstrFunction.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/RegexpSubstrFunction.java
index 7e8b389..296094e 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/RegexpSubstrFunction.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/RegexpSubstrFunction.java
@@ -95,7 +95,7 @@ public class RegexpSubstrFunction extends PrefixFunction {
         if (!getSourceStrExpression().evaluate(tuple, ptr)) {
             return false;
         }
-        String sourceStr = (String)PDataType.VARCHAR.toObject(ptr, getSourceStrExpression().getColumnModifier());
+        String sourceStr = (String)PDataType.VARCHAR.toObject(ptr, getSourceStrExpression().getSortOrder());
         if (sourceStr == null) {
             return false;
         }
@@ -104,7 +104,7 @@ public class RegexpSubstrFunction extends PrefixFunction {
         if (!offsetExpression.evaluate(tuple, ptr)) {
             return false;
         }
-        int offset = offsetExpression.getDataType().getCodec().decodeInt(ptr, offsetExpression.getColumnModifier());
+        int offset = offsetExpression.getDataType().getCodec().decodeInt(ptr, offsetExpression.getSortOrder());
 
         int strlen = sourceStr.length();
         // Account for 1 versus 0-based offset

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/ae131f56/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ReverseFunction.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ReverseFunction.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ReverseFunction.java
index b90fd7c..92db234 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ReverseFunction.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ReverseFunction.java
@@ -8,7 +8,7 @@ import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.phoenix.expression.Expression;
 import org.apache.phoenix.parse.FunctionParseNode.Argument;
 import org.apache.phoenix.parse.FunctionParseNode.BuiltInFunction;
-import org.apache.phoenix.schema.ColumnModifier;
+import org.apache.phoenix.schema.SortOrder;
 import org.apache.phoenix.schema.PDataType;
 import org.apache.phoenix.schema.tuple.Tuple;
 import org.apache.phoenix.util.StringUtil;
@@ -41,9 +41,9 @@ public class ReverseFunction extends ScalarFunction {
         byte[] target = new byte[targetOffset];
         int sourceOffset = ptr.getOffset(); 
         int endOffset = sourceOffset + ptr.getLength();
-        ColumnModifier modifier = arg.getColumnModifier();
+        SortOrder sortOrder = arg.getSortOrder();
         while (sourceOffset < endOffset) {
-            int nBytes = StringUtil.getBytesInChar(source[sourceOffset], modifier);
+            int nBytes = StringUtil.getBytesInChar(source[sourceOffset], sortOrder);
             targetOffset -= nBytes;
             System.arraycopy(source, sourceOffset, target, targetOffset, nBytes);
             sourceOffset += nBytes;
@@ -53,8 +53,8 @@ public class ReverseFunction extends ScalarFunction {
     }
 
     @Override
-    public ColumnModifier getColumnModifier() {
-        return getChildren().get(0).getColumnModifier();
+    public SortOrder getSortOrder() {
+        return getChildren().get(0).getSortOrder();
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/ae131f56/phoenix-core/src/main/java/org/apache/phoenix/expression/function/RoundDateExpression.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/RoundDateExpression.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/RoundDateExpression.java
index 8d171ca..d55935d 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/RoundDateExpression.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/RoundDateExpression.java
@@ -32,12 +32,14 @@ import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.hadoop.io.WritableUtils;
 
 import com.google.common.collect.Lists;
+
 import org.apache.phoenix.compile.KeyPart;
 import org.apache.phoenix.expression.Expression;
 import org.apache.phoenix.expression.LiteralExpression;
 import org.apache.phoenix.query.KeyRange;
 import org.apache.phoenix.schema.PColumn;
 import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.schema.SortOrder;
 import org.apache.phoenix.schema.PDataType.PDataCodec;
 import org.apache.phoenix.schema.tuple.Tuple;
 import org.apache.phoenix.util.ByteUtil;
@@ -134,7 +136,7 @@ public class RoundDateExpression extends ScalarFunction {
     public boolean evaluate(Tuple tuple, ImmutableBytesWritable ptr) {
         if (children.get(0).evaluate(tuple, ptr)) {
             PDataType dataType = getDataType();
-            long time = dataType.getCodec().decodeLong(ptr, children.get(0).getColumnModifier());
+            long time = dataType.getCodec().decodeLong(ptr, children.get(0).getSortOrder());
             long value = roundTime(time);
             
             Date d = new Date(value);
@@ -223,11 +225,11 @@ public class RoundDateExpression extends ScalarFunction {
                 ImmutableBytesWritable ptr = new ImmutableBytesWritable();
                 rhs.evaluate(null, ptr);
                 byte[] key = ByteUtil.copyKeyBytesIfNecessary(ptr);
-                // No need to take into account column modifier, because ROUND
+                // No need to take into account SortOrder, because ROUND
                 // always forces the value to be in ascending order
                 PDataCodec codec = getKeyRangeCodec(type);
                 int offset = ByteUtil.isInclusive(op) ? 1 : 0;
-                long value = codec.decodeLong(key, 0, null);
+                long value = codec.decodeLong(key, 0, SortOrder.getDefault());
                 byte[] nextKey = new byte[type.getByteSize()];
                 switch (op) {
                 case EQUAL:

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/ae131f56/phoenix-core/src/main/java/org/apache/phoenix/expression/function/RoundDecimalExpression.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/RoundDecimalExpression.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/RoundDecimalExpression.java
index a103077..e2d0356 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/RoundDecimalExpression.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/RoundDecimalExpression.java
@@ -94,7 +94,7 @@ public class RoundDecimalExpression extends ScalarFunction {
     public boolean evaluate(Tuple tuple, ImmutableBytesWritable ptr) {
         Expression childExpr = children.get(0);
         if(childExpr.evaluate(tuple, ptr)) {
-            BigDecimal value = (BigDecimal)PDataType.DECIMAL.toObject(ptr, childExpr.getColumnModifier());
+            BigDecimal value = (BigDecimal)PDataType.DECIMAL.toObject(ptr, childExpr.getSortOrder());
             BigDecimal scaledValue = value.setScale(scale, getRoundingMode());
             ptr.set(getDataType().toBytes(scaledValue));
             return true;

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/ae131f56/phoenix-core/src/main/java/org/apache/phoenix/expression/function/RoundTimestampExpression.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/RoundTimestampExpression.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/RoundTimestampExpression.java
index 77cff18..17253a5 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/RoundTimestampExpression.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/RoundTimestampExpression.java
@@ -29,7 +29,7 @@ import com.google.common.collect.Lists;
 import org.apache.phoenix.expression.CoerceExpression;
 import org.apache.phoenix.expression.Expression;
 import org.apache.phoenix.expression.LiteralExpression;
-import org.apache.phoenix.schema.ColumnModifier;
+import org.apache.phoenix.schema.SortOrder;
 import org.apache.phoenix.schema.PDataType;
 import org.apache.phoenix.schema.PDataType.PDataCodec;
 import org.apache.phoenix.schema.tuple.Tuple;
@@ -89,11 +89,11 @@ public class RoundTimestampExpression extends RoundDateExpression {
     @Override
     public boolean evaluate(Tuple tuple, ImmutableBytesWritable ptr) {
         if (children.get(0).evaluate(tuple, ptr)) {
-            ColumnModifier columnModifier = children.get(0).getColumnModifier();
+            SortOrder sortOrder = children.get(0).getSortOrder();
             PDataType dataType = getDataType();
-            int nanos = dataType.getNanos(ptr, columnModifier);
+            int nanos = dataType.getNanos(ptr, sortOrder);
             if(nanos >= HALF_OF_NANOS_IN_MILLI) {
-                long timeMillis = dataType.getMillis(ptr, columnModifier);
+                long timeMillis = dataType.getMillis(ptr, sortOrder);
                 Timestamp roundedTs = new Timestamp(timeMillis + 1);
                 byte[] byteValue = dataType.toBytes(roundedTs);
                 ptr.set(byteValue);

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/ae131f56/phoenix-core/src/main/java/org/apache/phoenix/expression/function/SqlTypeNameFunction.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/SqlTypeNameFunction.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/SqlTypeNameFunction.java
index 796ba83..9f6e8b9 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/SqlTypeNameFunction.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/SqlTypeNameFunction.java
@@ -64,7 +64,7 @@ public class SqlTypeNameFunction extends ScalarFunction {
         if (ptr.getLength() == 0) {
             return true;
         }
-        int sqlType = child.getDataType().getCodec().decodeInt(ptr, child.getColumnModifier());
+        int sqlType = child.getDataType().getCodec().decodeInt(ptr, child.getSortOrder());
         try {
             byte[] sqlTypeNameBytes = PDataType.fromTypeId(sqlType).getSqlTypeNameBytes();
             ptr.set(sqlTypeNameBytes);

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/ae131f56/phoenix-core/src/main/java/org/apache/phoenix/expression/function/StddevPopFunction.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/StddevPopFunction.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/StddevPopFunction.java
index de9d8ad..30f3106 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/StddevPopFunction.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/StddevPopFunction.java
@@ -61,9 +61,9 @@ public class StddevPopFunction extends DistinctValueWithCountAggregateFunction {
     public DistinctValueWithCountClientAggregator newClientAggregator() {
         if (children.get(0).getDataType() == PDataType.DECIMAL) {
             // Special Aggregators for DECIMAL datatype for more precision than double
-            return new DecimalStddevPopAggregator(children, getAggregatorExpression().getColumnModifier());
+            return new DecimalStddevPopAggregator(children, getAggregatorExpression().getSortOrder());
         }
-        return new StddevPopAggregator(children, getAggregatorExpression().getColumnModifier());
+        return new StddevPopAggregator(children, getAggregatorExpression().getSortOrder());
     }
     
     @Override

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/ae131f56/phoenix-core/src/main/java/org/apache/phoenix/expression/function/StddevSampFunction.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/StddevSampFunction.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/StddevSampFunction.java
index 1d1232e..f5b88a9 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/StddevSampFunction.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/StddevSampFunction.java
@@ -61,9 +61,9 @@ public class StddevSampFunction extends DistinctValueWithCountAggregateFunction
     public DistinctValueWithCountClientAggregator newClientAggregator() {
         if (children.get(0).getDataType() == PDataType.DECIMAL) {
             // Special Aggregators for DECIMAL datatype for more precision than double
-            return new DecimalStddevSampAggregator(children, getAggregatorExpression().getColumnModifier());
+            return new DecimalStddevSampAggregator(children, getAggregatorExpression().getSortOrder());
         }
-        return new StddevSampAggregator(children, getAggregatorExpression().getColumnModifier());
+        return new StddevSampAggregator(children, getAggregatorExpression().getSortOrder());
     }
     
     @Override

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/ae131f56/phoenix-core/src/main/java/org/apache/phoenix/expression/function/SubstrFunction.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/SubstrFunction.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/SubstrFunction.java
index 1ff7284..5efbc8f 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/SubstrFunction.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/SubstrFunction.java
@@ -30,7 +30,7 @@ import org.apache.phoenix.expression.Expression;
 import org.apache.phoenix.expression.LiteralExpression;
 import org.apache.phoenix.parse.FunctionParseNode.Argument;
 import org.apache.phoenix.parse.FunctionParseNode.BuiltInFunction;
-import org.apache.phoenix.schema.ColumnModifier;
+import org.apache.phoenix.schema.SortOrder;
 import org.apache.phoenix.schema.PDataType;
 import org.apache.phoenix.schema.tuple.Tuple;
 import org.apache.phoenix.util.StringUtil;
@@ -70,8 +70,6 @@ public class SubstrFunction extends PrefixFunction {
     }
 
     private void init() {
-        // TODO: when we have ColumnModifier.REVERSE, we'll need to negate offset,
-        // since the bytes are reverse and we'll want to work from the end.
         isOffsetConstant = getOffsetExpression() instanceof LiteralExpression;
         isLengthConstant = getLengthExpression() instanceof LiteralExpression;
         hasLengthExpression = !isLengthConstant || ((LiteralExpression)getLengthExpression()).getValue() != null;
@@ -100,7 +98,7 @@ public class SubstrFunction extends PrefixFunction {
         if (!offsetExpression.evaluate(tuple,  ptr)) {
             return false;
         }
-        int offset = offsetExpression.getDataType().getCodec().decodeInt(ptr, offsetExpression.getColumnModifier());
+        int offset = offsetExpression.getDataType().getCodec().decodeInt(ptr, offsetExpression.getSortOrder());
         
         int length = -1;
         if (hasLengthExpression) {
@@ -108,7 +106,7 @@ public class SubstrFunction extends PrefixFunction {
             if (!lengthExpression.evaluate(tuple, ptr)) {
                 return false;
             }
-            length = lengthExpression.getDataType().getCodec().decodeInt(ptr, lengthExpression.getColumnModifier());
+            length = lengthExpression.getDataType().getCodec().decodeInt(ptr, lengthExpression.getSortOrder());
             if (length <= 0) {
                 return false;
             }
@@ -120,8 +118,8 @@ public class SubstrFunction extends PrefixFunction {
         
         try {
             boolean isCharType = getStrExpression().getDataType() == PDataType.CHAR;
-            ColumnModifier columnModifier = getStrExpression().getColumnModifier();
-            int strlen = isCharType ? ptr.getLength() : StringUtil.calculateUTF8Length(ptr.get(), ptr.getOffset(), ptr.getLength(), columnModifier);
+            SortOrder sortOrder = getStrExpression().getSortOrder();
+            int strlen = isCharType ? ptr.getLength() : StringUtil.calculateUTF8Length(ptr.get(), ptr.getOffset(), ptr.getLength(), sortOrder);
             
             // Account for 1 versus 0-based offset
             offset = offset - (offset <= 0 ? 0 : 1);
@@ -134,8 +132,8 @@ public class SubstrFunction extends PrefixFunction {
             int maxLength = strlen - offset;
             length = length == -1 ? maxLength : Math.min(length,maxLength);
             
-            int byteOffset = isCharType ? offset : StringUtil.getByteLengthForUtf8SubStr(ptr.get(), ptr.getOffset(), offset, columnModifier);
-            int byteLength = isCharType ? length : StringUtil.getByteLengthForUtf8SubStr(ptr.get(), ptr.getOffset() + byteOffset, length, columnModifier);
+            int byteOffset = isCharType ? offset : StringUtil.getByteLengthForUtf8SubStr(ptr.get(), ptr.getOffset(), offset, sortOrder);
+            int byteLength = isCharType ? length : StringUtil.getByteLengthForUtf8SubStr(ptr.get(), ptr.getOffset() + byteOffset, length, sortOrder);
             ptr.set(ptr.get(), ptr.getOffset() + byteOffset, byteLength);
             return true;
         } catch (UnsupportedEncodingException e) {
@@ -167,8 +165,8 @@ public class SubstrFunction extends PrefixFunction {
     }
     
     @Override
-    public ColumnModifier getColumnModifier() {
-        return getStrExpression().getColumnModifier();
+    public SortOrder getSortOrder() {
+        return getStrExpression().getSortOrder();
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/ae131f56/phoenix-core/src/main/java/org/apache/phoenix/expression/function/SumAggregateFunction.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/SumAggregateFunction.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/SumAggregateFunction.java
index b12b297..ad61981 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/SumAggregateFunction.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/SumAggregateFunction.java
@@ -34,7 +34,7 @@ import org.apache.phoenix.expression.aggregator.NumberSumAggregator;
 import org.apache.phoenix.parse.FunctionParseNode.Argument;
 import org.apache.phoenix.parse.FunctionParseNode.BuiltInFunction;
 import org.apache.phoenix.parse.SumAggregateParseNode;
-import org.apache.phoenix.schema.ColumnModifier;
+import org.apache.phoenix.schema.SortOrder;
 import org.apache.phoenix.schema.PDataType;
 import org.apache.phoenix.schema.tuple.Tuple;
 
@@ -62,22 +62,22 @@ public class SumAggregateFunction extends DelegateConstantToCountAggregateFuncti
         super(childExpressions, delegate);
     }
     
-    private Aggregator newAggregator(final PDataType type, ColumnModifier columnModifier, ImmutableBytesWritable ptr) {
+    private Aggregator newAggregator(final PDataType type, SortOrder sortOrder, ImmutableBytesWritable ptr) {
         switch( type ) {
             case DECIMAL:
-                return new DecimalSumAggregator(columnModifier, ptr);
+                return new DecimalSumAggregator(sortOrder, ptr);
             case UNSIGNED_DOUBLE:
             case UNSIGNED_FLOAT:
             case DOUBLE:
             case FLOAT:
-                return new DoubleSumAggregator(columnModifier, ptr) {
+                return new DoubleSumAggregator(sortOrder, ptr) {
                     @Override
                     protected PDataType getInputDataType() {
                         return type;
                     }
                 };
             default:
-                return new NumberSumAggregator(columnModifier, ptr) {
+                return new NumberSumAggregator(sortOrder, ptr) {
                     @Override
                     protected PDataType getInputDataType() {
                         return type;
@@ -88,19 +88,19 @@ public class SumAggregateFunction extends DelegateConstantToCountAggregateFuncti
 
     @Override
     public Aggregator newClientAggregator() {
-        return newAggregator(getDataType(), null, null);
+        return newAggregator(getDataType(), SortOrder.getDefault(), null);
     }
     
     @Override
     public Aggregator newServerAggregator(Configuration conf) {
         Expression child = getAggregatorExpression();
-        return newAggregator(child.getDataType(), child.getColumnModifier(), null);
+        return newAggregator(child.getDataType(), child.getSortOrder(), null);
     }
     
     @Override
     public Aggregator newServerAggregator(Configuration conf, ImmutableBytesWritable ptr) {
         Expression child = getAggregatorExpression();
-        return newAggregator(child.getDataType(), child.getColumnModifier(), ptr);
+        return newAggregator(child.getDataType(), child.getSortOrder(), ptr);
     }
     
     @Override
@@ -116,7 +116,7 @@ public class SumAggregateFunction extends DelegateConstantToCountAggregateFuncti
                 ptr.set(PDataType.DECIMAL.toBytes(value));
             } else {
                 long constantLongValue = ((Number)constantValue).longValue();
-                long value = constantLongValue * type.getCodec().decodeLong(ptr, null);
+                long value = constantLongValue * type.getCodec().decodeLong(ptr, SortOrder.getDefault());
                 byte[] resultPtr = new byte[type.getByteSize()];
                 type.getCodec().encodeLong(value, resultPtr, 0);
                 ptr.set(resultPtr);

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/ae131f56/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ToCharFunction.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ToCharFunction.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ToCharFunction.java
index 4a07182..077e8ea 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ToCharFunction.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ToCharFunction.java
@@ -93,7 +93,7 @@ public class ToCharFunction extends ScalarFunction {
             return false;
         }
         PDataType type = expression.getDataType();
-        Object value = formatter.format(type.toObject(ptr, expression.getColumnModifier()));
+        Object value = formatter.format(type.toObject(ptr, expression.getSortOrder()));
         byte[] b = getDataType().toBytes(value);
         ptr.set(b);
         return true;

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/ae131f56/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ToDateFunction.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ToDateFunction.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ToDateFunction.java
index ed34794..435cffc 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ToDateFunction.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ToDateFunction.java
@@ -88,7 +88,7 @@ public class ToDateFunction extends ScalarFunction {
             return false;
         }
         PDataType type = expression.getDataType();
-        String dateStr = (String)type.toObject(ptr, expression.getColumnModifier());
+        String dateStr = (String)type.toObject(ptr, expression.getSortOrder());
         try {
             Object value = dateParser.parseObject(dateStr);
             byte[] byteValue = getDataType().toBytes(value);

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/ae131f56/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ToNumberFunction.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ToNumberFunction.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ToNumberFunction.java
index cc0a1b6..e859040 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ToNumberFunction.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ToNumberFunction.java
@@ -78,14 +78,14 @@ public class ToNumberFunction extends ScalarFunction {
 
         PDataType type = expression.getDataType();
         if (type.isCoercibleTo(PDataType.TIMESTAMP)) {
-        	Date date = (Date) type.toObject(ptr, expression.getColumnModifier());
+        	Date date = (Date) type.toObject(ptr, expression.getSortOrder());
         	BigDecimal time = new BigDecimal(date.getTime());
             byte[] byteValue = getDataType().toBytes(time);
             ptr.set(byteValue);
             return true;
         }
         
-        String stringValue = (String)type.toObject(ptr, expression.getColumnModifier());
+        String stringValue = (String)type.toObject(ptr, expression.getSortOrder());
         if (stringValue == null) {
             ptr.set(EMPTY_BYTE_ARRAY);
             return true;

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/ae131f56/phoenix-core/src/main/java/org/apache/phoenix/expression/function/TrimFunction.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/TrimFunction.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/TrimFunction.java
index 7350337..2d92703 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/TrimFunction.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/TrimFunction.java
@@ -27,7 +27,7 @@ import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.phoenix.expression.Expression;
 import org.apache.phoenix.parse.FunctionParseNode.Argument;
 import org.apache.phoenix.parse.FunctionParseNode.BuiltInFunction;
-import org.apache.phoenix.schema.ColumnModifier;
+import org.apache.phoenix.schema.SortOrder;
 import org.apache.phoenix.schema.PDataType;
 import org.apache.phoenix.schema.tuple.Tuple;
 import org.apache.phoenix.util.ByteUtil;
@@ -62,8 +62,8 @@ public class TrimFunction extends ScalarFunction {
     }
 
     @Override
-    public ColumnModifier getColumnModifier() {
-        return children.get(0).getColumnModifier();
+    public SortOrder getSortOrder() {
+        return children.get(0).getSortOrder();
     }    
 
     @Override
@@ -79,13 +79,13 @@ public class TrimFunction extends ScalarFunction {
         int offset = ptr.getOffset();
         int length = ptr.getLength();
         
-        ColumnModifier columnModifier = getColumnModifier();
-        int end = StringUtil.getFirstNonBlankCharIdxFromEnd(string, offset, length, columnModifier);
+        SortOrder sortOrder = getSortOrder();
+        int end = StringUtil.getFirstNonBlankCharIdxFromEnd(string, offset, length, sortOrder);
         if (end == offset - 1) {
             ptr.set(ByteUtil.EMPTY_BYTE_ARRAY);
             return true; 
         }
-        int head = StringUtil.getFirstNonBlankCharIdxFromStart(string, offset, length, columnModifier);
+        int head = StringUtil.getFirstNonBlankCharIdxFromStart(string, offset, length, sortOrder);
         ptr.set(string, head, end - head + 1);
         return true;
     }

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/ae131f56/phoenix-core/src/main/java/org/apache/phoenix/expression/function/UpperFunction.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/UpperFunction.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/UpperFunction.java
index af4c39c..af87860 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/UpperFunction.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/UpperFunction.java
@@ -28,7 +28,7 @@ public class UpperFunction extends ScalarFunction {
             return false;
         }
 
-        String sourceStr = (String)PDataType.VARCHAR.toObject(ptr, getStrExpression().getColumnModifier());
+        String sourceStr = (String)PDataType.VARCHAR.toObject(ptr, getStrExpression().getSortOrder());
         if (sourceStr == null) {
             return true;
         }

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/ae131f56/phoenix-core/src/main/java/org/apache/phoenix/index/IndexMaintainer.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/index/IndexMaintainer.java b/phoenix-core/src/main/java/org/apache/phoenix/index/IndexMaintainer.java
index 60e9401..f94b2aa 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/index/IndexMaintainer.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/index/IndexMaintainer.java
@@ -46,7 +46,6 @@ import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.WritableUtils;
 import org.apache.phoenix.client.KeyValueBuilder;
 import org.apache.phoenix.query.QueryConstants;
-import org.apache.phoenix.schema.ColumnModifier;
 import org.apache.phoenix.schema.PColumn;
 import org.apache.phoenix.schema.PColumnFamily;
 import org.apache.phoenix.schema.PDataType;
@@ -55,6 +54,7 @@ import org.apache.phoenix.schema.PTable;
 import org.apache.phoenix.schema.PTableType;
 import org.apache.phoenix.schema.RowKeySchema;
 import org.apache.phoenix.schema.SaltingUtil;
+import org.apache.phoenix.schema.SortOrder;
 import org.apache.phoenix.schema.ValueSchema;
 import org.apache.phoenix.schema.ValueSchema.Field;
 import org.apache.phoenix.util.BitSet;
@@ -104,7 +104,7 @@ public class IndexMaintainer implements Writable, Iterable<ColumnReference> {
                 maintainer.getIndexedColumnSizes().add(column.getByteSize());
                 maintainer.getIndexedColumns().add(new ColumnReference(column.getFamilyName().getBytes(), column.getName().getBytes()));
             }
-            if (indexColumn.getColumnModifier() != null) {
+            if (indexColumn.getSortOrder() == SortOrder.DESC) {
                 rowKeyMetaData.getDescIndexColumnBitSet().set(indexPos);
             }
         }
@@ -290,7 +290,7 @@ public class IndexMaintainer implements Writable, Iterable<ColumnReference> {
                 PDataType dataColumnType;
                 boolean isNullable = true;
                 boolean isDataColumnInverted = false;
-                ColumnModifier dataColumnModifier = null;
+                SortOrder dataSortOrder = SortOrder.getDefault();
                 if (dataPkPosition[i] == -1) {
                     dataColumnType = indexedColumnTypes.get(j);
                     ImmutableBytesPtr value = valueGetter.getLatestValue(iterator.next());
@@ -301,11 +301,11 @@ public class IndexMaintainer implements Writable, Iterable<ColumnReference> {
                     }
                     j++;
                } else {
-                   Field field = dataRowKeySchema.getField(dataPkPosition[i]);
+                    Field field = dataRowKeySchema.getField(dataPkPosition[i]);
                     dataColumnType = field.getDataType();
                     ptr.set(rowKeyPtr.get(), dataRowKeyLocator[0][i], dataRowKeyLocator[1][i]);
-                    dataColumnModifier = field.getColumnModifier();
-                    isDataColumnInverted = dataColumnModifier != null;
+                    dataSortOrder = field.getSortOrder();
+                    isDataColumnInverted = dataSortOrder != SortOrder.ASC;
                     isNullable = field.isNullable();
                 }
                 PDataType indexColumnType = IndexUtil.getIndexColumnDataType(isNullable, dataColumnType);
@@ -314,7 +314,7 @@ public class IndexMaintainer implements Writable, Iterable<ColumnReference> {
                     output.write(ptr.get(), ptr.getOffset(), ptr.getLength());
                 } else {
                     if (!isBytesComparable)  {
-                        indexColumnType.coerceBytes(ptr, dataColumnType, dataColumnModifier, null);
+                        indexColumnType.coerceBytes(ptr, dataColumnType, dataSortOrder, SortOrder.getDefault());
                     }
                     if (descIndexColumnBitSet.get(i) != isDataColumnInverted) {
                         writeInverted(ptr.get(), ptr.getOffset(), ptr.getLength(), output);
@@ -674,7 +674,7 @@ public class IndexMaintainer implements Writable, Iterable<ColumnReference> {
     
     private static void writeInverted(byte[] buf, int offset, int length, DataOutput output) throws IOException {
         for (int i = offset; i < offset + length; i++) {
-            byte b = ColumnModifier.SORT_DESC.apply(buf[i]);
+            byte b = SortOrder.invert(buf[i]);
             output.write(b);
         }
     }

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/ae131f56/phoenix-core/src/main/java/org/apache/phoenix/iterate/ExplainTable.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/iterate/ExplainTable.java b/phoenix-core/src/main/java/org/apache/phoenix/iterate/ExplainTable.java
index 01e7a2b..91c7d1d 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/iterate/ExplainTable.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/iterate/ExplainTable.java
@@ -37,9 +37,9 @@ import org.apache.phoenix.compile.ScanRanges;
 import org.apache.phoenix.compile.StatementContext;
 import org.apache.phoenix.query.KeyRange;
 import org.apache.phoenix.query.KeyRange.Bound;
-import org.apache.phoenix.schema.ColumnModifier;
 import org.apache.phoenix.schema.PDataType;
 import org.apache.phoenix.schema.RowKeySchema;
+import org.apache.phoenix.schema.SortOrder;
 import org.apache.phoenix.schema.TableRef;
 import org.apache.phoenix.util.StringUtil;
 
@@ -169,10 +169,10 @@ public abstract class ExplainTable {
         }
         ScanRanges scanRanges = context.getScanRanges();
         PDataType type = scanRanges.getSchema().getField(slotIndex).getDataType();
-        ColumnModifier modifier = tableRef.getTable().getPKColumns().get(slotIndex).getColumnModifier();
-        if (modifier != null) {
+        SortOrder sortOrder = tableRef.getTable().getPKColumns().get(slotIndex).getSortOrder();
+        if (sortOrder == SortOrder.DESC) {
             buf.append('~');
-            range = modifier.apply(range, 0, new byte[range.length], 0, range.length);
+            range = SortOrder.invert(range, 0, new byte[range.length], 0, range.length);
         }
         Format formatter = context.getConnection().getFormatter(type);
         buf.append(type.toStringLiteral(range, formatter));

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/ae131f56/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixDatabaseMetaData.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixDatabaseMetaData.java b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixDatabaseMetaData.java
index 6b7a0d9..4cb62fd 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixDatabaseMetaData.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixDatabaseMetaData.java
@@ -51,7 +51,7 @@ import org.apache.phoenix.iterate.MaterializedResultIterator;
 import org.apache.phoenix.iterate.ResultIterator;
 import org.apache.phoenix.parse.HintNode.Hint;
 import org.apache.phoenix.query.QueryConstants;
-import org.apache.phoenix.schema.ColumnModifier;
+import org.apache.phoenix.schema.SortOrder;
 import org.apache.phoenix.schema.PDataType;
 import org.apache.phoenix.schema.PDatum;
 import org.apache.phoenix.schema.PName;
@@ -162,7 +162,7 @@ public class PhoenixDatabaseMetaData implements DatabaseMetaData, org.apache.pho
     public static final String SCOPE_TABLE = "SCOPE_TABLE";
     public static final String SOURCE_DATA_TYPE = "SOURCE_DATA_TYPE";
     public static final String IS_AUTOINCREMENT = "IS_AUTOINCREMENT";
-    public static final String COLUMN_MODIFIER = "COLUMN_MODIFIER";
+    public static final String SORT_ORDER = "SORT_ORDER";
     public static final String IMMUTABLE_ROWS = "IMMUTABLE_ROWS";
     public static final byte[] IMMUTABLE_ROWS_BYTES = Bytes.toBytes(IMMUTABLE_ROWS);
     public static final String DEFAULT_COLUMN_FAMILY_NAME = "DEFAULT_COLUMN_FAMILY";
@@ -454,7 +454,7 @@ public class PhoenixDatabaseMetaData implements DatabaseMetaData, org.apache.pho
                 DatabaseMetaData.tableIndexOther + " TYPE,\n" + 
                 ORDINAL_POSITION + ",\n" +
                 COLUMN_NAME + ",\n" +
-                "CASE WHEN " + TABLE_CAT_NAME + " IS NOT NULL THEN null WHEN " + COLUMN_MODIFIER + " = " + ColumnModifier.toSystemValue(ColumnModifier.SORT_DESC) + " THEN 'D' ELSE 'A' END ASC_OR_DESC,\n" +
+                "CASE WHEN " + TABLE_CAT_NAME + " IS NOT NULL THEN null WHEN " + SORT_ORDER + " = " + (SortOrder.DESC.getSystemValue()) + " THEN 'D' ELSE 'A' END ASC_OR_DESC,\n" +
                 "null CARDINALITY,\n" +
                 "null PAGES,\n" +
                 "null FILTER_CONDITION,\n" +
@@ -605,7 +605,7 @@ public class PhoenixDatabaseMetaData implements DatabaseMetaData, org.apache.pho
                 COLUMN_NAME + "," +
                 "null as KEY_SEQ," +
                 "PK_NAME" + "," +
-                "CASE WHEN " + COLUMN_MODIFIER + " = " + ColumnModifier.toSystemValue(ColumnModifier.SORT_DESC) + " THEN 'D' ELSE 'A' END ASC_OR_DESC," +
+                "CASE WHEN " + SORT_ORDER + " = " + (SortOrder.DESC.getSystemValue()) + " THEN 'D' ELSE 'A' END ASC_OR_DESC," +
                 DATA_TYPE + "," + // include type info, though not in spec
                 SqlTypeNameFunction.NAME + "(" + DATA_TYPE + ") AS " + TYPE_NAME +
                 " from " + TYPE_SCHEMA_AND_TABLE + " " + TYPE_SCHEMA_AND_TABLE_ALIAS +
@@ -851,8 +851,8 @@ public class PhoenixDatabaseMetaData implements DatabaseMetaData, org.apache.pho
             return null;
         }
 		@Override
-		public ColumnModifier getColumnModifier() {
-			return null;
+		public SortOrder getSortOrder() {
+			return SortOrder.getDefault();
 		}
     };
     private static final RowProjector TABLE_TYPE_ROW_PROJECTOR = new RowProjector(Arrays.<ColumnProjector>asList(

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/ae131f56/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixParameterMetaData.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixParameterMetaData.java b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixParameterMetaData.java
index 2e33302..2f8e18d 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixParameterMetaData.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixParameterMetaData.java
@@ -26,7 +26,7 @@ import java.sql.SQLException;
 import org.apache.phoenix.exception.SQLExceptionCode;
 import org.apache.phoenix.exception.SQLExceptionInfo;
 import org.apache.phoenix.parse.BindParseNode;
-import org.apache.phoenix.schema.ColumnModifier;
+import org.apache.phoenix.schema.SortOrder;
 import org.apache.phoenix.schema.PDataType;
 import org.apache.phoenix.schema.PDatum;
 import org.apache.phoenix.schema.TypeMismatchException;
@@ -64,8 +64,8 @@ public class PhoenixParameterMetaData implements ParameterMetaData {
         }
         
         @Override
-        public ColumnModifier getColumnModifier() {
-            return null;
+        public SortOrder getSortOrder() {
+            return SortOrder.getDefault();
         }
         
         @Override

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/ae131f56/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixStatement.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixStatement.java b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixStatement.java
index 5a5703c..f7bcf3f 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixStatement.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixStatement.java
@@ -91,7 +91,7 @@ import org.apache.phoenix.parse.UpsertStatement;
 import org.apache.phoenix.query.QueryConstants;
 import org.apache.phoenix.query.QueryServices;
 import org.apache.phoenix.query.QueryServicesOptions;
-import org.apache.phoenix.schema.ColumnModifier;
+import org.apache.phoenix.schema.SortOrder;
 import org.apache.phoenix.schema.ExecuteQueryNotApplicableException;
 import org.apache.phoenix.schema.ExecuteUpdateNotApplicableException;
 import org.apache.phoenix.schema.MetaDataClient;
@@ -810,8 +810,8 @@ public class PhoenixStatement implements Statement, SQLCloseable, org.apache.pho
             return null;
         }
 		@Override
-		public ColumnModifier getColumnModifier() {
-			return null;
+		public SortOrder getSortOrder() {
+			return SortOrder.getDefault();
 		}
     };
     private static final RowProjector EXPLAIN_PLAN_ROW_PROJECTOR = new RowProjector(Arrays.<ColumnProjector>asList(

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/ae131f56/phoenix-core/src/main/java/org/apache/phoenix/parse/ColumnDef.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/ColumnDef.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/ColumnDef.java
index e79fab9..67e3a19 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/parse/ColumnDef.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/ColumnDef.java
@@ -23,10 +23,12 @@ import java.sql.SQLException;
 
 import org.apache.phoenix.exception.SQLExceptionCode;
 import org.apache.phoenix.exception.SQLExceptionInfo;
-import org.apache.phoenix.schema.ColumnModifier;
+import org.apache.phoenix.schema.SortOrder;
 import org.apache.phoenix.schema.PDataType;
 import org.apache.phoenix.util.SchemaUtil;
 
+import com.google.common.base.Preconditions;
+
 
 /**
  * 
@@ -42,13 +44,14 @@ public class ColumnDef {
     private final Integer maxLength;
     private final Integer scale;
     private final boolean isPK;
-    private final ColumnModifier columnModifier;
+    private final SortOrder sortOrder;
     private final boolean isArray;
     private final Integer arrSize;
  
     ColumnDef(ColumnName columnDefName, String sqlTypeName, boolean isArray, Integer arrSize, boolean isNull, Integer maxLength,
-    		            Integer scale, boolean isPK, ColumnModifier columnModifier) {
+    		            Integer scale, boolean isPK, SortOrder sortOrder) {
    	 try {
+         Preconditions.checkNotNull(sortOrder);
    	     PDataType localType = null;
          this.columnDefName = columnDefName;
          this.isArray = isArray;
@@ -127,7 +130,7 @@ public class ColumnDef {
          this.maxLength = maxLength;
          this.scale = scale;
          this.isPK = isPK;
-         this.columnModifier = columnModifier;
+         this.sortOrder = sortOrder;
          if(this.isArray) {
              this.dataType = localType;
          }
@@ -136,8 +139,8 @@ public class ColumnDef {
      }
     }
     ColumnDef(ColumnName columnDefName, String sqlTypeName, boolean isNull, Integer maxLength,
-            Integer scale, boolean isPK, ColumnModifier columnModifier) {
-    	this(columnDefName, sqlTypeName, false, 0, isNull, maxLength, scale, isPK, columnModifier);
+            Integer scale, boolean isPK, SortOrder sortOrder) {
+    	this(columnDefName, sqlTypeName, false, 0, isNull, maxLength, scale, isPK, sortOrder);
     }
 
     public ColumnName getColumnDefName() {
@@ -164,8 +167,8 @@ public class ColumnDef {
         return isPK;
     }
     
-    public ColumnModifier getColumnModifier() {
-    	return columnModifier;
+    public SortOrder getSortOrder() {
+    	return sortOrder;
     }
         
 	public boolean isArray() {

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/ae131f56/phoenix-core/src/main/java/org/apache/phoenix/parse/IndexKeyConstraint.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/IndexKeyConstraint.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/IndexKeyConstraint.java
index 2253bb4..6972c12 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/parse/IndexKeyConstraint.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/IndexKeyConstraint.java
@@ -24,16 +24,16 @@ import java.util.List;
 import org.apache.hadoop.hbase.util.Pair;
 
 import com.google.common.collect.ImmutableList;
-import org.apache.phoenix.schema.ColumnModifier;
+import org.apache.phoenix.schema.SortOrder;
 
 public class IndexKeyConstraint {
-    private final List<Pair<ColumnParseNode, ColumnModifier>> columnNameToModifier;
+    private final List<Pair<ColumnParseNode, SortOrder>> columnNameToSortOrder;
     
-    IndexKeyConstraint(List<Pair<ColumnParseNode, ColumnModifier>> columnNameAndModifier) {
-        this.columnNameToModifier = ImmutableList.copyOf(columnNameAndModifier);
+    IndexKeyConstraint(List<Pair<ColumnParseNode, SortOrder>> columnNameAndSortOrder) {
+        this.columnNameToSortOrder = ImmutableList.copyOf(columnNameAndSortOrder);
     }
 
-    public List<Pair<ColumnParseNode, ColumnModifier>> getColumns() {
-        return columnNameToModifier;
+    public List<Pair<ColumnParseNode, SortOrder>> getColumns() {
+        return columnNameToSortOrder;
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/ae131f56/phoenix-core/src/main/java/org/apache/phoenix/parse/ParseNodeFactory.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/ParseNodeFactory.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/ParseNodeFactory.java
index 7b7589f..c26285f 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/parse/ParseNodeFactory.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/ParseNodeFactory.java
@@ -44,7 +44,7 @@ import org.apache.phoenix.expression.function.FunctionExpression;
 import org.apache.phoenix.parse.FunctionParseNode.BuiltInFunction;
 import org.apache.phoenix.parse.FunctionParseNode.BuiltInFunctionInfo;
 import org.apache.phoenix.parse.JoinTableNode.JoinType;
-import org.apache.phoenix.schema.ColumnModifier;
+import org.apache.phoenix.schema.SortOrder;
 import org.apache.phoenix.schema.PDataType;
 import org.apache.phoenix.schema.PIndexState;
 import org.apache.phoenix.schema.PTableType;
@@ -245,17 +245,17 @@ public class ParseNodeFactory {
         return new PropertyName(familyName, propertyName);
     }
 
-    public ColumnDef columnDef(ColumnName columnDefName, String sqlTypeName, boolean isNull, Integer maxLength, Integer scale, boolean isPK, ColumnModifier columnModifier) {
-        return new ColumnDef(columnDefName, sqlTypeName, isNull, maxLength, scale, isPK, columnModifier);
+    public ColumnDef columnDef(ColumnName columnDefName, String sqlTypeName, boolean isNull, Integer maxLength, Integer scale, boolean isPK, SortOrder sortOrder) {
+        return new ColumnDef(columnDefName, sqlTypeName, isNull, maxLength, scale, isPK, sortOrder);
     }
     
     public ColumnDef columnDef(ColumnName columnDefName, String sqlTypeName, boolean isArray, Integer arrSize, boolean isNull, Integer maxLength, Integer scale, boolean isPK, 
-        	ColumnModifier columnModifier) {
-        return new ColumnDef(columnDefName, sqlTypeName, isArray, arrSize, isNull, maxLength, scale, isPK, columnModifier);
+        	SortOrder sortOrder) {
+        return new ColumnDef(columnDefName, sqlTypeName, isArray, arrSize, isNull, maxLength, scale, isPK, sortOrder);
     }
 
-    public PrimaryKeyConstraint primaryKey(String name, List<Pair<ColumnName, ColumnModifier>> columnNameAndModifier) {
-        return new PrimaryKeyConstraint(name, columnNameAndModifier);
+    public PrimaryKeyConstraint primaryKey(String name, List<Pair<ColumnName, SortOrder>> columnNameAndSortOrder) {
+        return new PrimaryKeyConstraint(name, columnNameAndSortOrder);
     }
     
     public CreateTableStatement createTable(TableName tableName, ListMultimap<String,Pair<String,Object>> props, List<ColumnDef> columns, PrimaryKeyConstraint pkConstraint, List<ParseNode> splits, PTableType tableType, boolean ifNotExists, TableName baseTableName, ParseNode tableTypeIdNode, int bindCount) {

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/ae131f56/phoenix-core/src/main/java/org/apache/phoenix/parse/PrimaryKeyConstraint.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/PrimaryKeyConstraint.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/PrimaryKeyConstraint.java
index 6719b0d..cbc4947 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/parse/PrimaryKeyConstraint.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/PrimaryKeyConstraint.java
@@ -27,33 +27,33 @@ import org.apache.hadoop.hbase.util.Pair;
 
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Maps;
-import org.apache.phoenix.schema.ColumnModifier;
+import org.apache.phoenix.schema.SortOrder;
 
 public class PrimaryKeyConstraint extends NamedNode {
-    public static final PrimaryKeyConstraint EMPTY = new PrimaryKeyConstraint(null, Collections.<Pair<ColumnName, ColumnModifier>>emptyList());
+    public static final PrimaryKeyConstraint EMPTY = new PrimaryKeyConstraint(null, Collections.<Pair<ColumnName, SortOrder>>emptyList());
 
-    private final List<Pair<ColumnName, ColumnModifier>> columns;
-    private final HashMap<ColumnName, Pair<ColumnName, ColumnModifier>> columnNameToModifier;
+    private final List<Pair<ColumnName, SortOrder>> columns;
+    private final HashMap<ColumnName, Pair<ColumnName, SortOrder>> columnNameToSortOrder;
     
-    PrimaryKeyConstraint(String name, List<Pair<ColumnName, ColumnModifier>> columns) {
+    PrimaryKeyConstraint(String name, List<Pair<ColumnName, SortOrder>> columns) {
         super(name);
-        this.columns = columns == null ? Collections.<Pair<ColumnName, ColumnModifier>>emptyList() : ImmutableList.copyOf(columns);
-        this.columnNameToModifier = Maps.newHashMapWithExpectedSize(this.columns.size());
-        for (Pair<ColumnName, ColumnModifier> p : this.columns) {
-            this.columnNameToModifier.put(p.getFirst(), p);
+        this.columns = columns == null ? Collections.<Pair<ColumnName, SortOrder>>emptyList() : ImmutableList.copyOf(columns);
+        this.columnNameToSortOrder = Maps.newHashMapWithExpectedSize(this.columns.size());
+        for (Pair<ColumnName, SortOrder> p : this.columns) {
+            this.columnNameToSortOrder.put(p.getFirst(), p);
         }
     }
 
-    public List<Pair<ColumnName, ColumnModifier>> getColumnNames() {
+    public List<Pair<ColumnName, SortOrder>> getColumnNames() {
         return columns;
     }
     
-    public Pair<ColumnName, ColumnModifier> getColumn(ColumnName columnName) {
-    	return columnNameToModifier.get(columnName);
+    public Pair<ColumnName, SortOrder> getColumn(ColumnName columnName) {
+    	return columnNameToSortOrder.get(columnName);
     }
     
     public boolean contains(ColumnName columnName) {
-        return columnNameToModifier.containsKey(columnName);
+        return columnNameToSortOrder.containsKey(columnName);
     }
     
     @Override

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/ae131f56/phoenix-core/src/main/java/org/apache/phoenix/query/KeyRange.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/KeyRange.java b/phoenix-core/src/main/java/org/apache/phoenix/query/KeyRange.java
index 11b4928..c05b60f 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/query/KeyRange.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/KeyRange.java
@@ -38,7 +38,7 @@ import org.apache.hadoop.io.WritableUtils;
 import com.google.common.base.Function;
 import com.google.common.collect.ComparisonChain;
 import com.google.common.collect.Lists;
-import org.apache.phoenix.schema.ColumnModifier;
+import org.apache.phoenix.schema.SortOrder;
 import org.apache.phoenix.util.ByteUtil;
 
 import edu.umd.cs.findbugs.annotations.NonNull;
@@ -550,7 +550,7 @@ public class KeyRange implements Writable {
     public KeyRange invert() {
         byte[] lower = this.getLowerRange();
         if (!this.lowerUnbound()) {
-            lower = ColumnModifier.SORT_DESC.apply(lower, 0, lower.length);
+            lower = SortOrder.invert(lower, 0, lower.length);
         }
         byte[] upper;
         if (this.isSingleKey()) {
@@ -558,7 +558,7 @@ public class KeyRange implements Writable {
         } else {
             upper = this.getUpperRange();
             if (!this.upperUnbound()) {
-                upper = ColumnModifier.SORT_DESC.apply(upper, 0, upper.length);
+                upper = SortOrder.invert(upper, 0, upper.length);
             }
         }
         return KeyRange.getKeyRange(lower, this.isLowerInclusive(), upper, this.isUpperInclusive());

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/ae131f56/phoenix-core/src/main/java/org/apache/phoenix/query/QueryConstants.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/QueryConstants.java b/phoenix-core/src/main/java/org/apache/phoenix/query/QueryConstants.java
index 779c4cd..28b77a8 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/query/QueryConstants.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/QueryConstants.java
@@ -26,7 +26,7 @@ import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.CACHE_SIZE;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.CHAR_OCTET_LENGTH;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.COLUMN_COUNT;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.COLUMN_DEF;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.COLUMN_MODIFIER;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SORT_ORDER;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.COLUMN_NAME;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.COLUMN_SIZE;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.CURRENT_VALUE;
@@ -184,7 +184,7 @@ public interface QueryConstants {
             SOURCE_DATA_TYPE + " INTEGER," + // supposed to be SHORT
             IS_AUTOINCREMENT + " VARCHAR," +
             // Columns added in 1.2.1
-            COLUMN_MODIFIER + " INTEGER," +
+            SORT_ORDER + " INTEGER," +
             SALT_BUCKETS + " INTEGER," +
             // Columns added in 2.0.0
             DATA_TABLE_NAME + " VARCHAR," +

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/ae131f56/phoenix-core/src/main/java/org/apache/phoenix/schema/DelegateColumn.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/DelegateColumn.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/DelegateColumn.java
index d4577c3..b53522e 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/DelegateColumn.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/DelegateColumn.java
@@ -40,8 +40,8 @@ public class DelegateColumn extends DelegateDatum implements PColumn {
     }
     
     @Override
-    public ColumnModifier getColumnModifier() {
-    	return getDelegate().getColumnModifier();
+    public SortOrder getSortOrder() {
+    	return getDelegate().getSortOrder();
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/ae131f56/phoenix-core/src/main/java/org/apache/phoenix/schema/DelegateDatum.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/DelegateDatum.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/DelegateDatum.java
index 5df896b..ff38503 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/DelegateDatum.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/DelegateDatum.java
@@ -52,8 +52,8 @@ public class DelegateDatum implements PDatum {
     }
     
 	@Override
-	public ColumnModifier getColumnModifier() {
-		return delegate.getColumnModifier();
+	public SortOrder getSortOrder() {
+		return delegate.getSortOrder();
 	}
 
     protected PDatum getDelegate() {

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/ae131f56/phoenix-core/src/main/java/org/apache/phoenix/schema/KeyValueSchema.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/KeyValueSchema.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/KeyValueSchema.java
index 89c8e81..818e629 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/KeyValueSchema.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/KeyValueSchema.java
@@ -67,7 +67,7 @@ public class KeyValueSchema extends ValueSchema {
         }
         
         public KeyValueSchemaBuilder addField(PDatum datum) {
-            super.addField(datum, fields.size() <  this.minNullable, null);
+            super.addField(datum, fields.size() <  this.minNullable, SortOrder.getDefault());
             return this;
         }
     }

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/ae131f56/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java
index bcb8c08..455f6e0 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java
@@ -25,7 +25,6 @@ import static com.google.common.collect.Sets.newLinkedHashSetWithExpectedSize;
 import static org.apache.phoenix.exception.SQLExceptionCode.INSUFFICIENT_MULTI_TENANT_COLUMNS;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.ARRAY_SIZE;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.COLUMN_COUNT;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.COLUMN_MODIFIER;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.COLUMN_NAME;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.COLUMN_SIZE;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.DATA_TABLE_NAME;
@@ -41,6 +40,7 @@ import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.NULLABLE;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.ORDINAL_POSITION;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.PK_NAME;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SALT_BUCKETS;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SORT_ORDER;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TABLE_CAT_NAME;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TABLE_NAME_NAME;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TABLE_SCHEM_NAME;
@@ -195,7 +195,7 @@ public class MetaDataClient {
         COLUMN_SIZE + "," +
         DECIMAL_DIGITS + "," +
         ORDINAL_POSITION + "," + 
-        COLUMN_MODIFIER + "," +
+        SORT_ORDER + "," +
         DATA_TABLE_NAME + "," + // write this both in the column and table rows for access by metadata APIs
         ARRAY_SIZE +
         ") VALUES (?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?)";
@@ -332,7 +332,7 @@ public class MetaDataClient {
             colUpsert.setInt(9, column.getScale());
         }
         colUpsert.setInt(10, column.getPosition() + 1);
-        colUpsert.setInt(11, ColumnModifier.toSystemValue(column.getColumnModifier()));
+        colUpsert.setInt(11, column.getSortOrder().getSystemValue());
         colUpsert.setString(12, parentTableName);
         if (column.getArraySize() == null) {
             colUpsert.setNull(13, Types.INTEGER);
@@ -345,13 +345,13 @@ public class MetaDataClient {
     private PColumn newColumn(int position, ColumnDef def, PrimaryKeyConstraint pkConstraint, String defaultColumnFamily) throws SQLException {
         try {
             ColumnName columnDefName = def.getColumnDefName();
-            ColumnModifier columnModifier = def.getColumnModifier();
+            SortOrder sortOrder = def.getSortOrder();
             boolean isPK = def.isPK();
             if (pkConstraint != null) {
-                Pair<ColumnName,ColumnModifier> pkColumnModifier = pkConstraint.getColumn(columnDefName);
-                if (pkColumnModifier != null) {
+                Pair<ColumnName, SortOrder> pkSortOrder = pkConstraint.getColumn(columnDefName);
+                if (pkSortOrder != null) {
                     isPK = true;
-                    columnModifier = pkColumnModifier.getSecond();
+                    sortOrder = pkSortOrder.getSecond();
                 }
             }
             
@@ -376,7 +376,7 @@ public class MetaDataClient {
             }
             
             PColumn column = new PColumnImpl(PNameFactory.newName(columnName), familyName, def.getDataType(),
-                    def.getMaxLength(), def.getScale(), def.isNull(), position, columnModifier, def.getArraySize());
+                    def.getMaxLength(), def.getScale(), def.isNull(), position, sortOrder, def.getArraySize());
             return column;
         } catch (IllegalArgumentException e) { // Based on precondition check in constructor
             throw new SQLException(e);
@@ -480,7 +480,7 @@ public class MetaDataClient {
         PrimaryKeyConstraint pk = statement.getIndexConstraint();
         TableName indexTableName = statement.getIndexTableName();
         
-        List<Pair<ColumnName, ColumnModifier>> indexedPkColumns = pk.getColumnNames();
+        List<Pair<ColumnName, SortOrder>> indexedPkColumns = pk.getColumnNames();
         List<ColumnName> includedColumns = statement.getIncludeColumns();
         TableRef tableRef = null;
         PTable table = null;
@@ -508,27 +508,27 @@ public class MetaDataClient {
                 } else {
                     unusedPkColumns = new LinkedHashSet<PColumn>(dataTable.getPKColumns());
                 }
-                List<Pair<ColumnName, ColumnModifier>> allPkColumns = Lists.newArrayListWithExpectedSize(unusedPkColumns.size());
+                List<Pair<ColumnName, SortOrder>> allPkColumns = Lists.newArrayListWithExpectedSize(unusedPkColumns.size());
                 List<ColumnDef> columnDefs = Lists.newArrayListWithExpectedSize(includedColumns.size() + indexedPkColumns.size());
                 
                 // First columns are the indexed ones
-                for (Pair<ColumnName, ColumnModifier> pair : indexedPkColumns) {
+                for (Pair<ColumnName, SortOrder> pair : indexedPkColumns) {
                     ColumnName colName = pair.getFirst();
                     PColumn col = resolver.resolveColumn(null, colName.getFamilyName(), colName.getColumnName()).getColumn();
                     unusedPkColumns.remove(col);
                     PDataType dataType = IndexUtil.getIndexColumnDataType(col);
                     colName = ColumnName.caseSensitiveColumnName(IndexUtil.getIndexColumnName(col));
-                    allPkColumns.add(new Pair<ColumnName, ColumnModifier>(colName, pair.getSecond()));
-                    columnDefs.add(FACTORY.columnDef(colName, dataType.getSqlTypeName(), col.isNullable(), col.getMaxLength(), col.getScale(), false, null));
+                    allPkColumns.add(new Pair<ColumnName, SortOrder>(colName, pair.getSecond()));
+                    columnDefs.add(FACTORY.columnDef(colName, dataType.getSqlTypeName(), col.isNullable(), col.getMaxLength(), col.getScale(), false, SortOrder.getDefault()));
                 }
                 
                 // Next all the PK columns from the data table that aren't indexed
                 if (!unusedPkColumns.isEmpty()) {
                     for (PColumn col : unusedPkColumns) {
                         ColumnName colName = ColumnName.caseSensitiveColumnName(IndexUtil.getIndexColumnName(col));
-                        allPkColumns.add(new Pair<ColumnName, ColumnModifier>(colName, col.getColumnModifier()));
+                        allPkColumns.add(new Pair<ColumnName, SortOrder>(colName, col.getSortOrder()));
                         PDataType dataType = IndexUtil.getIndexColumnDataType(col);
-                        columnDefs.add(FACTORY.columnDef(colName, dataType.getSqlTypeName(), col.isNullable(), col.getMaxLength(), col.getScale(), false, col.getColumnModifier()));
+                        columnDefs.add(FACTORY.columnDef(colName, dataType.getSqlTypeName(), col.isNullable(), col.getMaxLength(), col.getScale(), false, col.getSortOrder()));
                     }
                 }
                 pk = FACTORY.primaryKey(null, allPkColumns);
@@ -549,7 +549,7 @@ public class MetaDataClient {
                         if (!SchemaUtil.isPKColumn(col)) {
                             // Need to re-create ColumnName, since the above one won't have the column family name
                             colName = ColumnName.caseSensitiveColumnName(col.getFamilyName().getString(), IndexUtil.getIndexColumnName(col));
-                            columnDefs.add(FACTORY.columnDef(colName, col.getDataType().getSqlTypeName(), col.isNullable(), col.getMaxLength(), col.getScale(), false, col.getColumnModifier()));
+                            columnDefs.add(FACTORY.columnDef(colName, col.getDataType().getSqlTypeName(), col.isNullable(), col.getMaxLength(), col.getScale(), false, col.getSortOrder()));
                         }
                     }
                 }
@@ -665,8 +665,8 @@ public class MetaDataClient {
             
             PrimaryKeyConstraint pkConstraint = statement.getPrimaryKeyConstraint();
             String pkName = null;
-            List<Pair<ColumnName,ColumnModifier>> pkColumnsNames = Collections.<Pair<ColumnName,ColumnModifier>>emptyList();
-            Iterator<Pair<ColumnName,ColumnModifier>> pkColumnsIterator = Iterators.emptyIterator();
+            List<Pair<ColumnName,SortOrder>> pkColumnsNames = Collections.<Pair<ColumnName,SortOrder>>emptyList();
+            Iterator<Pair<ColumnName,SortOrder>> pkColumnsIterator = Iterators.emptyIterator();
             if (pkConstraint != null) {
                 pkColumnsNames = pkConstraint.getColumnNames();
                 pkColumnsIterator = pkColumnsNames.iterator();
@@ -858,7 +858,7 @@ public class MetaDataClient {
                     .build().buildException();
             }
             if (!pkColumnsNames.isEmpty() && pkColumnsNames.size() != pkColumns.size() - positionOffset) { // Then a column name in the primary key constraint wasn't resolved
-                Iterator<Pair<ColumnName,ColumnModifier>> pkColumnNamesIterator = pkColumnsNames.iterator();
+                Iterator<Pair<ColumnName,SortOrder>> pkColumnNamesIterator = pkColumnsNames.iterator();
                 while (pkColumnNamesIterator.hasNext()) {
                     ColumnName colName = pkColumnNamesIterator.next().getFirst();
                     ColumnDef colDef = findColumnDefOrNull(colDefs, colName);
@@ -1305,7 +1305,7 @@ public class MetaDataClient {
                                 int indexColPosition = index.getColumns().size();
                                 PDataType indexColDataType = IndexUtil.getIndexColumnDataType(column);
                                 ColumnName indexColName = ColumnName.caseSensitiveColumnName(IndexUtil.getIndexColumnName(column));
-                                ColumnDef indexColDef = FACTORY.columnDef(indexColName, indexColDataType.getSqlTypeName(), column.isNullable(), column.getMaxLength(), column.getScale(), true, column.getColumnModifier());
+                                ColumnDef indexColDef = FACTORY.columnDef(indexColName, indexColDataType.getSqlTypeName(), column.isNullable(), column.getMaxLength(), column.getScale(), true, column.getSortOrder());
                                 PColumn indexColumn = newColumn(indexColPosition, indexColDef, PrimaryKeyConstraint.EMPTY, index.getDefaultFamilyName() == null ? null : index.getDefaultFamilyName().getString());
                                 addColumnMutation(schemaName, index.getTableName().getString(), indexColumn, colUpsert, index.getParentTableName().getString());
                             }

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/ae131f56/phoenix-core/src/main/java/org/apache/phoenix/schema/PArrayDataType.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/PArrayDataType.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/PArrayDataType.java
index 8c755f8..2dd23df 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/PArrayDataType.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/PArrayDataType.java
@@ -121,8 +121,8 @@ public class PArrayDataType {
 	}
 
 	public Object toObject(byte[] bytes, int offset, int length, PDataType baseType, 
-			ColumnModifier columnModifier) {
-		return createPhoenixArray(bytes, offset, length, columnModifier,
+			SortOrder sortOrder) {
+		return createPhoenixArray(bytes, offset, length, sortOrder,
 				baseType);
 	}
 	
@@ -204,14 +204,14 @@ public class PArrayDataType {
 	}
 
 	public Object toObject(byte[] bytes, int offset, int length, PDataType baseType) {
-		return toObject(bytes, offset, length, baseType, null);
+		return toObject(bytes, offset, length, baseType, SortOrder.getDefault());
 	}
 	
 	public Object toObject(Object object, PDataType actualType) {
 		return object;
 	}
 
-	public Object toObject(Object object, PDataType actualType, ColumnModifier sortOrder) {
+	public Object toObject(Object object, PDataType actualType, SortOrder sortOrder) {
 		// How to use the sortOrder ? Just reverse the elements
 		return toObject(object, actualType);
 	}
@@ -274,7 +274,7 @@ public class PArrayDataType {
     }
 
 	private Object createPhoenixArray(byte[] bytes, int offset, int length,
-			ColumnModifier columnModifier, PDataType baseDataType) {
+			SortOrder sortOrder, PDataType baseDataType) {
 		if(bytes == null || bytes.length == 0) {
 			return null;
 		}
@@ -332,18 +332,18 @@ public class PArrayDataType {
 						byte[] val = new byte[elementLength];
 						buffer.get(val);
 						elements[i++] = baseDataType.toObject(val,
-								columnModifier);
+								sortOrder);
 					}
 				}
 				buffer.position(nextOff + initPos);
 				byte[] val = new byte[indexOffset - nextOff];
 				buffer.get(val);
-				elements[i++] = baseDataType.toObject(val, columnModifier);
+				elements[i++] = baseDataType.toObject(val, sortOrder);
 			} else {
 				byte[] val = new byte[indexOffset - valArrayPostion];
 				buffer.position(valArrayPostion + initPos);
 				buffer.get(val);
-				elements[i++] = baseDataType.toObject(val, columnModifier);
+				elements[i++] = baseDataType.toObject(val, sortOrder);
 			}
 		} else {
 			for (int i = 0; i < noOfElements; i++) {
@@ -354,7 +354,7 @@ public class PArrayDataType {
 					val = new byte[baseDataType.getByteSize()];
 				}
 				buffer.get(val);
-				elements[i] = baseDataType.toObject(val, columnModifier);
+				elements[i] = baseDataType.toObject(val, sortOrder);
 			}
 		}
 		return PArrayDataType

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/ae131f56/phoenix-core/src/main/java/org/apache/phoenix/schema/PColumnImpl.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/PColumnImpl.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/PColumnImpl.java
index 03e78f8..514612f 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/PColumnImpl.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/PColumnImpl.java
@@ -42,7 +42,7 @@ public class PColumnImpl implements PColumn {
     private Integer scale;
     private boolean nullable;
     private int position;
-    private ColumnModifier columnModifier;
+    private SortOrder sortOrder;
     private Integer arraySize;
 
     public PColumnImpl() {
@@ -55,13 +55,13 @@ public class PColumnImpl implements PColumn {
                        Integer scale,
                        boolean nullable,
                        int position,
-                       ColumnModifier sortOrder, Integer arrSize) {
+                       SortOrder sortOrder, Integer arrSize) {
         init(name, familyName, dataType, maxLength, scale, nullable, position, sortOrder, arrSize);
     }
 
     public PColumnImpl(PColumn column, int position) {
         this(column.getName(), column.getFamilyName(), column.getDataType(), column.getMaxLength(),
-                column.getScale(), column.isNullable(), position, column.getColumnModifier(), column.getArraySize());
+                column.getScale(), column.isNullable(), position, column.getSortOrder(), column.getArraySize());
     }
 
     private void init(PName name,
@@ -71,8 +71,9 @@ public class PColumnImpl implements PColumn {
             Integer scale,
             boolean nullable,
             int position,
-            ColumnModifier columnModifier,
+            SortOrder sortOrder,
             Integer arrSize) {
+    	Preconditions.checkNotNull(sortOrder);
         this.dataType = dataType;
         if (familyName == null) {
             // Allow nullable columns in PK, but only if they're variable length.
@@ -89,7 +90,7 @@ public class PColumnImpl implements PColumn {
         this.scale = scale;
         this.nullable = nullable;
         this.position = position;
-        this.columnModifier = columnModifier;
+        this.sortOrder = sortOrder;
         this.arraySize = arrSize;
     }
 
@@ -136,8 +137,8 @@ public class PColumnImpl implements PColumn {
     }
     
     @Override
-    public ColumnModifier getColumnModifier() {
-    	return columnModifier;
+    public SortOrder getSortOrder() {
+    	return sortOrder;
     }
 
     @Override
@@ -157,10 +158,10 @@ public class PColumnImpl implements PColumn {
         int scale = WritableUtils.readVInt(input);
         boolean nullable = input.readBoolean();
         int position = WritableUtils.readVInt(input);
-        ColumnModifier columnModifier = ColumnModifier.fromSystemValue(WritableUtils.readVInt(input));
+        SortOrder sortOrder = SortOrder.fromSystemValue(WritableUtils.readVInt(input));
         int arrSize = WritableUtils.readVInt(input);
         init(columnName, familyName, dataType, maxLength == NO_MAXLENGTH ? null : maxLength,
-                scale == NO_SCALE ? null : scale, nullable, position, columnModifier, arrSize == -1 ? null : arrSize);
+                scale == NO_SCALE ? null : scale, nullable, position, sortOrder, arrSize == -1 ? null : arrSize);
     }
 
     @Override
@@ -172,7 +173,7 @@ public class PColumnImpl implements PColumn {
         WritableUtils.writeVInt(output, scale == null ? NO_SCALE : scale);
         output.writeBoolean(nullable);
         WritableUtils.writeVInt(output, position);
-        WritableUtils.writeVInt(output, ColumnModifier.toSystemValue(columnModifier));
+        WritableUtils.writeVInt(output, sortOrder.getSystemValue());
         WritableUtils.writeVInt(output, arraySize == null ? -1 : arraySize);
     }