phoenix-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From jamestay...@apache.org
Subject git commit: PHOENIX-123 Fixed with array should not write any footer
Date Sun, 09 Mar 2014 10:16:25 GMT
Repository: incubator-phoenix
Updated Branches:
  refs/heads/master 0da3a52b9 -> 12e5658e9


PHOENIX-123 Fixed with array should not write any footer


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

Branch: refs/heads/master
Commit: 12e5658e94ec059154ef2c19799b37164357df1b
Parents: 0da3a52
Author: James Taylor <jamestaylor@apache.org>
Authored: Sun Mar 9 03:15:56 2014 -0700
Committer: James Taylor <jamestaylor@apache.org>
Committed: Sun Mar 9 03:15:56 2014 -0700

----------------------------------------------------------------------
 .../phoenix/compile/ExpressionProjector.java    |   3 +-
 .../apache/phoenix/compile/UpsertCompiler.java  |   2 +-
 .../ValueTypeIncompatibleException.java         |   2 +-
 .../expression/ArrayConstructorExpression.java  |   2 +-
 .../phoenix/expression/LiteralExpression.java   |   3 +
 .../function/ArrayLengthFunction.java           |   2 +-
 .../apache/phoenix/query/QueryConstants.java    |   7 +-
 .../apache/phoenix/schema/PArrayDataType.java   | 126 +++++------
 .../org/apache/phoenix/schema/PDataType.java    | 222 +++++++++++++------
 .../org/apache/phoenix/schema/PhoenixArray.java |  72 +++++-
 .../phoenix/schema/PDataTypeForArraysTest.java  |   4 +-
 11 files changed, 288 insertions(+), 157 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/12e5658e/phoenix-core/src/main/java/org/apache/phoenix/compile/ExpressionProjector.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/ExpressionProjector.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/ExpressionProjector.java
index f9a8316..db58332 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/ExpressionProjector.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/ExpressionProjector.java
@@ -21,7 +21,6 @@ package org.apache.phoenix.compile;
 import java.sql.SQLException;
 
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
-
 import org.apache.phoenix.expression.Expression;
 import org.apache.phoenix.schema.PDataType;
 import org.apache.phoenix.schema.tuple.Tuple;
