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

[10/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/PUnsignedSmallint.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PUnsignedSmallint.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PUnsignedSmallint.java
new file mode 100644
index 0000000..77ab263
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PUnsignedSmallint.java
@@ -0,0 +1,159 @@
+/*
+ * 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 PUnsignedSmallint extends PDataType<Short> {
+
+  public static final PUnsignedSmallint INSTANCE = new PUnsignedSmallint();
+
+  private PUnsignedSmallint() {
+    super("UNSIGNED_SMALLINT", 13, Short.class, new UnsignedShortCodec(), 17);
+  }
+
+  @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 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) {
+    if (object == null) {
+      throw newIllegalDataException(this + " may not be null");
+    }
+    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(String value) {
+    if (value == null || value.length() == 0) {
+      return null;
+    }
+    try {
+      Short b = Short.parseShort(value);
+      if (b.shortValue() < 0) {
+        throw newIllegalDataException("Value may not be negative(" + b + ")");
+      }
+      return b;
+    } catch (NumberFormatException e) {
+      throw newIllegalDataException(e);
+    }
+  }
+
+  @Override
+  public Object toObject(Object object, PDataType actualType) {
+    Short v = (Short) PSmallint.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) {
+    Short v = (Short) PSmallint.INSTANCE.toObject(b, o, l, actualType, sortOrder);
+    throwIfNonNegativeNumber(v);
+    return v;
+  }
+
+  @Override
+  public boolean isComparableTo(PDataType targetType) {
+    return PDecimal.INSTANCE.isComparableTo(targetType);
+  }
+
+  @Override
+  public boolean isCoercibleTo(PDataType targetType) {
+    return targetType.equals(this) || PUnsignedInt.INSTANCE.isCoercibleTo(targetType) || PSmallint.INSTANCE
+        .isCoercibleTo(targetType);
+  }
+
+  @Override
+  public boolean isCoercibleTo(PDataType targetType, Object value) {
+    return super.isCoercibleTo(targetType, value) || PSmallint.INSTANCE
+        .isCoercibleTo(targetType, value);
+  }
+
+  @Override
+  public int getResultSetSqlType() {
+    return PSmallint.INSTANCE.getResultSetSqlType();
+  }
+
+  @Override
+  public Object getSampleValue(Integer maxLength, Integer arrayLength) {
+    return ((Integer) RANDOM.get().nextInt(Short.MAX_VALUE)).shortValue();
+  }
+
+  static class UnsignedShortCodec extends PSmallint.ShortCodec {
+
+    @Override
+    public short decodeShort(byte[] b, int o, SortOrder sortOrder) {
+      Preconditions.checkNotNull(sortOrder);
+      checkForSufficientLength(b, o, Bytes.SIZEOF_SHORT);
+      if (sortOrder == SortOrder.DESC) {
+        b = SortOrder.invert(b, o, new byte[Bytes.SIZEOF_SHORT], 0, Bytes.SIZEOF_SHORT);
+      }
+      short v = Bytes.toShort(b, o);
+      if (v < 0) {
+        throw newIllegalDataException();
+      }
+      return v;
+    }
+
+    @Override
+    public int encodeShort(short v, byte[] b, int o) {
+      checkForSufficientLength(b, o, Bytes.SIZEOF_SHORT);
+      if (v < 0) {
+        throw newIllegalDataException();
+      }
+      Bytes.putShort(b, o, v);
+      return Bytes.SIZEOF_SHORT;
+    }
+  }
+}

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

http://git-wip-us.apache.org/repos/asf/phoenix/blob/04ef859b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PUnsignedTime.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PUnsignedTime.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PUnsignedTime.java
new file mode 100644
index 0000000..f738f44
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PUnsignedTime.java
@@ -0,0 +1,115 @@
+/*
+ * 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 java.sql.Time;
+import java.sql.Types;
+import java.text.Format;
+
+public class PUnsignedTime extends PDataType<Time> {
+
+  public static final PUnsignedTime INSTANCE = new PUnsignedTime();
+
+  private PUnsignedTime() {
+    super("UNSIGNED_TIME", 18, java.sql.Time.class, new PUnsignedDate.UnsignedDateCodec(), 13);
+  }
+
+  @Override
+  public byte[] toBytes(Object object) {
+    return PUnsignedDate.INSTANCE.toBytes(object);
+  }
+
+  @Override
+  public int toBytes(Object object, byte[] bytes, int offset) {
+    return PUnsignedDate.INSTANCE.toBytes(object, bytes, offset);
+  }
+
+  @Override
+  public Object toObject(byte[] b, int o, int l, PDataType actualType, SortOrder sortOrder,
+      Integer maxLength, Integer scale) {
+    java.sql.Time t = (java.sql.Time) PTime.INSTANCE.toObject(b, o, l, actualType, sortOrder);
+    throwIfNonNegativeDate(t);
+    return t;
+  }
+
+  @Override
+  public Object toObject(Object object, PDataType actualType) {
+    java.sql.Time t = (java.sql.Time) PTime.INSTANCE.toObject(object, actualType);
+    throwIfNonNegativeDate(t);
+    return t;
+  }
+
+  @Override
+  public boolean isCastableTo(PDataType targetType) {
+    return PUnsignedDate.INSTANCE.isCastableTo(targetType);
+  }
+
+  @Override
+  public boolean isCoercibleTo(PDataType targetType) {
+    return PUnsignedDate.INSTANCE.isCoercibleTo(targetType);
+  }
+
+  @Override
+  public boolean isCoercibleTo(PDataType targetType, Object value) {
+    return super.isCoercibleTo(targetType, value) || PTime.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 PTime.INSTANCE.compareTo(lhs, rhs, rhsType);
+  }
+
+  @Override
+  public Object toObject(String value) {
+    return PTime.INSTANCE.toObject(value);
+  }
+
+  @Override
+  public boolean isBytesComparableWith(PDataType otherType) {
+    return super.isBytesComparableWith(otherType) || otherType == PUnsignedDate.INSTANCE;
+  }
+
+  @Override
+  public String toStringLiteral(byte[] b, int offset, int length, Format formatter) {
+    return PUnsignedDate.INSTANCE.toStringLiteral(b, offset, length, formatter);
+  }
+
+  @Override
+  public int getResultSetSqlType() {
+    return Types.TIME;
+  }
+
+  @Override
+  public Object getSampleValue(Integer maxLength, Integer arrayLength) {
+    return new java.sql.Time(
+        (Long) PUnsignedLong.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/PUnsignedTimeArray.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PUnsignedTimeArray.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PUnsignedTimeArray.java
new file mode 100644
index 0000000..9d7584b
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PUnsignedTimeArray.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 PUnsignedTimeArray extends PArrayDataType<Time[]> {
+
+  public static final PUnsignedTimeArray INSTANCE = new PUnsignedTimeArray();
+
+  private PUnsignedTimeArray() {
+    super("UNSIGNED_TIME ARRAY", PDataType.ARRAY_TYPE_BASE + PUnsignedTime.INSTANCE.getSqlType(),
+        PhoenixArray.class, null, 39);
+  }
+
+  @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, PUnsignedTime.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, PUnsignedTime.INSTANCE, sortOrder, maxLength,
+        scale, PUnsignedTime.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(PUnsignedTime.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(PUnsignedTime.INSTANCE, arrayLength, maxLength);
+  }
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/04ef859b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PUnsignedTimestamp.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PUnsignedTimestamp.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PUnsignedTimestamp.java
new file mode 100644
index 0000000..21e6c50
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PUnsignedTimestamp.java
@@ -0,0 +1,151 @@
+/*
+ * 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.schema.SortOrder;
+import org.apache.phoenix.util.DateUtil;
+
+import java.sql.Timestamp;
+import java.sql.Types;
+import java.text.Format;
+
+public class PUnsignedTimestamp extends PDataType<Timestamp> {
+
+  public static final PUnsignedTimestamp INSTANCE = new PUnsignedTimestamp();
+
+  private PUnsignedTimestamp() {
+    super("UNSIGNED_TIMESTAMP", 19, java.sql.Timestamp.class, null, 12);
+  }
+
+  @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;
+    PUnsignedDate.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) {
+    java.sql.Timestamp ts = (java.sql.Timestamp) PTimestamp.INSTANCE.toObject(object, actualType);
+    throwIfNonNegativeDate(ts);
+    return ts;
+  }
+
+  @Override
+  public Object toObject(byte[] b, int o, int l, PDataType actualType, SortOrder sortOrder,
+      Integer maxLength, Integer scale) {
+    java.sql.Timestamp ts =
+        (java.sql.Timestamp) PTimestamp.INSTANCE.toObject(b, o, l, actualType, sortOrder);
+    throwIfNonNegativeDate(ts);
+    return ts;
+  }
+
+  @Override
+  public boolean isCastableTo(PDataType targetType) {
+    return PUnsignedDate.INSTANCE.isCastableTo(targetType);
+  }
+
+  @Override
+  public boolean isCoercibleTo(PDataType targetType) {
+    return targetType.equals(this) || PUnsignedDate.INSTANCE.isCoercibleTo(targetType);
+  }
+
+  @Override
+  public boolean isCoercibleTo(PDataType targetType, Object value) {
+    return super.isCoercibleTo(targetType, value) || PTimestamp.INSTANCE
+        .isCoercibleTo(targetType, value);
+  }
+
+  @Override
+  public boolean isFixedWidth() {
+    return true;
+  }
+
+  @Override
+  public Integer getByteSize() {
+    return PTimestamp.INSTANCE.getByteSize();
+  }
+
+  @Override
+  public int compareTo(Object lhs, Object rhs, PDataType rhsType) {
+    return PTimestamp.INSTANCE.compareTo(lhs, rhs, rhsType);
+  }
+
+  @Override
+  public Object toObject(String value) {
+    return PTimestamp.INSTANCE.toObject(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 =
+        PUnsignedLong.INSTANCE.getCodec().decodeLong(ptr.get(), ptr.getOffset(), sortOrder);
+    return millis;
+  }
+
+  @Override
+  public int getResultSetSqlType() {
+    return Types.TIMESTAMP;
+  }
+
+  @Override
+  public Object getSampleValue(Integer maxLength, Integer arrayLength) {
+    return new java.sql.Timestamp(
+        (Long) PUnsignedLong.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/PUnsignedTimestampArray.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PUnsignedTimestampArray.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PUnsignedTimestampArray.java
new file mode 100644
index 0000000..1159b5c
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PUnsignedTimestampArray.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.*;
+
+public class PUnsignedTimestampArray extends PArrayDataType<Timestamp[]> {
+
+  public static final PUnsignedTimestampArray INSTANCE = new PUnsignedTimestampArray();
+
+  private PUnsignedTimestampArray() {
+    super("UNSIGNED_TIMESTAMP ARRAY",
+        PDataType.ARRAY_TYPE_BASE + PUnsignedTimestamp.INSTANCE.getSqlType(), PhoenixArray.class,
+        null, 37);
+  }
+
+  @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, PUnsignedTimestamp.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, PUnsignedTimestamp.INSTANCE, sortOrder,
+        maxLength, scale, PUnsignedTimestamp.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(PUnsignedTimestamp.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(PUnsignedTimestamp.INSTANCE, arrayLength, maxLength);
+  }
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/04ef859b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PUnsignedTinyint.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PUnsignedTinyint.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PUnsignedTinyint.java
new file mode 100644
index 0000000..4c70c18
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PUnsignedTinyint.java
@@ -0,0 +1,155 @@
+/*
+ * 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 PUnsignedTinyint extends PDataType<Byte> {
+
+  public static final PUnsignedTinyint INSTANCE = new PUnsignedTinyint();
+
+  private PUnsignedTinyint() {
+    super("UNSIGNED_TINYINT", 11, Byte.class, new UnsignedByteCodec(), 18);
+  }
+
+  @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 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);
+      if (b.byteValue() < 0) {
+        throw newIllegalDataException("Value may not be negative(" + b + ")");
+      }
+      return b;
+    } catch (NumberFormatException e) {
+      throw newIllegalDataException(e);
+    }
+  }
+
+  @Override
+  public Object toObject(Object object, PDataType actualType) {
+    Byte v = (Byte) PTinyint.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) {
+    Byte v = (Byte) PTinyint.INSTANCE.toObject(b, o, l, actualType, sortOrder);
+    throwIfNonNegativeNumber(v);
+    return v;
+  }
+
+  @Override
+  public boolean isCoercibleTo(PDataType targetType) {
+    return targetType.equals(this) || PUnsignedSmallint.INSTANCE.isCoercibleTo(targetType)
+        || PTinyint.INSTANCE.isCoercibleTo(targetType);
+  }
+
+  @Override
+  public boolean isCoercibleTo(PDataType targetType, Object value) {
+    return super.isCoercibleTo(targetType, value) || PTinyint.INSTANCE
+        .isCoercibleTo(targetType, value);
+  }
+
+  @Override
+  public boolean isComparableTo(PDataType targetType) {
+    return PDecimal.INSTANCE.isComparableTo(targetType);
+  }
+
+  @Override
+  public int getResultSetSqlType() {
+    return PTinyint.INSTANCE.getResultSetSqlType();
+  }
+
+  @Override
+  public Object getSampleValue(Integer maxLength, Integer arrayLength) {
+    return ((Integer) RANDOM.get().nextInt(Byte.MAX_VALUE)).byteValue();
+  }
+
+  static class UnsignedByteCodec extends PTinyint.ByteCodec {
+
+    @Override
+    public byte decodeByte(byte[] b, int o, SortOrder sortOrder) {
+      Preconditions.checkNotNull(sortOrder);
+      checkForSufficientLength(b, o, Bytes.SIZEOF_BYTE);
+      byte v = b[o];
+      if (sortOrder == SortOrder.DESC) {
+        v = SortOrder.invert(v);
+      }
+      if (v < 0) {
+        throw newIllegalDataException();
+      }
+      return v;
+    }
+
+    @Override
+    public int encodeByte(byte v, byte[] b, int o) {
+      if (v < 0) {
+        throw newIllegalDataException();
+      }
+      Bytes.putByte(b, o, v);
+      return Bytes.SIZEOF_BYTE;
+    }
+  }
+}

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

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

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

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

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

http://git-wip-us.apache.org/repos/asf/phoenix/blob/04ef859b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PhoenixArray.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PhoenixArray.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PhoenixArray.java
new file mode 100644
index 0000000..dcfb2c3
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PhoenixArray.java
@@ -0,0 +1,644 @@
+/*
+ * 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 java.sql.Array;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.Arrays;
+import java.util.Map;
+
+import org.apache.phoenix.util.SQLCloseable;
+
+/**
+ * java.sql.Array implementation for Phoenix
+ */
+public class PhoenixArray implements Array,SQLCloseable {
+	PDataType baseType;
+	Object array;
+	int numElements;
+	Integer maxLength;
+  protected int hashCode = Integer.MIN_VALUE;
+	public PhoenixArray() {
+		// empty constructor
+	}
+	
+	public Integer getMaxLength() {
+	    return maxLength;
+	}
+
+    public boolean isPrimitiveType() {
+        return this.baseType.getCodec() != null;
+    }
+
+	private static Object[] coerceToNewLength(PDataType baseType, Object[] elements, int maxLength) {
+        Object[] resizedElements = new Object[elements.length];
+        for (int i = 0; i < elements.length; i++) {
+            int length = baseType.getMaxLength(elements[i]);
+            if (length == maxLength) {
+                resizedElements[i] = elements[i];
+            } else {
+                resizedElements[i] = baseType.pad(elements[i],maxLength);
+            }
+        }
+        return resizedElements;
+	}
+	private static Object[] coerceToEqualLength(PDataType baseType, Object[] elements) {
+	    if (elements == null || elements.length == 0) {
+	        return elements;
+	    }
+	    Object element = elements[0];
+	    int maxLength = baseType.getMaxLength(element);
+	    boolean resizeElements = false;
+	    for (int i = 1; i < elements.length; i++) {
+	        int length = baseType.getMaxLength(elements[i]);
+	        if (length > maxLength) {
+	            maxLength = length;
+	            resizeElements = true;
+	        } else if (length < maxLength) {
+	            resizeElements = true;
+	        }
+	    }
+	    if (!resizeElements) {
+	        return elements;
+	    }
+	    return coerceToNewLength(baseType, elements, maxLength);
+	}
+	
+	public PhoenixArray(PDataType baseType, Object[] elements) {
+		// As we are dealing with primitive types and only the Boxed objects
+		this.baseType = baseType;
+		if (baseType.isFixedWidth()) {
+		    if (baseType.getByteSize() == null) {
+    		    elements = coerceToEqualLength(baseType, elements);
+    		    if (elements != null && elements.length > 0) {
+    		        this.maxLength = baseType.getMaxLength(elements[0]);
+    		    }
+		    } else {
+		        maxLength = baseType.getByteSize();
+		    }
+		}
+    this.array = convertObjectArrayToPrimitiveArray(elements);
+		this.numElements = elements.length;
+	}
+	
+	public PhoenixArray(PhoenixArray pArr, Integer desiredMaxLength) {
+	    this.baseType = pArr.baseType;
+	    Object[] elements = (Object[])pArr.array;
+        if (baseType.isFixedWidth()) {
+            if (baseType.getByteSize() == null) {
+                elements = coerceToNewLength(baseType, (Object[])pArr.array, desiredMaxLength);
+                maxLength = desiredMaxLength;
+            } else {
+                maxLength = baseType.getByteSize();
+            }
+        }
+        this.array = convertObjectArrayToPrimitiveArray(elements);
+        this.numElements = elements.length;
+    }
+
+  public Object convertObjectArrayToPrimitiveArray(Object[] elements) {
+    return elements;
+	}
+	
+	@Override
+	public void free() throws SQLException {
+	}
+
+	@Override
+	public Object getArray() throws SQLException {
+		return array;
+	}
+	
+	@Override
+	public void close() throws SQLException {
+		this.array = null;
+	}
+
+	@Override
+	public Object getArray(Map<String, Class<?>> map) throws SQLException {
+		throw new UnsupportedOperationException("Currently not supported");
+	}
+
+	@Override
+	public Object getArray(long index, int count) throws SQLException {
+		if(index < 1) {
+			throw new IllegalArgumentException("Index cannot be less than 1");
+		}
+		// Get the set of elements from the given index to the specified count
+		Object[] intArr = (Object[]) array;
+		boundaryCheck(index, count, intArr);
+		Object[] newArr = new Object[count];
+		// Add checks() here.
+		int i = 0;
+		for (int j = (int) index; j < count; j++) {
+			newArr[i] = intArr[j];
+			i++;
+		}
+		return newArr;
+	}
+
+	private void boundaryCheck(long index, int count, Object[] arr) {
+		if ((--index) + count > arr.length) {
+			throw new IllegalArgumentException("The array index is out of range of the total number of elements in the array " + arr.length);
+		}
+	}
+
+	@Override
+	public Object getArray(long index, int count, Map<String, Class<?>> map)
+			throws SQLException {
+		if(map != null && !map.isEmpty()) {
+			throw new UnsupportedOperationException("Currently not supported");
+		}
+		return null;
+	}
+
+	@Override
+	public int getBaseType() throws SQLException {
+		return baseType.getSqlType();
+	}
+
+	@Override
+	public String getBaseTypeName() throws SQLException {
+		return baseType.getSqlTypeName();
+	}
+
+	@Override
+	public ResultSet getResultSet() throws SQLException {
+		throw new UnsupportedOperationException("Currently not supported");
+	}
+
+	@Override
+	public ResultSet getResultSet(Map<String, Class<?>> arg0)
+			throws SQLException {
+		throw new UnsupportedOperationException("Currently not supported");
+	}
+
+	@Override
+	public ResultSet getResultSet(long arg0, int arg1) throws SQLException {
+		throw new UnsupportedOperationException("Currently not supported");
+	}
+
+	@Override
+	public ResultSet getResultSet(long arg0, int arg1,
+			Map<String, Class<?>> arg2) throws SQLException {
+		throw new UnsupportedOperationException("Currently not supported");
+	}
+
+  /**
+   * Return the value in position {@code index} from the underlying array. Used to work around
+   * casting and reflection while enabling primitive arrays.
+   */
+  public Object getElement(int index) {
+    return ((Object[]) array)[index];
+  }
+
+	public int getDimensions() {
+		return this.numElements;
+	}
+	
+	public int estimateByteSize(int pos) {
+	    if(((Object[])array)[pos] == null) {
+	        return 0;
+	    }
+		return this.baseType.estimateByteSize(((Object[])array)[pos]);
+	}
+	
+	public Integer getMaxLength(int pos) {
+	    return this.baseType.getMaxLength(((Object[])array)[pos]);
+	}
+	
+	public byte[] toBytes(int pos) {
+		return this.baseType.toBytes(((Object[])array)[pos]);
+	}
+	
+	public boolean isNull(int pos) {
+	    if(this.baseType.toBytes(((Object[])array)[pos]).length == 0) {
+	        return true;
+	    } else {
+	        return false;
+	    }
+	}
+	
+	@Override
+	public boolean equals(Object obj) {
+    if (obj == null) return false;
+    if (this == obj) return true;
+    if (!(obj instanceof PhoenixArray)) return false;
+    PhoenixArray oArray = (PhoenixArray) obj;
+    if (numElements != oArray.numElements) return false;
+    if (baseType.getSqlType() != oArray.baseType.getSqlType()) return false;
+    return Arrays.deepEquals((Object[]) array, (Object[]) oArray.array);
+	}
+
+	@Override
+	public int hashCode() {
+    // implementation based on commons.lang.HashCodeBuilder, except the hashcode is cached and
+    // reused for a given instance.
+    if (hashCode != Integer.MIN_VALUE) return hashCode;
+    hashCode = 17;
+    hashCode = hashCode * 37 + numElements;
+    hashCode = hashCode * 37 + baseType.getSqlType();
+    hashCode = hashCode * 37 + Arrays.deepHashCode((Object[]) array);
+    return hashCode;
+	}
+
+	public static class PrimitiveIntPhoenixArray extends PhoenixArray {
+		private int[] intArr;
+		public PrimitiveIntPhoenixArray(PDataType dataType, Object[] elements) {
+			super(dataType, elements);
+		}
+		@Override
+		public Object convertObjectArrayToPrimitiveArray(Object[] elements) {
+			intArr = new int[elements.length];
+			int i = 0;
+			for(Object o : elements) {
+			    if (o != null) {
+			        intArr[i] = (Integer)o;
+			    }
+			    i++;
+			}
+      return intArr;
+		}
+		
+		@Override
+        public int estimateByteSize(int pos) {
+			return this.baseType.estimateByteSize(intArr[pos]);
+		}
+		
+		@Override
+        public byte[] toBytes(int pos) {
+			return this.baseType.toBytes(intArr[pos]);
+		}
+		
+		@Override
+		public boolean equals(Object obj) {
+      if (obj == null) return false;
+      if (this == obj) return true;
+      if (this.getClass() != obj.getClass()) return false;
+      PrimitiveIntPhoenixArray oArray = (PrimitiveIntPhoenixArray) obj;
+      if (numElements != oArray.numElements) return false;
+      if (baseType.getSqlType() != oArray.baseType.getSqlType()) return false;
+      return Arrays.equals((int[]) array, (int[]) oArray.array);
+		}
+
+    @Override
+    public int hashCode() {
+      if (hashCode != Integer.MIN_VALUE) return hashCode;
+      hashCode = 17;
+      hashCode = hashCode * 37 + numElements;
+      hashCode = hashCode * 37 + baseType.getSqlType();
+      hashCode = hashCode * 37 + Arrays.hashCode((int[]) array);
+      return hashCode;
+    }
+
+    @Override
+    public Object getElement(int index) {
+      return ((int[]) array)[index];
+    }
+	}
+	
+	public static class PrimitiveShortPhoenixArray extends PhoenixArray {
+		private short[] shortArr;
+		public PrimitiveShortPhoenixArray(PDataType dataType, Object[] elements) {
+			super(dataType, elements);
+		}
+		@Override
+		public Object convertObjectArrayToPrimitiveArray(Object[] elements) {
+			shortArr = new short[elements.length];
+			int i = 0;
+            for(Object o : elements) {
+                if (o != null) {
+                    shortArr[i] = (Short)o;
+                }
+                i++;
+            }
+      return shortArr;
+		}
+		
+		@Override
+        public int estimateByteSize(int pos) {
+			return this.baseType.estimateByteSize(shortArr[pos]);
+		}
+		
+		@Override
+        public byte[] toBytes(int pos) {
+			return this.baseType.toBytes(shortArr[pos]);
+		}
+		
+		@Override
+		public boolean equals(Object obj) {
+      if (obj == null) return false;
+      if (this == obj) return true;
+      if (this.getClass() != obj.getClass()) return false;
+      PrimitiveShortPhoenixArray oArray = (PrimitiveShortPhoenixArray) obj;
+      if (numElements != oArray.numElements) return false;
+      if (baseType.getSqlType() != oArray.baseType.getSqlType()) return false;
+      return Arrays.equals((short[]) array, (short[]) oArray.array);
+		}
+
+    @Override
+    public int hashCode() {
+      if (hashCode != Integer.MIN_VALUE) return hashCode;
+      hashCode = 17;
+      hashCode = hashCode * 37 + numElements;
+      hashCode = hashCode * 37 + baseType.getSqlType();
+      hashCode = hashCode * 37 + Arrays.hashCode((short[]) array);
+      return hashCode;
+    }
+
+    @Override
+    public Object getElement(int index) {
+      return ((short[]) array)[index];
+    }
+  }
+	
+	public static class PrimitiveLongPhoenixArray extends PhoenixArray {
+		private long[] longArr;
+		public PrimitiveLongPhoenixArray(PDataType dataType, Object[] elements) {
+			super(dataType, elements);
+		}
+		@Override
+		public Object convertObjectArrayToPrimitiveArray(Object[] elements) {
+			longArr = new long[elements.length];
+			int i = 0;
+            for(Object o : elements) {
+                if (o != null) {
+                    longArr[i] = (Long)o;
+                }
+                i++;
+            }
+      return longArr;
+		}
+		@Override
+        public int estimateByteSize(int pos) {
+			return this.baseType.estimateByteSize(longArr[pos]);
+		}
+		
+		@Override
+        public byte[] toBytes(int pos) {
+			return this.baseType.toBytes(longArr[pos]);
+		}
+		
+		@Override
+		public boolean equals(Object obj) {
+      if (obj == null) return false;
+      if (this == obj) return true;
+      if (this.getClass() != obj.getClass()) return false;
+      PrimitiveLongPhoenixArray oArray = (PrimitiveLongPhoenixArray) obj;
+      if (numElements != oArray.numElements) return false;
+      if (baseType.getSqlType() != oArray.baseType.getSqlType()) return false;
+      return Arrays.equals((long[]) array, (long[]) oArray.array);
+		}
+
+    @Override
+    public int hashCode() {
+      if (hashCode != Integer.MIN_VALUE) return hashCode;
+      hashCode = 17;
+      hashCode = hashCode * 37 + numElements;
+      hashCode = hashCode * 37 + baseType.getSqlType();
+      hashCode = hashCode * 37 + Arrays.hashCode((long[]) array);
+      return hashCode;
+    }
+
+    @Override
+    public Object getElement(int index) {
+      return ((long[]) array)[index];
+    }
+	}
+	
+	public static class PrimitiveDoublePhoenixArray extends PhoenixArray {
+		private double[] doubleArr;
+		public PrimitiveDoublePhoenixArray(PDataType dataType, Object[] elements) {
+			super(dataType, elements);
+		}
+		@Override
+		public Object convertObjectArrayToPrimitiveArray(Object[] elements) {
+			doubleArr = new double[elements.length];
+			int i = 0;
+			for (Object o : elements) {
+			    if (o != null) {
+			        doubleArr[i] = (Double) o;
+			    }
+			    i++;
+			}
+      return doubleArr;
+		}
+		
+		@Override
+        public int estimateByteSize(int pos) {
+			return this.baseType.estimateByteSize(doubleArr[pos]);
+		}
+		
+		@Override
+        public byte[] toBytes(int pos) {
+			return this.baseType.toBytes(doubleArr[pos]);
+		}
+		
+		@Override
+		public boolean equals(Object obj) {
+      if (obj == null) return false;
+      if (this == obj) return true;
+      if (this.getClass() != obj.getClass()) return false;
+      PrimitiveDoublePhoenixArray oArray = (PrimitiveDoublePhoenixArray) obj;
+      if (numElements != oArray.numElements) return false;
+      if (baseType.getSqlType() != oArray.baseType.getSqlType()) return false;
+      return Arrays.equals((double[]) array, (double[]) oArray.array);
+		}
+
+    @Override
+    public int hashCode() {
+      if (hashCode != Integer.MIN_VALUE) return hashCode;
+      hashCode = 17;
+      hashCode = hashCode * 37 + numElements;
+      hashCode = hashCode * 37 + baseType.getSqlType();
+      hashCode = hashCode * 37 + Arrays.hashCode((double[]) array);
+      return hashCode;
+    }
+
+    @Override
+    public Object getElement(int index) {
+      return ((double[]) array)[index];
+    }
+	}
+	
+	public static class PrimitiveFloatPhoenixArray extends PhoenixArray {
+		private float[] floatArr;
+		public PrimitiveFloatPhoenixArray(PDataType dataType, Object[] elements) {
+			super(dataType, elements);
+		}
+		@Override
+		public Object convertObjectArrayToPrimitiveArray(Object[] elements) {
+			floatArr = new float[elements.length];
+			int i = 0;
+            for(Object o : elements) {
+                if (o != null) {
+                    floatArr[i] = (Float)o;
+                }
+                i++;
+            }
+      return floatArr;
+		}
+		
+		@Override
+        public int estimateByteSize(int pos) {
+			return this.baseType.estimateByteSize(floatArr[pos]);
+		}
+		
+		@Override
+        public byte[] toBytes(int pos) {
+			return this.baseType.toBytes(floatArr[pos]);
+		}
+		
+		@Override
+		public boolean equals(Object obj) {
+      if (obj == null) return false;
+      if (this == obj) return true;
+      if (this.getClass() != obj.getClass()) return false;
+      PrimitiveFloatPhoenixArray oArray = (PrimitiveFloatPhoenixArray) obj;
+      if (numElements != oArray.numElements) return false;
+      if (baseType.getSqlType() != oArray.baseType.getSqlType()) return false;
+      return Arrays.equals((float[]) array, (float[]) oArray.array);
+		}
+
+    @Override
+    public int hashCode() {
+      if (hashCode != Integer.MIN_VALUE) return hashCode;
+      hashCode = 17;
+      hashCode = hashCode * 37 + numElements;
+      hashCode = hashCode * 37 + baseType.getSqlType();
+      hashCode = hashCode * 37 + Arrays.hashCode((float[]) array);
+      return hashCode;
+    }
+
+    @Override
+    public Object getElement(int index) {
+      return ((float[]) array)[index];
+    }
+	}
+	
+	public static class PrimitiveBytePhoenixArray extends PhoenixArray {
+		private byte[] byteArr;
+		public PrimitiveBytePhoenixArray(PDataType dataType, Object[] elements) {
+			super(dataType, elements);
+		}
+		@Override
+		public Object convertObjectArrayToPrimitiveArray(Object[] elements) {
+			byteArr = new byte[elements.length];
+			int i = 0;
+            for(Object o : elements) {
+                if (o != null) {
+                    byteArr[i] = (Byte)o;
+                }
+                i++;
+            }
+      return byteArr;
+		}
+		
+		@Override
+        public int estimateByteSize(int pos) {
+			return this.baseType.estimateByteSize(byteArr[pos]);
+		}
+		
+		@Override
+        public byte[] toBytes(int pos) {
+			return this.baseType.toBytes(byteArr[pos]);
+		}
+		
+		@Override
+		public boolean equals(Object obj) {
+      if (obj == null) return false;
+      if (this == obj) return true;
+      if (this.getClass() != obj.getClass()) return false;
+      PrimitiveBytePhoenixArray oArray = (PrimitiveBytePhoenixArray) obj;
+      if (numElements != oArray.numElements) return false;
+      if (baseType.getSqlType() != oArray.baseType.getSqlType()) return false;
+      return Arrays.equals((byte[]) array, (byte[]) oArray.array);
+		}
+
+    @Override
+    public int hashCode() {
+      if (hashCode != Integer.MIN_VALUE) return hashCode;
+      hashCode = 17;
+      hashCode = hashCode * 37 + numElements;
+      hashCode = hashCode * 37 + baseType.getSqlType();
+      hashCode = hashCode * 37 + Arrays.hashCode((byte[]) array);
+      return hashCode;
+    }
+
+    @Override
+    public Object getElement(int index) {
+      return ((byte[]) array)[index];
+    }
+	}
+	
+	public static class PrimitiveBooleanPhoenixArray extends PhoenixArray {
+		private boolean[] booleanArr;
+		public PrimitiveBooleanPhoenixArray(PDataType dataType, Object[] elements) {
+			super(dataType, elements);
+		}
+		@Override
+		public Object convertObjectArrayToPrimitiveArray(Object[] elements) {
+			booleanArr = new boolean[elements.length];
+			int i = 0;
+            for(Object o : elements) {
+                if (o != null) {
+                    booleanArr[i] = (Boolean)o;
+                }
+                i++;
+            }
+      return booleanArr;
+		}
+		
+		@Override
+        public int estimateByteSize(int pos) {
+			return this.baseType.estimateByteSize(booleanArr[pos]);
+		}
+		
+		@Override
+        public byte[] toBytes(int pos) {
+			return this.baseType.toBytes(booleanArr[pos]);
+		}
+		
+		@Override
+		public boolean equals(Object obj) {
+      if (obj == null) return false;
+      if (this == obj) return true;
+      if (this.getClass() != obj.getClass()) return false;
+      PrimitiveBooleanPhoenixArray oArray = (PrimitiveBooleanPhoenixArray) obj;
+      if (numElements != oArray.numElements) return false;
+      if (baseType.getSqlType() != oArray.baseType.getSqlType()) return false;
+      return Arrays.equals((boolean[]) array, (boolean[]) oArray.array);
+		}
+
+    @Override
+    public int hashCode() {
+      if (hashCode != Integer.MIN_VALUE) return hashCode;
+      hashCode = 17;
+      hashCode = hashCode * 37 + numElements;
+      hashCode = hashCode * 37 + baseType.getSqlType();
+      hashCode = hashCode * 37 + Arrays.hashCode((boolean[]) array);
+      return hashCode;
+    }
+
+    @Override
+    public Object getElement(int index) {
+      return ((boolean[]) array)[index];
+    }
+	}
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/04ef859b/phoenix-core/src/main/java/org/apache/phoenix/util/ByteUtil.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/util/ByteUtil.java b/phoenix-core/src/main/java/org/apache/phoenix/util/ByteUtil.java
index 82be510..1e3516d 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/util/ByteUtil.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/ByteUtil.java
@@ -32,7 +32,7 @@ import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.io.WritableUtils;
 import org.apache.phoenix.hbase.index.util.ImmutableBytesPtr;
 import org.apache.phoenix.query.KeyRange;
-import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.schema.types.PDataType;
 import org.apache.phoenix.schema.SortOrder;
 
 import com.google.common.base.Preconditions;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/04ef859b/phoenix-core/src/main/java/org/apache/phoenix/util/CSVCommonsLoader.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/util/CSVCommonsLoader.java b/phoenix-core/src/main/java/org/apache/phoenix/util/CSVCommonsLoader.java
index 0bf366e..a82dc3c 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/util/CSVCommonsLoader.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/CSVCommonsLoader.java
@@ -28,7 +28,7 @@ import org.apache.commons.csv.CSVRecord;
 import org.apache.phoenix.exception.SQLExceptionCode;
 import org.apache.phoenix.exception.SQLExceptionInfo;
 import org.apache.phoenix.jdbc.PhoenixConnection;
-import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.schema.types.PDataType;
 import org.apache.phoenix.util.csv.CsvUpsertExecutor;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/04ef859b/phoenix-core/src/main/java/org/apache/phoenix/util/ColumnInfo.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/util/ColumnInfo.java b/phoenix-core/src/main/java/org/apache/phoenix/util/ColumnInfo.java
index 79eaeb0..55865c0 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/util/ColumnInfo.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/ColumnInfo.java
@@ -13,7 +13,7 @@ package org.apache.phoenix.util;
 import java.util.List;
 
 import org.apache.phoenix.query.QueryConstants;
-import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.schema.types.PDataType;
 
 import com.google.common.base.Preconditions;
 import com.google.common.base.Splitter;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/04ef859b/phoenix-core/src/main/java/org/apache/phoenix/util/ExpressionUtil.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/util/ExpressionUtil.java b/phoenix-core/src/main/java/org/apache/phoenix/util/ExpressionUtil.java
index eac396b..6032aec 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/util/ExpressionUtil.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/ExpressionUtil.java
@@ -19,7 +19,7 @@ import org.apache.phoenix.expression.LiteralExpression;
 import org.apache.phoenix.expression.function.CurrentDateFunction;
 import org.apache.phoenix.expression.function.CurrentTimeFunction;
 import org.apache.phoenix.expression.function.FunctionExpression;
-import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.schema.types.PDataType;
 
 import com.google.common.collect.Lists;
 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/04ef859b/phoenix-core/src/main/java/org/apache/phoenix/util/IndexUtil.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/util/IndexUtil.java b/phoenix-core/src/main/java/org/apache/phoenix/util/IndexUtil.java
index b6d2fce..ae2d2c4 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/util/IndexUtil.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/IndexUtil.java
@@ -67,11 +67,13 @@ import org.apache.phoenix.query.QueryConstants;
 import org.apache.phoenix.schema.ColumnFamilyNotFoundException;
 import org.apache.phoenix.schema.ColumnNotFoundException;
 import org.apache.phoenix.schema.ColumnRef;
+import org.apache.phoenix.schema.types.PDecimal;
 import org.apache.phoenix.schema.KeyValueSchema;
 import org.apache.phoenix.schema.PColumn;
 import org.apache.phoenix.schema.PColumnFamily;
-import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.schema.types.PDataType;
 import org.apache.phoenix.schema.PTable;
+import org.apache.phoenix.schema.types.PVarchar;
 import org.apache.phoenix.schema.TableNotFoundException;
 import org.apache.phoenix.schema.TableRef;
 import org.apache.phoenix.schema.tuple.ResultTuple;
@@ -106,12 +108,12 @@ public class IndexUtil {
             return dataType;
         }
         // for fixed length numeric types and boolean
-        if (dataType.isCastableTo(PDataType.DECIMAL)) {
-            return PDataType.DECIMAL;
+        if (dataType.isCastableTo(PDecimal.INSTANCE)) {
+            return PDecimal.INSTANCE;
         }
         // for CHAR
-        if (dataType.isCoercibleTo(PDataType.VARCHAR)) {
-            return PDataType.VARCHAR;
+        if (dataType.isCoercibleTo(PVarchar.INSTANCE)) {
+            return PVarchar.INSTANCE;
         }
         throw new IllegalArgumentException("Unsupported non nullable index type " + dataType);
     }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/04ef859b/phoenix-core/src/main/java/org/apache/phoenix/util/MetaDataUtil.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/util/MetaDataUtil.java b/phoenix-core/src/main/java/org/apache/phoenix/util/MetaDataUtil.java
index d5c7a18..c1aa2cc 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/util/MetaDataUtil.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/MetaDataUtil.java
@@ -49,8 +49,11 @@ import org.apache.phoenix.hbase.index.util.VersionUtil;
 import org.apache.phoenix.jdbc.PhoenixConnection;
 import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
 import org.apache.phoenix.query.QueryConstants;
-import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.schema.types.PBoolean;
+import org.apache.phoenix.schema.types.PDataType;
+import org.apache.phoenix.schema.types.PLong;
 import org.apache.phoenix.schema.PName;
+import org.apache.phoenix.schema.types.PSmallint;
 import org.apache.phoenix.schema.PTable;
 import org.apache.phoenix.schema.PTableType;
 import org.apache.phoenix.schema.SequenceKey;
@@ -159,7 +162,7 @@ public class MetaDataUtil {
         if (kvs != null) {
             for (Cell kv : kvs) { // list is not ordered, so search. TODO: we could potentially assume the position
                 if (Bytes.compareTo(kv.getQualifierArray(), kv.getQualifierOffset(), kv.getQualifierLength(), PhoenixDatabaseMetaData.TABLE_SEQ_NUM_BYTES, 0, PhoenixDatabaseMetaData.TABLE_SEQ_NUM_BYTES.length) == 0) {
-                    return PDataType.LONG.getCodec().decodeLong(kv.getValueArray(), kv.getValueOffset(), SortOrder.getDefault());
+                    return PLong.INSTANCE.getCodec().decodeLong(kv.getValueArray(), kv.getValueOffset(), SortOrder.getDefault());
                 }
             }
         }
@@ -251,7 +254,7 @@ public class MetaDataUtil {
     
     public static boolean isMultiTenant(Mutation m, KeyValueBuilder builder, ImmutableBytesWritable ptr) {
         if (getMutationValue(m, PhoenixDatabaseMetaData.MULTI_TENANT_BYTES, builder, ptr)) {
-            return Boolean.TRUE.equals(PDataType.BOOLEAN.toObject(ptr));
+            return Boolean.TRUE.equals(PBoolean.INSTANCE.toObject(ptr));
         }
         return false;
     }
@@ -307,7 +310,7 @@ public class MetaDataUtil {
     }
 
     public static PDataType getViewIndexIdDataType() {
-        return PDataType.SMALLINT;
+        return PSmallint.INSTANCE;
     }
 
     public static String getViewIndexIdColumnName() {
@@ -326,7 +329,7 @@ public class MetaDataUtil {
         byte[] physicalIndexName = MetaDataUtil.getViewIndexPhysicalName(physicalTableName);
         try {
             HTableDescriptor desc = connection.getQueryServices().getTableDescriptor(physicalIndexName);
-            return desc != null && Boolean.TRUE.equals(PDataType.BOOLEAN.toObject(desc.getValue(IS_VIEW_INDEX_TABLE_PROP_BYTES)));
+            return desc != null && Boolean.TRUE.equals(PBoolean.INSTANCE.toObject(desc.getValue(IS_VIEW_INDEX_TABLE_PROP_BYTES)));
         } catch (TableNotFoundException e) {
             return false;
         }
@@ -344,7 +347,7 @@ public class MetaDataUtil {
         byte[] physicalIndexName = MetaDataUtil.getLocalIndexPhysicalName(physicalTableName);
         try {
             HTableDescriptor desc = connection.getQueryServices().getTableDescriptor(physicalIndexName);
-            return desc != null && Boolean.TRUE.equals(PDataType.BOOLEAN.toObject(desc.getValue(IS_LOCAL_INDEX_TABLE_PROP_BYTES)));
+            return desc != null && Boolean.TRUE.equals(PBoolean.INSTANCE.toObject(desc.getValue(IS_LOCAL_INDEX_TABLE_PROP_BYTES)));
         } catch (TableNotFoundException e) {
             return false;
         }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/04ef859b/phoenix-core/src/main/java/org/apache/phoenix/util/NumberUtil.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/util/NumberUtil.java b/phoenix-core/src/main/java/org/apache/phoenix/util/NumberUtil.java
index 7889a89..56e0137 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/util/NumberUtil.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/NumberUtil.java
@@ -19,7 +19,7 @@ package org.apache.phoenix.util;
 
 import java.math.BigDecimal;
 
-import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.schema.types.PDataType;
 
 /**
  * Utility methods for numbers like decimal, long, etc.

http://git-wip-us.apache.org/repos/asf/phoenix/blob/04ef859b/phoenix-core/src/main/java/org/apache/phoenix/util/PhoenixRuntime.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/util/PhoenixRuntime.java b/phoenix-core/src/main/java/org/apache/phoenix/util/PhoenixRuntime.java
index 0e2bae3..269eda5 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/util/PhoenixRuntime.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/PhoenixRuntime.java
@@ -18,7 +18,7 @@
 package org.apache.phoenix.util;
 
 import static com.google.common.base.Preconditions.checkNotNull;
-import static org.apache.phoenix.schema.PDataType.ARRAY_TYPE_SUFFIX;
+import static org.apache.phoenix.schema.types.PDataType.ARRAY_TYPE_SUFFIX;
 
 import java.io.File;
 import java.io.FileReader;
@@ -66,7 +66,7 @@ import org.apache.phoenix.schema.KeyValueSchema.KeyValueSchemaBuilder;
 import org.apache.phoenix.schema.MetaDataClient;
 import org.apache.phoenix.schema.PColumn;
 import org.apache.phoenix.schema.PColumnFamily;
-import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.schema.types.PDataType;
 import org.apache.phoenix.schema.PTable;
 import org.apache.phoenix.schema.PTableKey;
 import org.apache.phoenix.schema.PTableType;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/04ef859b/phoenix-core/src/main/java/org/apache/phoenix/util/ScanUtil.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/util/ScanUtil.java b/phoenix-core/src/main/java/org/apache/phoenix/util/ScanUtil.java
index db1c661..2dfa573 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/util/ScanUtil.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/ScanUtil.java
@@ -45,15 +45,15 @@ import org.apache.phoenix.filter.SkipScanFilter;
 import org.apache.phoenix.query.KeyRange;
 import org.apache.phoenix.query.KeyRange.Bound;
 import org.apache.phoenix.query.QueryConstants;
-import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.schema.types.PDataType;
 import org.apache.phoenix.schema.PName;
 import org.apache.phoenix.schema.PNameFactory;
+import org.apache.phoenix.schema.types.PVarbinary;
 import org.apache.phoenix.schema.RowKeySchema;
 import org.apache.phoenix.schema.ValueSchema.Field;
 
 import com.google.common.collect.Lists;
 
-
 /**
  * 
  * Various utilities for scans
@@ -437,7 +437,7 @@ public class ScanUtil {
     public static ScanRanges newScanRanges(List<Mutation> mutations) throws SQLException {
         List<KeyRange> keys = Lists.newArrayListWithExpectedSize(mutations.size());
         for (Mutation m : mutations) {
-            keys.add(PDataType.VARBINARY.getKeyRange(m.getRow()));
+            keys.add(PVarbinary.INSTANCE.getKeyRange(m.getRow()));
         }
         ScanRanges keyRanges = ScanRanges.create(SchemaUtil.VAR_BINARY_SCHEMA, Collections.singletonList(keys), ScanUtil.SINGLE_COLUMN_SLOT_SPAN);
         return keyRanges;