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

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

http://git-wip-us.apache.org/repos/asf/phoenix/blob/04ef859b/phoenix-core/src/main/java/org/apache/phoenix/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
new file mode 100644
index 0000000..6b2dc84
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PDecimal.java
@@ -0,0 +1,396 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.phoenix.schema.types;
+
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.phoenix.query.QueryConstants;
+import org.apache.phoenix.schema.SortOrder;
+import org.apache.phoenix.util.ByteUtil;
+import org.apache.phoenix.util.NumberUtil;
+
+import java.math.BigDecimal;
+import java.sql.Timestamp;
+import java.sql.Types;
+import java.text.Format;
+
+public class PDecimal extends PDataType<BigDecimal> {
+
+  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 PDecimal() {
+    super("DECIMAL", Types.DECIMAL, BigDecimal.class, null, 8);
+  }
+
+  @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;
+  }
+
+  @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);
+  }
+
+  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;
+  }
+
+  @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);
+  }
+
+  @Override
+  public Integer getMaxLength(Object o) {
+    if (o == null) {
+      return MAX_PRECISION;
+    }
+    BigDecimal v = (BigDecimal) o;
+    return v.precision();
+  }
+
+  @Override
+  public Integer getScale(Object o) {
+    return null;
+  }
+
+  @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 = actualType.getCodec().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);
+  }
+
+  @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);
+  }
+
+  @Override
+  public boolean isFixedWidth() {
+    return false;
+  }
+
+  @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 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 isSizeCompatible(ImmutableBytesWritable ptr, Object value, PDataType srcType,
+      Integer maxLength, Integer scale, Integer desiredMaxLength, Integer desiredScale) {
+    if (ptr.getLength() == 0) {
+      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 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);
+  }
+
+  @Override
+  public Object getSampleValue(Integer maxLength, Integer arrayLength) {
+    return new BigDecimal((Long) PLong.INSTANCE.getSampleValue(maxLength, arrayLength));
+  }
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/04ef859b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PDecimalArray.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PDecimalArray.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PDecimalArray.java
new file mode 100644
index 0000000..6874ac3
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PDecimalArray.java
@@ -0,0 +1,110 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.phoenix.schema.types;
+
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.phoenix.schema.SortOrder;
+
+import java.math.BigDecimal;
+import java.sql.Types;
+
+public class PDecimalArray extends PArrayDataType<BigDecimal[]> {
+
+  public static final PDecimalArray INSTANCE = new PDecimalArray();
+
+  private PDecimalArray() {
+    super("DECIMAL ARRAY", PDataType.ARRAY_TYPE_BASE + PDecimal.INSTANCE.getSqlType(),
+        PhoenixArray.class, null, 35);
+  }
+
+  @Override
+  public boolean isArrayType() {
+    return true;
+  }
+
+  @Override
+  public boolean isFixedWidth() {
+    return false;
+  }
+
+  @Override
+  public int compareTo(Object lhs, Object rhs, PDataType rhsType) {
+    return compareTo(lhs, rhs);
+  }
+
+  @Override
+  public Integer getByteSize() {
+    return null;
+  }
+
+  @Override
+  public byte[] toBytes(Object object) {
+    return toBytes(object, SortOrder.ASC);
+  }
+
+  @Override
+  public byte[] toBytes(Object object, SortOrder sortOrder) {
+    return toBytes(object, PDecimal.INSTANCE, sortOrder);
+  }
+
+  @Override
+  public Object toObject(byte[] bytes, int offset, int length,
+      PDataType actualType, SortOrder sortOrder, Integer maxLength,
+      Integer scale) {
+    return toObject(bytes, offset, length, PDecimal.INSTANCE, sortOrder, maxLength, scale,
+        PDecimal.INSTANCE);
+  }
+
+  @Override
+  public void coerceBytes(ImmutableBytesWritable ptr, Object object, PDataType actualType,
+      Integer maxLength, Integer scale, SortOrder actualModifer, Integer desiredMaxLength,
+      Integer desiredScale, SortOrder desiredModifier) {
+    coerceBytes(ptr, object, actualType, maxLength, scale, desiredMaxLength, desiredScale,
+        this, actualModifer, desiredModifier);
+  }
+
+  @Override
+  public boolean isCoercibleTo(PDataType targetType) {
+    return isCoercibleTo(targetType, this);
+  }
+
+  @Override
+  public boolean isCoercibleTo(PDataType targetType, Object value) {
+    if (value == null) {
+      return true;
+    }
+    PhoenixArray pArr = (PhoenixArray) value;
+    Object[] decimalArr = (Object[]) pArr.array;
+    for (Object i : decimalArr) {
+      if (!super.isCoercibleTo(PDecimal.INSTANCE, i)) {
+        return false;
+      }
+    }
+    return true;
+  }
+
+  @Override
+  public int getResultSetSqlType() {
+    return Types.ARRAY;
+  }
+
+  @Override
+  public Object getSampleValue(Integer maxLength, Integer arrayLength) {
+    return getSampleValue(PDecimal.INSTANCE, arrayLength, maxLength);
+  }
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/04ef859b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PDouble.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PDouble.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PDouble.java
new file mode 100644
index 0000000..d8e3c4f
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PDouble.java
@@ -0,0 +1,311 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.phoenix.schema.types;
+
+import com.google.common.base.Preconditions;
+import com.google.common.primitives.Doubles;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.phoenix.schema.SortOrder;
+
+import java.math.BigDecimal;
+import java.sql.Types;
+
+public class PDouble extends PDataType<Double> {
+
+  public static final PDouble INSTANCE = new PDouble();
+
+  private PDouble() {
+    super("DOUBLE", Types.DOUBLE, Double.class, new DoubleCodec(), 7);
+  }
+
+  @Override
+  public int compareTo(Object lhs, Object rhs, PDataType rhsType) {
+    if (rhsType == PDecimal.INSTANCE) {
+      return -((BigDecimal) rhs).compareTo(BigDecimal.valueOf(((Number) lhs).doubleValue()));
+    }
+    return Doubles.compare(((Number) lhs).doubleValue(), ((Number) rhs).doubleValue());
+  }
+
+  @Override
+  public boolean isFixedWidth() {
+    return true;
+  }
+
+  @Override
+  public Integer getByteSize() {
+    return Bytes.SIZEOF_DOUBLE;
+  }
+
+  @Override
+  public Integer getScale(Object o) {
+    if (o == null) {
+      return null;
+    }
+    Double v = (Double) o;
+    BigDecimal bd = BigDecimal.valueOf(v);
+    return bd.scale() == 0 ? null : bd.scale();
+  }
+
+  @Override
+  public Integer getMaxLength(Object o) {
+    if (o == null) {
+      return null;
+    }
+    Double v = (Double) o;
+    BigDecimal db = BigDecimal.valueOf(v);
+    return db.precision();
+  }
+
+  @Override
+  public byte[] toBytes(Object object) {
+    byte[] b = new byte[Bytes.SIZEOF_DOUBLE];
+    toBytes(object, b, 0);
+    return b;
+  }
+
+  @Override
+  public int toBytes(Object object, byte[] bytes, int offset) {
+    if (object == null) {
+      throw newIllegalDataException(this + " may not be null");
+    }
+    return this.getCodec().encodeDouble(((Number) object).doubleValue(),
+        bytes, offset);
+  }
+
+  @Override
+  public Object toObject(String value) {
+    if (value == null || value.length() == 0) {
+      return null;
+    }
+    try {
+      return Double.parseDouble(value);
+    } catch (NumberFormatException e) {
+      throw newIllegalDataException(e);
+    }
+  }
+
+  @Override
+  public Object toObject(Object object, PDataType actualType) {
+    if (object == null) {
+      return null;
+    }
+    double de;
+    if (equalsAny(actualType, PDouble.INSTANCE, PUnsignedDouble.INSTANCE)) {
+      return object;
+    } else if (equalsAny(actualType, PFloat.INSTANCE, PUnsignedFloat.INSTANCE)) {
+      de = (Float) object;
+      return de;
+    } else if (equalsAny(actualType, PLong.INSTANCE, PUnsignedLong.INSTANCE)) {
+      de = (Long) object;
+      return de;
+    } else if (equalsAny(actualType, PInteger.INSTANCE, PUnsignedInt.INSTANCE)) {
+      de = (Integer) object;
+      return de;
+    } else if (equalsAny(actualType, PTinyint.INSTANCE, PUnsignedTinyint.INSTANCE)) {
+      de = (Byte) object;
+      return de;
+    } else if (equalsAny(actualType, PSmallint.INSTANCE, PUnsignedSmallint.INSTANCE)) {
+      de = (Short) object;
+      return de;
+    } else if (actualType == PDecimal.INSTANCE) {
+      BigDecimal d = (BigDecimal) object;
+      return d.doubleValue();
+    }
+    return throwConstraintViolationException(actualType, this);
+  }
+
+  @Override
+  public Double toObject(byte[] b, int o, int l, PDataType actualType,
+      SortOrder sortOrder, Integer maxLength, Integer scale) {
+    if (l <= 0) {
+      return null;
+    }
+    if (equalsAny(actualType, PDouble.INSTANCE, PUnsignedDouble.INSTANCE, PFloat.INSTANCE,
+        PUnsignedFloat.INSTANCE, PLong.INSTANCE, PUnsignedLong.INSTANCE, PInteger.INSTANCE,
+        PUnsignedInt.INSTANCE, PSmallint.INSTANCE, PUnsignedSmallint.INSTANCE, PTinyint.INSTANCE,
+        PUnsignedTinyint.INSTANCE)) {
+      return actualType.getCodec().decodeDouble(b, o, sortOrder);
+    } else if (actualType == PDecimal.INSTANCE) {
+      BigDecimal bd = (BigDecimal) actualType.toObject(b, o, l, actualType, sortOrder);
+      return bd.doubleValue();
+    }
+    throwConstraintViolationException(actualType, this);
+    return null;
+  }
+
+  @Override
+  public boolean isCoercibleTo(PDataType targetType, Object value) {
+    if (value != null) {
+      double d = (Double) value;
+      if (targetType.equals(PUnsignedDouble.INSTANCE)) {
+        return d >= 0;
+      } else if (targetType.equals(PFloat.INSTANCE)) {
+        return Double.isNaN(d)
+            || d == Double.POSITIVE_INFINITY
+            || d == Double.NEGATIVE_INFINITY
+            || (d >= -Float.MAX_VALUE && d <= Float.MAX_VALUE);
+      } else if (targetType.equals(PUnsignedFloat.INSTANCE)) {
+        return Double.isNaN(d) || d == Double.POSITIVE_INFINITY
+            || (d >= 0 && d <= Float.MAX_VALUE);
+      } else if (targetType.equals(PUnsignedLong.INSTANCE)) {
+        return (d >= 0 && d <= Long.MAX_VALUE);
+      } else if (targetType.equals(PLong.INSTANCE)) {
+        return (d >= Long.MIN_VALUE && d <= Long.MAX_VALUE);
+      } else if (targetType.equals(PUnsignedInt.INSTANCE)) {
+        return (d >= 0 && d <= Integer.MAX_VALUE);
+      } else if (targetType.equals(PInteger.INSTANCE)) {
+        return (d >= Integer.MIN_VALUE && d <= Integer.MAX_VALUE);
+      } else if (targetType.equals(PUnsignedSmallint.INSTANCE)) {
+        return (d >= 0 && d <= Short.MAX_VALUE);
+      } else if (targetType.equals(PSmallint.INSTANCE)) {
+        return (d >= Short.MIN_VALUE && d <= Short.MAX_VALUE);
+      } else if (targetType.equals(PTinyint.INSTANCE)) {
+        return (d >= Byte.MIN_VALUE && d < Byte.MAX_VALUE);
+      } else if (targetType.equals(PUnsignedTinyint.INSTANCE)) {
+        return (d >= 0 && d < Byte.MAX_VALUE);
+      }
+    }
+    return super.isCoercibleTo(targetType, value);
+  }
+
+  @Override
+  public boolean isCoercibleTo(PDataType targetType) {
+    return equalsAny(targetType, this, PDecimal.INSTANCE, PVarbinary.INSTANCE, PBinary.INSTANCE);
+  }
+
+  @Override
+  public Object getSampleValue(Integer maxLength, Integer arrayLength) {
+    return RANDOM.get().nextDouble();
+  }
+
+  static class DoubleCodec extends BaseCodec {
+
+    @Override
+    public long decodeLong(byte[] b, int o, SortOrder sortOrder) {
+      double v = decodeDouble(b, o, sortOrder);
+      if (v < Long.MIN_VALUE || v > Long.MAX_VALUE) {
+        throw newIllegalDataException(
+            "Value " + v + " cannot be cast to Long without changing its value");
+      }
+      return (long) v;
+    }
+
+    @Override
+    public int decodeInt(byte[] b, int o, SortOrder sortOrder) {
+      double v = decodeDouble(b, o, sortOrder);
+      if (v < Integer.MIN_VALUE || v > Integer.MAX_VALUE) {
+        throw newIllegalDataException(
+            "Value " + v + " cannot be cast to Integer without changing its value");
+      }
+      return (int) v;
+    }
+
+    @Override
+    public byte decodeByte(byte[] b, int o, SortOrder sortOrder) {
+      double v = decodeDouble(b, o, sortOrder);
+      if (v < Byte.MIN_VALUE || v > Byte.MAX_VALUE) {
+        throw newIllegalDataException(
+            "Value " + v + " cannot be cast to Byte without changing its value");
+      }
+      return (byte) v;
+    }
+
+    @Override
+    public short decodeShort(byte[] b, int o, SortOrder sortOrder) {
+      double v = decodeDouble(b, o, sortOrder);
+      if (v < Short.MIN_VALUE || v > Short.MAX_VALUE) {
+        throw newIllegalDataException(
+            "Value " + v + " cannot be cast to Short without changing its value");
+      }
+      return (short) v;
+    }
+
+    @Override
+    public double decodeDouble(byte[] b, int o, SortOrder sortOrder) {
+      Preconditions.checkNotNull(sortOrder);
+      checkForSufficientLength(b, o, Bytes.SIZEOF_LONG);
+      if (sortOrder == SortOrder.DESC) {
+        for (int i = o; i < Bytes.SIZEOF_LONG; i++) {
+          b[i] = (byte) (b[i] ^ 0xff);
+        }
+      }
+      long l = Bytes.toLong(b, o);
+      l--;
+      l ^= (~l >> Long.SIZE - 1) | Long.MIN_VALUE;
+      return Double.longBitsToDouble(l);
+    }
+
+    @Override
+    public float decodeFloat(byte[] b, int o, SortOrder sortOrder) {
+      double v = decodeDouble(b, o, sortOrder);
+      if (Double.isNaN(v) || v == Double.NEGATIVE_INFINITY
+          || v == Double.POSITIVE_INFINITY
+          || (v >= -Float.MAX_VALUE && v <= Float.MAX_VALUE)) {
+        return (float) v;
+      } else {
+        throw newIllegalDataException(
+            "Value " + v + " cannot be cast to Float without changing its value");
+      }
+
+    }
+
+    @Override
+    public int encodeShort(short v, byte[] b, int o) {
+      return encodeDouble(v, b, o);
+    }
+
+    @Override
+    public int encodeLong(long v, byte[] b, int o) {
+      return encodeDouble(v, b, o);
+    }
+
+    @Override
+    public int encodeInt(int v, byte[] b, int o) {
+      return encodeDouble(v, b, o);
+    }
+
+    @Override
+    public int encodeByte(byte v, byte[] b, int o) {
+      return encodeDouble(v, b, o);
+    }
+
+    @Override
+    public int encodeDouble(double v, byte[] b, int o) {
+      checkForSufficientLength(b, o, Bytes.SIZEOF_LONG);
+      long l = Double.doubleToLongBits(v);
+      l = (l ^ ((l >> Long.SIZE - 1) | Long.MIN_VALUE)) + 1;
+      Bytes.putLong(b, o, l);
+      return Bytes.SIZEOF_LONG;
+    }
+
+    @Override
+    public int encodeFloat(float v, byte[] b, int o) {
+      return encodeDouble(v, b, o);
+    }
+
+    @Override
+    public PhoenixArrayFactory getPhoenixArrayFactory() {
+      return new PhoenixArrayFactory() {
+        @Override
+        public PhoenixArray newArray(PDataType type, Object[] elements) {
+          return new PhoenixArray.PrimitiveDoublePhoenixArray(type, elements);
+        }
+      };
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/04ef859b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PDoubleArray.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PDoubleArray.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PDoubleArray.java
new file mode 100644
index 0000000..7ecec61
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PDoubleArray.java
@@ -0,0 +1,108 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.phoenix.schema.types;
+
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.phoenix.schema.SortOrder;
+
+import java.sql.Types;
+
+public class PDoubleArray extends PArrayDataType<double[]> {
+
+  public static final PDoubleArray INSTANCE = new PDoubleArray();
+
+  private PDoubleArray() {
+    super("DOUBLE ARRAY", PDataType.ARRAY_TYPE_BASE + PDouble.INSTANCE.getSqlType(),
+        PhoenixArray.class, null, 34);
+  }
+
+  @Override
+  public boolean isArrayType() {
+    return true;
+  }
+
+  @Override
+  public boolean isFixedWidth() {
+    return false;
+  }
+
+  @Override
+  public int compareTo(Object lhs, Object rhs, PDataType rhsType) {
+    return compareTo(lhs, rhs);
+  }
+
+  @Override
+  public Integer getByteSize() {
+    return null;
+  }
+
+  @Override
+  public byte[] toBytes(Object object) {
+    return toBytes(object, SortOrder.ASC);
+  }
+
+  @Override
+  public byte[] toBytes(Object object, SortOrder sortOrder) {
+    return toBytes(object, PDouble.INSTANCE, sortOrder);
+  }
+
+  @Override
+  public Object toObject(byte[] bytes, int offset, int length,
+      PDataType actualType, SortOrder sortOrder, Integer maxLength, Integer scale) {
+    return toObject(bytes, offset, length, PDouble.INSTANCE, sortOrder, maxLength, scale,
+        PDouble.INSTANCE);
+  }
+
+  @Override
+  public boolean isCoercibleTo(PDataType targetType) {
+    return isCoercibleTo(targetType, this);
+  }
+
+  @Override
+  public boolean isCoercibleTo(PDataType targetType, Object value) {
+    if (value == null) {
+      return true;
+    }
+    PhoenixArray.PrimitiveDoublePhoenixArray pArr = (PhoenixArray.PrimitiveDoublePhoenixArray) value;
+    double[] doubleArr = (double[]) pArr.array;
+    for (double i : doubleArr) {
+      if (!super.isCoercibleTo(PDouble.INSTANCE, i)) {
+        return false;
+      }
+    }
+    return true;
+  }
+
+  @Override
+  public void coerceBytes(ImmutableBytesWritable ptr, Object object, PDataType actualType,
+      Integer maxLength, Integer scale, SortOrder actualModifer, Integer desiredMaxLength,
+      Integer desiredScale, SortOrder desiredModifier) {
+    coerceBytes(ptr, object, actualType, maxLength, scale, desiredMaxLength, desiredScale,
+        this, actualModifer, desiredModifier);
+  }
+
+  @Override
+  public int getResultSetSqlType() {
+    return Types.ARRAY;
+  }
+
+  @Override
+  public Object getSampleValue(Integer maxLength, Integer arrayLength) {
+    return getSampleValue(PDouble.INSTANCE, arrayLength, maxLength);
+  }
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/04ef859b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PFloat.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PFloat.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PFloat.java
new file mode 100644
index 0000000..608b270
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PFloat.java
@@ -0,0 +1,308 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.phoenix.schema.types;
+
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.phoenix.schema.SortOrder;
+
+import java.math.BigDecimal;
+import java.sql.Types;
+
+public class PFloat extends PDataType<Float> {
+
+  public static final PFloat INSTANCE = new PFloat();
+
+  private PFloat() {
+    super("FLOAT", Types.FLOAT, Float.class, new FloatCodec(), 6);
+  }
+
+  @Override
+  public int compareTo(Object lhs, Object rhs, PDataType rhsType) {
+    return PDouble.INSTANCE.compareTo(lhs, rhs, rhsType);
+  }
+
+  @Override
+  public boolean isFixedWidth() {
+    return true;
+  }
+
+  @Override
+  public Integer getByteSize() {
+    return Bytes.SIZEOF_FLOAT;
+  }
+
+  @Override
+  public Integer getScale(Object o) {
+    if (o == null) {
+      return null;
+    }
+    Float v = (Float) o;
+    BigDecimal bd = BigDecimal.valueOf(v);
+    return bd.scale() == 0 ? null : bd.scale();
+  }
+
+  @Override
+  public Integer getMaxLength(Object o) {
+    if (o == null) {
+      return null;
+    }
+    Float v = (Float) o;
+    BigDecimal bd = BigDecimal.valueOf(v);
+    return bd.precision();
+  }
+
+  @Override
+  public byte[] toBytes(Object object) {
+    byte[] b = new byte[Bytes.SIZEOF_FLOAT];
+    toBytes(object, b, 0);
+    return b;
+  }
+
+  @Override
+  public int toBytes(Object object, byte[] bytes, int offset) {
+    if (object == null) {
+      throw newIllegalDataException(this + " may not be null");
+    }
+    return this.getCodec().encodeFloat(((Number) object).floatValue(),
+        bytes, offset);
+  }
+
+  @Override
+  public Object toObject(String value) {
+    if (value == null || value.length() == 0) {
+      return null;
+    }
+    try {
+      return Float.parseFloat(value);
+    } catch (NumberFormatException e) {
+      throw newIllegalDataException(e);
+    }
+  }
+
+  @Override
+  public Object toObject(Object object, PDataType actualType) {
+    if (object == null) {
+      return null;
+    }
+    float f;
+    if (equalsAny(actualType, PFloat.INSTANCE, PUnsignedFloat.INSTANCE)) {
+      return object;
+    } else if (equalsAny(actualType, PDouble.INSTANCE, PUnsignedDouble.INSTANCE)) {
+      double d = (Double) object;
+      if (Double.isNaN(d)
+          || d == Double.POSITIVE_INFINITY
+          || d == Double.NEGATIVE_INFINITY
+          || (d >= -Float.MAX_VALUE && d <= Float.MAX_VALUE)) {
+        return (float) d;
+      } else {
+        throw newIllegalDataException(
+            actualType + " value " + d + " cannot be cast to Float without changing its value");
+      }
+    } else if (equalsAny(actualType, PLong.INSTANCE, PUnsignedLong.INSTANCE)) {
+      f = (Long) object;
+      return f;
+    } else if (equalsAny(actualType, PInteger.INSTANCE, PUnsignedInt.INSTANCE)) {
+      f = (Integer) object;
+      return f;
+    } else if (equalsAny(actualType, PTinyint.INSTANCE, PUnsignedTinyint.INSTANCE)) {
+      f = (Byte) object;
+      return f;
+    } else if (equalsAny(actualType, PSmallint.INSTANCE, PUnsignedSmallint.INSTANCE)) {
+      f = (Short) object;
+      return f;
+    } else if (actualType == PDecimal.INSTANCE) {
+      BigDecimal dl = (BigDecimal) object;
+      return dl.floatValue();
+    }
+    return throwConstraintViolationException(actualType, this);
+  }
+
+  @Override
+  public Float toObject(byte[] b, int o, int l, PDataType actualType, SortOrder sortOrder,
+      Integer maxLength, Integer scale) {
+    if (l <= 0) {
+      return null;
+    }
+    if (equalsAny(actualType, PFloat.INSTANCE, PUnsignedFloat.INSTANCE, PDouble.INSTANCE,
+        PUnsignedDouble.INSTANCE, PLong.INSTANCE, PUnsignedLong.INSTANCE, PInteger.INSTANCE,
+        PUnsignedInt.INSTANCE, PSmallint.INSTANCE, PUnsignedSmallint.INSTANCE, PTinyint.INSTANCE,
+        PUnsignedTinyint.INSTANCE)) {
+      return actualType.getCodec().decodeFloat(b, o, sortOrder);
+    } else if (actualType == PDecimal.INSTANCE) {
+      BigDecimal bd = (BigDecimal) actualType.toObject(b, o, l, actualType, sortOrder);
+      return bd.floatValue();
+    }
+
+    throwConstraintViolationException(actualType, this);
+    return null;
+  }
+
+  @Override
+  public boolean isCoercibleTo(PDataType targetType, Object value) {
+    if (value != null) {
+      float f = (Float) value;
+      if (targetType.equals(PUnsignedFloat.INSTANCE)) {
+        return f >= 0;
+      } else if (targetType.equals(PUnsignedLong.INSTANCE)) {
+        return (f >= 0 && f <= Long.MAX_VALUE);
+      } else if (targetType.equals(PLong.INSTANCE)) {
+        return (f >= Long.MIN_VALUE && f <= Long.MAX_VALUE);
+      } else if (targetType.equals(PUnsignedInt.INSTANCE)) {
+        return (f >= 0 && f <= Integer.MAX_VALUE);
+      } else if (targetType.equals(PInteger.INSTANCE)) {
+        return (f >= Integer.MIN_VALUE && f <= Integer.MAX_VALUE);
+      } else if (targetType.equals(PUnsignedSmallint.INSTANCE)) {
+        return (f >= 0 && f <= Short.MAX_VALUE);
+      } else if (targetType.equals(PSmallint.INSTANCE)) {
+        return (f >= Short.MIN_VALUE && f <= Short.MAX_VALUE);
+      } else if (targetType.equals(PTinyint.INSTANCE)) {
+        return (f >= Byte.MIN_VALUE && f < Byte.MAX_VALUE);
+      } else if (targetType.equals(PUnsignedTinyint.INSTANCE)) {
+        return (f >= 0 && f < Byte.MAX_VALUE);
+      }
+    }
+    return super.isCoercibleTo(targetType, value);
+  }
+
+  @Override
+  public boolean isCoercibleTo(PDataType targetType) {
+    return this.equals(targetType) || PDouble.INSTANCE.isCoercibleTo(targetType);
+  }
+
+  @Override
+  public Object getSampleValue(Integer maxLength, Integer arrayLength) {
+    return RANDOM.get().nextFloat();
+  }
+
+  static class FloatCodec extends BaseCodec {
+
+    @Override
+    public long decodeLong(byte[] b, int o, SortOrder sortOrder) {
+      float v = decodeFloat(b, o, sortOrder);
+      if (v < Long.MIN_VALUE || v > Long.MAX_VALUE) {
+        throw newIllegalDataException(
+            "Value " + v + " cannot be cast to Long without changing its value");
+      }
+      return (long) v;
+    }
+
+    @Override
+    public int decodeInt(byte[] b, int o, SortOrder sortOrder) {
+      float v = decodeFloat(b, o, sortOrder);
+      if (v < Integer.MIN_VALUE || v > Integer.MAX_VALUE) {
+        throw newIllegalDataException(
+            "Value " + v + " cannot be cast to Integer without changing its value");
+      }
+      return (int) v;
+    }
+
+    @Override
+    public byte decodeByte(byte[] b, int o, SortOrder sortOrder) {
+      float v = decodeFloat(b, o, sortOrder);
+      if (v < Byte.MIN_VALUE || v > Byte.MAX_VALUE) {
+        throw newIllegalDataException(
+            "Value " + v + " cannot be cast to Byte without changing its value");
+      }
+      return (byte) v;
+    }
+
+    @Override
+    public short decodeShort(byte[] b, int o, SortOrder sortOrder) {
+      float v = decodeFloat(b, o, sortOrder);
+      if (v < Short.MIN_VALUE || v > Short.MAX_VALUE) {
+        throw newIllegalDataException(
+            "Value " + v + " cannot be cast to Short without changing its value");
+      }
+      return (short) v;
+    }
+
+    @Override
+    public double decodeDouble(byte[] b, int o,
+        SortOrder sortOrder) {
+      return decodeFloat(b, o, sortOrder);
+    }
+
+    @Override
+    public float decodeFloat(byte[] b, int o, SortOrder sortOrder) {
+      Preconditions.checkNotNull(sortOrder);
+      checkForSufficientLength(b, o, Bytes.SIZEOF_INT);
+      if (sortOrder == SortOrder.DESC) {
+        for (int i = o; i < Bytes.SIZEOF_INT; i++) {
+          b[i] = (byte) (b[i] ^ 0xff);
+        }
+      }
+      int i = Bytes.toInt(b, o);
+      i--;
+      i ^= (~i >> Integer.SIZE - 1) | Integer.MIN_VALUE;
+      return Float.intBitsToFloat(i);
+    }
+
+    @Override
+    public int encodeShort(short v, byte[] b, int o) {
+      return encodeFloat(v, b, o);
+    }
+
+    @Override
+    public int encodeLong(long v, byte[] b, int o) {
+      return encodeFloat(v, b, o);
+    }
+
+    @Override
+    public int encodeInt(int v, byte[] b, int o) {
+      return encodeFloat(v, b, o);
+    }
+
+    @Override
+    public int encodeByte(byte v, byte[] b, int o) {
+      return encodeFloat(v, b, o);
+    }
+
+    @Override
+    public int encodeDouble(double v, byte[] b, int o) {
+      if (Double.isNaN(v) || v == Double.POSITIVE_INFINITY
+          || v == Double.NEGATIVE_INFINITY
+          || (v >= -Float.MAX_VALUE && v <= Float.MAX_VALUE)) {
+        return encodeFloat((float) v, b, o);
+      } else {
+        throw newIllegalDataException(
+            "Value " + v + " cannot be encoded as an Float without changing its value");
+      }
+
+    }
+
+    @Override
+    public int encodeFloat(float v, byte[] b, int o) {
+      checkForSufficientLength(b, o, Bytes.SIZEOF_FLOAT);
+      int i = Float.floatToIntBits(v);
+      i = (i ^ ((i >> Integer.SIZE - 1) | Integer.MIN_VALUE)) + 1;
+      Bytes.putInt(b, o, i);
+      return Bytes.SIZEOF_FLOAT;
+    }
+
+    @Override
+    public PhoenixArrayFactory getPhoenixArrayFactory() {
+      return new PhoenixArrayFactory() {
+        @Override
+        public PhoenixArray newArray(PDataType type, Object[] elements) {
+          return new PhoenixArray.PrimitiveFloatPhoenixArray(type, elements);
+        }
+      };
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/04ef859b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PFloatArray.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PFloatArray.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PFloatArray.java
new file mode 100644
index 0000000..5e8ddb3
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PFloatArray.java
@@ -0,0 +1,109 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.phoenix.schema.types;
+
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.phoenix.schema.SortOrder;
+
+import java.sql.Types;
+
+public class PFloatArray extends PArrayDataType<float[]> {
+
+  public static final PFloatArray INSTANCE = new PFloatArray();
+
+  private PFloatArray() {
+    super("FLOAT ARRAY", PDataType.ARRAY_TYPE_BASE + PFloat.INSTANCE.getSqlType(),
+        PhoenixArray.class, null, 33);
+  }
+
+  @Override
+  public boolean isArrayType() {
+    return true;
+  }
+
+  @Override
+  public boolean isFixedWidth() {
+    return false;
+  }
+
+  @Override
+  public int compareTo(Object lhs, Object rhs, PDataType rhsType) {
+    return compareTo(lhs, rhs);
+  }
+
+  @Override
+  public Integer getByteSize() {
+    return null;
+  }
+
+  @Override
+  public byte[] toBytes(Object object) {
+    return toBytes(object, SortOrder.ASC);
+  }
+
+  @Override
+  public byte[] toBytes(Object object, SortOrder sortOrder) {
+    return toBytes(object, PFloat.INSTANCE, sortOrder);
+  }
+
+  @Override
+  public Object toObject(byte[] bytes, int offset, int length,
+      PDataType actualType, SortOrder sortOrder, Integer maxLength,
+      Integer scale) {
+    return toObject(bytes, offset, length, PFloat.INSTANCE, sortOrder, maxLength, scale,
+        PFloat.INSTANCE);
+  }
+
+  @Override
+  public boolean isCoercibleTo(PDataType targetType) {
+    return isCoercibleTo(targetType, this);
+  }
+
+  @Override
+  public boolean isCoercibleTo(PDataType targetType, Object value) {
+    if (value == null) {
+      return true;
+    }
+    PhoenixArray.PrimitiveFloatPhoenixArray pArr = (PhoenixArray.PrimitiveFloatPhoenixArray) value;
+    float[] floatArr = (float[]) pArr.array;
+    for (float i : floatArr) {
+      if (!super.isCoercibleTo(PFloat.INSTANCE, i)) {
+        return false;
+      }
+    }
+    return true;
+  }
+
+  @Override
+  public int getResultSetSqlType() {
+    return Types.ARRAY;
+  }
+
+  @Override
+  public void coerceBytes(ImmutableBytesWritable ptr, Object object, PDataType actualType,
+      Integer maxLength, Integer scale, SortOrder actualModifer, Integer desiredMaxLength,
+      Integer desiredScale, SortOrder desiredModifier) {
+    coerceBytes(ptr, object, actualType, maxLength, scale, desiredMaxLength, desiredScale,
+        this, actualModifer, desiredModifier);
+  }
+
+  @Override
+  public Object getSampleValue(Integer maxLength, Integer arrayLength) {
+    return getSampleValue(PFloat.INSTANCE, arrayLength, maxLength);
+  }
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/04ef859b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PInteger.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PInteger.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PInteger.java
new file mode 100644
index 0000000..c4760b5
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PInteger.java
@@ -0,0 +1,275 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.phoenix.schema.types;
+
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.phoenix.schema.SortOrder;
+
+import java.math.BigDecimal;
+import java.sql.Types;
+
+public class PInteger extends PDataType<Integer> {
+
+  public static final PInteger INSTANCE = new PInteger();
+
+  private PInteger() {
+    super("INTEGER", Types.INTEGER, Integer.class, new IntCodec(), 3);
+  }
+
+  @Override
+  public Integer getScale(Object o) {
+    return ZERO;
+  }
+
+  @Override
+  public byte[] toBytes(Object object) {
+    byte[] b = new byte[Bytes.SIZEOF_INT];
+    toBytes(object, b, 0);
+    return b;
+  }
+
+  @Override
+  public int toBytes(Object object, byte[] b, int o) {
+    if (object == null) {
+      throw newIllegalDataException(this + " may not be null");
+    }
+    return this.getCodec().encodeInt(((Number) object).intValue(), b, o);
+  }
+
+  @Override
+  public Object toObject(Object object, PDataType actualType) {
+    Object o = PLong.INSTANCE.toObject(object, actualType);
+    if (!(o instanceof Long) || o == null) {
+      return o;
+    }
+    long l = (Long) o;
+    if (l < Integer.MIN_VALUE || l > Integer.MAX_VALUE) {
+      throw newIllegalDataException(
+          actualType + " value " + l + " cannot be cast to Integer without changing its value");
+    }
+    int v = (int) l;
+    return v;
+  }
+
+  @Override
+  public Integer toObject(byte[] b, int o, int l, PDataType actualType,
+      SortOrder sortOrder, Integer maxLength, Integer scale) {
+    if (l == 0) {
+      return null;
+    }
+    if (equalsAny(actualType, PLong.INSTANCE, PUnsignedLong.INSTANCE, PInteger.INSTANCE,
+        PUnsignedInt.INSTANCE, PSmallint.INSTANCE, PUnsignedSmallint.INSTANCE, PTinyint.INSTANCE,
+        PUnsignedTinyint.INSTANCE, PFloat.INSTANCE, PUnsignedFloat.INSTANCE, PDouble.INSTANCE,
+        PUnsignedDouble.INSTANCE)) {
+      return actualType.getCodec().decodeInt(b, o, sortOrder);
+    } else if (actualType == PDecimal.INSTANCE) {
+      BigDecimal bd = (BigDecimal) actualType.toObject(b, o, l, actualType, sortOrder);
+      return bd.intValueExact();
+    }
+    throwConstraintViolationException(actualType, this);
+    return null;
+  }
+
+  @Override
+  public boolean isCoercibleTo(PDataType targetType, Object value) {
+    if (value != null) {
+      int i;
+      if (equalsAny(targetType, PUnsignedDouble.INSTANCE, PUnsignedFloat.INSTANCE,
+          PUnsignedLong.INSTANCE, PUnsignedInt.INSTANCE)) {
+        i = (Integer) value;
+        return i >= 0;
+      } else if (targetType.equals(PUnsignedSmallint.INSTANCE)) {
+        i = (Integer) value;
+        return (i >= 0 && i <= Short.MAX_VALUE);
+      } else if (targetType.equals(PSmallint.INSTANCE)) {
+        i = (Integer) value;
+        return (i >= Short.MIN_VALUE && i <= Short.MAX_VALUE);
+      } else if (targetType.equals(PTinyint.INSTANCE)) {
+        i = (Integer) value;
+        return (i >= Byte.MIN_VALUE && i <= Byte.MAX_VALUE);
+      } else if (targetType.equals(PUnsignedTinyint.INSTANCE)) {
+        i = (Integer) value;
+        return (i >= 0 && i < Byte.MAX_VALUE);
+      }
+    }
+    return super.isCoercibleTo(targetType, value);
+  }
+
+  @Override
+  public boolean isCoercibleTo(PDataType targetType) {
+    return equalsAny(targetType, this, PFloat.INSTANCE) || PLong.INSTANCE.isCoercibleTo(targetType);
+  }
+
+  @Override
+  public boolean isFixedWidth() {
+    return true;
+  }
+
+  @Override
+  public Integer getByteSize() {
+    return Bytes.SIZEOF_INT;
+  }
+
+  @Override
+  public Integer getMaxLength(Object o) {
+    return INT_PRECISION;
+  }
+
+  @Override
+  public int compareTo(Object lhs, Object rhs, PDataType rhsType) {
+    return PLong.INSTANCE.compareTo(lhs, rhs, rhsType);
+  }
+
+  @Override
+  public boolean isComparableTo(PDataType targetType) {
+    return PDecimal.INSTANCE.isComparableTo(targetType);
+  }
+
+  @Override
+  public Object toObject(String value) {
+    if (value == null || value.length() == 0) {
+      return null;
+    }
+    try {
+      return Integer.parseInt(value);
+    } catch (NumberFormatException e) {
+      throw newIllegalDataException(e);
+    }
+  }
+
+  @Override
+  public Object getSampleValue(Integer maxLength, Integer arrayLength) {
+    return RANDOM.get().nextInt();
+  }
+
+  static class IntCodec extends BaseCodec {
+
+    @Override
+    public long decodeLong(byte[] b, int o, SortOrder sortOrder) {
+      return decodeInt(b, o, sortOrder);
+    }
+
+    @Override
+    public float decodeFloat(byte[] b, int o, SortOrder sortOrder) {
+      return decodeInt(b, o, sortOrder);
+    }
+
+    @Override
+    public double decodeDouble(byte[] b, int o,
+        SortOrder sortOrder) {
+      return decodeInt(b, o, sortOrder);
+    }
+
+    @Override
+    public int decodeInt(byte[] bytes, int o, SortOrder sortOrder) {
+      Preconditions.checkNotNull(sortOrder);
+      checkForSufficientLength(bytes, o, Bytes.SIZEOF_INT);
+      int v;
+      if (sortOrder == SortOrder.ASC) {
+        v = bytes[o] ^ 0x80; // Flip sign bit back
+        for (int i = 1; i < Bytes.SIZEOF_INT; i++) {
+          v = (v << 8) + (bytes[o + i] & 0xff);
+        }
+      } else {
+        v = bytes[o] ^ 0xff ^ 0x80; // Flip sign bit back
+        for (int i = 1; i < Bytes.SIZEOF_INT; i++) {
+          v = (v << 8) + ((bytes[o + i] ^ 0xff) & 0xff);
+        }
+      }
+      return v;
+    }
+
+    @Override
+    public int encodeInt(int v, byte[] b, int o) {
+      checkForSufficientLength(b, o, Bytes.SIZEOF_INT);
+      b[o + 0] = (byte) ((v >> 24) ^ 0x80); // Flip sign bit so that INTEGER is binary comparable
+      b[o + 1] = (byte) (v >> 16);
+      b[o + 2] = (byte) (v >> 8);
+      b[o + 3] = (byte) v;
+      return Bytes.SIZEOF_INT;
+    }
+
+    @Override
+    public int encodeFloat(float v, byte[] b, int o) {
+      if (v < Integer.MIN_VALUE || v > Integer.MAX_VALUE) {
+        throw newIllegalDataException(
+            "Value " + v + " cannot be encoded as an Integer without changing its value");
+      }
+      return encodeInt((int) v, b, o);
+    }
+
+    @Override
+    public int encodeDouble(double v, byte[] b, int o) {
+      if (v < Integer.MIN_VALUE || v > Integer.MAX_VALUE) {
+        throw newIllegalDataException(
+            "Value " + v + " cannot be encoded as an Integer without changing its value");
+      }
+      return encodeInt((int) v, b, o);
+    }
+
+    @Override
+    public int encodeLong(long v, byte[] b, int o) {
+      if (v < Integer.MIN_VALUE || v > Integer.MAX_VALUE) {
+        throw newIllegalDataException(
+            "Value " + v + " cannot be encoded as an Integer without changing its value");
+      }
+      return encodeInt((int) v, b, o);
+    }
+
+    @Override
+    public byte decodeByte(byte[] b, int o, SortOrder sortOrder) {
+      int v = decodeInt(b, o, sortOrder);
+      if (v < Byte.MIN_VALUE || v > Byte.MAX_VALUE) {
+        throw newIllegalDataException(
+            "Value " + v + " cannot be cast to Byte without changing its value");
+      }
+      return (byte) v;
+    }
+
+    @Override
+    public short decodeShort(byte[] b, int o, SortOrder sortOrder) {
+      int v = decodeInt(b, o, sortOrder);
+      if (v < Short.MIN_VALUE || v > Short.MAX_VALUE) {
+        throw newIllegalDataException(
+            "Value " + v + " cannot be cast to Short without changing its value");
+      }
+      return (short) v;
+    }
+
+    @Override
+    public int encodeByte(byte v, byte[] b, int o) {
+      return encodeInt(v, b, o);
+    }
+
+    @Override
+    public int encodeShort(short v, byte[] b, int o) {
+      return encodeInt(v, b, o);
+    }
+
+    @Override
+    public PhoenixArrayFactory getPhoenixArrayFactory() {
+      return new PhoenixArrayFactory() {
+        @Override
+        public PhoenixArray newArray(PDataType type, Object[] elements) {
+          return new PhoenixArray.PrimitiveIntPhoenixArray(type, elements);
+        }
+      };
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/04ef859b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PIntegerArray.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PIntegerArray.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PIntegerArray.java
new file mode 100644
index 0000000..28a5c4c
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PIntegerArray.java
@@ -0,0 +1,109 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.phoenix.schema.types;
+
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.phoenix.schema.SortOrder;
+
+import java.sql.Types;
+
+public class PIntegerArray extends PArrayDataType<int[]> {
+
+  public static final PIntegerArray INSTANCE = new PIntegerArray();
+
+  private PIntegerArray() {
+    super("INTEGER ARRAY", PDataType.ARRAY_TYPE_BASE + PInteger.INSTANCE.getSqlType(),
+        PhoenixArray.class, null, 24);
+  }
+
+  @Override
+  public boolean isArrayType() {
+    return true;
+  }
+
+  @Override
+  public boolean isFixedWidth() {
+    return false;
+  }
+
+  @Override
+  public int compareTo(Object lhs, Object rhs, PDataType rhsType) {
+    return compareTo(lhs, rhs);
+  }
+
+  @Override
+  public Integer getByteSize() {
+    return null;
+  }
+
+  @Override
+  public byte[] toBytes(Object object) {
+    return toBytes(object, SortOrder.ASC);
+  }
+
+  @Override
+  public byte[] toBytes(Object object, SortOrder sortOrder) {
+    return toBytes(object, PInteger.INSTANCE, sortOrder);
+  }
+
+  @Override
+  public Object toObject(byte[] bytes, int offset, int length,
+      PDataType actualType, SortOrder sortOrder, Integer maxLength,
+      Integer scale) {
+    return toObject(bytes, offset, length, PInteger.INSTANCE, sortOrder, maxLength, scale,
+        PInteger.INSTANCE);
+  }
+
+  @Override
+  public boolean isCoercibleTo(PDataType targetType) {
+    return isCoercibleTo(targetType, this);
+  }
+
+  @Override
+  public void coerceBytes(ImmutableBytesWritable ptr, Object object, PDataType actualType,
+      Integer maxLength, Integer scale, SortOrder actualModifer, Integer desiredMaxLength,
+      Integer desiredScale, SortOrder desiredModifier) {
+    coerceBytes(ptr, object, actualType, maxLength, scale, desiredMaxLength, desiredScale,
+        this, actualModifer, desiredModifier);
+  }
+
+  @Override
+  public boolean isCoercibleTo(PDataType targetType, Object value) {
+    if (value == null) {
+      return true;
+    }
+    PhoenixArray.PrimitiveIntPhoenixArray pArr = (PhoenixArray.PrimitiveIntPhoenixArray) value;
+    int[] intArr = (int[]) pArr.array;
+    for (int i : intArr) {
+      if (!super.isCoercibleTo(PInteger.INSTANCE, i)) {
+        return false;
+      }
+    }
+    return true;
+  }
+
+  @Override
+  public int getResultSetSqlType() {
+    return Types.ARRAY;
+  }
+
+  @Override
+  public Object getSampleValue(Integer maxLength, Integer arrayLength) {
+    return getSampleValue(PInteger.INSTANCE, arrayLength, maxLength);
+  }
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/04ef859b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PLong.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PLong.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PLong.java
new file mode 100644
index 0000000..1bef844
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PLong.java
@@ -0,0 +1,331 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.phoenix.schema.types;
+
+import com.google.common.base.Preconditions;
+import com.google.common.primitives.Doubles;
+import com.google.common.primitives.Longs;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.phoenix.schema.SortOrder;
+
+import java.math.BigDecimal;
+import java.sql.Types;
+
+public class PLong extends PDataType<Long> {
+
+  public static final PLong INSTANCE = new PLong();
+
+  private PLong() {
+    super("BIGINT", Types.BIGINT, Long.class, new LongCodec(), 2);
+  }
+
+  @Override
+  public Integer getScale(Object o) {
+    return ZERO;
+  }
+
+  @Override
+  public byte[] toBytes(Object object) {
+    byte[] b = new byte[Bytes.SIZEOF_LONG];
+    toBytes(object, b, 0);
+    return b;
+  }
+
+  @Override
+  public int toBytes(Object object, byte[] b, int o) {
+    if (object == null) {
+      throw newIllegalDataException(this + " may not be null");
+    }
+    return this.getCodec().encodeLong(((Number) object).longValue(), b, o);
+  }
+
+  @Override
+  public Object toObject(Object object, PDataType actualType) {
+    if (object == null) {
+      return null;
+    }
+    long s;
+    if (equalsAny(actualType, PLong.INSTANCE, PUnsignedLong.INSTANCE)) {
+      return object;
+    } else if (equalsAny(actualType, PUnsignedInt.INSTANCE,
+        PInteger.INSTANCE)) {
+      s = (Integer) object;
+      return s;
+    } else if (equalsAny(actualType, PTinyint.INSTANCE, PUnsignedTinyint.INSTANCE)) {
+      s = (Byte) object;
+      return s;
+    } else if (equalsAny(actualType, PSmallint.INSTANCE, PUnsignedSmallint.INSTANCE)) {
+      s = (Short) object;
+      return s;
+    } else if (equalsAny(actualType, PFloat.INSTANCE, PUnsignedFloat.INSTANCE)) {
+      Float f = (Float) object;
+      if (f > Long.MAX_VALUE || f < Long.MIN_VALUE) {
+        throw newIllegalDataException(
+            actualType + " value " + f + " cannot be cast to Long without changing its value");
+      }
+      s = f.longValue();
+      return s;
+    } else if (equalsAny(actualType, PDouble.INSTANCE, PUnsignedDouble.INSTANCE)) {
+      Double de = (Double) object;
+      if (de > Long.MAX_VALUE || de < Long.MIN_VALUE) {
+        throw newIllegalDataException(
+            actualType + " value " + de + " cannot be cast to Long without changing its value");
+      }
+      s = de.longValue();
+      return s;
+    } else if (actualType == PDecimal.INSTANCE) {
+      BigDecimal d = (BigDecimal) object;
+      return d.longValueExact();
+    } else if (equalsAny(actualType, PDate.INSTANCE, PUnsignedDate.INSTANCE, PTime.INSTANCE,
+        PUnsignedTime.INSTANCE)) {
+      java.util.Date date = (java.util.Date) object;
+      return date.getTime();
+    }
+    return throwConstraintViolationException(actualType, this);
+  }
+
+  @Override
+  public Long toObject(byte[] b, int o, int l, PDataType actualType, SortOrder sortOrder,
+      Integer maxLength, Integer scale) {
+    if (l == 0) {
+      return null;
+    }
+    if (equalsAny(actualType, PLong.INSTANCE, PUnsignedLong.INSTANCE,
+        PInteger.INSTANCE, PUnsignedInt.INSTANCE, PSmallint.INSTANCE,
+        PUnsignedSmallint.INSTANCE, PTinyint.INSTANCE, PUnsignedTinyint.INSTANCE, PFloat.INSTANCE,
+        PUnsignedFloat.INSTANCE, PDouble.INSTANCE, PUnsignedDouble.INSTANCE, PDate.INSTANCE,
+        PUnsignedDate.INSTANCE, PTime.INSTANCE, PUnsignedTime.INSTANCE)) {
+      return actualType.getCodec().decodeLong(b, o, sortOrder);
+    } else if (actualType == PDecimal.INSTANCE) {
+      BigDecimal bd = (BigDecimal) actualType.toObject(b, o, l, actualType, sortOrder);
+      return bd.longValueExact();
+    }
+    throwConstraintViolationException(actualType, this);
+    return null;
+  }
+
+  @Override
+  public boolean isCoercibleTo(PDataType targetType) {
+    // In general, don't allow conversion of LONG to INTEGER. There are times when
+    // we check isComparableTo for a more relaxed check and then throw a runtime
+    // exception if we overflow
+    return equalsAny(targetType, this, PDecimal.INSTANCE, PVarbinary.INSTANCE, PBinary.INSTANCE, PDouble.INSTANCE);
+  }
+
+  @Override
+  public boolean isComparableTo(PDataType targetType) {
+    return PDecimal.INSTANCE.isComparableTo(targetType);
+  }
+
+  @Override
+  public boolean isCoercibleTo(PDataType targetType, Object value) {
+    if (value != null) {
+      long l;
+      if (equalsAny(targetType, PUnsignedDouble.INSTANCE, PUnsignedFloat.INSTANCE,
+          PUnsignedLong.INSTANCE)) {
+        l = (Long) value;
+        return l >= 0;
+      } else if (targetType.equals(PUnsignedInt.INSTANCE)) {
+        l = (Long) value;
+        return (l >= 0 && l <= Integer.MAX_VALUE);
+      } else if (targetType.equals(PInteger.INSTANCE)) {
+        l = (Long) value;
+        return (l >= Integer.MIN_VALUE && l <= Integer.MAX_VALUE);
+      } else if (targetType.equals(PUnsignedSmallint.INSTANCE)) {
+        l = (Long) value;
+        return (l >= 0 && l <= Short.MAX_VALUE);
+      } else if (targetType.equals(PSmallint.INSTANCE)) {
+        l = (Long) value;
+        return (l >= Short.MIN_VALUE && l <= Short.MAX_VALUE);
+      } else if (targetType.equals(PTinyint.INSTANCE)) {
+        l = (Long) value;
+        return (l >= Byte.MIN_VALUE && l < Byte.MAX_VALUE);
+      } else if (targetType.equals(PUnsignedTinyint.INSTANCE)) {
+        l = (Long) value;
+        return (l >= 0 && l < Byte.MAX_VALUE);
+      }
+    }
+    return super.isCoercibleTo(targetType, value);
+  }
+
+  @Override
+  public boolean isCastableTo(PDataType targetType) {
+    return super.isCastableTo(targetType) || targetType.isCoercibleTo(PTimestamp.INSTANCE);
+  }
+
+  @Override
+  public boolean isFixedWidth() {
+    return true;
+  }
+
+  @Override
+  public Integer getByteSize() {
+    return Bytes.SIZEOF_LONG;
+  }
+
+  @Override
+  public Integer getMaxLength(Object o) {
+    return LONG_PRECISION;
+  }
+
+  @Override
+  public int compareTo(Object lhs, Object rhs, PDataType rhsType) {
+    if (rhsType == PDecimal.INSTANCE) {
+      return -((BigDecimal) rhs).compareTo(BigDecimal.valueOf(((Number) lhs).longValue()));
+    } else if (equalsAny(rhsType, PDouble.INSTANCE, PFloat.INSTANCE, PUnsignedDouble.INSTANCE, PUnsignedFloat.INSTANCE)) {
+      return Doubles.compare(((Number) lhs).doubleValue(), ((Number) rhs).doubleValue());
+    }
+    return Longs.compare(((Number) lhs).longValue(), ((Number) rhs).longValue());
+  }
+
+  @Override
+  public Object toObject(String value) {
+    if (value == null || value.length() == 0) {
+      return null;
+    }
+    try {
+      return Long.parseLong(value);
+    } catch (NumberFormatException e) {
+      throw newIllegalDataException(e);
+    }
+  }
+
+  @Override
+  public Object getSampleValue(Integer maxLength, Integer arrayLength) {
+    return RANDOM.get().nextLong();
+  }
+
+  static class LongCodec extends BaseCodec {
+
+    @Override
+    public float decodeFloat(byte[] b, int o, SortOrder sortOrder) {
+      return decodeLong(b, o, sortOrder);
+    }
+
+    @Override
+    public double decodeDouble(byte[] b, int o, SortOrder sortOrder) {
+      return decodeLong(b, o, sortOrder);
+    }
+
+    @Override
+    public long decodeLong(byte[] bytes, int o, SortOrder sortOrder) {
+      Preconditions.checkNotNull(sortOrder);
+      checkForSufficientLength(bytes, o, Bytes.SIZEOF_LONG);
+      long v;
+      byte b = bytes[o];
+      if (sortOrder == SortOrder.ASC) {
+        v = b ^ 0x80; // Flip sign bit back
+        for (int i = 1; i < Bytes.SIZEOF_LONG; i++) {
+          b = bytes[o + i];
+          v = (v << 8) + (b & 0xff);
+        }
+      } else {
+        b = (byte) (b ^ 0xff);
+        v = b ^ 0x80; // Flip sign bit back
+        for (int i = 1; i < Bytes.SIZEOF_LONG; i++) {
+          b = bytes[o + i];
+          b ^= 0xff;
+          v = (v << 8) + (b & 0xff);
+        }
+      }
+      return v;
+    }
+
+    @Override
+    public int decodeInt(byte[] b, int o, SortOrder sortOrder) {
+      long v = decodeLong(b, o, sortOrder);
+      if (v < Integer.MIN_VALUE || v > Integer.MAX_VALUE) {
+        throw newIllegalDataException(
+            "Value " + v + " cannot be cast to Integer without changing its value");
+      }
+      return (int) v;
+    }
+
+    @Override
+    public int encodeFloat(float v, byte[] b, int o) {
+      if (v < Long.MIN_VALUE || v > Long.MAX_VALUE) {
+        throw newIllegalDataException(
+            "Value " + v + " cannot be encoded as an Long without changing its value");
+      }
+      return encodeLong((long) v, b, o);
+    }
+
+    @Override
+    public int encodeDouble(double v, byte[] b, int o) {
+      if (v < Long.MIN_VALUE || v > Long.MAX_VALUE) {
+        throw newIllegalDataException(
+            "Value " + v + " cannot be encoded as an Long without changing its value");
+      }
+      return encodeLong((long) v, b, o);
+    }
+
+    @Override
+    public int encodeLong(long v, byte[] b, int o) {
+      checkForSufficientLength(b, o, Bytes.SIZEOF_LONG);
+      b[o + 0] = (byte) ((v >> 56) ^ 0x80); // Flip sign bit so that INTEGER is binary comparable
+      b[o + 1] = (byte) (v >> 48);
+      b[o + 2] = (byte) (v >> 40);
+      b[o + 3] = (byte) (v >> 32);
+      b[o + 4] = (byte) (v >> 24);
+      b[o + 5] = (byte) (v >> 16);
+      b[o + 6] = (byte) (v >> 8);
+      b[o + 7] = (byte) v;
+      return Bytes.SIZEOF_LONG;
+    }
+
+    @Override
+    public byte decodeByte(byte[] b, int o, SortOrder sortOrder) {
+      long v = decodeLong(b, o, sortOrder);
+      if (v < Byte.MIN_VALUE || v > Byte.MAX_VALUE) {
+        throw newIllegalDataException(
+            "Value " + v + " cannot be cast to Byte without changing its value");
+      }
+      return (byte) v;
+    }
+
+    @Override
+    public short decodeShort(byte[] b, int o, SortOrder sortOrder) {
+      long v = decodeLong(b, o, sortOrder);
+      if (v < Short.MIN_VALUE || v > Short.MAX_VALUE) {
+        throw newIllegalDataException(
+            "Value " + v + " cannot be cast to Short without changing its value");
+      }
+      return (short) v;
+    }
+
+    @Override
+    public int encodeByte(byte v, byte[] b, int o) {
+      return encodeLong(v, b, o);
+    }
+
+    @Override
+    public int encodeShort(short v, byte[] b, int o) {
+      return encodeLong(v, b, o);
+    }
+
+    @Override
+    public PhoenixArrayFactory getPhoenixArrayFactory() {
+      return new PhoenixArrayFactory() {
+        @Override
+        public PhoenixArray newArray(PDataType type, Object[] elements) {
+          return new PhoenixArray.PrimitiveLongPhoenixArray(type, elements);
+        }
+      };
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/04ef859b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PLongArray.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PLongArray.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PLongArray.java
new file mode 100644
index 0000000..0670792
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PLongArray.java
@@ -0,0 +1,109 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.phoenix.schema.types;
+
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.phoenix.schema.SortOrder;
+
+import java.sql.Types;
+
+public class PLongArray extends PArrayDataType<long[]> {
+
+  public static final PLongArray INSTANCE = new PLongArray();
+
+  private PLongArray() {
+    super("BIGINT ARRAY", PDataType.ARRAY_TYPE_BASE + PLong.INSTANCE.getSqlType(),
+        PhoenixArray.class, null, 30);
+  }
+
+  @Override
+  public boolean isArrayType() {
+    return true;
+  }
+
+  @Override
+  public boolean isFixedWidth() {
+    return false;
+  }
+
+  @Override
+  public int compareTo(Object lhs, Object rhs, PDataType rhsType) {
+    return compareTo(lhs, rhs);
+  }
+
+  @Override
+  public Integer getByteSize() {
+    return null;
+  }
+
+  @Override
+  public byte[] toBytes(Object object) {
+    return toBytes(object, SortOrder.ASC);
+  }
+
+  @Override
+  public byte[] toBytes(Object object, SortOrder sortOrder) {
+    return toBytes(object, PLong.INSTANCE, sortOrder);
+  }
+
+  @Override
+  public Object toObject(byte[] bytes, int offset, int length,
+      PDataType actualType, SortOrder sortOrder, Integer maxLength,
+      Integer scale) {
+    return toObject(bytes, offset, length, PLong.INSTANCE, sortOrder, maxLength, scale,
+        PLong.INSTANCE);
+  }
+
+  @Override
+  public boolean isCoercibleTo(PDataType targetType) {
+    return isCoercibleTo(targetType, this);
+  }
+
+  @Override
+  public boolean isCoercibleTo(PDataType targetType, Object value) {
+    if (value == null) {
+      return true;
+    }
+    PhoenixArray.PrimitiveLongPhoenixArray pArr = (PhoenixArray.PrimitiveLongPhoenixArray) value;
+    long[] longArr = (long[]) pArr.array;
+    for (long i : longArr) {
+      if (!super.isCoercibleTo(PLong.INSTANCE, i)) {
+        return false;
+      }
+    }
+    return true;
+  }
+
+  @Override
+  public int getResultSetSqlType() {
+    return Types.ARRAY;
+  }
+
+  @Override
+  public void coerceBytes(ImmutableBytesWritable ptr, Object object, PDataType actualType,
+      Integer maxLength, Integer scale, SortOrder actualModifer, Integer desiredMaxLength,
+      Integer desiredScale, SortOrder desiredModifier) {
+    coerceBytes(ptr, object, actualType, maxLength, scale, desiredMaxLength, desiredScale,
+        this, actualModifer, desiredModifier);
+  }
+
+  @Override
+  public Object getSampleValue(Integer maxLength, Integer arrayLength) {
+    return getSampleValue(PLong.INSTANCE, arrayLength, maxLength);
+  }
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/04ef859b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PSmallint.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PSmallint.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PSmallint.java
new file mode 100644
index 0000000..6ba631a
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PSmallint.java
@@ -0,0 +1,259 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.phoenix.schema.types;
+
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.phoenix.schema.SortOrder;
+
+import java.math.BigDecimal;
+import java.sql.Types;
+
+public class PSmallint extends PDataType<Short> {
+
+  public static final PSmallint INSTANCE = new PSmallint();
+
+  private PSmallint() {
+    super("SMALLINT", Types.SMALLINT, Short.class, new ShortCodec(), 4);
+  }
+
+  @Override
+    public Integer getScale(Object o) {
+      return ZERO;
+    }
+
+    @Override
+    public int compareTo(Object lhs, Object rhs, PDataType rhsType) {
+      return PLong.INSTANCE.compareTo(lhs, rhs, rhsType);
+    }
+
+    @Override
+    public boolean isComparableTo(PDataType targetType) {
+      return PDecimal.INSTANCE.isComparableTo(targetType);
+    }
+
+    @Override
+    public boolean isFixedWidth() {
+      return true;
+    }
+
+    @Override
+    public Integer getByteSize() {
+      return Bytes.SIZEOF_SHORT;
+    }
+
+    @Override
+    public Integer getMaxLength(Object o) {
+      return SHORT_PRECISION;
+    }
+
+    @Override
+    public byte[] toBytes(Object object) {
+      byte[] b = new byte[Bytes.SIZEOF_SHORT];
+      toBytes(object, b, 0);
+      return b;
+    }
+
+    @Override
+    public int toBytes(Object object, byte[] bytes, int offset) {
+      if (object == null) {
+        throw newIllegalDataException(this + " may not be null");
+      }
+      return this.getCodec().encodeShort(((Number)object).shortValue(), bytes, offset);
+    }
+
+    @Override
+    public Object toObject(Object object, PDataType actualType) {
+      Object o = PLong.INSTANCE.toObject(object, actualType);
+      if (!(o instanceof Long) || o == null) {
+        return o;
+      }
+      long l = (Long)o;
+      if (l < Short.MIN_VALUE || l > Short.MAX_VALUE) {
+        throw newIllegalDataException(actualType + " value " + l + " cannot be cast to Short without changing its value");
+      }
+      short s = (short)l;
+      return s;
+    }
+
+    @Override
+    public Short toObject(byte[] b, int o, int l, PDataType actualType, SortOrder sortOrder, Integer maxLength, Integer scale) {
+      if (l == 0) {
+        return null;
+      }
+      if (equalsAny(actualType, PSmallint.INSTANCE, PUnsignedSmallint.INSTANCE, PTinyint.INSTANCE,
+          PUnsignedTinyint.INSTANCE, PLong.INSTANCE, PUnsignedLong.INSTANCE, PInteger.INSTANCE,
+          PUnsignedInt.INSTANCE, PFloat.INSTANCE, PUnsignedFloat.INSTANCE, PDouble.INSTANCE,
+          PUnsignedDouble.INSTANCE)) {
+        return actualType.getCodec().decodeShort(b, o, sortOrder);
+      } else if (actualType == PDecimal.INSTANCE) {
+        BigDecimal bd = (BigDecimal)actualType.toObject(b, o, l, actualType, sortOrder);
+        return bd.shortValueExact();
+      }
+      throwConstraintViolationException(actualType,this);
+      return null;
+    }
+
+    @Override
+    public Object toObject(String value) {
+      if (value == null || value.length() == 0) {
+        return null;
+      }
+      try {
+        return Short.parseShort(value);
+      } catch (NumberFormatException e) {
+        throw newIllegalDataException(e);
+      }
+    }
+
+    @Override
+    public boolean isCoercibleTo(PDataType targetType, Object value) {
+      if (value != null) {
+        short i;
+        if (equalsAny(targetType, PUnsignedDouble.INSTANCE, PUnsignedFloat.INSTANCE,
+            PUnsignedLong.INSTANCE, PUnsignedInt.INSTANCE, PUnsignedSmallint.INSTANCE)) {
+          i = (Short) value;
+          return i >= 0;
+        } else if (targetType == PUnsignedTinyint.INSTANCE) {
+          i = (Short) value;
+          return (i >= 0 && i <= Byte.MAX_VALUE);
+        } else if (targetType == PTinyint.INSTANCE) {
+          i = (Short) value;
+          return (i >= Byte.MIN_VALUE && i <= Byte.MAX_VALUE);
+        }
+      }
+      return super.isCoercibleTo(targetType, value);
+    }
+
+    @Override
+    public boolean isCoercibleTo(PDataType targetType) {
+      return this.equals(targetType) || PInteger.INSTANCE.isCoercibleTo(targetType);
+    }
+
+    @Override
+    public Object getSampleValue(Integer maxLength, Integer arrayLength) {
+      return ((Integer) PInteger.INSTANCE.getSampleValue(maxLength, arrayLength)).shortValue();
+    }
+
+  static class ShortCodec extends BaseCodec {
+
+      @Override
+      public long decodeLong(byte[] b, int o, SortOrder sortOrder) {
+        return decodeShort(b, o, sortOrder);
+      }
+
+      @Override
+      public int decodeInt(byte[] b, int o, SortOrder sortOrder) {
+        return decodeShort(b, o, sortOrder);
+      }
+
+      @Override
+      public byte decodeByte(byte[] b, int o, SortOrder sortOrder) {
+        short v = decodeShort(b, o, sortOrder);
+        if (v < Byte.MIN_VALUE || v > Byte.MAX_VALUE) {
+            throw newIllegalDataException("Value " + v + " cannot be cast to Byte without changing its value");
+        }
+        return (byte)v;
+      }
+
+      @Override
+      public short decodeShort(byte[] b, int o, SortOrder sortOrder) {
+    	Preconditions.checkNotNull(sortOrder);
+        checkForSufficientLength(b, o, Bytes.SIZEOF_SHORT);
+        int v;
+        if (sortOrder == SortOrder.ASC) {
+            v = b[o] ^ 0x80; // Flip sign bit back
+            for (int i = 1; i < Bytes.SIZEOF_SHORT; i++) {
+                v = (v << 8) + (b[o + i] & 0xff);
+            }
+        } else {
+            v = b[o] ^ 0xff ^ 0x80; // Flip sign bit back
+            for (int i = 1; i < Bytes.SIZEOF_SHORT; i++) {
+                v = (v << 8) + ((b[o + i] ^ 0xff) & 0xff);
+            }
+        }
+        return (short)v;
+      }
+
+      @Override
+      public int encodeShort(short v, byte[] b, int o) {
+          checkForSufficientLength(b, o, Bytes.SIZEOF_SHORT);
+          b[o + 0] = (byte) ((v >> 8) ^ 0x80); // Flip sign bit so that Short is binary comparable
+          b[o + 1] = (byte) v;
+          return Bytes.SIZEOF_SHORT;
+      }
+
+      @Override
+      public int encodeLong(long v, byte[] b, int o) {
+          if (v < Short.MIN_VALUE || v > Short.MAX_VALUE) {
+              throw newIllegalDataException("Value " + v + " cannot be encoded as an Short without changing its value");
+          }
+          return encodeShort((short)v,b,o);
+      }
+
+      @Override
+      public int encodeInt(int v, byte[] b, int o) {
+        if (v < Short.MIN_VALUE || v > Short.MAX_VALUE) {
+          throw newIllegalDataException("Value " + v + " cannot be encoded as an Short without changing its value");
+        }
+        return encodeShort((short)v,b,o);
+      }
+
+      @Override
+      public int encodeByte(byte v, byte[] b, int o) {
+        return encodeShort(v,b,o);
+      }
+
+      @Override
+      public float decodeFloat(byte[] b, int o, SortOrder sortOrder) {
+          return decodeShort(b, o, sortOrder);
+      }
+
+      @Override
+      public double decodeDouble(byte[] b, int o,
+              SortOrder sortOrder) {
+          return decodeShort(b, o, sortOrder);
+      }
+
+      @Override
+      public int encodeDouble(double v, byte[] b, int o) {
+          if (v < Short.MIN_VALUE || v > Short.MAX_VALUE) {
+              throw newIllegalDataException("Value " + v + " cannot be encoded as an Short without changing its value");
+          }
+          return encodeShort((short)v,b,o);
+      }
+
+      @Override
+      public int encodeFloat(float v, byte[] b, int o) {
+          if (v < Short.MIN_VALUE || v > Short.MAX_VALUE) {
+              throw newIllegalDataException("Value " + v + " cannot be encoded as an Short without changing its value");
+          }
+          return encodeShort((short)v,b,o);
+      }
+
+      @Override
+      public PhoenixArrayFactory getPhoenixArrayFactory() {
+          return new PhoenixArrayFactory() {
+              @Override
+              public PhoenixArray newArray(PDataType type, Object[] elements) {
+                  return new PhoenixArray.PrimitiveShortPhoenixArray(type, elements);
+              }
+          };
+      }
+    }
+}