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

[11/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/PSmallintArray.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PSmallintArray.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PSmallintArray.java
new file mode 100644
index 0000000..98da723
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PSmallintArray.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 PSmallintArray extends PArrayDataType<short[]> {
+
+  public static final PSmallintArray INSTANCE = new PSmallintArray();
+
+  private PSmallintArray() {
+    super("SMALLINT ARRAY", PDataType.ARRAY_TYPE_BASE + PSmallint.INSTANCE.getSqlType(),
+        PhoenixArray.class, null, 31);
+  }
+
+  @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, PSmallint.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, PSmallint.INSTANCE, sortOrder, maxLength, scale,
+        PSmallint.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.PrimitiveShortPhoenixArray pArr = (PhoenixArray.PrimitiveShortPhoenixArray) value;
+    short[] shortArr = (short[]) pArr.array;
+    for (short i : shortArr) {
+      if (!super.isCoercibleTo(PSmallint.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(PSmallint.INSTANCE, arrayLength, maxLength);
+  }
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/04ef859b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PTime.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PTime.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PTime.java
new file mode 100644
index 0000000..319f801
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PTime.java
@@ -0,0 +1,137 @@
+/*
+ * 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.util.Bytes;
+import org.apache.phoenix.schema.SortOrder;
+import org.apache.phoenix.util.DateUtil;
+
+import java.math.BigDecimal;
+import java.sql.Time;
+import java.sql.Types;
+import java.text.Format;
+
+public class PTime extends PDataType<Time> {
+
+  public static final PTime INSTANCE = new PTime();
+
+  private PTime() {
+    super("TIME", Types.TIME, java.sql.Time.class, new PDate.DateCodec(), 10);
+  }
+
+  @Override
+  public byte[] toBytes(Object object) {
+    return PDate.INSTANCE.toBytes(object);
+  }
+
+  @Override
+  public int toBytes(Object object, byte[] bytes, int offset) {
+    return PDate.INSTANCE.toBytes(object, bytes, offset);
+  }
+
+  @Override
+  public java.sql.Time toObject(byte[] b, int o, int l, PDataType actualType, SortOrder sortOrder,
+      Integer maxLength, Integer scale) {
+    if (l == 0) {
+      return null;
+    }
+    if (equalsAny(actualType, PTimestamp.INSTANCE, PUnsignedTimestamp.INSTANCE, PDate.INSTANCE,
+        PUnsignedDate.INSTANCE, PTime.INSTANCE, PUnsignedTime.INSTANCE, PLong.INSTANCE,
+        PUnsignedLong.INSTANCE)) {
+      return new java.sql.Time(actualType.getCodec().decodeLong(b, o, sortOrder));
+    } else if (actualType == PDecimal.INSTANCE) {
+      BigDecimal bd = (BigDecimal) actualType.toObject(b, o, l, actualType, sortOrder);
+      return new java.sql.Time(bd.longValueExact());
+    }
+    throwConstraintViolationException(actualType, this);
+    return null;
+  }
+
+  @Override
+  public Object toObject(Object object, PDataType actualType) {
+    if (object == null) {
+      return null;
+    }
+    if (equalsAny(actualType, PDate.INSTANCE, PUnsignedDate.INSTANCE)) {
+      return new java.sql.Time(((java.util.Date) object).getTime());
+    } else if (equalsAny(actualType, PTimestamp.INSTANCE, PUnsignedTimestamp.INSTANCE)) {
+      return new java.sql.Time(((java.sql.Timestamp) object).getTime());
+    } else if (equalsAny(actualType, PTime.INSTANCE, PUnsignedTime.INSTANCE)) {
+      return object;
+    } else if (equalsAny(actualType, PLong.INSTANCE, PUnsignedLong.INSTANCE)) {
+      return new java.sql.Time((Long) object);
+    } else if (actualType == PDecimal.INSTANCE) {
+      return new java.sql.Time(((BigDecimal) object).longValueExact());
+    }
+    return throwConstraintViolationException(actualType, this);
+  }
+
+  @Override
+  public boolean isCastableTo(PDataType targetType) {
+    return PDate.INSTANCE.isCastableTo(targetType);
+  }
+
+  @Override
+  public boolean isCoercibleTo(PDataType targetType) {
+    return PDate.INSTANCE.isCoercibleTo(targetType);
+  }
+
+  @Override
+  public boolean isCoercibleTo(PDataType targetType, Object value) {
+    return PDate.INSTANCE.isCoercibleTo(targetType, value);
+  }
+
+  @Override
+  public boolean isFixedWidth() {
+    return true;
+  }
+
+  @Override
+  public Integer getByteSize() {
+    return Bytes.SIZEOF_LONG;
+  }
+
+  @Override
+  public int compareTo(Object lhs, Object rhs, PDataType rhsType) {
+    return PDate.INSTANCE.compareTo(lhs, rhs, rhsType);
+  }
+
+  @Override
+  public Object toObject(String value) {
+    if (value == null || value.length() == 0) {
+      return null;
+    }
+    return DateUtil.parseTime(value);
+  }
+
+  @Override
+  public boolean isBytesComparableWith(PDataType otherType) {
+    return super.isBytesComparableWith(otherType) || otherType.equals(PDate.INSTANCE);
+  }
+
+  @Override
+  public String toStringLiteral(byte[] b, int offset, int length, Format formatter) {
+    // TODO: different default formatter for TIME?
+    return PDate.INSTANCE.toStringLiteral(b, offset, length, formatter);
+  }
+
+  @Override
+  public Object getSampleValue(Integer maxLength, Integer arrayLength) {
+    return new java.sql.Time((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/PTimeArray.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PTimeArray.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PTimeArray.java
new file mode 100644
index 0000000..e0a7655
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PTimeArray.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.sql.Time;
+import java.sql.Types;
+
+public class PTimeArray extends PArrayDataType<Time[]> {
+
+  public static final PTimeArray INSTANCE = new PTimeArray();
+
+  private PTimeArray() {
+    super("TIME ARRAY", PDataType.ARRAY_TYPE_BASE + PTime.INSTANCE.getSqlType(), PhoenixArray.class,
+        null, 38);
+  }
+
+  @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, PTime.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, PTime.INSTANCE, sortOrder, maxLength, scale,
+        PTime.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 pArr = (PhoenixArray) value;
+    Object[] timeArr = (Object[]) pArr.array;
+    for (Object i : timeArr) {
+      if (!super.isCoercibleTo(PTime.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(PTime.INSTANCE, arrayLength, maxLength);
+  }
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/04ef859b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PTimestamp.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PTimestamp.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PTimestamp.java
new file mode 100644
index 0000000..2b95611
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PTimestamp.java
@@ -0,0 +1,209 @@
+/*
+ * 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.hadoop.hbase.util.Bytes;
+import org.apache.phoenix.query.QueryConstants;
+import org.apache.phoenix.schema.SortOrder;
+import org.apache.phoenix.util.DateUtil;
+
+import java.math.BigDecimal;
+import java.sql.Timestamp;
+import java.sql.Types;
+import java.text.Format;
+
+public class PTimestamp extends PDataType<Timestamp> {
+
+  public static final PTimestamp INSTANCE = new PTimestamp();
+
+  private PTimestamp() {
+    super("TIMESTAMP", Types.TIMESTAMP, java.sql.Timestamp.class,
+        new PDate.DateCodec(), 9);
+  }
+
+  @Override
+  public byte[] toBytes(Object object) {
+    if (object == null) {
+      throw newIllegalDataException(this + " may not be null");
+    }
+    byte[] bytes = new byte[getByteSize()];
+    toBytes(object, bytes, 0);
+    return bytes;
+  }
+
+  @Override
+  public int toBytes(Object object, byte[] bytes, int offset) {
+    if (object == null) {
+      throw newIllegalDataException(this + " may not be null");
+    }
+    java.sql.Timestamp value = (java.sql.Timestamp) object;
+    PDate.INSTANCE.getCodec().encodeLong(value.getTime(), bytes, offset);
+
+            /*
+             * By not getting the stuff that got spilled over from the millis part,
+             * it leaves the timestamp's byte representation saner - 8 bytes of millis | 4 bytes of nanos.
+             * Also, it enables timestamp bytes to be directly compared with date/time bytes.
+             */
+    Bytes.putInt(bytes, offset + Bytes.SIZEOF_LONG, value.getNanos() % 1000000);
+    return getByteSize();
+  }
+
+  @Override
+  public Object toObject(Object object, PDataType actualType) {
+    if (object == null) {
+      return null;
+    }
+    if (equalsAny(actualType, PDate.INSTANCE, PUnsignedDate.INSTANCE, PTime.INSTANCE,
+        PUnsignedTime.INSTANCE)) {
+      return new java.sql.Timestamp(((java.util.Date) object).getTime());
+    } else if (equalsAny(actualType, PTimestamp.INSTANCE, PUnsignedTimestamp.INSTANCE)) {
+      return object;
+    } else if (equalsAny(actualType, PLong.INSTANCE, PUnsignedLong.INSTANCE)) {
+      return new java.sql.Timestamp((Long) object);
+    } else if (actualType == PDecimal.INSTANCE) {
+      BigDecimal bd = (BigDecimal) object;
+      long ms = bd.longValue();
+      int nanos =
+          (bd.remainder(BigDecimal.ONE).multiply(QueryConstants.BD_MILLIS_NANOS_CONVERSION))
+              .intValue();
+      return DateUtil.getTimestamp(ms, nanos);
+    }
+    return throwConstraintViolationException(actualType, this);
+  }
+
+  @Override
+  public java.sql.Timestamp toObject(byte[] b, int o, int l, PDataType actualType,
+      SortOrder sortOrder, Integer maxLength, Integer scale) {
+    if (actualType == null || l == 0) {
+      return null;
+    }
+    java.sql.Timestamp v;
+    if (equalsAny(actualType, PTimestamp.INSTANCE, PUnsignedTimestamp.INSTANCE)) {
+      long millisDeserialized =
+          (actualType == PTimestamp.INSTANCE ? PDate.INSTANCE : PUnsignedDate.INSTANCE).getCodec()
+              .decodeLong(b, o, sortOrder);
+      v = new java.sql.Timestamp(millisDeserialized);
+      int nanosDeserialized =
+          PUnsignedInt.INSTANCE.getCodec().decodeInt(b, o + Bytes.SIZEOF_LONG, sortOrder);
+                /*
+                 * There was a bug in serialization of timestamps which was causing the sub-second millis part
+                 * of time stamp to be present both in the LONG and INT bytes. Having the <100000 check
+                 * makes this serialization fix backward compatible.
+                 */
+      v.setNanos(
+          nanosDeserialized < 1000000 ? v.getNanos() + nanosDeserialized : nanosDeserialized);
+      return v;
+    } else if (equalsAny(actualType, PDate.INSTANCE, PUnsignedDate.INSTANCE, PTime.INSTANCE,
+        PUnsignedTime.INSTANCE, PLong.INSTANCE, PUnsignedLong.INSTANCE)) {
+      return new java.sql.Timestamp(actualType.getCodec().decodeLong(b, o, sortOrder));
+    } else if (actualType == PDecimal.INSTANCE) {
+      BigDecimal bd = (BigDecimal) actualType.toObject(b, o, l, actualType, sortOrder);
+      long ms = bd.longValue();
+      int nanos = (bd.remainder(BigDecimal.ONE).multiply(QueryConstants.BD_MILLIS_NANOS_CONVERSION))
+          .intValue();
+      v = DateUtil.getTimestamp(ms, nanos);
+      return v;
+    }
+    throwConstraintViolationException(actualType, this);
+    return null;
+  }
+
+  @Override
+  public boolean isCastableTo(PDataType targetType) {
+    return PDate.INSTANCE.isCastableTo(targetType);
+  }
+
+  @Override
+  public boolean isCoercibleTo(PDataType targetType) {
+    return equalsAny(targetType, this, PVarbinary.INSTANCE, PBinary.INSTANCE);
+  }
+
+  @Override
+  public boolean isCoercibleTo(PDataType targetType, Object value) {
+    if (value != null) {
+      if (targetType.equals(PUnsignedTimestamp.INSTANCE)) {
+        return ((java.util.Date) value).getTime() >= 0;
+      } else if (equalsAny(targetType, PUnsignedDate.INSTANCE, PUnsignedTime.INSTANCE)) {
+        return ((java.util.Date) value).getTime() >= 0
+            && ((java.sql.Timestamp) value).getNanos() == 0;
+      } else if (equalsAny(targetType, PDate.INSTANCE, PTime.INSTANCE)) {
+        return ((java.sql.Timestamp) value).getNanos() == 0;
+      }
+    }
+    return super.isCoercibleTo(targetType, value);
+  }
+
+  @Override
+  public boolean isFixedWidth() {
+    return true;
+  }
+
+  @Override
+  public Integer getByteSize() {
+    return MAX_TIMESTAMP_BYTES;
+  }
+
+  @Override
+  public int compareTo(Object lhs, Object rhs, PDataType rhsType) {
+    if (equalsAny(rhsType, PTimestamp.INSTANCE, PUnsignedTimestamp.INSTANCE)) {
+      return ((java.sql.Timestamp) lhs).compareTo((java.sql.Timestamp) rhs);
+    }
+    int c = ((java.util.Date) rhs).compareTo((java.util.Date) lhs);
+    if (c != 0) return c;
+    return ((java.sql.Timestamp) lhs).getNanos();
+  }
+
+  @Override
+  public Object toObject(String value) {
+    if (value == null || value.length() == 0) {
+      return null;
+    }
+    return DateUtil.parseTimestamp(value);
+  }
+
+  @Override
+  public String toStringLiteral(byte[] b, int offset, int length, Format formatter) {
+    java.sql.Timestamp value = (java.sql.Timestamp) toObject(b, offset, length);
+    if (formatter == null || formatter == DateUtil.DEFAULT_DATE_FORMATTER) {
+      // If default formatter has not been overridden,
+      // use one that displays milliseconds.
+      formatter = DateUtil.DEFAULT_MS_DATE_FORMATTER;
+    }
+    return "'" + super.toStringLiteral(b, offset, length, formatter) + "." + value.getNanos() + "'";
+  }
+
+  @Override
+  public int getNanos(ImmutableBytesWritable ptr, SortOrder sortOrder) {
+    int nanos = PUnsignedInt.INSTANCE.getCodec()
+        .decodeInt(ptr.get(), ptr.getOffset() + PLong.INSTANCE.getByteSize(), sortOrder);
+    return nanos;
+  }
+
+  @Override
+  public long getMillis(ImmutableBytesWritable ptr, SortOrder sortOrder) {
+    long millis = PLong.INSTANCE.getCodec().decodeLong(ptr.get(), ptr.getOffset(), sortOrder);
+    return millis;
+  }
+
+  @Override
+  public Object getSampleValue(Integer maxLength, Integer arrayLength) {
+    return new java.sql.Timestamp(
+        (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/PTimestampArray.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PTimestampArray.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PTimestampArray.java
new file mode 100644
index 0000000..d44bda9
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PTimestampArray.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.*;
+
+public class PTimestampArray extends PArrayDataType<Timestamp[]> {
+
+  public static final PTimestampArray INSTANCE = new PTimestampArray();
+
+  private PTimestampArray() {
+    super("TIMESTAMP ARRAY", PDataType.ARRAY_TYPE_BASE + PTimestamp.INSTANCE.getSqlType(),
+        PhoenixArray.class, null, 36);
+  }
+
+  @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, PTimestamp.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, PTimestamp.INSTANCE, sortOrder, maxLength, scale,
+        PTimestamp.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 pArr = (PhoenixArray) value;
+    Object[] timeStampArr = (Object[]) pArr.array;
+    for (Object i : timeStampArr) {
+      if (!super.isCoercibleTo(PTimestamp.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(PTimestamp.INSTANCE, arrayLength, maxLength);
+  }
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/04ef859b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PTinyint.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PTinyint.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PTinyint.java
new file mode 100644
index 0000000..681b41b
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PTinyint.java
@@ -0,0 +1,253 @@
+/*
+ * 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 PTinyint extends PDataType<Byte> {
+
+  public static final PTinyint INSTANCE = new PTinyint();
+
+  private PTinyint() {
+    super("TINYINT", Types.TINYINT, Byte.class, new ByteCodec(), 5);
+  }
+
+  @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_BYTE;
+  }
+
+  @Override
+  public Integer getMaxLength(Object o) {
+    return BYTE_PRECISION;
+  }
+
+  @Override
+  public byte[] toBytes(Object object) {
+    byte[] b = new byte[Bytes.SIZEOF_BYTE];
+    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().encodeByte(((Number) object).byteValue(), bytes, offset);
+  }
+
+  @Override
+  public Object toObject(String value) {
+    if (value == null || value.length() == 0) {
+      return null;
+    }
+    try {
+      Byte b = Byte.parseByte(value);
+      return b;
+    } catch (NumberFormatException e) {
+      throw newIllegalDataException(e);
+    }
+  }
+
+  @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 < Byte.MIN_VALUE || l > Byte.MAX_VALUE) {
+      throw newIllegalDataException(
+          actualType + " value " + l + " cannot be cast to Byte without changing its value");
+    }
+    return (byte) l;
+  }
+
+  @Override
+  public Byte 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().decodeByte(b, o, sortOrder);
+    } else if (actualType == PDecimal.INSTANCE) {
+      BigDecimal bd = (BigDecimal) actualType.toObject(b, o, l, actualType, sortOrder);
+      return bd.byteValueExact();
+    }
+    throwConstraintViolationException(actualType, this);
+    return null;
+  }
+
+  @Override
+  public boolean isCoercibleTo(PDataType targetType, Object value) {
+    if (value != null) {
+      if (equalsAny(targetType, this, PUnsignedDouble.INSTANCE, PUnsignedFloat.INSTANCE,
+          PUnsignedLong.INSTANCE, PUnsignedInt.INSTANCE, PUnsignedSmallint.INSTANCE,
+          PUnsignedTinyint.INSTANCE)) {
+        byte i = (Byte) value;
+        return i >= 0;
+      }
+    }
+    return super.isCoercibleTo(targetType, value);
+  }
+
+  @Override
+  public boolean isCoercibleTo(PDataType targetType) {
+    return this.equals(targetType) || PSmallint.INSTANCE.isCoercibleTo(targetType);
+  }
+
+  @Override
+  public Object getSampleValue(Integer maxLength, Integer arrayLength) {
+    return ((Integer) PInteger.INSTANCE.getSampleValue(maxLength, arrayLength))
+        .byteValue();
+  }
+
+  static class ByteCodec extends BaseCodec {
+
+    @Override
+    public long decodeLong(byte[] b, int o, SortOrder sortOrder) {
+      return decodeByte(b, o, sortOrder);
+    }
+
+    @Override
+    public int decodeInt(byte[] b, int o, SortOrder sortOrder) {
+      return decodeByte(b, o, sortOrder);
+    }
+
+    @Override
+    public byte decodeByte(byte[] b, int o, SortOrder sortOrder) {
+      Preconditions.checkNotNull(sortOrder);
+      checkForSufficientLength(b, o, Bytes.SIZEOF_BYTE);
+      int v;
+      if (sortOrder == SortOrder.ASC) {
+        v = b[o] ^ 0x80; // Flip sign bit back
+      } else {
+        v = b[o] ^ 0xff ^ 0x80; // Flip sign bit back
+      }
+      return (byte) v;
+    }
+
+    @Override
+    public short decodeShort(byte[] b, int o, SortOrder sortOrder) {
+      return decodeByte(b, o, sortOrder);
+    }
+
+    @Override
+    public int encodeShort(short v, byte[] b, int o) {
+      checkForSufficientLength(b, o, Bytes.SIZEOF_BYTE);
+      if (v < Byte.MIN_VALUE || v > Byte.MAX_VALUE) {
+        throw newIllegalDataException(
+            "Value " + v + " cannot be encoded as an Byte without changing its value");
+      }
+      return encodeByte((byte) v, b, o);
+    }
+
+    @Override
+    public int encodeLong(long v, byte[] b, int o) {
+      if (v < Byte.MIN_VALUE || v > Byte.MAX_VALUE) {
+        throw newIllegalDataException(
+            "Value " + v + " cannot be encoded as an Byte without changing its value");
+      }
+      return encodeByte((byte) v, b, o);
+    }
+
+    @Override
+    public int encodeInt(int v, byte[] b, int o) {
+      if (v < Byte.MIN_VALUE || v > Byte.MAX_VALUE) {
+        throw newIllegalDataException(
+            "Value " + v + " cannot be encoded as an Byte without changing its value");
+      }
+      return encodeByte((byte) v, b, o);
+    }
+
+    @Override
+    public int encodeByte(byte v, byte[] b, int o) {
+      checkForSufficientLength(b, o, Bytes.SIZEOF_BYTE);
+      b[o] = (byte) (v ^ 0x80); // Flip sign bit so that Short is binary comparable
+      return Bytes.SIZEOF_BYTE;
+    }
+
+    @Override
+    public double decodeDouble(byte[] b, int o, SortOrder sortOrder) {
+      return decodeByte(b, o, sortOrder);
+    }
+
+    @Override
+    public float decodeFloat(byte[] b, int o, SortOrder sortOrder) {
+      return decodeByte(b, o, sortOrder);
+    }
+
+    @Override
+    public int encodeFloat(float v, byte[] b, int o) {
+      if (v < Byte.MIN_VALUE || v > Byte.MAX_VALUE) {
+        throw newIllegalDataException(
+            "Value " + v + " cannot be encoded as an Byte without changing its value");
+      }
+      return encodeByte((byte) v, b, o);
+    }
+
+    @Override
+    public int encodeDouble(double v, byte[] b, int o) {
+      if (v < Byte.MIN_VALUE || v > Byte.MAX_VALUE) {
+        throw newIllegalDataException(
+            "Value " + v + " cannot be encoded as an Byte without changing its value");
+      }
+      return encodeByte((byte) v, b, o);
+    }
+
+    @Override
+    public PhoenixArrayFactory getPhoenixArrayFactory() {
+      return new PhoenixArrayFactory() {
+        @Override
+        public PhoenixArray newArray(PDataType type, Object[] elements) {
+          return new PhoenixArray.PrimitiveBytePhoenixArray(type, elements);
+        }
+      };
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/04ef859b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PTinyintArray.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PTinyintArray.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PTinyintArray.java
new file mode 100644
index 0000000..e0b300d
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PTinyintArray.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 PTinyintArray extends PArrayDataType<byte[]> {
+
+  public static final PTinyintArray INSTANCE = new PTinyintArray();
+
+  private PTinyintArray() {
+    super("TINYINT ARRAY", PDataType.ARRAY_TYPE_BASE + PTinyint.INSTANCE.getSqlType(),
+        PhoenixArray.class, null, 32);
+  }
+
+  @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, PTinyint.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, PTinyint.INSTANCE, sortOrder, maxLength, scale,
+        PTinyint.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.PrimitiveBytePhoenixArray pArr = (PhoenixArray.PrimitiveBytePhoenixArray) value;
+    byte[] byteArr = (byte[]) pArr.array;
+    for (byte i : byteArr) {
+      if (!super.isCoercibleTo(PTinyint.INSTANCE, i)) {
+        return false;
+      }
+    }
+    return true;
+  }
+
+  @Override
+  public int getResultSetSqlType() {
+    return Types.ARRAY;
+  }
+
+  @Override
+  public Object getSampleValue(Integer maxLength, Integer arrayLength) {
+    return getSampleValue(PTinyint.INSTANCE, arrayLength, maxLength);
+  }
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/04ef859b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PUnsignedDate.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PUnsignedDate.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PUnsignedDate.java
new file mode 100644
index 0000000..8b63fbb
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PUnsignedDate.java
@@ -0,0 +1,164 @@
+/*
+ * 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 org.apache.phoenix.util.DateUtil;
+
+import java.sql.Types;
+import java.sql.Date;
+import java.text.Format;
+
+public class PUnsignedDate extends PDataType<Date> {
+
+  public static final PUnsignedDate INSTANCE = new PUnsignedDate();
+
+  private PUnsignedDate() {
+    super("UNSIGNED_DATE", 19, Date.class,
+        new UnsignedDateCodec(), 14); // After TIMESTAMP and DATE to ensure toLiteral finds those first
+  }
+
+  @Override
+  public byte[] toBytes(Object object) {
+    if (object == null) {
+      throw newIllegalDataException(this + " may not be null");
+    }
+    byte[] bytes = new byte[getByteSize()];
+    toBytes(object, bytes, 0);
+    return bytes;
+  }
+
+  @Override
+  public int toBytes(Object object, byte[] bytes, int offset) {
+    if (object == null) {
+      throw newIllegalDataException(this + " may not be null");
+    }
+    getCodec().encodeLong(((java.util.Date) object).getTime(), bytes, offset);
+    return this.getByteSize();
+  }
+
+  @Override
+  public Object toObject(Object object, PDataType actualType) {
+    Date d = (Date) PDate.INSTANCE.toObject(object, actualType);
+    throwIfNonNegativeDate(d);
+    return d;
+  }
+
+  @Override
+  public Object toObject(byte[] b, int o, int l, PDataType actualType, SortOrder sortOrder, Integer maxLength, Integer scale) {
+    Date d = (Date) PDate.INSTANCE.toObject(b, o, l, actualType, sortOrder);
+    throwIfNonNegativeDate(d);
+    return d;
+  }
+
+  @Override
+  public boolean isCastableTo(PDataType targetType) {
+    return PDate.INSTANCE.isCastableTo(targetType);
+  }
+
+  @Override
+  public boolean isCoercibleTo(PDataType targetType) {
+    return equalsAny(targetType, this, PUnsignedTime.INSTANCE, PUnsignedTimestamp.INSTANCE)
+        || PDate.INSTANCE.isCoercibleTo(targetType);
+  }
+
+  @Override
+  public boolean isCoercibleTo(PDataType targetType, Object value) {
+    return super.isCoercibleTo(targetType, value) || PDate.INSTANCE.isCoercibleTo(targetType, value);
+  }
+
+  @Override
+  public boolean isFixedWidth() {
+    return true;
+  }
+
+  @Override
+  public Integer getByteSize() {
+    return PDate.INSTANCE.getByteSize();
+  }
+
+  @Override
+  public int compareTo(Object lhs, Object rhs, PDataType rhsType) {
+    return PDate.INSTANCE.compareTo(lhs, rhs, rhsType);
+  }
+
+  @Override
+  public Object toObject(String value) {
+    return PDate.INSTANCE.toObject(value);
+  }
+
+  @Override
+  public boolean isBytesComparableWith(PDataType otherType) {
+    return super.isBytesComparableWith(otherType) || otherType.equals(PUnsignedTime.INSTANCE);
+  }
+
+  @Override
+  public String toStringLiteral(byte[] b, int offset, int length, Format formatter) {
+    // Can't delegate, as the super.toStringLiteral calls this.toBytes
+    if (formatter == null || formatter == DateUtil.DEFAULT_DATE_FORMATTER) {
+      // If default formatter has not been overridden,
+      // use one that displays milliseconds.
+      formatter = DateUtil.DEFAULT_MS_DATE_FORMATTER;
+    }
+    return "'" + super.toStringLiteral(b, offset, length, formatter) + "'";
+  }
+
+  @Override
+  public void coerceBytes(ImmutableBytesWritable ptr, Object object, PDataType actualType,
+      Integer maxLength, Integer scale, SortOrder actualModifier,
+      Integer desiredMaxLength, Integer desiredScale,
+      SortOrder expectedModifier) {
+    if (ptr.getLength() > 0 && actualType == PUnsignedTimestamp.INSTANCE
+        && actualModifier == expectedModifier) {
+      ptr.set(ptr.get(), ptr.getOffset(), getByteSize());
+      return;
+    }
+    super.coerceBytes(ptr, object, actualType, maxLength, scale, actualModifier, desiredMaxLength,
+        desiredScale, expectedModifier);
+  }
+
+  @Override
+  public int getResultSetSqlType() {
+    return Types.DATE;
+  }
+
+  @Override
+  public Object getSampleValue(Integer maxLength, Integer arrayLength) {
+    return new Date((Long) PUnsignedLong.INSTANCE.getSampleValue(maxLength, arrayLength));
+  }
+
+  static class UnsignedDateCodec extends PUnsignedLong.UnsignedLongCodec {
+
+    @Override
+    public int decodeInt(byte[] b, int o, SortOrder sortOrder) {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public PhoenixArrayFactory getPhoenixArrayFactory() {
+      return new PhoenixArrayFactory() {
+
+        @Override
+        public PhoenixArray newArray(PDataType type, Object[] elements) {
+          return new PhoenixArray(type, elements);
+        }
+      };
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/04ef859b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PUnsignedDateArray.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PUnsignedDateArray.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PUnsignedDateArray.java
new file mode 100644
index 0000000..859c93b
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PUnsignedDateArray.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.sql.Types;
+import java.util.Date;
+
+public class PUnsignedDateArray extends PArrayDataType<Date> {
+
+  public static final PUnsignedDateArray INSTANCE = new PUnsignedDateArray();
+
+  private PUnsignedDateArray() {
+    super("UNSIGNED_DATE ARRAY", PDataType.ARRAY_TYPE_BASE + PUnsignedDate.INSTANCE.getSqlType(),
+        PhoenixArray.class, null, 41);
+  }
+
+  @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, PUnsignedDate.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, PUnsignedDate.INSTANCE, sortOrder, maxLength,
+        scale, PUnsignedDate.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 pArr = (PhoenixArray) value;
+    Object[] dateArr = (Object[]) pArr.array;
+    for (Object i : dateArr) {
+      if (!super.isCoercibleTo(PUnsignedDate.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(PUnsignedDate.INSTANCE, arrayLength, maxLength);
+  }
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/04ef859b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PUnsignedDouble.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PUnsignedDouble.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PUnsignedDouble.java
new file mode 100644
index 0000000..c1fd7ba
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PUnsignedDouble.java
@@ -0,0 +1,158 @@
+/*
+ * 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;
+
+public class PUnsignedDouble extends PDataType<PDouble> {
+
+  public static final PUnsignedDouble INSTANCE = new PUnsignedDouble();
+
+  private PUnsignedDouble() {
+    super("UNSIGNED_DOUBLE", 15, Double.class, new UnsignedDoubleCodec(), 20);
+  }
+
+  @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) {
+    return PDouble.INSTANCE.getScale(o);
+  }
+
+  @Override
+  public Integer getMaxLength(Object o) {
+    return PDouble.INSTANCE.getMaxLength(o);
+  }
+
+  @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 {
+      Double d = Double.parseDouble(value);
+      if (d.doubleValue() < 0) {
+        throw newIllegalDataException("Value may not be negative("
+            + d + ")");
+      }
+      return d;
+    } catch (NumberFormatException e) {
+      throw newIllegalDataException(e);
+    }
+  }
+
+  @Override
+  public Object toObject(Object object, PDataType actualType) {
+    Double v = (Double) PDouble.INSTANCE.toObject(object, actualType);
+    throwIfNonNegativeNumber(v);
+    return v;
+  }
+
+  @Override
+  public Object toObject(byte[] b, int o, int l, PDataType actualType, SortOrder sortOrder,
+      Integer maxLength, Integer scale) {
+    Double v = (Double) PDouble.INSTANCE.toObject(b, o, l, actualType, sortOrder);
+    throwIfNonNegativeNumber(v);
+    return v;
+  }
+
+  @Override
+  public boolean isCoercibleTo(PDataType targetType, Object value) {
+    return super.isCoercibleTo(targetType, value) || PDouble.INSTANCE
+        .isCoercibleTo(targetType, value);
+  }
+
+  @Override
+  public boolean isCoercibleTo(PDataType targetType) {
+    return this.equals(targetType) || PDouble.INSTANCE.isCoercibleTo(targetType);
+  }
+
+  @Override
+  public int getResultSetSqlType() {
+    return PDouble.INSTANCE.getResultSetSqlType();
+  }
+
+  @Override
+  public Object getSampleValue(Integer maxLength, Integer arrayLength) {
+    return Math.abs((Double) PDouble.INSTANCE.getSampleValue(maxLength, arrayLength));
+  }
+
+  static class UnsignedDoubleCodec extends PDouble.DoubleCodec {
+
+    @Override
+    public int encodeDouble(double v, byte[] b, int o) {
+      checkForSufficientLength(b, o, Bytes.SIZEOF_DOUBLE);
+      if (v < 0) {
+        throw newIllegalDataException();
+      }
+      Bytes.putDouble(b, o, v);
+      return Bytes.SIZEOF_DOUBLE;
+    }
+
+    @Override
+    public double decodeDouble(byte[] b, int o, SortOrder sortOrder) {
+      Preconditions.checkNotNull(sortOrder);
+      checkForSufficientLength(b, o, Bytes.SIZEOF_DOUBLE);
+      if (sortOrder == SortOrder.DESC) {
+        b = SortOrder.invert(b, o, new byte[Bytes.SIZEOF_DOUBLE], 0, Bytes.SIZEOF_DOUBLE);
+      }
+      double v = Bytes.toDouble(b, o);
+      if (v < 0) {
+        throw newIllegalDataException();
+      }
+      return v;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/04ef859b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PUnsignedDoubleArray.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PUnsignedDoubleArray.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PUnsignedDoubleArray.java
new file mode 100644
index 0000000..50c69bf
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PUnsignedDoubleArray.java
@@ -0,0 +1,112 @@
+/*
+ * 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 PUnsignedDoubleArray extends PArrayDataType<double[]> {
+
+  public static final PUnsignedDoubleArray INSTANCE = new PUnsignedDoubleArray();
+
+  private PUnsignedDoubleArray() {
+    super("UNSIGNED_DOUBLE ARRAY", PDataType.ARRAY_TYPE_BASE + PUnsignedDouble.INSTANCE.getSqlType(),
+        PhoenixArray.class, null, 47);
+  }
+
+  @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, PUnsignedDouble.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, PUnsignedDouble.INSTANCE, sortOrder, maxLength,
+        scale, PUnsignedDouble.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 pArr = (PhoenixArray) value;
+    Object[] doubleArr = (Object[]) pArr.array;
+    for (Object i : doubleArr) {
+      if (!super.isCoercibleTo(PUnsignedDouble.INSTANCE, i) && (!super.isCoercibleTo(
+          PUnsignedTimestamp.INSTANCE, i))
+          && (!super.isCoercibleTo(PUnsignedTime.INSTANCE, i)) && (!super
+          .isCoercibleTo(PUnsignedDate.INSTANCE, i))) {
+        return false;
+      }
+    }
+    return true;
+  }
+
+  @Override
+  public int getResultSetSqlType() {
+    return Types.ARRAY;
+  }
+
+  @Override
+  public Object getSampleValue(Integer maxLength, Integer arrayLength) {
+    return getSampleValue(PUnsignedDouble.INSTANCE, arrayLength, maxLength);
+  }
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/04ef859b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PUnsignedFloat.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PUnsignedFloat.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PUnsignedFloat.java
new file mode 100644
index 0000000..c04b9c7
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PUnsignedFloat.java
@@ -0,0 +1,152 @@
+/*
+ * 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;
+
+public class PUnsignedFloat extends PDataType<PFloat> {
+
+  public static final PUnsignedFloat INSTANCE = new PUnsignedFloat();
+
+  private PUnsignedFloat() {
+    super("UNSIGNED_FLOAT", 14, Float.class, new UnsignedFloatCodec(), 19);
+  }
+
+  @Override
+  public int compareTo(Object lhs, Object rhs, PDataType rhsType) {
+    return PFloat.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) {
+    return PFloat.INSTANCE.getScale(o);
+  }
+
+  @Override
+  public Integer getMaxLength(Object o) {
+    return PFloat.INSTANCE.getMaxLength(o);
+  }
+
+  @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 {
+      Float f = Float.parseFloat(value);
+      if (f.floatValue() < 0) {
+        throw newIllegalDataException("Value may not be negative("
+            + f + ")");
+      }
+      return f;
+    } catch (NumberFormatException e) {
+      throw newIllegalDataException(e);
+    }
+  }
+
+  @Override
+  public Object toObject(Object object, PDataType actualType) {
+    Float v = (Float) PFloat.INSTANCE.toObject(object, actualType);
+    throwIfNonNegativeNumber(v);
+    return v;
+  }
+
+  @Override
+  public Object toObject(byte[] b, int o, int l, PDataType actualType, SortOrder sortOrder,
+      Integer maxLength, Integer scale) {
+    Float v = (Float) PFloat.INSTANCE.toObject(b, o, l, actualType, sortOrder);
+    throwIfNonNegativeNumber(v);
+    return v;
+  }
+
+  @Override
+  public boolean isCoercibleTo(PDataType targetType, Object value) {
+    return super.isCoercibleTo(targetType) || PFloat.INSTANCE.isCoercibleTo(targetType, value);
+  }
+
+  @Override
+  public boolean isCoercibleTo(PDataType targetType) {
+    return this.equals(targetType) || PUnsignedDouble.INSTANCE.isCoercibleTo(targetType) || PFloat.INSTANCE
+        .isCoercibleTo(targetType);
+  }
+
+  @Override
+  public int getResultSetSqlType() {
+    return PFloat.INSTANCE.getResultSetSqlType();
+  }
+
+  @Override
+  public Object getSampleValue(Integer maxLength, Integer arrayLength) {
+    return Math.abs((Float) PFloat.INSTANCE.getSampleValue(maxLength, arrayLength));
+  }
+
+  static class UnsignedFloatCodec extends PFloat.FloatCodec {
+
+    @Override
+    public int encodeFloat(float v, byte[] b, int o) {
+      checkForSufficientLength(b, o, Bytes.SIZEOF_FLOAT);
+      if (v < 0) {
+        throw newIllegalDataException();
+      }
+      Bytes.putFloat(b, o, v);
+      return Bytes.SIZEOF_FLOAT;
+    }
+
+    @Override
+    public float decodeFloat(byte[] b, int o, SortOrder sortOrder) {
+      Preconditions.checkNotNull(sortOrder);
+      checkForSufficientLength(b, o, Bytes.SIZEOF_FLOAT);
+      if (sortOrder == SortOrder.DESC) {
+        b = SortOrder.invert(b, o, new byte[Bytes.SIZEOF_FLOAT], 0, Bytes.SIZEOF_FLOAT);
+      }
+      float v = Bytes.toFloat(b, o);
+      if (v < 0) {
+        throw newIllegalDataException();
+      }
+      return v;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/04ef859b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PUnsignedFloatArray.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PUnsignedFloatArray.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PUnsignedFloatArray.java
new file mode 100644
index 0000000..db9d075
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PUnsignedFloatArray.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 PUnsignedFloatArray extends PArrayDataType<float[]> {
+
+  public static final PUnsignedFloatArray INSTANCE = new PUnsignedFloatArray();
+
+  private PUnsignedFloatArray() {
+    super("UNSIGNED_FLOAT ARRAY", PDataType.ARRAY_TYPE_BASE + PUnsignedFloat.INSTANCE.getSqlType(),
+        PhoenixArray.class, null, 46);
+  }
+
+  @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, PUnsignedFloat.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, PUnsignedFloat.INSTANCE, sortOrder, maxLength,
+        scale, PUnsignedFloat.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 pArr = (PhoenixArray) value;
+    Object[] floatArr = (Object[]) pArr.array;
+    for (Object i : floatArr) {
+      if (!super.isCoercibleTo(PUnsignedFloat.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(PUnsignedFloat.INSTANCE, arrayLength, maxLength);
+  }
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/04ef859b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PUnsignedInt.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PUnsignedInt.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PUnsignedInt.java
new file mode 100644
index 0000000..8bd5e6d
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PUnsignedInt.java
@@ -0,0 +1,178 @@
+/*
+ * 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.hadoop.hbase.util.Order;
+import org.apache.phoenix.schema.SortOrder;
+
+/**
+ * Unsigned integer type that restricts values to be from 0 to {@link Integer#MAX_VALUE}
+ * inclusive. May be used to map to existing HTable values created through
+ * {@link org.apache.hadoop.hbase.util.Bytes#toBytes(int)}
+ * as long as all values are non negative (the leading sign bit of negative numbers would cause
+ * them to sort ahead of positive numbers when they're used as part of the row key when using the
+ * HBase utility methods).
+ */
+public class PUnsignedInt extends PDataType<Integer> {
+
+  public static final PUnsignedInt INSTANCE = new PUnsignedInt();
+
+  private PUnsignedInt() {
+    super("UNSIGNED_INT", 9 /* no constant available in Types */, Integer.class,
+        new UnsignedIntCodec(), 16);
+  }
+
+  @Override
+  public boolean isOrderPreserving() {
+    return true;
+  }
+
+  @Override
+  public Order getOrder() {
+    return Order.ASCENDING;
+  }
+
+  @Override
+  public boolean isSkippable() {
+    return true;
+  }
+
+  @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) {
+    Integer v = (Integer) PInteger.INSTANCE.toObject(object, actualType);
+    throwIfNonNegativeNumber(v);
+    return v;
+  }
+
+  @Override
+  public Object toObject(byte[] b, int o, int l, PDataType actualType, SortOrder sortOrder,
+      Integer maxLength, Integer scale) {
+    Integer v =
+        (Integer) PInteger.INSTANCE.toObject(b, o, l, actualType, sortOrder);
+    throwIfNonNegativeNumber(v);
+    return v;
+  }
+
+  @Override
+  public boolean isCoercibleTo(PDataType targetType) {
+    return targetType.equals(this) || targetType.equals(PUnsignedFloat.INSTANCE)
+        || PUnsignedLong.INSTANCE.isCoercibleTo(targetType)
+        || PInteger.INSTANCE.isCoercibleTo(targetType);
+  }
+
+  @Override
+  public boolean isCoercibleTo(PDataType targetType, Object value) {
+    return super.isCoercibleTo(targetType, value) || PInteger.INSTANCE
+        .isCoercibleTo(targetType, value);
+  }
+
+  @Override
+  public boolean isFixedWidth() {
+    return true;
+  }
+
+  @Override
+  public Integer getByteSize() {
+    return Bytes.SIZEOF_INT;
+  }
+
+  @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 {
+      Integer i = Integer.parseInt(value);
+      if (i.intValue() < 0) {
+        throw newIllegalDataException("Value may not be negative(" + i + ")");
+      }
+      return i;
+    } catch (NumberFormatException e) {
+      throw newIllegalDataException(e);
+    }
+  }
+
+  @Override
+  public int getResultSetSqlType() {
+    return PInteger.INSTANCE.getResultSetSqlType();
+  }
+
+  @Override
+  public Object getSampleValue(Integer maxLength, Integer arrayLength) {
+    return Math.abs((Integer) PInteger.INSTANCE.getSampleValue(maxLength, arrayLength));
+  }
+
+  static class UnsignedIntCodec extends PInteger.IntCodec {
+
+    @Override
+    public int decodeInt(byte[] b, int o, SortOrder sortOrder) {
+      Preconditions.checkNotNull(sortOrder);
+      checkForSufficientLength(b, o, Bytes.SIZEOF_INT);
+      if (sortOrder == SortOrder.DESC) {
+        b = SortOrder.invert(b, o, new byte[Bytes.SIZEOF_INT], 0, Bytes.SIZEOF_INT);
+      }
+      int v = Bytes.toInt(b, o);
+      if (v < 0) {
+        throw newIllegalDataException();
+      }
+      return v;
+    }
+
+    @Override
+    public int encodeInt(int v, byte[] b, int o) {
+      checkForSufficientLength(b, o, Bytes.SIZEOF_INT);
+      if (v < 0) {
+        throw newIllegalDataException();
+      }
+      Bytes.putInt(b, o, v);
+      return Bytes.SIZEOF_INT;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/04ef859b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PUnsignedIntArray.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PUnsignedIntArray.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PUnsignedIntArray.java
new file mode 100644
index 0000000..11e5f10
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PUnsignedIntArray.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 PUnsignedIntArray extends PArrayDataType<int[]> {
+
+  public static final PUnsignedIntArray INSTANCE = new PUnsignedIntArray();
+
+  private PUnsignedIntArray() {
+    super("UNSIGNED_INT ARRAY", PDataType.ARRAY_TYPE_BASE + PUnsignedInt.INSTANCE.getSqlType(),
+        PhoenixArray.class, null, 43);
+  }
+
+  @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, PUnsignedInt.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, PUnsignedInt.INSTANCE, sortOrder, maxLength,
+        scale, PUnsignedInt.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 pArr = (PhoenixArray) value;
+    Object[] intArr = (Object[]) pArr.array;
+    for (Object i : intArr) {
+      if (!super.isCoercibleTo(PUnsignedInt.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(PUnsignedInt.INSTANCE, arrayLength, maxLength);
+  }
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/04ef859b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PUnsignedLong.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PUnsignedLong.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PUnsignedLong.java
new file mode 100644
index 0000000..a0ead11
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PUnsignedLong.java
@@ -0,0 +1,193 @@
+/*
+ * 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.hadoop.hbase.util.Order;
+import org.apache.phoenix.schema.SortOrder;
+
+import java.math.BigDecimal;
+
+/**
+ * Unsigned long type that restricts values to be from 0 to {@link Long#MAX_VALUE}
+ * inclusive. May be used to map to existing HTable values created through
+ * {@link org.apache.hadoop.hbase.util.Bytes#toBytes(long)}
+ * as long as all values are non negative (the leading sign bit of negative numbers would cause
+ * them to sort ahead of positive numbers when they're used as part of the row key when using the
+ * HBase utility methods).
+ */
+public class PUnsignedLong extends PDataType<Long> {
+
+  public static final PUnsignedLong INSTANCE = new PUnsignedLong();
+
+  private PUnsignedLong() {
+    super("UNSIGNED_LONG", 10 /* no constant available in Types */, Long.class,
+        new UnsignedLongCodec(), 15);
+  }
+
+  @Override
+  public boolean isOrderPreserving() {
+    return true;
+  }
+
+  @Override
+  public Order getOrder() {
+    return Order.ASCENDING;
+  }
+
+  @Override
+  public boolean isSkippable() {
+    return true;
+  }
+
+  @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) {
+    Long v = (Long) PLong.INSTANCE.toObject(object, actualType);
+    throwIfNonNegativeNumber(v);
+    return v;
+  }
+
+  @Override
+  public Object toObject(byte[] b, int o, int l, PDataType actualType, SortOrder sortOrder,
+      Integer maxLength, Integer scale) {
+    Long v = (Long) PLong.INSTANCE.toObject(b, o, l, actualType, sortOrder);
+    throwIfNonNegativeNumber(v);
+    return v;
+  }
+
+  @Override
+  public boolean isCoercibleTo(PDataType targetType) {
+    return targetType == this || targetType == PUnsignedDouble.INSTANCE || PLong.INSTANCE
+        .isCoercibleTo(targetType);
+  }
+
+  @Override
+  public boolean isCoercibleTo(PDataType targetType, Object value) {
+    return super.isCoercibleTo(targetType, value) || PLong.INSTANCE.isCoercibleTo(targetType, value);
+  }
+
+  @Override
+  public boolean isFixedWidth() {
+    return true;
+  }
+
+  @Override
+  public Integer getByteSize() {
+    return Bytes.SIZEOF_LONG;
+  }
+
+  @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 boolean isComparableTo(PDataType targetType) {
+    return PDecimal.INSTANCE.isComparableTo(targetType);
+  }
+
+  @Override
+  public Object toObject(String value) {
+    if (value == null || value.length() == 0) {
+      return null;
+    }
+    try {
+      Long l = Long.parseLong(value);
+      if (l.longValue() < 0) {
+        throw newIllegalDataException("Value may not be negative(" + l + ")");
+      }
+      return l;
+    } catch (NumberFormatException e) {
+      throw newIllegalDataException(e);
+    }
+  }
+
+  @Override
+  public int getResultSetSqlType() {
+    return PLong.INSTANCE.getResultSetSqlType();
+  }
+
+  @Override
+  public Object getSampleValue(Integer maxLength, Integer arrayLength) {
+    return Math.abs((Long) PLong.INSTANCE.getSampleValue(maxLength, arrayLength));
+  }
+
+  static class UnsignedLongCodec extends PLong.LongCodec {
+
+    @Override
+    public long decodeLong(byte[] b, int o, SortOrder sortOrder) {
+      Preconditions.checkNotNull(sortOrder);
+      checkForSufficientLength(b, o, Bytes.SIZEOF_LONG);
+      long v = 0;
+      if (sortOrder == SortOrder.ASC) {
+        for (int i = o; i < o + Bytes.SIZEOF_LONG; i++) {
+          v <<= 8;
+          v ^= b[i] & 0xFF;
+        }
+      } else {
+        for (int i = o; i < o + Bytes.SIZEOF_LONG; i++) {
+          v <<= 8;
+          v ^= (b[i] & 0xFF) ^ 0xFF;
+        }
+      }
+      if (v < 0) {
+        throw newIllegalDataException();
+      }
+      return v;
+    }
+
+    @Override
+    public int encodeLong(long v, byte[] b, int o) {
+      checkForSufficientLength(b, o, Bytes.SIZEOF_LONG);
+      if (v < 0) {
+        throw newIllegalDataException();
+      }
+      Bytes.putLong(b, o, v);
+      return Bytes.SIZEOF_LONG;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/04ef859b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PUnsignedLongArray.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PUnsignedLongArray.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PUnsignedLongArray.java
new file mode 100644
index 0000000..dcbe210
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PUnsignedLongArray.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 PUnsignedLongArray extends PArrayDataType<long[]> {
+
+  public static final PUnsignedLongArray INSTANCE = new PUnsignedLongArray();
+
+  private PUnsignedLongArray() {
+    super("UNSIGNED_LONG ARRAY", PDataType.ARRAY_TYPE_BASE + PUnsignedLong.INSTANCE.getSqlType(),
+        PhoenixArray.class, null, 42);
+  }
+
+  @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, PUnsignedLong.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, PUnsignedLong.INSTANCE, sortOrder, maxLength,
+        scale, PUnsignedLong.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 pArr = (PhoenixArray) value;
+    Object[] longArr = (Object[]) pArr.array;
+    for (Object i : longArr) {
+      if (!super.isCoercibleTo(PUnsignedLong.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(PUnsignedLong.INSTANCE, arrayLength, maxLength);
+  }
+}