You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@phoenix.apache.org by sa...@apache.org on 2016/11/04 22:13:39 UTC

[12/50] [abbrv] phoenix git commit: PHOENIX-3396 Valid Multi-byte strings whose total byte size is greater than the max char limit cannot be inserted into VARCHAR fields in the PK

http://git-wip-us.apache.org/repos/asf/phoenix/blob/bb88e9f5/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PDecimal.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PDecimal.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PDecimal.java
index 17910de..9fff730 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PDecimal.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PDecimal.java
@@ -35,384 +35,385 @@ import com.google.common.base.Preconditions;
 
 public class PDecimal extends PRealNumber<BigDecimal> {
 
-  public static final PDecimal INSTANCE = new PDecimal();
+    public static final PDecimal INSTANCE = new PDecimal();
 
-  private static final BigDecimal MIN_DOUBLE_AS_BIG_DECIMAL =
-      BigDecimal.valueOf(-Double.MAX_VALUE);
-  private static final BigDecimal MAX_DOUBLE_AS_BIG_DECIMAL =
-      BigDecimal.valueOf(Double.MAX_VALUE);
-  private static final BigDecimal MIN_FLOAT_AS_BIG_DECIMAL =
-      BigDecimal.valueOf(-Float.MAX_VALUE);
-  private static final BigDecimal MAX_FLOAT_AS_BIG_DECIMAL =
-      BigDecimal.valueOf(Float.MAX_VALUE);
+    private static final BigDecimal MIN_DOUBLE_AS_BIG_DECIMAL =
+            BigDecimal.valueOf(-Double.MAX_VALUE);
+    private static final BigDecimal MAX_DOUBLE_AS_BIG_DECIMAL =
+            BigDecimal.valueOf(Double.MAX_VALUE);
+    private static final BigDecimal MIN_FLOAT_AS_BIG_DECIMAL =
+            BigDecimal.valueOf(-Float.MAX_VALUE);
+    private static final BigDecimal MAX_FLOAT_AS_BIG_DECIMAL =
+            BigDecimal.valueOf(Float.MAX_VALUE);
 
-  private PDecimal() {
-    super("DECIMAL", Types.DECIMAL, BigDecimal.class, null, 8);
-  }
-
-  @Override
-  public byte[] toBytes(Object object) {
-    if (object == null) {
-      return ByteUtil.EMPTY_BYTE_ARRAY;
+    private PDecimal() {
+        super("DECIMAL", Types.DECIMAL, BigDecimal.class, null, 8);
     }
-    BigDecimal v = (BigDecimal) object;
-    v = NumberUtil.normalize(v);
-    int len = getLength(v);
-    byte[] result = new byte[Math.min(len, MAX_BIG_DECIMAL_BYTES)];
-    PDataType.toBytes(v, result, 0, len);
-    return result;
-  }
 
-  @Override
-  public int toBytes(Object object, byte[] bytes, int offset) {
-    if (object == null) {
-      return 0;
+    @Override
+    public byte[] toBytes(Object object) {
+        if (object == null) {
+            return ByteUtil.EMPTY_BYTE_ARRAY;
+        }
+        BigDecimal v = (BigDecimal) object;
+        v = NumberUtil.normalize(v);
+        int len = getLength(v);
+        byte[] result = new byte[Math.min(len, MAX_BIG_DECIMAL_BYTES)];
+        PDataType.toBytes(v, result, 0, len);
+        return result;
     }
-    BigDecimal v = (BigDecimal) object;
-    v = NumberUtil.normalize(v);
-    int len = getLength(v);
-    return PDataType.toBytes(v, bytes, offset, len);
-  }
 
-  private int getLength(BigDecimal v) {
-    int signum = v.signum();
-    if (signum == 0) { // Special case for zero
-      return 1;
+    @Override
+    public int toBytes(Object object, byte[] bytes, int offset) {
+        if (object == null) {
+            return 0;
+        }
+        BigDecimal v = (BigDecimal) object;
+        v = NumberUtil.normalize(v);
+        int len = getLength(v);
+        return PDataType.toBytes(v, bytes, offset, len);
     }
-            /*
-             * Size of DECIMAL includes:
-             * 1) one byte for exponent
-             * 2) one byte for terminal byte if negative
-             * 3) one byte for every two digits with the following caveats:
-             *    a) add one to round up in the case when there is an odd number of digits
-             *    b) add one in the case that the scale is odd to account for 10x of lowest significant digit
-             *       (basically done to increase the range of exponents that can be represented)
-             */
-    return (signum < 0 ? 2 : 1) + (v.precision() + 1 + (v.scale() % 2 == 0 ? 0 : 1)) / 2;
-  }
 
-  @Override
-  public int estimateByteSize(Object o) {
-    if (o == null) {
-      return 1;
+    private int getLength(BigDecimal v) {
+        int signum = v.signum();
+        if (signum == 0) { // Special case for zero
+            return 1;
+        }
+        /*
+         * Size of DECIMAL includes:
+         * 1) one byte for exponent
+         * 2) one byte for terminal byte if negative
+         * 3) one byte for every two digits with the following caveats:
+         *    a) add one to round up in the case when there is an odd number of digits
+         *    b) add one in the case that the scale is odd to account for 10x of lowest significant digit
+         *       (basically done to increase the range of exponents that can be represented)
+         */
+        return (signum < 0 ? 2 : 1) + (v.precision() + 1 + (v.scale() % 2 == 0 ? 0 : 1)) / 2;
     }
-    BigDecimal v = (BigDecimal) o;
-    // TODO: should we strip zeros and round here too?
-    return Math.min(getLength(v), MAX_BIG_DECIMAL_BYTES);
-  }
 
-  @Override
-  public Integer getMaxLength(Object o) {
-    if (o == null) {
-      return MAX_PRECISION;
+    @Override
+    public int estimateByteSize(Object o) {
+        if (o == null) {
+            return 1;
+        }
+        BigDecimal v = (BigDecimal) o;
+        // TODO: should we strip zeros and round here too?
+        return Math.min(getLength(v), MAX_BIG_DECIMAL_BYTES);
     }
-    BigDecimal v = (BigDecimal) o;
-    return v.precision();
-  }
 
-  @Override
-  public Integer getScale(Object o) {
-    return null;
-  }
+    @Override
+    public Integer getMaxLength(Object o) {
+        if (o == null) {
+            return MAX_PRECISION;
+        }
+        BigDecimal v = (BigDecimal) o;
+        return v.precision();
+    }
 
-  @Override
-  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;
+    @Override
+    public Integer getScale(Object o) {
+        return null;
     }
-    if (actualType == PDecimal.INSTANCE) {
-      if (sortOrder == SortOrder.DESC) {
-        b = SortOrder.invert(b, o, new byte[l], 0, l);
-        o = 0;
-      }
-      return toBigDecimal(b, o, l);
-    } else if (equalsAny(actualType, PDate.INSTANCE, PTime.INSTANCE, PUnsignedDate.INSTANCE,
-        PUnsignedTime.INSTANCE, PLong.INSTANCE, PUnsignedLong.INSTANCE, PInteger.INSTANCE,
-        PUnsignedInt.INSTANCE, PSmallint.INSTANCE, PUnsignedSmallint.INSTANCE, PTinyint.INSTANCE,
-        PUnsignedTinyint.INSTANCE)) {
-      return BigDecimal.valueOf(actualType.getCodec().decodeLong(b, o, sortOrder));
-    } else if (equalsAny(actualType, PFloat.INSTANCE, PUnsignedFloat.INSTANCE)) {
-      return BigDecimal.valueOf(actualType.getCodec().decodeFloat(b, o, sortOrder));
-    } else if (equalsAny(actualType, PDouble.INSTANCE, PUnsignedDouble.INSTANCE)) {
-      return BigDecimal.valueOf(actualType.getCodec().decodeDouble(b, o, sortOrder));
-    } else if (equalsAny(actualType, PTimestamp.INSTANCE, PUnsignedTimestamp.INSTANCE)) {
-      long millisPart = DateUtil.getCodecFor(actualType).decodeLong(b, o, sortOrder);
-      int nanoPart = PUnsignedInt.INSTANCE.getCodec().decodeInt(b, o + Bytes.SIZEOF_LONG, sortOrder);
-      BigDecimal nanosPart = BigDecimal.valueOf(
-          (nanoPart % QueryConstants.MILLIS_TO_NANOS_CONVERTOR)
-              / QueryConstants.MILLIS_TO_NANOS_CONVERTOR);
-      return BigDecimal.valueOf(millisPart).add(nanosPart);
-    } else if (actualType == PBoolean.INSTANCE) {
-      return (Boolean) PBoolean.INSTANCE.toObject(b, o, l, actualType, sortOrder) ?
-          BigDecimal.ONE :
-          BigDecimal.ZERO;
+
+    @Override
+    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;
+        }
+        if (actualType == PDecimal.INSTANCE) {
+            if (sortOrder == SortOrder.DESC) {
+                b = SortOrder.invert(b, o, new byte[l], 0, l);
+                o = 0;
+            }
+            return toBigDecimal(b, o, l);
+        } else if (equalsAny(actualType, PDate.INSTANCE, PTime.INSTANCE, PUnsignedDate.INSTANCE,
+                PUnsignedTime.INSTANCE, PLong.INSTANCE, PUnsignedLong.INSTANCE, PInteger.INSTANCE,
+                PUnsignedInt.INSTANCE, PSmallint.INSTANCE, PUnsignedSmallint.INSTANCE, PTinyint.INSTANCE,
+                PUnsignedTinyint.INSTANCE)) {
+            return BigDecimal.valueOf(actualType.getCodec().decodeLong(b, o, sortOrder));
+        } else if (equalsAny(actualType, PFloat.INSTANCE, PUnsignedFloat.INSTANCE)) {
+            return BigDecimal.valueOf(actualType.getCodec().decodeFloat(b, o, sortOrder));
+        } else if (equalsAny(actualType, PDouble.INSTANCE, PUnsignedDouble.INSTANCE)) {
+            return BigDecimal.valueOf(actualType.getCodec().decodeDouble(b, o, sortOrder));
+        } else if (equalsAny(actualType, PTimestamp.INSTANCE, PUnsignedTimestamp.INSTANCE)) {
+            long millisPart = DateUtil.getCodecFor(actualType).decodeLong(b, o, sortOrder);
+            int nanoPart = PUnsignedInt.INSTANCE.getCodec().decodeInt(b, o + Bytes.SIZEOF_LONG, sortOrder);
+            BigDecimal nanosPart = BigDecimal.valueOf(
+                    (nanoPart % QueryConstants.MILLIS_TO_NANOS_CONVERTOR)
+                    / QueryConstants.MILLIS_TO_NANOS_CONVERTOR);
+            return BigDecimal.valueOf(millisPart).add(nanosPart);
+        } else if (actualType == PBoolean.INSTANCE) {
+            return (Boolean) PBoolean.INSTANCE.toObject(b, o, l, actualType, sortOrder) ?
+                    BigDecimal.ONE :
+                        BigDecimal.ZERO;
+        }
+        return throwConstraintViolationException(actualType, this);
     }
-    return throwConstraintViolationException(actualType, this);
-  }
 
-  @Override
-  public Object toObject(Object object, PDataType actualType) {
-    if (object == null) {
-      return null;
+    @Override
+    public Object toObject(Object object, PDataType actualType) {
+        if (object == null) {
+            return null;
+        }
+        if (equalsAny(actualType, PInteger.INSTANCE, PUnsignedInt.INSTANCE)) {
+            return BigDecimal.valueOf((Integer) object);
+        } else if (equalsAny(actualType, PLong.INSTANCE, PUnsignedLong.INSTANCE)) {
+            return BigDecimal.valueOf((Long) object);
+        } else if (equalsAny(actualType, PSmallint.INSTANCE, PUnsignedSmallint.INSTANCE)) {
+            return BigDecimal.valueOf((Short) object);
+        } else if (equalsAny(actualType, PTinyint.INSTANCE, PUnsignedTinyint.INSTANCE)) {
+            return BigDecimal.valueOf((Byte) object);
+        } else if (equalsAny(actualType, PFloat.INSTANCE, PUnsignedFloat.INSTANCE)) {
+            return BigDecimal.valueOf((Float) object);
+        } else if (equalsAny(actualType, PDouble.INSTANCE, PUnsignedDouble.INSTANCE)) {
+            return BigDecimal.valueOf((Double) object);
+        } else if (actualType == PDecimal.INSTANCE) {
+            return object;
+        } else if (equalsAny(actualType, PDate.INSTANCE, PUnsignedDate.INSTANCE, PTime.INSTANCE,
+                PUnsignedTime.INSTANCE)) {
+            java.util.Date d = (java.util.Date) object;
+            return BigDecimal.valueOf(d.getTime());
+        } else if (equalsAny(actualType, PTimestamp.INSTANCE,
+                PUnsignedTimestamp.INSTANCE)) {
+            Timestamp ts = (Timestamp) object;
+            long millisPart = ts.getTime();
+            BigDecimal nanosPart = BigDecimal.valueOf(
+                    (ts.getNanos() % QueryConstants.MILLIS_TO_NANOS_CONVERTOR)
+                    / QueryConstants.MILLIS_TO_NANOS_CONVERTOR);
+            BigDecimal value = BigDecimal.valueOf(millisPart).add(nanosPart);
+            return value;
+        } else if (actualType == PBoolean.INSTANCE) {
+            return ((Boolean) object) ? BigDecimal.ONE : BigDecimal.ZERO;
+        }
+        return throwConstraintViolationException(actualType, this);
     }
-    if (equalsAny(actualType, PInteger.INSTANCE, PUnsignedInt.INSTANCE)) {
-      return BigDecimal.valueOf((Integer) object);
-    } else if (equalsAny(actualType, PLong.INSTANCE, PUnsignedLong.INSTANCE)) {
-      return BigDecimal.valueOf((Long) object);
-    } else if (equalsAny(actualType, PSmallint.INSTANCE, PUnsignedSmallint.INSTANCE)) {
-      return BigDecimal.valueOf((Short) object);
-    } else if (equalsAny(actualType, PTinyint.INSTANCE, PUnsignedTinyint.INSTANCE)) {
-      return BigDecimal.valueOf((Byte) object);
-    } else if (equalsAny(actualType, PFloat.INSTANCE, PUnsignedFloat.INSTANCE)) {
-      return BigDecimal.valueOf((Float) object);
-    } else if (equalsAny(actualType, PDouble.INSTANCE, PUnsignedDouble.INSTANCE)) {
-      return BigDecimal.valueOf((Double) object);
-    } else if (actualType == PDecimal.INSTANCE) {
-      return object;
-    } else if (equalsAny(actualType, PDate.INSTANCE, PUnsignedDate.INSTANCE, PTime.INSTANCE,
-        PUnsignedTime.INSTANCE)) {
-      java.util.Date d = (java.util.Date) object;
-      return BigDecimal.valueOf(d.getTime());
-    } else if (equalsAny(actualType, PTimestamp.INSTANCE,
-        PUnsignedTimestamp.INSTANCE)) {
-      Timestamp ts = (Timestamp) object;
-      long millisPart = ts.getTime();
-      BigDecimal nanosPart = BigDecimal.valueOf(
-          (ts.getNanos() % QueryConstants.MILLIS_TO_NANOS_CONVERTOR)
-              / QueryConstants.MILLIS_TO_NANOS_CONVERTOR);
-      BigDecimal value = BigDecimal.valueOf(millisPart).add(nanosPart);
-      return value;
-    } else if (actualType == PBoolean.INSTANCE) {
-      return ((Boolean) object) ? BigDecimal.ONE : BigDecimal.ZERO;
+
+    @Override
+    public boolean isFixedWidth() {
+        return false;
     }
-    return throwConstraintViolationException(actualType, this);
-  }
 
-  @Override
-  public boolean isFixedWidth() {
-    return false;
-  }
+    @Override
+    public Integer getByteSize() {
+        return MAX_BIG_DECIMAL_BYTES;
+    }
 
-  @Override
-  public Integer getByteSize() {
-    return MAX_BIG_DECIMAL_BYTES;
-  }
+    @Override
+    public int compareTo(Object lhs, Object rhs, PDataType rhsType) {
+        if (rhsType == PDecimal.INSTANCE) {
+            return ((BigDecimal) lhs).compareTo((BigDecimal) rhs);
+        }
+        return -rhsType.compareTo(rhs, lhs, this);
+    }
 
-  @Override
-  public int compareTo(Object lhs, Object rhs, PDataType rhsType) {
-    if (rhsType == PDecimal.INSTANCE) {
-      return ((BigDecimal) lhs).compareTo((BigDecimal) rhs);
+    @Override
+    public boolean isCastableTo(PDataType targetType) {
+        return super.isCastableTo(targetType) || targetType.isCoercibleTo(
+                PTimestamp.INSTANCE) || targetType.equals(PBoolean.INSTANCE);
     }
-    return -rhsType.compareTo(rhs, lhs, this);
-  }
 
-  @Override
-  public boolean isCastableTo(PDataType targetType) {
-    return super.isCastableTo(targetType) || targetType.isCoercibleTo(
-        PTimestamp.INSTANCE) || targetType.equals(PBoolean.INSTANCE);
-  }
+    @Override
+    public boolean isCoercibleTo(PDataType targetType, Object value) {
+        if (value != null) {
+            BigDecimal bd;
+            if (equalsAny(targetType, PUnsignedLong.INSTANCE, PUnsignedInt.INSTANCE,
+                    PUnsignedSmallint.INSTANCE, PUnsignedTinyint.INSTANCE)) {
+                bd = (BigDecimal) value;
+                if (bd.signum() == -1) {
+                    return false;
+                }
+            } else if (targetType.equals(PLong.INSTANCE)) {
+                bd = (BigDecimal) value;
+                try {
+                    bd.longValueExact();
+                    return true;
+                } catch (ArithmeticException e) {
+                    return false;
+                }
+            } else if (targetType.equals(PInteger.INSTANCE)) {
+                bd = (BigDecimal) value;
+                try {
+                    bd.intValueExact();
+                    return true;
+                } catch (ArithmeticException e) {
+                    return false;
+                }
+            } else if (targetType.equals(PSmallint.INSTANCE)) {
+                bd = (BigDecimal) value;
+                try {
+                    bd.shortValueExact();
+                    return true;
+                } catch (ArithmeticException e) {
+                    return false;
+                }
+            } else if (targetType.equals(PTinyint.INSTANCE)) {
+                bd = (BigDecimal) value;
+                try {
+                    bd.byteValueExact();
+                    return true;
+                } catch (ArithmeticException e) {
+                    return false;
+                }
+            } else if (targetType.equals(PUnsignedFloat.INSTANCE)) {
+                bd = (BigDecimal) value;
+                try {
+                    BigDecimal maxFloat = MAX_FLOAT_AS_BIG_DECIMAL;
+                    boolean isNegtive = (bd.signum() == -1);
+                    return bd.compareTo(maxFloat) <= 0 && !isNegtive;
+                } catch (Exception e) {
+                    return false;
+                }
+            } else if (targetType.equals(PFloat.INSTANCE)) {
+                bd = (BigDecimal) value;
+                try {
+                    BigDecimal maxFloat = MAX_FLOAT_AS_BIG_DECIMAL;
+                    // Float.MIN_VALUE should not be used here, as this is the
+                    // smallest in terms of closest to zero.
+                    BigDecimal minFloat = MIN_FLOAT_AS_BIG_DECIMAL;
+                    return bd.compareTo(maxFloat) <= 0 && bd.compareTo(minFloat) >= 0;
+                } catch (Exception e) {
+                    return false;
+                }
+            } else if (targetType.equals(PUnsignedDouble.INSTANCE)) {
+                bd = (BigDecimal) value;
+                try {
+                    BigDecimal maxDouble = MAX_DOUBLE_AS_BIG_DECIMAL;
+                    boolean isNegtive = (bd.signum() == -1);
+                    return bd.compareTo(maxDouble) <= 0 && !isNegtive;
+                } catch (Exception e) {
+                    return false;
+                }
+            } else if (targetType.equals(PDouble.INSTANCE)) {
+                bd = (BigDecimal) value;
+                try {
+                    BigDecimal maxDouble = MAX_DOUBLE_AS_BIG_DECIMAL;
+                    BigDecimal minDouble = MIN_DOUBLE_AS_BIG_DECIMAL;
+                    return bd.compareTo(maxDouble) <= 0 && bd.compareTo(minDouble) >= 0;
+                } catch (Exception e) {
+                    return false;
+                }
+            }
+        }
+        return super.isCoercibleTo(targetType, value);
+    }
 
-  @Override
-  public boolean isCoercibleTo(PDataType targetType, Object value) {
-    if (value != null) {
-      BigDecimal bd;
-      if (equalsAny(targetType, PUnsignedLong.INSTANCE, PUnsignedInt.INSTANCE,
-          PUnsignedSmallint.INSTANCE, PUnsignedTinyint.INSTANCE)) {
-        bd = (BigDecimal) value;
-        if (bd.signum() == -1) {
-          return false;
+    @Override
+    public boolean isSizeCompatible(ImmutableBytesWritable ptr, Object value, PDataType srcType,
+            SortOrder sortOrder, Integer maxLength, Integer scale, Integer desiredMaxLength, Integer desiredScale) {
+        if (ptr.getLength() == 0) {
+            return true;
         }
-      } else if (targetType.equals(PLong.INSTANCE)) {
-        bd = (BigDecimal) value;
-        try {
-          bd.longValueExact();
-          return true;
-        } catch (ArithmeticException e) {
-          return false;
+        // Any numeric type fits into a DECIMAL
+        if (srcType != PDecimal.INSTANCE) {
+            if(!srcType.isCoercibleTo(this)) {
+                throw new IllegalArgumentException(TypeMismatchException.newException(srcType, this));
+            }
+            return true;
         }
-      } else if (targetType.equals(PInteger.INSTANCE)) {
-        bd = (BigDecimal) value;
-        try {
-          bd.intValueExact();
-          return true;
-        } catch (ArithmeticException e) {
-          return false;
+        // Use the scale from the value if provided, as it prevents a deserialization.
+        // The maxLength and scale for the underlying expression are ignored, because they
+        // are not relevant in this case: for example a DECIMAL(10,2) may be assigned to a
+        // DECIMAL(5,0) as long as the value fits.
+        if (value != null) {
+            BigDecimal v = (BigDecimal) value;
+            maxLength = v.precision();
+            scale = v.scale();
+        } else {
+            this.coerceBytes(ptr, value, srcType, maxLength, scale, SortOrder.getDefault(), desiredMaxLength, desiredScale, sortOrder, true);
+            int[] v = getDecimalPrecisionAndScale(ptr.get(), ptr.getOffset(), ptr.getLength());
+            maxLength = v[0];
+            scale = v[1];
         }
-      } else if (targetType.equals(PSmallint.INSTANCE)) {
-        bd = (BigDecimal) value;
-        try {
-          bd.shortValueExact();
-          return true;
-        } catch (ArithmeticException e) {
-          return false;
+        if (desiredMaxLength != null && desiredScale != null && maxLength != null && scale != null &&
+                ((desiredScale == null && desiredMaxLength < maxLength) ||
+                        (desiredMaxLength - desiredScale) < (maxLength - scale))) {
+            return false;
         }
-      } else if (targetType.equals(PTinyint.INSTANCE)) {
-        bd = (BigDecimal) value;
-        try {
-          bd.byteValueExact();
-          return true;
-        } catch (ArithmeticException e) {
-          return false;
-        }
-      } else if (targetType.equals(PUnsignedFloat.INSTANCE)) {
-        bd = (BigDecimal) value;
-        try {
-          BigDecimal maxFloat = MAX_FLOAT_AS_BIG_DECIMAL;
-          boolean isNegtive = (bd.signum() == -1);
-          return bd.compareTo(maxFloat) <= 0 && !isNegtive;
-        } catch (Exception e) {
-          return false;
+        return true;
+    }
+
+    @Override
+    public void coerceBytes(ImmutableBytesWritable ptr, Object object, PDataType actualType,
+            Integer maxLength, Integer scale, SortOrder actualModifier, Integer desiredMaxLength, Integer desiredScale,
+            SortOrder expectedModifier) {
+        if (desiredScale == null) {
+            // deiredScale not available, or we do not have scale requirement, delegate to parents.
+            super.coerceBytes(ptr, object, actualType, maxLength, scale, actualModifier, desiredMaxLength,
+                    desiredScale, expectedModifier);
+            return;
         }
-      } else if (targetType.equals(PFloat.INSTANCE)) {
-        bd = (BigDecimal) value;
-        try {
-          BigDecimal maxFloat = MAX_FLOAT_AS_BIG_DECIMAL;
-          // Float.MIN_VALUE should not be used here, as this is the
-          // smallest in terms of closest to zero.
-          BigDecimal minFloat = MIN_FLOAT_AS_BIG_DECIMAL;
-          return bd.compareTo(maxFloat) <= 0 && bd.compareTo(minFloat) >= 0;
-        } catch (Exception e) {
-          return false;
+        if (ptr.getLength() == 0) {
+            return;
         }
-      } else if (targetType.equals(PUnsignedDouble.INSTANCE)) {
-        bd = (BigDecimal) value;
-        try {
-          BigDecimal maxDouble = MAX_DOUBLE_AS_BIG_DECIMAL;
-          boolean isNegtive = (bd.signum() == -1);
-          return bd.compareTo(maxDouble) <= 0 && !isNegtive;
-        } catch (Exception e) {
-          return false;
+        if (scale == null) {
+            if (object != null) {
+                BigDecimal v = (BigDecimal) object;
+                scale = v.scale();
+            } else {
+                int[] v = getDecimalPrecisionAndScale(ptr.get(), ptr.getOffset(), ptr.getLength());
+                scale = v[1];
+            }
         }
-      } else if (targetType.equals(PDouble.INSTANCE)) {
-        bd = (BigDecimal) value;
-        try {
-          BigDecimal maxDouble = MAX_DOUBLE_AS_BIG_DECIMAL;
-          BigDecimal minDouble = MIN_DOUBLE_AS_BIG_DECIMAL;
-          return bd.compareTo(maxDouble) <= 0 && bd.compareTo(minDouble) >= 0;
-        } catch (Exception e) {
-          return false;
+        if (this == actualType && scale <= desiredScale) {
+            // No coerce and rescale necessary
+            return;
+        } else {
+            BigDecimal decimal;
+            // Rescale is necessary.
+            if (object != null) { // value object is passed in.
+                decimal = (BigDecimal) toObject(object, actualType);
+            } else { // only value bytes is passed in, need to convert to object first.
+                decimal = (BigDecimal) toObject(ptr);
+            }
+            decimal = decimal.setScale(desiredScale, BigDecimal.ROUND_DOWN);
+            ptr.set(toBytes(decimal));
         }
-      }
     }
-    return super.isCoercibleTo(targetType, value);
-  }
 
-  @Override
-  public boolean isSizeCompatible(ImmutableBytesWritable ptr, Object value, PDataType srcType,
-      Integer maxLength, Integer scale, Integer desiredMaxLength, Integer desiredScale) {
-    if (ptr.getLength() == 0) {
-      return true;
-    }
-    // Any numeric type fits into a DECIMAL
-    if (srcType != PDecimal.INSTANCE) {
-        if(!srcType.isCoercibleTo(this)) {
-            throw new IllegalArgumentException(TypeMismatchException.newException(srcType, this));
+    @Override
+    public Object toObject(String value) {
+        if (value == null || value.length() == 0) {
+            return null;
+        }
+        try {
+            return new BigDecimal(value);
+        } catch (NumberFormatException e) {
+            throw newIllegalDataException(e);
         }
-        return true;
-    }
-    // Use the scale from the value if provided, as it prevents a deserialization.
-    // The maxLength and scale for the underlying expression are ignored, because they
-    // are not relevant in this case: for example a DECIMAL(10,2) may be assigned to a
-    // DECIMAL(5,0) as long as the value fits.
-    if (value != null) {
-      BigDecimal v = (BigDecimal) value;
-      maxLength = v.precision();
-      scale = v.scale();
-    } else {
-      int[] v = getDecimalPrecisionAndScale(ptr.get(), ptr.getOffset(), ptr.getLength());
-      maxLength = v[0];
-      scale = v[1];
-    }
-    if (desiredMaxLength != null && desiredScale != null && maxLength != null && scale != null &&
-        ((desiredScale == null && desiredMaxLength < maxLength) ||
-            (desiredMaxLength - desiredScale) < (maxLength - scale))) {
-      return false;
-    }
-    return true;
-  }
-
-  @Override
-  public void coerceBytes(ImmutableBytesWritable ptr, Object object, PDataType actualType,
-      Integer maxLength, Integer scale, SortOrder actualModifier, Integer desiredMaxLength, Integer desiredScale,
-      SortOrder expectedModifier) {
-    if (desiredScale == null) {
-      // deiredScale not available, or we do not have scale requirement, delegate to parents.
-      super.coerceBytes(ptr, object, actualType, maxLength, scale, actualModifier, desiredMaxLength,
-          desiredScale, expectedModifier);
-      return;
-    }
-    if (ptr.getLength() == 0) {
-      return;
-    }
-    if (scale == null) {
-      if (object != null) {
-        BigDecimal v = (BigDecimal) object;
-        scale = v.scale();
-      } else {
-        int[] v = getDecimalPrecisionAndScale(ptr.get(), ptr.getOffset(), ptr.getLength());
-        scale = v[1];
-      }
-    }
-    if (this == actualType && scale <= desiredScale) {
-      // No coerce and rescale necessary
-      return;
-    } else {
-      BigDecimal decimal;
-      // Rescale is necessary.
-      if (object != null) { // value object is passed in.
-        decimal = (BigDecimal) toObject(object, actualType);
-      } else { // only value bytes is passed in, need to convert to object first.
-        decimal = (BigDecimal) toObject(ptr);
-      }
-      decimal = decimal.setScale(desiredScale, BigDecimal.ROUND_DOWN);
-      ptr.set(toBytes(decimal));
     }
-  }
 
-  @Override
-  public Object toObject(String value) {
-    if (value == null || value.length() == 0) {
-      return null;
-    }
-    try {
-      return new BigDecimal(value);
-    } catch (NumberFormatException e) {
-      throw newIllegalDataException(e);
+    @Override
+    public Integer estimateByteSizeFromLength(Integer length) {
+        // No association of runtime byte size from decimal precision.
+        return null;
     }
-  }
-
-  @Override
-  public Integer estimateByteSizeFromLength(Integer length) {
-    // No association of runtime byte size from decimal precision.
-    return null;
-  }
 
-  @Override
-  public String toStringLiteral(byte[] b, int offset, int length, Format formatter) {
-    if (formatter == null) {
-      BigDecimal o = (BigDecimal) toObject(b, offset, length);
-      return o.toPlainString();
+    @Override
+    public String toStringLiteral(byte[] b, int offset, int length, Format formatter) {
+        if (formatter == null) {
+            BigDecimal o = (BigDecimal) toObject(b, offset, length);
+            return o.toPlainString();
+        }
+        return super.toStringLiteral(b, offset, length, formatter);
     }
-    return super.toStringLiteral(b, offset, length, formatter);
-  }
 
-  @Override
-  public String toStringLiteral(Object o, Format formatter) {
-      if (formatter == null) {
-          if(o == null) {
-              return String.valueOf(o);
-          }
-          return ((BigDecimal)o).toPlainString();
+    @Override
+    public String toStringLiteral(Object o, Format formatter) {
+        if (formatter == null) {
+            if(o == null) {
+                return String.valueOf(o);
+            }
+            return ((BigDecimal)o).toPlainString();
         }
         return super.toStringLiteral(o, formatter);
-  }
+    }
 
-  @Override
-  public Object getSampleValue(Integer maxLength, Integer arrayLength) {
-    return new BigDecimal((Long) PLong.INSTANCE.getSampleValue(maxLength, arrayLength));
-  }
+    @Override
+    public Object getSampleValue(Integer maxLength, Integer arrayLength) {
+        return new BigDecimal((Long) PLong.INSTANCE.getSampleValue(maxLength, arrayLength));
+    }
 
     // take details from org.apache.phoenix.schema.types.PDataType#toBigDecimal(byte[], int, int)
     @Override

http://git-wip-us.apache.org/repos/asf/phoenix/blob/bb88e9f5/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PVarbinary.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PVarbinary.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PVarbinary.java
index aafa1c6..d96650d 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PVarbinary.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PVarbinary.java
@@ -20,7 +20,6 @@ package org.apache.phoenix.schema.types;
 import java.sql.Types;
 import java.text.Format;
 
-import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.hadoop.hbase.util.Base64;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.phoenix.schema.SortOrder;
@@ -28,149 +27,138 @@ import org.apache.phoenix.util.ByteUtil;
 
 public class PVarbinary extends PBinaryBase {
 
-  public static final PVarbinary INSTANCE = new PVarbinary();
+    public static final PVarbinary INSTANCE = new PVarbinary();
 
-  private PVarbinary() {
-    super("VARBINARY", Types.VARBINARY, byte[].class, null, 22);
-  }
+    private PVarbinary() {
+        super("VARBINARY", Types.VARBINARY, byte[].class, null, 22);
+    }
+
+    @Override
+    public byte[] toBytes(Object object) {
+        if (object == null) {
+            return ByteUtil.EMPTY_BYTE_ARRAY;
+        }
+        return (byte[]) object;
+    }
 
-  @Override
-  public byte[] toBytes(Object object) {
-    if (object == null) {
-      return ByteUtil.EMPTY_BYTE_ARRAY;
+    @Override
+    public int toBytes(Object object, byte[] bytes, int offset) {
+        if (object == null) {
+            return 0;
+        }
+        byte[] o = (byte[]) object;
+        // assumes there's enough room
+        System.arraycopy(bytes, offset, o, 0, o.length);
+        return o.length;
     }
-    return (byte[]) object;
-  }
 
-  @Override
-  public int toBytes(Object object, byte[] bytes, int offset) {
-    if (object == null) {
-      return 0;
+    /**
+     * Override because we must always create a new byte array
+     */
+    @Override
+    public byte[] toBytes(Object object, SortOrder sortOrder) {
+        byte[] bytes = toBytes(object);
+        // Override because we need to allocate a new buffer in this case
+        if (sortOrder == SortOrder.DESC) {
+            return SortOrder.invert(bytes, 0, new byte[bytes.length], 0, bytes.length);
+        }
+        return bytes;
     }
-    byte[] o = (byte[]) object;
-    // assumes there's enough room
-    System.arraycopy(bytes, offset, o, 0, o.length);
-    return o.length;
-  }
-
-  /**
-   * Override because we must always create a new byte array
-   */
-  @Override
-  public byte[] toBytes(Object object, SortOrder sortOrder) {
-    byte[] bytes = toBytes(object);
-    // Override because we need to allocate a new buffer in this case
-    if (sortOrder == SortOrder.DESC) {
-      return SortOrder.invert(bytes, 0, new byte[bytes.length], 0, bytes.length);
+
+    @Override
+    public Object toObject(byte[] bytes, int offset, int length, PDataType actualType,
+            SortOrder sortOrder, Integer maxLength, Integer scale) {
+        if (length == 0) {
+            return null;
+        }
+        if (offset == 0 && bytes.length == length && sortOrder == SortOrder.ASC) {
+            return bytes;
+        }
+        byte[] bytesCopy = new byte[length];
+        System.arraycopy(bytes, offset, bytesCopy, 0, length);
+        if (sortOrder == SortOrder.DESC) {
+            bytesCopy = SortOrder.invert(bytes, offset, bytesCopy, 0, length);
+            offset = 0;
+        }
+        return bytesCopy;
     }
-    return bytes;
-  }
-
-  @Override
-  public Object toObject(byte[] bytes, int offset, int length, PDataType actualType,
-      SortOrder sortOrder, Integer maxLength, Integer scale) {
-    if (length == 0) {
-      return null;
+
+    @Override
+    public Object toObject(Object object, PDataType actualType) {
+        return actualType.toBytes(object);
     }
-    if (offset == 0 && bytes.length == length && sortOrder == SortOrder.ASC) {
-      return bytes;
+
+    @Override
+    public boolean isFixedWidth() {
+        return false;
     }
-    byte[] bytesCopy = new byte[length];
-    System.arraycopy(bytes, offset, bytesCopy, 0, length);
-    if (sortOrder == SortOrder.DESC) {
-      bytesCopy = SortOrder.invert(bytes, offset, bytesCopy, 0, length);
-      offset = 0;
+
+    @Override
+    public int estimateByteSize(Object o) {
+        byte[] value = (byte[]) o;
+        return value == null ? 1 : value.length;
     }
-    return bytesCopy;
-  }
-
-  @Override
-  public Object toObject(Object object, PDataType actualType) {
-    return actualType.toBytes(object);
-  }
-
-  @Override
-  public boolean isFixedWidth() {
-    return false;
-  }
-
-  @Override
-  public int estimateByteSize(Object o) {
-    byte[] value = (byte[]) o;
-    return value == null ? 1 : value.length;
-  }
-
-  @Override
-  public Integer getByteSize() {
-    return null;
-  }
-
-  @Override
-  public boolean isCoercibleTo(PDataType targetType) {
-    return equalsAny(targetType, this, PBinary.INSTANCE);
-  }
-
-  @Override
-  public boolean isSizeCompatible(ImmutableBytesWritable ptr, Object value, PDataType srcType,
-      Integer maxLength, Integer scale, Integer desiredMaxLength,
-      Integer desiredScale) {
-    if (ptr.getLength() != 0 && srcType.equals(PBinary.INSTANCE) && maxLength != null
-        && desiredMaxLength != null) {
-      return maxLength <= desiredMaxLength;
+
+    @Override
+    public Integer getByteSize() {
+        return null;
     }
-    return true;
-  }
-
-  @Override
-  public int compareTo(Object lhs, Object rhs, PDataType rhsType) {
-    if (lhs == null && rhs == null) {
-      return 0;
-    } else if (lhs == null) {
-      return -1;
-    } else if (rhs == null) {
-      return 1;
+
+    @Override
+    public boolean isCoercibleTo(PDataType targetType) {
+        return equalsAny(targetType, this, PBinary.INSTANCE);
     }
-    if (equalsAny(rhsType, this, PBinary.INSTANCE)) {
-      return Bytes.compareTo((byte[]) lhs, (byte[]) rhs);
-    } else {
-      byte[] rhsBytes = rhsType.toBytes(rhs);
-      return Bytes.compareTo((byte[]) lhs, rhsBytes);
+
+    @Override
+    public int compareTo(Object lhs, Object rhs, PDataType rhsType) {
+        if (lhs == null && rhs == null) {
+            return 0;
+        } else if (lhs == null) {
+            return -1;
+        } else if (rhs == null) {
+            return 1;
+        }
+        if (equalsAny(rhsType, this, PBinary.INSTANCE)) {
+            return Bytes.compareTo((byte[]) lhs, (byte[]) rhs);
+        } else {
+            byte[] rhsBytes = rhsType.toBytes(rhs);
+            return Bytes.compareTo((byte[]) lhs, rhsBytes);
+        }
     }
-  }
 
-  @Override
-  public Object toObject(String value) {
-    if (value == null || value.length() == 0) {
-      return null;
+    @Override
+    public Object toObject(String value) {
+        if (value == null || value.length() == 0) {
+            return null;
+        }
+        return Base64.decode(value);
     }
-    return Base64.decode(value);
-  }
-
-  @Override
-  public String toStringLiteral(byte[] b, int o, int length, Format formatter) {
-    StringBuilder buf = new StringBuilder();
-    buf.append('[');
-    if (length > 0) {
-        for (int i = o; i < length; i++) {
-          buf.append(0xFF & b[i]);
-          buf.append(',');
+
+    @Override
+    public String toStringLiteral(byte[] b, int o, int length, Format formatter) {
+        StringBuilder buf = new StringBuilder();
+        buf.append('[');
+        if (length > 0) {
+            for (int i = o; i < length; i++) {
+                buf.append(0xFF & b[i]);
+                buf.append(',');
+            }
+            buf.setLength(buf.length()-1);
         }
-        buf.setLength(buf.length()-1);
+        buf.append(']');
+        return buf.toString();
+    }
+
+    @Override
+    public String toStringLiteral(Object o, Format formatter) {
+        return toStringLiteral((byte[])o, 0, ((byte[]) o).length, formatter);
+    }
+
+    @Override
+    public Object getSampleValue(Integer maxLength, Integer arrayLength) {
+        int length = maxLength != null && maxLength > 0 ? maxLength : 1;
+        byte[] b = new byte[length];
+        RANDOM.get().nextBytes(b);
+        return b;
     }
-    buf.append(']');
-    return buf.toString();
-  }
-
-  @Override
-  public String toStringLiteral(Object o, Format formatter) {
-      return toStringLiteral((byte[])o, 0, ((byte[]) o).length, formatter);
-  }
-  
-  @Override
-  public Object getSampleValue(Integer maxLength, Integer arrayLength) {
-    int length = maxLength != null && maxLength > 0 ? maxLength : 1;
-    byte[] b = new byte[length];
-    RANDOM.get().nextBytes(b);
-    return b;
-  }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/bb88e9f5/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PVarchar.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PVarchar.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PVarchar.java
index 2575115..0ddf622 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PVarchar.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PVarchar.java
@@ -30,134 +30,142 @@ import com.google.common.base.Preconditions;
 
 public class PVarchar extends PDataType<String> {
 
-  public static final PVarchar INSTANCE = new PVarchar();
-
-  private PVarchar() {
-    super("VARCHAR", Types.VARCHAR, String.class, null, 0);
-  }
-
-  @Override
-  public byte[] toBytes(Object object) {
-    // TODO: consider using avro UTF8 object instead of String
-    // so that we get get the size easily
-    if (object == null) {
-      return ByteUtil.EMPTY_BYTE_ARRAY;
-    }
-    return Bytes.toBytes((String) object);
-  }
-
-  @Override
-  public int toBytes(Object object, byte[] bytes, int offset) {
-    if (object == null) {
-      return 0;
-    }
-    byte[] b = toBytes(object); // TODO: no byte[] allocation: use CharsetEncoder
-    System.arraycopy(b, 0, bytes, offset, b.length);
-    return b.length;
-  }
-
-  @Override
-  public Object toObject(byte[] bytes, int offset, int length, PDataType actualType,
-      SortOrder sortOrder, Integer maxLength, Integer scale) {
-    if (length == 0) {
-      return null;
-    }
-    if (!actualType.isCoercibleTo(this)) {
-      throwConstraintViolationException(actualType, this);
-    }
-    if (sortOrder == SortOrder.DESC) {
-      bytes = SortOrder.invert(bytes, offset, length);
-      offset = 0;
-    }
-    return Bytes.toString(bytes, offset, length);
-  }
-
-  @Override
-  public Object toObject(Object object, PDataType actualType) {
-    if (equalsAny(actualType, this, PChar.INSTANCE)) {
-      String s = (String) object;
-      return s == null || s.length() > 0 ? s : null;
-    }
-    return throwConstraintViolationException(actualType, this);
-  }
-
-  @Override
-  public boolean isCoercibleTo(PDataType targetType) {
-    return equalsAny(targetType, this, PChar.INSTANCE, PVarbinary.INSTANCE, PBinary.INSTANCE);
-  }
-
-  @Override
-  public boolean isCoercibleTo(PDataType targetType, Object value) {
-    if (isCoercibleTo(targetType)) {
-      if (targetType.equals(PChar.INSTANCE)) {
-        return value != null;
-      }
-      return true;
-    }
-    return false;
-  }
-
-  @Override
-  public boolean isSizeCompatible(ImmutableBytesWritable ptr, Object value, PDataType srcType,
-      Integer maxLength, Integer scale, Integer desiredMaxLength,
-      Integer desiredScale) {
-    if (ptr.getLength() != 0 && maxLength != null && desiredMaxLength != null) {
-      return maxLength <= desiredMaxLength;
-    }
-    return true;
-  }
-
-  @Override
-  public boolean isFixedWidth() {
-    return false;
-  }
-
-  @Override
-  public int estimateByteSize(Object o) {
-    String value = (String) o;
-    return value == null ? 1 : value.length();
-  }
-
-  @Override
-  public Integer getByteSize() {
-    return null;
-  }
-
-  @Override
-  public int compareTo(Object lhs, Object rhs, PDataType rhsType) {
-    return ((String) lhs).compareTo((String) rhs);
-  }
-
-  @Override
-  public Object toObject(String value) {
-    return value;
-  }
-
-  @Override
-  public boolean isBytesComparableWith(PDataType otherType) {
-    return super.isBytesComparableWith(otherType) || otherType == PChar.INSTANCE;
-  }
-
-  @Override
-  public String toStringLiteral(Object o, Format formatter) {
-    if (formatter != null) {
-      return "'" + formatter.format(o) + "'";
-    }
-    return null == o ? String.valueOf(o) : "'" + StringUtil.escapeStringConstant(o.toString()) + "'";
-  }
-
-  private char[] sampleChars = new char[1];
-
-  @Override
-  public Object getSampleValue(Integer maxLength, Integer arrayLength) {
-    Preconditions.checkArgument(maxLength == null || maxLength >= 0);
-    int length = maxLength != null ? maxLength : 1;
-    if (length != sampleChars.length) {
-      sampleChars = new char[length];
-    }
-    for (int i = 0; i < length; i++) {
-      sampleChars[i] = (char) RANDOM.get().nextInt(Byte.MAX_VALUE);
-    }
-    return new String(sampleChars);
-  }
+    public static final PVarchar INSTANCE = new PVarchar();
+
+    private PVarchar() {
+        super("VARCHAR", Types.VARCHAR, String.class, null, 0);
+    }
+
+    @Override
+    public byte[] toBytes(Object object) {
+        // TODO: consider using avro UTF8 object instead of String
+        // so that we get get the size easily
+        if (object == null) {
+            return ByteUtil.EMPTY_BYTE_ARRAY;
+        }
+        return Bytes.toBytes((String) object);
+    }
+
+    @Override
+    public int toBytes(Object object, byte[] bytes, int offset) {
+        if (object == null) {
+            return 0;
+        }
+        byte[] b = toBytes(object); // TODO: no byte[] allocation: use CharsetEncoder
+        System.arraycopy(b, 0, bytes, offset, b.length);
+        return b.length;
+    }
+
+    @Override
+    public Object toObject(byte[] bytes, int offset, int length, PDataType actualType,
+            SortOrder sortOrder, Integer maxLength, Integer scale) {
+        if (length == 0) {
+            return null;
+        }
+        if (!actualType.isCoercibleTo(this)) {
+            throwConstraintViolationException(actualType, this);
+        }
+        if (sortOrder == SortOrder.DESC) {
+            bytes = SortOrder.invert(bytes, offset, length);
+            offset = 0;
+        }
+        return Bytes.toString(bytes, offset, length);
+    }
+
+    @Override
+    public Object toObject(Object object, PDataType actualType) {
+        if (equalsAny(actualType, this, PChar.INSTANCE)) {
+            String s = (String) object;
+            return s == null || s.length() > 0 ? s : null;
+        }
+        return throwConstraintViolationException(actualType, this);
+    }
+
+    @Override
+    public boolean isCoercibleTo(PDataType targetType) {
+        return equalsAny(targetType, this, PChar.INSTANCE, PVarbinary.INSTANCE, PBinary.INSTANCE);
+    }
+
+    @Override
+    public boolean isCoercibleTo(PDataType targetType, Object value) {
+        if (isCoercibleTo(targetType)) {
+            if (targetType.equals(PChar.INSTANCE)) {
+                return value != null;
+            }
+            return true;
+        }
+        return false;
+    }
+
+    @Override
+    public boolean isSizeCompatible(ImmutableBytesWritable ptr, Object value, PDataType srcType,
+            SortOrder sortOrder, Integer maxLength, Integer scale,
+            Integer desiredMaxLength, Integer desiredScale) {
+        if (ptr.getLength() != 0 && desiredMaxLength != null) {
+            if (maxLength == null) {
+                if (value != null) { // Use value if provided
+                    maxLength = value.toString().length();
+                } else {
+                    coerceBytes(ptr, value, srcType, maxLength, scale, sortOrder, desiredMaxLength, desiredScale, sortOrder, true);
+                    maxLength = StringUtil.calculateUTF8Length(ptr.get(), ptr.getOffset(), ptr.getLength(), sortOrder);
+                }
+            }
+            return maxLength <= desiredMaxLength;
+        }
+        return true;
+    }
+
+    @Override
+    public boolean isFixedWidth() {
+        return false;
+    }
+
+    @Override
+    public int estimateByteSize(Object o) {
+        String value = (String) o;
+        return value == null ? 1 : value.length();
+    }
+
+    @Override
+    public Integer getByteSize() {
+        return null;
+    }
+
+    @Override
+    public int compareTo(Object lhs, Object rhs, PDataType rhsType) {
+        return ((String) lhs).compareTo((String) rhs);
+    }
+
+    @Override
+    public Object toObject(String value) {
+        return value;
+    }
+
+    @Override
+    public boolean isBytesComparableWith(PDataType otherType) {
+        return super.isBytesComparableWith(otherType) || otherType == PChar.INSTANCE;
+    }
+
+    @Override
+    public String toStringLiteral(Object o, Format formatter) {
+        if (formatter != null) {
+            return "'" + formatter.format(o) + "'";
+        }
+        return null == o ? String.valueOf(o) : "'" + StringUtil.escapeStringConstant(o.toString()) + "'";
+    }
+
+    private char[] sampleChars = new char[1];
+
+    @Override
+    public Object getSampleValue(Integer maxLength, Integer arrayLength) {
+        Preconditions.checkArgument(maxLength == null || maxLength >= 0);
+        int length = maxLength != null ? maxLength : 1;
+        if (length != sampleChars.length) {
+            sampleChars = new char[length];
+        }
+        for (int i = 0; i < length; i++) {
+            sampleChars[i] = (char) RANDOM.get().nextInt(Byte.MAX_VALUE);
+        }
+        return new String(sampleChars);
+    }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/bb88e9f5/phoenix-core/src/main/java/org/apache/phoenix/util/SchemaUtil.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/util/SchemaUtil.java b/phoenix-core/src/main/java/org/apache/phoenix/util/SchemaUtil.java
index b53daea..5fc7564 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/util/SchemaUtil.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/SchemaUtil.java
@@ -44,6 +44,7 @@ import javax.annotation.Nullable;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.phoenix.exception.SQLExceptionCode;
@@ -419,8 +420,16 @@ public class SchemaUtil {
     }
 
     public static String toString(PDataType type, byte[] value) {
+        return toString(type, value, 0, value.length);
+    }
+
+    public static String toString(PDataType type, ImmutableBytesWritable value) {
+        return toString(type, value.get(), value.getOffset(), value.getLength());
+    }
+
+    public static String toString(PDataType type, byte[] value, int offset, int length) {
         boolean isString = type.isCoercibleTo(PVarchar.INSTANCE);
-        return isString ? ("'" + type.toObject(value).toString() + "'") : type.toObject(value).toString();
+        return isString ? ("'" + type.toObject(value).toString() + "'") : type.toObject(value, offset, length).toString();
     }
 
     public static byte[] getEmptyColumnFamily(PName defaultColumnFamily, List<PColumnFamily> families) {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/bb88e9f5/phoenix-core/src/test/java/org/apache/phoenix/schema/MutationTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/schema/MutationTest.java b/phoenix-core/src/test/java/org/apache/phoenix/schema/MutationTest.java
index ccbda54..ce2e22f 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/schema/MutationTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/schema/MutationTest.java
@@ -20,6 +20,7 @@ package org.apache.phoenix.schema;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
 
 import java.sql.Connection;
 import java.sql.DriverManager;
@@ -30,8 +31,10 @@ import java.util.List;
 import org.apache.hadoop.hbase.client.Durability;
 import org.apache.hadoop.hbase.client.Mutation;
 import org.apache.hadoop.hbase.util.Pair;
+import org.apache.phoenix.exception.SQLExceptionCode;
 import org.apache.phoenix.jdbc.PhoenixConnection;
 import org.apache.phoenix.query.BaseConnectionlessQueryTest;
+import org.apache.phoenix.schema.types.PVarchar;
 import org.junit.Test;
 
 public class MutationTest extends BaseConnectionlessQueryTest {
@@ -70,5 +73,56 @@ public class MutationTest extends BaseConnectionlessQueryTest {
             }
         }
     }
+    
+    @Test
+    public void testSizeConstraint() throws Exception {
+        Connection conn = DriverManager.getConnection(getUrl());
+        try {
+            int maxLength1 = 3;
+            int maxLength2 = 20;
+            conn.setAutoCommit(false);
+            String bvalue = "01234567890123456789";
+            assertEquals(20,PVarchar.INSTANCE.toBytes(bvalue).length);
+            String value = "\u6fb4\u7c96\u87e4\u092f\u893b\u9143\u5ca4\u8c66\ud311\u85b0\u9129\u813c\u056a\ub066\u7889\u7889\u7889\u7889\u7889\u7889";
+            assertTrue(value.length() <= maxLength2 && value.getBytes().length > maxLength2);
+            conn.createStatement().execute("CREATE TABLE t1 (k1 char(" + maxLength1 + ") not null, k2 varchar(" + maxLength2 + "), "
+                    + "v1 varchar(" + maxLength2 + "), v2 varbinary(" + maxLength2 + "), v3 binary(" + maxLength2 + "), constraint pk primary key (k1, k2))");
+            conn.createStatement().execute("UPSERT INTO t1 VALUES('a','" + value + "', '" + value + "','" + bvalue + "','" + bvalue + "')");
+            try {
+                conn.createStatement().execute("UPSERT INTO t1(k1,v1) VALUES('abcd','" + value + "')");
+                fail();
+            } catch (SQLException e) {
+                assertEquals(SQLExceptionCode.DATA_EXCEEDS_MAX_CAPACITY.getErrorCode(),e.getErrorCode());
+            }
+            try {
+                conn.createStatement().execute("UPSERT INTO t1(k1,v2) VALUES('b','" + value + "')");
+                fail();
+            } catch (SQLException e) {
+                assertEquals(SQLExceptionCode.DATA_EXCEEDS_MAX_CAPACITY.getErrorCode(),e.getErrorCode());
+            }
+            try {
+                conn.createStatement().execute("UPSERT INTO t1(k1,v3) VALUES('b','" + value + "')");
+                fail();
+            } catch (SQLException e) {
+                assertEquals(SQLExceptionCode.DATA_EXCEEDS_MAX_CAPACITY.getErrorCode(),e.getErrorCode());
+            }
+            value = "\u6fb4\u7c96\u87e4\u092f\u893b\u9143\u5ca4\u8c66\ud311\u85b0\u9129\u813c\u056a\ub066\u7889\u7889\u7889\u7889\u7889\u7889\u7889\u7889\u7889";
+            assertTrue(value.length() > maxLength2);
+            try {
+                conn.createStatement().execute("UPSERT INTO t1(k1,k2) VALUES('a','" + value + "')");
+                fail();
+            } catch (SQLException e) {
+                assertEquals(SQLExceptionCode.DATA_EXCEEDS_MAX_CAPACITY.getErrorCode(),e.getErrorCode());
+            }
+            try {
+                conn.createStatement().execute("UPSERT INTO t1(k1,v1) VALUES('a','" + value + "')");
+                fail();
+            } catch (SQLException e) {
+                assertEquals(SQLExceptionCode.DATA_EXCEEDS_MAX_CAPACITY.getErrorCode(),e.getErrorCode());
+            }
+        } finally {
+            conn.close();
+        }
+    }
 
 }