@@ -73,7 +72,7 @@ public class ExpressionProjector implements ColumnProjector {
             if (ptr.getLength() == 0) {
                 return null;
             }        
-            return type.toObject(ptr, expression.getDataType(), expression.getSortOrder());
+            return type.toObject(ptr, expression.getDataType(), expression.getSortOrder(), expression.getMaxLength(), expression.getScale());
         } catch (RuntimeException e) {
             // FIXME: Expression.evaluate does not throw SQLException
             // so this will unwrap throws from that.

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/12e5658e/phoenix-core/src/main/java/org/apache/phoenix/compile/UpsertCompiler.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/UpsertCompiler.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/UpsertCompiler.java
index c4a2230..ccb1fbe 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/UpsertCompiler.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/UpsertCompiler.java
@@ -676,7 +676,7 @@ public class UpsertCompiler {
                     constantExpression.evaluate(tuple, ptr);
                     Object value = null;
                     if (constantExpression.getDataType() != null) {
-                        value = constantExpression.getDataType().toObject(ptr, constantExpression.getSortOrder());
+                        value = constantExpression.getDataType().toObject(ptr, constantExpression.getSortOrder(), constantExpression.getMaxLength(), constantExpression.getScale());
                         if (!constantExpression.getDataType().isCoercibleTo(column.getDataType(), value)) { 
                             throw TypeMismatchException.newException(
                                 constantExpression.getDataType(), column.getDataType(), "expression: "

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/12e5658e/phoenix-core/src/main/java/org/apache/phoenix/exception/ValueTypeIncompatibleException.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/exception/ValueTypeIncompatibleException.java b/phoenix-core/src/main/java/org/apache/phoenix/exception/ValueTypeIncompatibleException.java
index 3adf415..470d5b9 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/exception/ValueTypeIncompatibleException.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/exception/ValueTypeIncompatibleException.java
@@ -31,6 +31,6 @@ public class ValueTypeIncompatibleException extends IllegalDataException {
     }
 
     private static String getTypeDisplayString(PDataType type, Integer precision, Integer scale) {
-        return type.toString() + "(" + precision + "," + scale + ")";
+        return type.toString() + "(" + precision + (scale == null ? "" : ("," + scale + ")"));
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/12e5658e/phoenix-core/src/main/java/org/apache/phoenix/expression/ArrayConstructorExpression.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/ArrayConstructorExpression.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/ArrayConstructorExpression.java
index 008d267..85eb735 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/expression/ArrayConstructorExpression.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/ArrayConstructorExpression.java
@@ -115,8 +115,8 @@ public class ArrayConstructorExpression extends BaseCompoundExpression {
                 PArrayDataType.writeEndSeperatorForVarLengthArray(oStream);
                 noOfElements = PArrayDataType.serailizeOffsetArrayIntoStream(oStream, byteStream, noOfElements,
                         offsetPos[offsetPos.length - 1], offsetPos);
+                PArrayDataType.serializeHeaderInfoIntoStream(oStream, noOfElements);
             }
-            PArrayDataType.serializeHeaderInfoIntoStream(oStream, noOfElements);
             ptr.set(byteStream.getBuffer(), 0, byteStream.size());
             return true;
         } catch (IOException e) {

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/12e5658e/phoenix-core/src/main/java/org/apache/phoenix/expression/LiteralExpression.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/LiteralExpression.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/LiteralExpression.java
index d382518..9c1c604 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/expression/LiteralExpression.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/LiteralExpression.java
@@ -29,6 +29,7 @@ import org.apache.phoenix.exception.SQLExceptionInfo;
 import org.apache.phoenix.expression.visitor.ExpressionVisitor;
 import org.apache.phoenix.schema.IllegalDataException;
 import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.schema.PhoenixArray;
 import org.apache.phoenix.schema.SortOrder;
 import org.apache.phoenix.schema.TypeMismatchException;
 import org.apache.phoenix.schema.tuple.Tuple;
@@ -163,6 +164,8 @@ public class LiteralExpression extends BaseTerminalExpression {
                 } else if (value != null) {
                     maxLength = ((String)value).length();
                 }
+            } else if (type.isArrayType()) {
+                maxLength = ((PhoenixArray)value).getMaxLength();
             }
             if (b.length == 0) {
                 return TYPED_NULL_EXPRESSIONS[type.ordinal()];

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/12e5658e/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ArrayLengthFunction.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ArrayLengthFunction.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ArrayLengthFunction.java
index 941b1e3..1988e9d 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ArrayLengthFunction.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ArrayLengthFunction.java
@@ -50,7 +50,7 @@ public class ArrayLengthFunction extends ScalarFunction {
 		PDataType baseType = PDataType.fromTypeId(children.get(0).getDataType()
 				.getSqlType()
 				- PDataType.ARRAY_TYPE_BASE);
-		int length = PArrayDataType.getArrayLength(ptr, baseType);
+		int length = PArrayDataType.getArrayLength(ptr, baseType, arrayExpr.getMaxLength());
 		byte[] lengthBuf = new byte[PDataType.INTEGER.getByteSize()];
 		PDataType.INTEGER.getCodec().encodeInt(length, lengthBuf, 0);
 		ptr.set(lengthBuf);

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/12e5658e/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 7bb2c19..6224fb0 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
@@ -83,6 +83,7 @@ import org.apache.phoenix.hbase.index.util.ImmutableBytesPtr;
 import org.apache.phoenix.schema.MetaDataSplitPolicy;
 import org.apache.phoenix.schema.PName;
 import org.apache.phoenix.schema.PNameFactory;
+import org.apache.phoenix.util.ByteUtil;
 
 
 /**
@@ -117,9 +118,9 @@ public interface QueryConstants {
     public final static PName AGG_COLUMN_NAME = SINGLE_COLUMN_NAME;
     public final static PName AGG_COLUMN_FAMILY_NAME = SINGLE_COLUMN_FAMILY_NAME;
     
-    public static final byte[] ARRAY_VALUE_COLUMN_FAMILY = Bytes.toBytes("_a");
-    // TODO: use empty byte array so as not to accidentally conflict with any other columns
-    public static final byte[] ARRAY_VALUE_COLUMN_QUALIFIER = ARRAY_VALUE_COLUMN_FAMILY;
+    public static final byte[] ARRAY_VALUE_COLUMN_FAMILY = Bytes.toBytes("a");
+    // Use empty byte array for column qualifier so as not to accidentally conflict with any other columns
+    public static final byte[] ARRAY_VALUE_COLUMN_QUALIFIER = ByteUtil.EMPTY_BYTE_ARRAY;
 
     public static final byte[] TRUE = new byte[] {1};
 

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/12e5658e/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 d6d823d..865a281 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
@@ -29,6 +29,8 @@ import org.apache.phoenix.util.ByteUtil;
 import org.apache.phoenix.util.StringUtil;
 import org.apache.phoenix.util.TrustedByteArrayOutputStream;
 
+import com.google.common.base.Objects;
+
 /**
  * The datatype for PColummns that are Arrays. Any variable length array would follow the below order. 
  * Every element would be seperated by a seperator byte '0'. Null elements are counted and once a first 
@@ -50,15 +52,16 @@ public class PArrayDataType {
 		if(object == null) {
 			throw new ConstraintViolationException(this + " may not be null");
 		}
-		Pair<Integer, Integer> nullsVsNullRepeationCounter = new Pair<Integer, Integer>();
-        int size = estimateByteSize(object, nullsVsNullRepeationCounter,
-                PDataType.fromTypeId((baseType.getSqlType() + PDataType.ARRAY_TYPE_BASE)));
-        int noOfElements = ((PhoenixArray)object).numElements;
+		PhoenixArray arr = ((PhoenixArray)object);
+        int noOfElements = arr.numElements;
         if(noOfElements == 0) {
         	return ByteUtil.EMPTY_BYTE_ARRAY;
         }
         TrustedByteArrayOutputStream byteStream = null;
 		if (!baseType.isFixedWidth()) {
+	        Pair<Integer, Integer> nullsVsNullRepeationCounter = new Pair<Integer, Integer>();
+	        int size = estimateByteSize(object, nullsVsNullRepeationCounter,
+	                PDataType.fromTypeId((baseType.getSqlType() + PDataType.ARRAY_TYPE_BASE)));
 		    size += ((2 * Bytes.SIZEOF_BYTE) + (noOfElements - nullsVsNullRepeationCounter.getFirst()) * Bytes.SIZEOF_BYTE)
 		                                + (nullsVsNullRepeationCounter.getSecond() * 2 * Bytes.SIZEOF_BYTE);
 		    // Assume an offset array that fit into Short.MAX_VALUE.  Also not considering nulls that could be > 255
@@ -67,8 +70,9 @@ public class PArrayDataType {
 		    // Here the int for noofelements, byte for the version, int for the offsetarray position and 2 bytes for the end seperator
             byteStream = new TrustedByteArrayOutputStream(size + capacity + Bytes.SIZEOF_INT + Bytes.SIZEOF_BYTE +  Bytes.SIZEOF_INT);
 		} else {
+		    int size = arr.getMaxLength() * noOfElements;
 		    // Here the int for noofelements, byte for the version
-		    byteStream = new TrustedByteArrayOutputStream(size + Bytes.SIZEOF_INT+ Bytes.SIZEOF_BYTE);
+		    byteStream = new TrustedByteArrayOutputStream(size);
 		}
 		DataOutputStream oStream = new DataOutputStream(byteStream);
 		return createArrayBytes(byteStream, oStream, (PhoenixArray)object, noOfElements, baseType, 0);
@@ -242,7 +246,9 @@ public class PArrayDataType {
 	    if (ptr.getLength() == 0 || value == null) {
 	        return;
 	    }
-	    
+	    if (Objects.equal(maxLength, desiredMaxLength)) {
+	        return;
+	    }
 	    // TODO: handle bit inversion
 	    // TODO: handle coerce between different types
 	    // TODO: validate that maxLength and desiredMaxLength come through as expected
@@ -250,21 +256,10 @@ public class PArrayDataType {
 	    // or bit inversion
 	    //FIXME: don't write number of elements in the case of fixed width arrays as it will mess up sort order
 	    PhoenixArray pArr = (PhoenixArray) value;
-        Object[] arr = (Object[]) pArr.array;
+	    pArr = new PhoenixArray(pArr, desiredMaxLength);
         PDataType baseType = PDataType.fromTypeId(actualType.getSqlType()
                 - PDataType.ARRAY_TYPE_BASE);
-        if (baseType.isFixedWidth()) {
-            boolean createNewArray = false;
-            for (int i = 0; i < arr.length; i++) {
-                if (baseType.getMaxLength(arr[i]) < desiredMaxLength) {
-                    createNewArray = true;
-                    break;
-                }
-            }
-            if (createNewArray) { 
-                rewriteArrayBytes(ptr, desiredMaxLength, desiredMaxLength, baseType); 
-            }
-        }
+	    ptr.set(toBytes(pArr, baseType));
 	}
 
 
@@ -274,29 +269,28 @@ public class PArrayDataType {
 	}
 
 	public Object toObject(byte[] bytes, int offset, int length, PDataType baseType, 
-			SortOrder sortOrder) {
+			SortOrder sortOrder, Integer maxLength, Integer scale) {
 		return createPhoenixArray(bytes, offset, length, sortOrder,
-				baseType);
+				baseType, maxLength);
 	}
 
     public static void positionAtArrayElement(ImmutableBytesWritable ptr, int arrayIndex, PDataType baseDataType,
             Integer byteSize) {
         byte[] bytes = ptr.get();
         int initPos = ptr.getOffset();
-        int noOfElements = 0;
-        noOfElements = Bytes.toInt(bytes, (ptr.getOffset() + ptr.getLength() - (Bytes.SIZEOF_BYTE + Bytes.SIZEOF_INT)),
-                Bytes.SIZEOF_INT);
-        if (arrayIndex >= noOfElements) {
-            ptr.set(ByteUtil.EMPTY_BYTE_ARRAY);
-            return;
-        }
-        boolean useShort = true;
-        if (noOfElements < 0) {
-            noOfElements = -noOfElements;
-            useShort = false;
-        }
-
         if (!baseDataType.isFixedWidth()) {
+            int noOfElements = Bytes.toInt(bytes, (ptr.getOffset() + ptr.getLength() - (Bytes.SIZEOF_BYTE + Bytes.SIZEOF_INT)),
+                    Bytes.SIZEOF_INT);
+            boolean useShort = true;
+            if (noOfElements < 0) {
+                noOfElements = -noOfElements;
+                useShort = false;
+            }
+            if (arrayIndex >= noOfElements) {
+                ptr.set(ByteUtil.EMPTY_BYTE_ARRAY);
+                return;
+            }
+
             int indexOffset = Bytes.toInt(bytes,
                     (ptr.getOffset() + ptr.getLength() - (Bytes.SIZEOF_BYTE + 2 * Bytes.SIZEOF_INT))) + ptr.getOffset();
             if(arrayIndex >= noOfElements) {
@@ -321,10 +315,12 @@ public class PArrayDataType {
                 ptr.set(bytes, currOffset + initPos, elementLength);
             }
         } else {
-            if (byteSize != null) {
-                ptr.set(bytes, ptr.getOffset() + (arrayIndex * byteSize), byteSize);
+            int elemByteSize = (byteSize == null ? baseDataType.getByteSize() : byteSize);
+            int offset = arrayIndex * elemByteSize;
+            if (offset >= ptr.getLength()) {
+                ptr.set(ByteUtil.EMPTY_BYTE_ARRAY);
             } else {
-                ptr.set(bytes, ptr.getOffset() + (arrayIndex * baseDataType.getByteSize()), baseDataType.getByteSize());
+                ptr.set(bytes, ptr.getOffset() + offset, elemByteSize);
             }
         }
     }
@@ -350,10 +346,6 @@ public class PArrayDataType {
         return offset;
     }
 
-	public Object toObject(byte[] bytes, int offset, int length, PDataType baseType) {
-		return toObject(bytes, offset, length, baseType, SortOrder.getDefault());
-	}
-	
 	public Object toObject(Object object, PDataType actualType) {
 		return object;
 	}
@@ -396,6 +388,7 @@ public class PArrayDataType {
                 PArrayDataType.writeEndSeperatorForVarLengthArray(oStream);
                 noOfElements = PArrayDataType.serailizeOffsetArrayIntoStream(oStream, byteStream, noOfElements,
                         offsetPos[offsetPos.length - 1], offsetPos);
+                serializeHeaderInfoIntoStream(oStream, noOfElements);
             } else {
                 for (int i = 0; i < noOfElements; i++) {
                     byte[] bytes = array.toBytes(i);
@@ -411,7 +404,6 @@ public class PArrayDataType {
                     }
                 }
             }
-            serializeHeaderInfoIntoStream(oStream, noOfElements);
             ImmutableBytesWritable ptr = new ImmutableBytesWritable();
             ptr.set(byteStream.getBuffer(), 0, byteStream.size());
             return ByteUtil.copyKeyBytesIfNecessary(ptr);
@@ -487,22 +479,22 @@ public class PArrayDataType {
     // a null null null b c null d would be
     // 65 0 0 3 66 0 67 0 0 1 68 0 0 0
 	// Follow the above example to understand how this works
-    private Object createPhoenixArray(byte[] bytes, int offset, int length, SortOrder sortOrder, PDataType baseDataType) {
+    private Object createPhoenixArray(byte[] bytes, int offset, int length, SortOrder sortOrder, PDataType baseDataType, Integer maxLength) {
         if (bytes == null || bytes.length == 0) { return null; }
-        ByteBuffer buffer = ByteBuffer.wrap(bytes, offset, length);
-        int initPos = buffer.position();
-        buffer.position((buffer.limit() - (Bytes.SIZEOF_BYTE + Bytes.SIZEOF_INT)));
-        int noOfElemPos = buffer.position();
-        int noOfElements = buffer.getInt();
-        boolean useShort = true;
-        int baseSize = Bytes.SIZEOF_SHORT;
-        if (noOfElements < 0) {
-            noOfElements = -noOfElements;
-            baseSize = Bytes.SIZEOF_INT;
-            useShort = false;
-        }
-        Object[] elements = (Object[])java.lang.reflect.Array.newInstance(baseDataType.getJavaClass(), noOfElements);
+        Object[] elements;
         if (!baseDataType.isFixedWidth()) {
+            ByteBuffer buffer = ByteBuffer.wrap(bytes, offset, length);
+            int initPos = buffer.position();
+            buffer.position((buffer.limit() - (Bytes.SIZEOF_BYTE + Bytes.SIZEOF_INT)));
+            int noOfElements = buffer.getInt();
+            boolean useShort = true;
+            int baseSize = Bytes.SIZEOF_SHORT;
+            if (noOfElements < 0) {
+                noOfElements = -noOfElements;
+                baseSize = Bytes.SIZEOF_INT;
+                useShort = false;
+            }
+            elements = (Object[])java.lang.reflect.Array.newInstance(baseDataType.getJavaClass(), noOfElements);
             buffer.position(buffer.limit() - (Bytes.SIZEOF_BYTE + (2 * Bytes.SIZEOF_INT)));
             int indexOffset = buffer.getInt();
             buffer.position(initPos);
@@ -548,18 +540,13 @@ public class PArrayDataType {
                 }
             }
         } else {
-            buffer.position(initPos);
+            int elemLength = (maxLength == null ? baseDataType.getByteSize() : maxLength);
+            int noOfElements = length / elemLength;
+            elements = (Object[])java.lang.reflect.Array.newInstance(baseDataType.getJavaClass(), noOfElements);
+            ImmutableBytesWritable ptr = new ImmutableBytesWritable();
             for (int i = 0; i < noOfElements; i++) {
-                byte[] val;
-                if (baseDataType == PDataType.CHAR) {
-                    int maxLength = (noOfElemPos - initPos)/noOfElements;
-                    // Should be char array
-                    val = new byte[maxLength];
-                } else {
-                    val = new byte[baseDataType.getByteSize()];
-                }
-                buffer.get(val);
-                elements[i] = baseDataType.toObject(val, sortOrder);
+                ptr.set(bytes, offset + i * elemLength, elemLength);
+                elements[i] = baseDataType.toObject(ptr, sortOrder);
             }
         }
         return PArrayDataType.instantiatePhoenixArray(baseDataType, elements);
@@ -579,10 +566,11 @@ public class PArrayDataType {
 	}
 
 	public static int getArrayLength(ImmutableBytesWritable ptr,
-			PDataType baseType) {
+			PDataType baseType, Integer maxLength) {
 		byte[] bytes = ptr.get();
 		if(baseType.isFixedWidth()) {
-			return ((ptr.getLength() - (Bytes.SIZEOF_BYTE + Bytes.SIZEOF_INT))/baseType.getByteSize());
+		    int elemLength = maxLength == null ? baseType.getByteSize() : maxLength;
+			return (ptr.getLength() / elemLength);
 		}
 		return Bytes.toInt(bytes, (ptr.getOffset() + ptr.getLength() - (Bytes.SIZEOF_BYTE + Bytes.SIZEOF_INT)));
 	}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/12e5658e/phoenix-core/src/main/java/org/apache/phoenix/schema/PDataType.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/PDataType.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/PDataType.java
index 211b3b0..8abc54b 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/PDataType.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/PDataType.java
@@ -31,6 +31,7 @@ import java.util.Map;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.hadoop.hbase.util.Base64;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.phoenix.exception.ValueTypeIncompatibleException;
 import org.apache.phoenix.query.KeyRange;
 import org.apache.phoenix.query.QueryConstants;
 import org.apache.phoenix.util.ByteUtil;
@@ -39,6 +40,7 @@ import org.apache.phoenix.util.NumberUtil;
 import org.apache.phoenix.util.StringUtil;
 
 import com.google.common.base.Preconditions;
+import com.google.common.base.Strings;
 import com.google.common.collect.ImmutableMap;
 import com.google.common.math.LongMath;
 import com.google.common.primitives.Booleans;
@@ -79,7 +81,7 @@ public enum PDataType {
         }
 
         @Override
-        public Object toObject(byte[] bytes, int offset, int length, PDataType actualType, SortOrder sortOrder) {
+        public Object toObject(byte[] bytes, int offset, int length, PDataType actualType, SortOrder sortOrder, Integer maxLength, Integer scale) {
             if (!actualType.isCoercibleTo(this)) {
                 throw new ConstraintViolationException(actualType + " cannot be coerced to " + this);
             }
@@ -178,6 +180,21 @@ public enum PDataType {
      */
     CHAR("CHAR", Types.CHAR, String.class, null) { // Delegate to VARCHAR
         @Override
+        public Object pad(Object object, int maxLength) {
+            String s = (String) object;
+            if (s == null) {
+                return s;
+            }
+            if (s.length() == maxLength) {
+                return object;
+            }
+            if (s.length() > maxLength) {
+                throw new ValueTypeIncompatibleException(this,maxLength,null);
+            }
+            return Strings.padEnd(s, maxLength, ' ');
+        }
+        
+        @Override
         public byte[] toBytes(Object object) {
             if (object == null) {
                 throw new ConstraintViolationException(this + " may not be null");
@@ -202,7 +219,7 @@ public enum PDataType {
         }
 
         @Override
-        public Object toObject(byte[] bytes, int offset, int length, PDataType actualType, SortOrder sortOrder) {
+        public Object toObject(byte[] bytes, int offset, int length, PDataType actualType, SortOrder sortOrder, Integer maxLength, Integer scale) {
             if (!actualType.isCoercibleTo(this)) { // TODO: have isCoercibleTo that takes bytes, offset?
                 throw new ConstraintViolationException(actualType + " cannot be coerced to " + this);
             }
@@ -378,7 +395,7 @@ public enum PDataType {
         }
 
         @Override
-        public Long toObject(byte[] b, int o, int l, PDataType actualType, SortOrder sortOrder) {
+        public Long toObject(byte[] b, int o, int l, PDataType actualType, SortOrder sortOrder, Integer maxLength, Integer scale) {
             if (l == 0) {
                 return null;
             }
@@ -527,7 +544,7 @@ public enum PDataType {
         }
 
         @Override
-        public Integer toObject(byte[] b, int o, int l, PDataType actualType, SortOrder sortOrder) {
+        public Integer toObject(byte[] b, int o, int l, PDataType actualType, SortOrder sortOrder, Integer maxLength, Integer scale) {
             if (l == 0) {
                 return null;
             }
@@ -685,7 +702,7 @@ public enum PDataType {
       }
 
       @Override
-      public Short toObject(byte[] b, int o, int l, PDataType actualType, SortOrder sortOrder) {
+      public Short toObject(byte[] b, int o, int l, PDataType actualType, SortOrder sortOrder, Integer maxLength, Integer scale) {
           if (l == 0) {
               return null;
           }
@@ -826,7 +843,7 @@ public enum PDataType {
       }
       
       @Override
-      public Byte toObject(byte[] b, int o, int l, PDataType actualType, SortOrder sortOrder) {
+      public Byte toObject(byte[] b, int o, int l, PDataType actualType, SortOrder sortOrder, Integer maxLength, Integer scale) {
           if (l == 0) {
               return null;
           }
@@ -987,7 +1004,7 @@ public enum PDataType {
         }
         
         @Override
-        public Float toObject(byte[] b, int o, int l, PDataType actualType, SortOrder sortOrder) {
+        public Float toObject(byte[] b, int o, int l, PDataType actualType, SortOrder sortOrder, Integer maxLength, Integer scale) {
             if (l <= 0) {
                 return null;
             }
@@ -1156,7 +1173,7 @@ public enum PDataType {
         }
         
         @Override
-        public Double toObject(byte[] b, int o, int l, PDataType actualType, SortOrder sortOrder) {
+        public Double toObject(byte[] b, int o, int l, PDataType actualType, SortOrder sortOrder, Integer maxLength, Integer scale) {
             if (l <= 0) {
                 return null;
             }
@@ -1302,7 +1319,7 @@ public enum PDataType {
         }
 
         @Override
-        public Object toObject(byte[] b, int o, int l, PDataType actualType, SortOrder sortOrder) {
+        public Object toObject(byte[] b, int o, int l, PDataType actualType, SortOrder sortOrder, Integer maxLength, Integer scale) {
             Preconditions.checkNotNull(sortOrder);        	
             if (l == 0) {
                 return null;
@@ -1650,7 +1667,7 @@ public enum PDataType {
         }
 
         @Override
-        public Timestamp toObject(byte[] b, int o, int l, PDataType actualType, SortOrder sortOrder) {
+        public Timestamp toObject(byte[] b, int o, int l, PDataType actualType, SortOrder sortOrder, Integer maxLength, Integer scale) {
             if (actualType == null || l == 0) {
                 return null;
             }
@@ -1779,7 +1796,7 @@ public enum PDataType {
         }
 
         @Override
-        public Time toObject(byte[] b, int o, int l, PDataType actualType, SortOrder sortOrder) {
+        public Time toObject(byte[] b, int o, int l, PDataType actualType, SortOrder sortOrder, Integer maxLength, Integer scale) {
             if (l == 0) {
                 return null;
             }
@@ -1922,7 +1939,7 @@ public enum PDataType {
         }
 
         @Override
-        public Date toObject(byte[] b, int o, int l, PDataType actualType, SortOrder sortOrder) {
+        public Date toObject(byte[] b, int o, int l, PDataType actualType, SortOrder sortOrder, Integer maxLength, Integer scale) {
             if (l == 0) {
                 return null;
             }
@@ -2059,7 +2076,7 @@ public enum PDataType {
         }
 
         @Override
-        public Object toObject(byte[] b, int o, int l, PDataType actualType, SortOrder sortOrder) {
+        public Object toObject(byte[] b, int o, int l, PDataType actualType, SortOrder sortOrder, Integer maxLength, Integer scale) {
             Timestamp ts = (Timestamp) TIMESTAMP.toObject(b, o, l, actualType, sortOrder);
             throwIfNonNegativeDate(ts);
             return ts;
@@ -2136,7 +2153,7 @@ public enum PDataType {
         }
 
         @Override
-        public Object toObject(byte[] b, int o, int l, PDataType actualType, SortOrder sortOrder) {
+        public Object toObject(byte[] b, int o, int l, PDataType actualType, SortOrder sortOrder, Integer maxLength, Integer scale) {
             Time t = (Time)TIME.toObject(b, o, l, actualType, sortOrder);
             throwIfNonNegativeDate(t);
             return t;
@@ -2223,7 +2240,7 @@ public enum PDataType {
         }
 
         @Override
-        public Object toObject(byte[] b, int o, int l, PDataType actualType, SortOrder sortOrder) {
+        public Object toObject(byte[] b, int o, int l, PDataType actualType, SortOrder sortOrder, Integer maxLength, Integer scale) {
             Date d = (Date)DATE.toObject(b,o,l,actualType, sortOrder);
             throwIfNonNegativeDate(d);
             return d;
@@ -2326,7 +2343,7 @@ public enum PDataType {
         }
 
         @Override
-        public Object toObject(byte[] b, int o, int l, PDataType actualType, SortOrder sortOrder) {
+        public Object toObject(byte[] b, int o, int l, PDataType actualType, SortOrder sortOrder, Integer maxLength, Integer scale) {
             Long v = (Long)LONG.toObject(b, o, l, actualType, sortOrder);
             throwIfNonNegativeNumber(v);
             return v;
@@ -2422,7 +2439,7 @@ public enum PDataType {
         }
 
         @Override
-        public Object toObject(byte[] b, int o, int l, PDataType actualType, SortOrder sortOrder) {
+        public Object toObject(byte[] b, int o, int l, PDataType actualType, SortOrder sortOrder, Integer maxLength, Integer scale) {
             Integer v = (Integer)INTEGER.toObject(b, o, l, actualType, sortOrder);
             throwIfNonNegativeNumber(v);
             return v;
@@ -2547,7 +2564,7 @@ public enum PDataType {
       }
       
       @Override
-      public Object toObject(byte[] b, int o, int l, PDataType actualType, SortOrder sortOrder) {
+      public Object toObject(byte[] b, int o, int l, PDataType actualType, SortOrder sortOrder, Integer maxLength, Integer scale) {
           Short v = (Short)SMALLINT.toObject(b, o, l, actualType, sortOrder);
           throwIfNonNegativeNumber(v);
           return v;
@@ -2638,7 +2655,7 @@ public enum PDataType {
       }
       
       @Override
-      public Object toObject(byte[] b, int o, int l, PDataType actualType, SortOrder sortOrder) {
+      public Object toObject(byte[] b, int o, int l, PDataType actualType, SortOrder sortOrder, Integer maxLength, Integer scale) {
           Byte v = (Byte)TINYINT.toObject(b, o, l, actualType, sortOrder);
           throwIfNonNegativeNumber(v);
           return v;
@@ -2733,7 +2750,7 @@ public enum PDataType {
         }
         
         @Override
-        public Object toObject(byte[] b, int o, int l, PDataType actualType, SortOrder sortOrder) {
+        public Object toObject(byte[] b, int o, int l, PDataType actualType, SortOrder sortOrder, Integer maxLength, Integer scale) {
             Float v = (Float)FLOAT.toObject(b, o, l, actualType, sortOrder);
             throwIfNonNegativeNumber(v);
             return v;
@@ -2826,7 +2843,7 @@ public enum PDataType {
         }
         
         @Override
-        public Object toObject(byte[] b, int o, int l, PDataType actualType, SortOrder sortOrder) {
+        public Object toObject(byte[] b, int o, int l, PDataType actualType, SortOrder sortOrder, Integer maxLength, Integer scale) {
             Double v = (Double)DOUBLE.toObject(b, o, l, actualType, sortOrder);
             throwIfNonNegativeNumber(v);
             return v;
@@ -2879,7 +2896,7 @@ public enum PDataType {
         }
 
         @Override
-        public Object toObject(byte[] bytes, int offset, int length, PDataType targetType, SortOrder sortOrder) {
+        public Object toObject(byte[] bytes, int offset, int length, PDataType targetType, SortOrder sortOrder, Integer maxLength, Integer scale) {
             if (length > 1) {
                 throw new IllegalDataException("BOOLEAN may only be a single byte");
             }
@@ -2962,7 +2979,7 @@ public enum PDataType {
         }
 
         @Override
-        public Object toObject(byte[] bytes, int offset, int length, PDataType actualType, SortOrder sortOrder) {
+        public Object toObject(byte[] bytes, int offset, int length, PDataType actualType, SortOrder sortOrder, Integer maxLength, Integer scale) {
             if (length == 0) {
                 return null;
             }
@@ -3062,6 +3079,24 @@ public enum PDataType {
     },
     BINARY("BINARY", Types.BINARY, byte[].class, null) {
         @Override
+        public Object pad(Object object, int maxLength) {
+            byte[] b = (byte[]) object;
+            if (b == null) {
+                return null;
+            }
+            if (b.length == maxLength) {
+                return object;
+            }
+            if (b.length > maxLength) {
+                throw new ValueTypeIncompatibleException(this,maxLength,null);
+            }
+            byte[] newBytes = new byte[maxLength];
+            System.arraycopy(b, 0, newBytes, 0, b.length);
+            
+            return newBytes;
+        }
+        
+        @Override
         public byte[] toBytes(Object object) { // Deligate to VARBINARY
             if (object == null) {
                 throw new ConstraintViolationException(this + " may not be null");
@@ -3088,7 +3123,7 @@ public enum PDataType {
         }
 
         @Override
-        public Object toObject(byte[] bytes, int offset, int length, PDataType actualType, SortOrder sortOrder) {
+        public Object toObject(byte[] bytes, int offset, int length, PDataType actualType, SortOrder sortOrder, Integer maxLength, Integer scale) {
             if (!actualType.isCoercibleTo(this)) {
                 throw new ConstraintViolationException(actualType + " cannot be coerced to " + this);
             }
@@ -3218,8 +3253,9 @@ public enum PDataType {
 		
 		@Override
 		public Object toObject(byte[] bytes, int offset, int length,
-				PDataType actualType, SortOrder sortOrder) {
-			return pDataTypeForArray.toObject(bytes, offset, length, PDataType.INTEGER, sortOrder);
+				PDataType actualType, SortOrder sortOrder, Integer maxLength, Integer scale) {
+
+			return pDataTypeForArray.toObject(bytes, offset, length, PDataType.INTEGER, sortOrder, maxLength, scale);
 		}
 		
 		@Override
@@ -3282,8 +3318,9 @@ public enum PDataType {
 		
 		@Override
 		public Object toObject(byte[] bytes, int offset, int length,
-				PDataType actualType, SortOrder sortOrder) {
-			return pDataTypeForArray.toObject(bytes, offset, length, PDataType.BOOLEAN, sortOrder);
+				PDataType actualType, SortOrder sortOrder, Integer maxLength, Integer scale) {
+
+			return pDataTypeForArray.toObject(bytes, offset, length, PDataType.BOOLEAN, sortOrder, maxLength, scale);
 		}
 		
 		@Override
@@ -3346,8 +3383,9 @@ public enum PDataType {
 		
 		@Override
 		public Object toObject(byte[] bytes, int offset, int length,
-				PDataType actualType, SortOrder sortOrder) {
-			return pDataTypeForArray.toObject(bytes, offset, length, PDataType.VARCHAR, sortOrder);
+				PDataType actualType, SortOrder sortOrder, Integer maxLength, Integer scale) {
+
+			return pDataTypeForArray.toObject(bytes, offset, length, PDataType.VARCHAR, sortOrder, maxLength, scale);
 		}
 		
 		@Override
@@ -3423,8 +3461,9 @@ public enum PDataType {
 		
 		@Override
 		public Object toObject(byte[] bytes, int offset, int length,
-				PDataType actualType, SortOrder sortOrder) {
-			return pDataTypeForArray.toObject(bytes, offset, length, PDataType.VARBINARY, sortOrder);
+				PDataType actualType, SortOrder sortOrder, Integer maxLength, Integer scale) {
+
+			return pDataTypeForArray.toObject(bytes, offset, length, PDataType.VARBINARY, sortOrder, maxLength, scale);
 		}
 		
 		@Override
@@ -3500,8 +3539,9 @@ public enum PDataType {
 		
 		@Override
 		public Object toObject(byte[] bytes, int offset, int length,
-				PDataType actualType, SortOrder sortOrder) {
-			return pDataTypeForArray.toObject(bytes, offset, length, PDataType.BINARY, sortOrder);
+				PDataType actualType, SortOrder sortOrder, Integer maxLength, Integer scale) {
+
+			return pDataTypeForArray.toObject(bytes, offset, length, PDataType.BINARY, sortOrder, maxLength, scale);
 		}
 		
 		@Override
@@ -3579,8 +3619,9 @@ public enum PDataType {
 		
 		@Override
 		public Object toObject(byte[] bytes, int offset, int length,
-				PDataType actualType, SortOrder sortOrder) {
-			return pDataTypeForArray.toObject(bytes, offset, length, PDataType.CHAR, sortOrder);
+				PDataType actualType, SortOrder sortOrder, Integer maxLength, Integer scale) {
+
+			return pDataTypeForArray.toObject(bytes, offset, length, PDataType.CHAR, sortOrder, maxLength, scale);
 		}
 		
 		@Override
@@ -3656,8 +3697,9 @@ public enum PDataType {
 		
 		@Override
 		public Object toObject(byte[] bytes, int offset, int length,
-				PDataType actualType, SortOrder sortOrder) {
-			return pDataTypeForArray.toObject(bytes, offset, length, PDataType.LONG, sortOrder);
+				PDataType actualType, SortOrder sortOrder, Integer maxLength, Integer scale) {
+
+			return pDataTypeForArray.toObject(bytes, offset, length, PDataType.LONG, sortOrder, maxLength, scale);
 		}
 		
 		@Override
@@ -3719,8 +3761,9 @@ public enum PDataType {
 		
 		@Override
 		public Object toObject(byte[] bytes, int offset, int length,
-				PDataType actualType, SortOrder sortOrder) {
-			return pDataTypeForArray.toObject(bytes, offset, length, PDataType.SMALLINT, sortOrder);
+				PDataType actualType, SortOrder sortOrder, Integer maxLength, Integer scale) {
+
+			return pDataTypeForArray.toObject(bytes, offset, length, PDataType.SMALLINT, sortOrder, maxLength, scale);
 		}
 		
 		@Override
@@ -3782,8 +3825,9 @@ public enum PDataType {
 		
 		@Override
 		public Object toObject(byte[] bytes, int offset, int length,
-				PDataType actualType, SortOrder sortOrder) {
-			return pDataTypeForArray.toObject(bytes, offset, length, PDataType.TINYINT, sortOrder);
+				PDataType actualType, SortOrder sortOrder, Integer maxLength, Integer scale) {
+
+			return pDataTypeForArray.toObject(bytes, offset, length, PDataType.TINYINT, sortOrder, maxLength, scale);
 		}
 		
 		@Override
@@ -3846,8 +3890,9 @@ public enum PDataType {
 		
 		@Override
 		public Object toObject(byte[] bytes, int offset, int length,
-				PDataType actualType, SortOrder sortOrder) {
-			return pDataTypeForArray.toObject(bytes, offset, length, PDataType.FLOAT, sortOrder);
+				PDataType actualType, SortOrder sortOrder, Integer maxLength, Integer scale) {
+
+			return pDataTypeForArray.toObject(bytes, offset, length, PDataType.FLOAT, sortOrder, maxLength, scale);
 		}
 	
 		@Override
@@ -3910,8 +3955,9 @@ public enum PDataType {
 		
 		@Override
 		public Object toObject(byte[] bytes, int offset, int length,
-				PDataType actualType, SortOrder sortOrder) {
-			return pDataTypeForArray.toObject(bytes, offset, length, PDataType.DOUBLE, sortOrder);
+				PDataType actualType, SortOrder sortOrder, Integer maxLength, Integer scale) {
+
+			return pDataTypeForArray.toObject(bytes, offset, length, PDataType.DOUBLE, sortOrder, maxLength, scale);
 		}
 		
 		@Override
@@ -3974,8 +4020,9 @@ public enum PDataType {
 		
 		@Override
 		public Object toObject(byte[] bytes, int offset, int length,
-				PDataType actualType, SortOrder sortOrder) {
-			return pDataTypeForArray.toObject(bytes, offset, length, PDataType.DECIMAL, sortOrder);
+				PDataType actualType, SortOrder sortOrder, Integer maxLength, Integer scale) {
+
+			return pDataTypeForArray.toObject(bytes, offset, length, PDataType.DECIMAL, sortOrder, maxLength, scale);
 		}
 	
 		@Override
@@ -4046,8 +4093,9 @@ public enum PDataType {
 		
 		@Override
 		public Object toObject(byte[] bytes, int offset, int length,
-				PDataType actualType, SortOrder sortOrder) {
-			return pDataTypeForArray.toObject(bytes, offset, length, PDataType.TIMESTAMP, sortOrder);
+				PDataType actualType, SortOrder sortOrder, Integer maxLength, Integer scale) {
+
+			return pDataTypeForArray.toObject(bytes, offset, length, PDataType.TIMESTAMP, sortOrder, maxLength, scale);
 		}
 		
 		@Override
@@ -4110,8 +4158,9 @@ public enum PDataType {
 		
 		@Override
 		public Object toObject(byte[] bytes, int offset, int length,
-				PDataType actualType, SortOrder sortOrder) {
-			return pDataTypeForArray.toObject(bytes, offset, length, PDataType.UNSIGNED_TIMESTAMP, sortOrder);
+				PDataType actualType, SortOrder sortOrder, Integer maxLength, Integer scale) {
+
+			return pDataTypeForArray.toObject(bytes, offset, length, PDataType.UNSIGNED_TIMESTAMP, sortOrder, maxLength, scale);
 		}
 		
 		@Override
@@ -4173,8 +4222,9 @@ public enum PDataType {
 		
 		@Override
 		public Object toObject(byte[] bytes, int offset, int length,
-				PDataType actualType, SortOrder sortOrder) {
-			return pDataTypeForArray.toObject(bytes, offset, length, PDataType.TIME, sortOrder);
+				PDataType actualType, SortOrder sortOrder, Integer maxLength, Integer scale) {
+
+			return pDataTypeForArray.toObject(bytes, offset, length, PDataType.TIME, sortOrder, maxLength, scale);
 		}
 		
 		@Override
@@ -4236,8 +4286,9 @@ public enum PDataType {
 		
 		@Override
 		public Object toObject(byte[] bytes, int offset, int length,
-				PDataType actualType, SortOrder sortOrder) {
-			return pDataTypeForArray.toObject(bytes, offset, length, PDataType.UNSIGNED_TIME, sortOrder);
+				PDataType actualType, SortOrder sortOrder, Integer maxLength, Integer scale) {
+
+			return pDataTypeForArray.toObject(bytes, offset, length, PDataType.UNSIGNED_TIME, sortOrder, maxLength, scale);
 		}
 		
 		@Override
@@ -4299,8 +4350,9 @@ public enum PDataType {
 		
 		@Override
 		public Object toObject(byte[] bytes, int offset, int length,
-				PDataType actualType, SortOrder sortOrder) {
-			return pDataTypeForArray.toObject(bytes, offset, length, PDataType.DATE, sortOrder);
+				PDataType actualType, SortOrder sortOrder, Integer maxLength, Integer scale) {
+
+			return pDataTypeForArray.toObject(bytes, offset, length, PDataType.DATE, sortOrder, maxLength, scale);
 		}
 		
 		@Override
@@ -4362,8 +4414,9 @@ public enum PDataType {
 		
 		@Override
 		public Object toObject(byte[] bytes, int offset, int length,
-				PDataType actualType, SortOrder sortOrder) {
-			return pDataTypeForArray.toObject(bytes, offset, length, PDataType.UNSIGNED_DATE, sortOrder);
+				PDataType actualType, SortOrder sortOrder, Integer maxLength, Integer scale) {
+
+			return pDataTypeForArray.toObject(bytes, offset, length, PDataType.UNSIGNED_DATE, sortOrder, maxLength, scale);
 		}
 		
 		@Override
@@ -4425,8 +4478,9 @@ public enum PDataType {
 		
 		@Override
 		public Object toObject(byte[] bytes, int offset, int length,
-				PDataType actualType, SortOrder sortOrder) {
-			return pDataTypeForArray.toObject(bytes, offset, length, PDataType.UNSIGNED_LONG, sortOrder);
+				PDataType actualType, SortOrder sortOrder, Integer maxLength, Integer scale) {
+
+			return pDataTypeForArray.toObject(bytes, offset, length, PDataType.UNSIGNED_LONG, sortOrder, maxLength, scale);
 		}
 		
 		@Override
@@ -4488,8 +4542,9 @@ public enum PDataType {
 		
 		@Override
 		public Object toObject(byte[] bytes, int offset, int length,
-				PDataType actualType, SortOrder sortOrder) {
-			return pDataTypeForArray.toObject(bytes, offset, length, PDataType.UNSIGNED_INT, sortOrder);
+				PDataType actualType, SortOrder sortOrder, Integer maxLength, Integer scale) {
+
+			return pDataTypeForArray.toObject(bytes, offset, length, PDataType.UNSIGNED_INT, sortOrder, maxLength, scale);
 		}
 
 		@Override
@@ -4552,8 +4607,9 @@ public enum PDataType {
 		
 		@Override
 		public Object toObject(byte[] bytes, int offset, int length,
-				PDataType actualType, SortOrder sortOrder) {
-			return pDataTypeForArray.toObject(bytes, offset, length, PDataType.UNSIGNED_SMALLINT, sortOrder);
+				PDataType actualType, SortOrder sortOrder, Integer maxLength, Integer scale) {
+
+			return pDataTypeForArray.toObject(bytes, offset, length, PDataType.UNSIGNED_SMALLINT, sortOrder, maxLength, scale);
 		}
 		
 		@Override
@@ -4616,8 +4672,9 @@ public enum PDataType {
 		
 		@Override
 		public Object toObject(byte[] bytes, int offset, int length,
-				PDataType actualType, SortOrder sortOrder) {
-			return pDataTypeForArray.toObject(bytes, offset, length, PDataType.UNSIGNED_TINYINT, sortOrder);
+				PDataType actualType, SortOrder sortOrder, Integer maxLength, Integer scale) {
+
+			return pDataTypeForArray.toObject(bytes, offset, length, PDataType.UNSIGNED_TINYINT, sortOrder, maxLength, scale);
 		}
 		
 		@Override
@@ -4678,8 +4735,9 @@ public enum PDataType {
 		
 		@Override
 		public Object toObject(byte[] bytes, int offset, int length,
-				PDataType actualType, SortOrder sortOrder) {
-			return pDataTypeForArray.toObject(bytes, offset, length, PDataType.UNSIGNED_FLOAT, sortOrder);
+				PDataType actualType, SortOrder sortOrder, Integer maxLength, Integer scale) {
+
+			return pDataTypeForArray.toObject(bytes, offset, length, PDataType.UNSIGNED_FLOAT, sortOrder, maxLength, scale);
 		}
 
 		@Override
@@ -4742,8 +4800,9 @@ public enum PDataType {
 		
 		@Override
 		public Object toObject(byte[] bytes, int offset, int length,
-				PDataType actualType, SortOrder sortOrder) {
-			return pDataTypeForArray.toObject(bytes, offset, length, PDataType.UNSIGNED_DOUBLE, sortOrder);
+				PDataType actualType, SortOrder sortOrder, Integer maxLength, Integer scale) {
+
+			return pDataTypeForArray.toObject(bytes, offset, length, PDataType.UNSIGNED_DOUBLE, sortOrder, maxLength, scale);
 		}
 		
 		@Override
@@ -4777,6 +4836,7 @@ public enum PDataType {
     private final byte[] sqlTypeNameBytes;
     private final PDataCodec codec;
     final PArrayDataType pDataTypeForArray = new PArrayDataType();
+    
     private PDataType(String sqlTypeName, int sqlType, Class clazz, PDataCodec codec) {
         this.sqlTypeName = sqlTypeName;
         this.sqlType = sqlType;
@@ -6359,7 +6419,11 @@ public enum PDataType {
     /*
      * Each enum must override this to define the set of objects it may create
      */
-    public abstract Object toObject(byte[] bytes, int offset, int length, PDataType actualType, SortOrder sortOrder);
+    public abstract Object toObject(byte[] bytes, int offset, int length, PDataType actualType, SortOrder sortOrder, Integer maxLength, Integer scale);
+    
+    public final Object toObject(byte[] bytes, int offset, int length, PDataType actualType, SortOrder sortOrder) {
+        return toObject(bytes, offset, length, actualType, sortOrder, null, null);
+    }
     
     public final Object toObject(byte[] bytes, int offset, int length, PDataType actualType) {
         return toObject(bytes, offset, length, actualType, SortOrder.getDefault());
@@ -6373,6 +6437,14 @@ public enum PDataType {
         return this.toObject(ptr.get(), ptr.getOffset(), ptr.getLength(), actualType, sortOrder);
     }
     
+    public final Object toObject(ImmutableBytesWritable ptr, PDataType actualType, SortOrder sortOrder, Integer maxLength, Integer scale) {
+        return this.toObject(ptr.get(), ptr.getOffset(), ptr.getLength(), actualType, sortOrder, maxLength, scale);
+    }
+    
+    public final Object toObject(ImmutableBytesWritable ptr, SortOrder sortOrder, Integer maxLength, Integer scale) {
+        return this.toObject(ptr.get(), ptr.getOffset(), ptr.getLength(), this, sortOrder, maxLength, scale);
+    }
+    
     public final Object toObject(ImmutableBytesWritable ptr) {
         return toObject(ptr.get(), ptr.getOffset(), ptr.getLength());
     }
@@ -6579,5 +6651,9 @@ public enum PDataType {
     public long getMillis(ImmutableBytesWritable ptr, SortOrder sortOrder) {
         throw new UnsupportedOperationException("Operation not supported for type " + this);
     }
+
+    public Object pad(Object object, int maxLength) {
+        return object;
+    }
     
 }

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/12e5658e/phoenix-core/src/main/java/org/apache/phoenix/schema/PhoenixArray.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/PhoenixArray.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/PhoenixArray.java
index c44bf36..b2826a8 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/PhoenixArray.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/PhoenixArray.java
@@ -33,21 +33,85 @@ public class PhoenixArray implements Array,SQLCloseable {
 	Object array;
 	int numElements;
 	Integer maxLength;
+	
 	public PhoenixArray() {
 		// empty constructor
 	}
 	
+	public Integer getMaxLength() {
+	    return maxLength;
+	}
+
+	private static Object[] coerceToNewLength(PDataType baseType, Object[] elements, int maxLength) {
+        Object[] resizedElements = new Object[elements.length];
+        for (int i = 0; i < elements.length; i++) {
+            int length = baseType.getMaxLength(elements[i]);
+            if (length == maxLength) {
+                resizedElements[i] = elements[i];
+            } else {
+                resizedElements[i] = baseType.pad(elements[i],maxLength);
+            }
+        }
+        return resizedElements;
+	}
+	private static Object[] coerceToEqualLength(PDataType baseType, Object[] elements) {
+	    if (elements == null || elements.length == 0) {
+	        return elements;
+	    }
+	    Object element = elements[0];
+	    int maxLength = baseType.getMaxLength(element);
+	    boolean resizeElements = false;
+	    for (int i = 1; i < elements.length; i++) {
+	        int length = baseType.getMaxLength(elements[i]);
+	        if (length > maxLength) {
+	            maxLength = length;
+	            resizeElements = true;
+	        } else if (length < maxLength) {
+	            resizeElements = true;
+	        }
+	    }
+	    if (!resizeElements) {
+	        return elements;
+	    }
+	    return coerceToNewLength(baseType, elements, maxLength);
+	}
+	
 	public PhoenixArray(PDataType baseType, Object[] elements) {
 		// As we are dealing with primitive types and only the Boxed objects
 		this.baseType = baseType;
-		this.array = convertObjectArrayToPrimitiveArray(elements);
+		if (baseType.isFixedWidth()) {
+		    if (baseType.getByteSize() == null) {
+    		    elements = coerceToEqualLength(baseType, elements);
+    		    if (elements != null && elements.length > 0) {
+    		        this.maxLength = baseType.getMaxLength(elements[0]);
+    		    }
+		    } else {
+		        maxLength = baseType.getByteSize();
+		    }
+		}
+        this.array = convertObjectArrayToPrimitiveArray(elements);
 		this.numElements = elements.length;
 	}
 	
-	public Object convertObjectArrayToPrimitiveArray(Object[] elements) {
-		return elements;
-	}
+	public PhoenixArray(PhoenixArray pArr, Integer desiredMaxLength) {
+	    this.baseType = pArr.baseType;
+	    Object[] elements = (Object[])pArr.array;
+        if (baseType.isFixedWidth()) {
+            if (baseType.getByteSize() == null) {
+                elements = coerceToNewLength(baseType, (Object[])pArr.array, desiredMaxLength);
+                maxLength = desiredMaxLength;
+            } else {
+                maxLength = baseType.getByteSize();
+            }
+        }
+        this.array = convertObjectArrayToPrimitiveArray(elements);
+        this.numElements = elements.length;
+    }
 
+    public Object convertObjectArrayToPrimitiveArray(Object[] elements) {
+	    return elements; 
+	}
+	
 	@Override
 	public void free() throws SQLException {
 	}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/12e5658e/phoenix-core/src/test/java/org/apache/phoenix/schema/PDataTypeForArraysTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/schema/PDataTypeForArraysTest.java b/phoenix-core/src/test/java/org/apache/phoenix/schema/PDataTypeForArraysTest.java
index 6c5f8ca..d3e2d0c 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/schema/PDataTypeForArraysTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/schema/PDataTypeForArraysTest.java
@@ -339,7 +339,7 @@ public class PDataTypeForArraysTest {
 				PDataType.VARCHAR, strArr);
 		byte[] bytes = PDataType.VARCHAR_ARRAY.toBytes(arr);
 		ImmutableBytesWritable ptr = new ImmutableBytesWritable(bytes);
-		int result = PArrayDataType.getArrayLength(ptr, PDataType.VARCHAR);
+		int result = PArrayDataType.getArrayLength(ptr, PDataType.VARCHAR, null);
 		assertEquals(5, result);
 	}
 
@@ -663,7 +663,7 @@ public class PDataTypeForArraysTest {
 		PDataType.LONG_ARRAY.toObject(arr, PDataType.LONG_ARRAY);
 		byte[] bytes = PDataType.LONG_ARRAY.toBytes(arr);
 		ImmutableBytesWritable ptr = new ImmutableBytesWritable(bytes);
-		int length = PArrayDataType.getArrayLength(ptr, PDataType.LONG);
+		int length = PArrayDataType.getArrayLength(ptr, PDataType.LONG, null);
 		assertEquals(4, length);
 	}
 


Mime
View raw message