You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@phoenix.apache.org by ja...@apache.org on 2016/10/28 06:12:43 UTC

[1/3] phoenix git commit: PHOENIX-3421 Column name lookups fail when on an indexed table

Repository: phoenix
Updated Branches:
  refs/heads/4.x-HBase-1.1 837d114b2 -> 48e828ccb


PHOENIX-3421 Column name lookups fail when on an indexed table


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

Branch: refs/heads/4.x-HBase-1.1
Commit: 48e828ccbbb763b3c5bf2217eb284200f1748940
Parents: faa2cd0
Author: James Taylor <ja...@apache.org>
Authored: Thu Oct 27 23:09:09 2016 -0700
Committer: James Taylor <ja...@apache.org>
Committed: Thu Oct 27 23:11:39 2016 -0700

----------------------------------------------------------------------
 .../org/apache/phoenix/util/PhoenixRuntime.java | 34 +++++++++++++++
 .../phoenix/compile/QueryOptimizerTest.java     |  3 +-
 .../apache/phoenix/util/PhoenixRuntimeTest.java | 44 ++++++++++++++++++++
 3 files changed, 79 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/48e828cc/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 764d135..5dd4592 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
@@ -801,6 +801,33 @@ public class PhoenixRuntime {
     private static String addQuotes(String str) {
         return str == null ? str : "\"" + str + "\"";
     }
+    
+    /**
+    * Get the column family, column name pairs that make up the row key of the table that will be queried.
+    * @param conn - connection used to generate the query plan. Caller should take care of closing the connection appropriately.
+    * @param plan - query plan to get info for.
+    * @return the pairs of column family name and column name columns in the data table that make up the row key for
+    * the table used in the query plan. Column family names are optional and hence the first part of the pair is nullable.
+    * Column names and family names are enclosed in double quotes to allow for case sensitivity and for presence of 
+    * special characters. Salting column and view index id column are not included. If the connection is tenant specific 
+    * and the table used by the query plan is multi-tenant, then the tenant id column is not included as well.
+    * @throws SQLException
+    */
+    public static List<Pair<String, String>> getPkColsForSql(Connection conn, QueryPlan plan) throws SQLException {
+        checkNotNull(plan);
+        checkNotNull(conn);
+        List<PColumn> pkColumns = getPkColumns(plan.getTableRef().getTable(), conn, true);
+        List<Pair<String, String>> columns = Lists.newArrayListWithExpectedSize(pkColumns.size());
+        String columnName;
+        String familyName;
+        for (PColumn pCol : pkColumns ) {
+            columnName = addQuotes(pCol.getName().getString());
+            familyName = pCol.getFamilyName() != null ? addQuotes(pCol.getFamilyName().getString()) : null;
+            columns.add(new Pair<String, String>(familyName, columnName));
+        }
+        return columns;
+    }
+
     /**
      *
      * @param columns - Initialized empty list to be filled with the pairs of column family name and column name for columns that are used 
@@ -815,6 +842,7 @@ public class PhoenixRuntime {
      * names correspond to the index table.
      * @throws SQLException
      */
+    @Deprecated
     public static void getPkColsForSql(List<Pair<String, String>> columns, QueryPlan plan, Connection conn, boolean forDataTable) throws SQLException {
         checkNotNull(columns);
         checkNotNull(plan);
@@ -843,6 +871,7 @@ public class PhoenixRuntime {
      * types correspond to the index table.
      * @throws SQLException
      */
+    @Deprecated
     public static void getPkColsDataTypesForSql(List<Pair<String, String>> columns, List<String> dataTypes, QueryPlan plan, Connection conn, boolean forDataTable) throws SQLException {
         checkNotNull(columns);
         checkNotNull(dataTypes);
@@ -1022,6 +1051,11 @@ public class PhoenixRuntime {
         // normalize and remove quotes from family and column names before looking up.
         familyName = SchemaUtil.normalizeIdentifier(familyName);
         columnName = SchemaUtil.normalizeIdentifier(columnName);
+        // Column names are always for the data table, so we must translate them if
+        // we're dealing with an index table.
+        if (table.getType() == PTableType.INDEX) {
+            columnName = IndexUtil.getIndexColumnName(familyName, columnName);
+        }
         PColumn pColumn = null;
         if (familyName != null) {
             PColumnFamily family = table.getColumnFamily(familyName);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/48e828cc/phoenix-core/src/test/java/org/apache/phoenix/compile/QueryOptimizerTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/compile/QueryOptimizerTest.java b/phoenix-core/src/test/java/org/apache/phoenix/compile/QueryOptimizerTest.java
index 47dfcd3..b3a845c 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/compile/QueryOptimizerTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/compile/QueryOptimizerTest.java
@@ -641,8 +641,7 @@ public class QueryOptimizerTest extends BaseConnectionlessQueryTest {
         Connection conn = stmt.getConnection();
         QueryPlan plan = PhoenixRuntime.getOptimizedQueryPlan(stmt);
         
-        List<Pair<String, String>> columns = new ArrayList<Pair<String, String>>();
-        PhoenixRuntime.getPkColsForSql(columns, plan, conn, true);
+        List<Pair<String, String>> columns = PhoenixRuntime.getPkColsForSql(conn, plan);
         assertEquals(expectedPkCols, Joiner.on(",").join(getColumnNames(columns)));
         List<String> dataTypes = new ArrayList<String>();
         columns = new ArrayList<Pair<String,String>>();

http://git-wip-us.apache.org/repos/asf/phoenix/blob/48e828cc/phoenix-core/src/test/java/org/apache/phoenix/util/PhoenixRuntimeTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/util/PhoenixRuntimeTest.java b/phoenix-core/src/test/java/org/apache/phoenix/util/PhoenixRuntimeTest.java
index fe81120..783ab17 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/util/PhoenixRuntimeTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/util/PhoenixRuntimeTest.java
@@ -18,6 +18,7 @@
 
 package org.apache.phoenix.util;
 
+import static org.junit.Assert.assertArrayEquals;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNotNull;
@@ -26,6 +27,7 @@ import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
 import java.sql.Connection;
+import java.sql.Date;
 import java.sql.DriverManager;
 import java.sql.PreparedStatement;
 import java.sql.SQLException;
@@ -39,6 +41,7 @@ import org.apache.hadoop.hbase.util.Pair;
 import org.apache.phoenix.compile.QueryPlan;
 import org.apache.phoenix.expression.Expression;
 import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
+import org.apache.phoenix.jdbc.PhoenixStatement;
 import org.apache.phoenix.query.BaseConnectionlessQueryTest;
 import org.apache.phoenix.query.QueryConstants;
 import org.apache.phoenix.schema.PTable;
@@ -102,6 +105,47 @@ public class PhoenixRuntimeTest extends BaseConnectionlessQueryTest {
     }
     
     @Test
+    public void testGetPkColsEncodeDecode() throws Exception {
+        Connection conn = DriverManager.getConnection(getUrl());
+        String ddl = "CREATE TABLE t (\n" + 
+                "TENANT_ID VARCHAR NOT NULL,\n" +
+                "PARENT_ID CHAR(15) NOT NULL,\n" + 
+                "CREATED_DATE DATE NOT NULL,\n" + 
+                "ENTITY_HISTORY_ID CHAR(15) NOT NULL,\n" + 
+                "DATA_TYPE VARCHAR,\n" + 
+                "OLDVAL_STRING VARCHAR,\n" + 
+                "NEWVAL_STRING VARCHAR\n" + 
+                "CONSTRAINT PK PRIMARY KEY(TENANT_ID, PARENT_ID, CREATED_DATE DESC, ENTITY_HISTORY_ID))"
+                + " MULTI_TENANT = true, IMMUTABLE_ROWS = true";
+        conn.createStatement().execute(ddl);
+        String indexDDL = "CREATE INDEX i ON t (CREATED_DATE, PARENT_ID) INCLUDE (DATA_TYPE, OLDVAL_STRING, NEWVAL_STRING)";
+        conn.createStatement().execute(indexDDL);
+        
+        String tenantId = "111111111111111";
+        String parentId = "222222222222222";
+        Date createdDate = new Date(System.currentTimeMillis());
+        String ehId = "333333333333333";
+        
+        Object[] values = new Object[] {tenantId, createdDate, parentId, ehId};
+        QueryPlan plan = conn.createStatement().unwrap(PhoenixStatement.class).optimizeQuery("SELECT PARENT_ID FROM T WHERE CREATED_DATE > CURRENT_DATE()-1 AND TENANT_ID = '111111111111111'");
+        List<Pair<String,String>> pkColumns = PhoenixRuntime.getPkColsForSql(conn, plan);
+        String fullTableName = plan.getTableRef().getTable().getName().getString();
+        assertEquals("I", fullTableName);
+        byte[] encodedValues = PhoenixRuntime.encodeValues(conn, fullTableName, values, pkColumns);
+        Object[] decodedValues = PhoenixRuntime.decodeValues(conn, fullTableName, encodedValues, pkColumns);
+        assertArrayEquals(values, decodedValues);
+        
+        plan = conn.createStatement().unwrap(PhoenixStatement.class).optimizeQuery("SELECT /*+ NO_INDEX */ ENTITY_HISTORY_ID FROM T");
+        pkColumns = PhoenixRuntime.getPkColsForSql(conn, plan);
+        values = new Object[] {tenantId, parentId, createdDate, ehId};
+        fullTableName = plan.getTableRef().getTable().getName().getString();
+        assertEquals("T", fullTableName);
+        encodedValues = PhoenixRuntime.encodeValues(conn, fullTableName, values, pkColumns);
+        decodedValues = PhoenixRuntime.decodeValues(conn, fullTableName, encodedValues, pkColumns);
+        assertArrayEquals(values, decodedValues);
+    }
+    
+    @Test
     public void testGetPkColsDataTypes() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl(), new Properties());
         int i = 0;


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

Posted by ja...@apache.org.
PHOENIX-3396 Valid Multi-byte strings whose total byte size is greater than the max char limit cannot be inserted into VARCHAR fields in the PK


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

Branch: refs/heads/4.x-HBase-1.1
Commit: faa2cd04396b0e112f2967cc3aa5b2b508b00979
Parents: 837d114
Author: James Taylor <ja...@apache.org>
Authored: Thu Oct 27 20:31:42 2016 -0700
Committer: James Taylor <ja...@apache.org>
Committed: Thu Oct 27 23:11:39 2016 -0700

----------------------------------------------------------------------
 .../phoenix/end2end/ArithmeticQueryIT.java      |  11 +-
 .../apache/phoenix/end2end/UpsertSelectIT.java  |  56 +-
 .../apache/phoenix/compile/UpsertCompiler.java  |   8 +-
 .../UngroupedAggregateRegionObserver.java       | 369 +++++-----
 .../exception/DataExceedsCapacityException.java |  14 +-
 .../phoenix/exception/SQLExceptionInfo.java     |   9 +-
 .../function/ArrayConcatFunction.java           |   5 +-
 .../function/ArrayModifierFunction.java         |   8 +-
 .../phoenix/index/PhoenixIndexBuilder.java      |   4 +-
 .../org/apache/phoenix/parse/ColumnDef.java     |   4 +-
 .../org/apache/phoenix/schema/PTableImpl.java   |  47 +-
 .../phoenix/schema/types/PArrayDataType.java    |  11 +-
 .../apache/phoenix/schema/types/PBinary.java    | 340 +++++-----
 .../phoenix/schema/types/PBinaryBase.java       |  17 +
 .../org/apache/phoenix/schema/types/PChar.java  |  15 +-
 .../apache/phoenix/schema/types/PDataType.java  |   5 +-
 .../apache/phoenix/schema/types/PDecimal.java   | 669 ++++++++++---------
 .../apache/phoenix/schema/types/PVarbinary.java | 248 ++++---
 .../apache/phoenix/schema/types/PVarchar.java   | 268 ++++----
 .../org/apache/phoenix/util/SchemaUtil.java     |  11 +-
 .../org/apache/phoenix/schema/MutationTest.java |  54 ++
 21 files changed, 1154 insertions(+), 1019 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/faa2cd04/phoenix-core/src/it/java/org/apache/phoenix/end2end/ArithmeticQueryIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ArithmeticQueryIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ArithmeticQueryIT.java
index 5ad356b..c297441 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ArithmeticQueryIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ArithmeticQueryIT.java
@@ -225,11 +225,16 @@ public class ArithmeticQueryIT extends ParallelStatsDisabledIT {
             assertTrue(rs.next());
             assertEquals(new BigDecimal("100.3"), rs.getBigDecimal(1));
             assertFalse(rs.next());
-            // source and target in same table, values scheme incompatible.
+            // source and target in same table, values scheme incompatible. should throw
             query = "UPSERT INTO " + source + "(pk, col4) SELECT pk, col1 from " + source;
             stmt = conn.prepareStatement(query);
-            stmt.execute();
-            conn.commit();
+            try {
+                stmt.execute();
+                conn.commit();
+                fail();
+            } catch (SQLException e) {
+                assertEquals(SQLExceptionCode.DATA_EXCEEDS_MAX_CAPACITY.getErrorCode(), e.getErrorCode());
+            }
             query = "SELECT col4 FROM " + source;
             stmt = conn.prepareStatement(query);
             rs = stmt.executeQuery();

http://git-wip-us.apache.org/repos/asf/phoenix/blob/faa2cd04/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpsertSelectIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpsertSelectIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpsertSelectIT.java
index 3561274..763f11b 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpsertSelectIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpsertSelectIT.java
@@ -19,6 +19,7 @@ package org.apache.phoenix.end2end;
 
 import static org.apache.phoenix.util.PhoenixRuntime.TENANT_ID_ATTRIB;
 import static org.apache.phoenix.util.PhoenixRuntime.UPSERT_BATCH_SIZE_ATTRIB;
+import static org.apache.phoenix.util.TestUtil.ATABLE_NAME;
 import static org.apache.phoenix.util.TestUtil.A_VALUE;
 import static org.apache.phoenix.util.TestUtil.B_VALUE;
 import static org.apache.phoenix.util.TestUtil.CUSTOM_ENTITY_DATA_FULL_NAME;
@@ -29,7 +30,6 @@ import static org.apache.phoenix.util.TestUtil.ROW7;
 import static org.apache.phoenix.util.TestUtil.ROW8;
 import static org.apache.phoenix.util.TestUtil.ROW9;
 import static org.apache.phoenix.util.TestUtil.TEST_PROPERTIES;
-import static org.apache.phoenix.util.TestUtil.ATABLE_NAME;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNotNull;
@@ -1375,6 +1375,60 @@ public class UpsertSelectIT extends BaseClientManagedTimeIT {
         assertEquals("[[128,0,0,54], [128,0,4,0]]", rs.getArray(2).toString());
     }
 
+    @Test
+    public void testUpsertSelectWithMultiByteCharsNoAutoCommit() throws Exception {
+        testUpsertSelectWithMultiByteChars(false);
+    }
+
+    @Test
+    public void testUpsertSelectWithMultiByteCharsAutoCommit() throws Exception {
+        testUpsertSelectWithMultiByteChars(true);
+    }
+
+    private void testUpsertSelectWithMultiByteChars(boolean autoCommit) throws Exception {
+        long ts = nextTimestamp();
+        Properties props = new Properties();
+        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts));
+        Connection conn = DriverManager.getConnection(getUrl(), props);
+        conn.setAutoCommit(autoCommit);
+        conn.createStatement().execute(
+                "create table t1 (id bigint not null primary key, v varchar(20))");
+        conn.close();
+
+        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 10));
+        conn = DriverManager.getConnection(getUrl(), props);
+        conn.setAutoCommit(autoCommit);
+        conn.createStatement().execute("upsert into t1 values (1, 'foo')");
+        conn.commit();
+
+        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 15));
+        conn = DriverManager.getConnection(getUrl(), props);
+        conn.setAutoCommit(autoCommit);
+        conn.createStatement().execute(
+                "upsert into t1(id, v) select id, '\u6fb4\u7c96\u87e4\u092f\u893b\u9143\u5ca4\u8c66\ud311\u85b0\u9129\u813c\u056a\ub066\u7889\u7889\u7889\u7889\u7889\u7889' from t1 WHERE id = 1");
+        conn.commit();
+
+        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 20));
+        conn = DriverManager.getConnection(getUrl(), props);
+        conn.setAutoCommit(autoCommit);
+        ResultSet rs = conn.createStatement().executeQuery("select * from t1");
+
+        assertTrue(rs.next());
+        assertEquals(1, rs.getLong(1));
+        assertEquals("\u6fb4\u7c96\u87e4\u092f\u893b\u9143\u5ca4\u8c66\ud311\u85b0\u9129\u813c\u056a\ub066\u7889\u7889\u7889\u7889\u7889\u7889", rs.getString(2));
+
+        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 25));
+        conn = DriverManager.getConnection(getUrl(), props);
+        conn.setAutoCommit(autoCommit);
+        try {
+            conn.createStatement().execute(
+                    "upsert into t1(id, v) select id, '\u6fb4\u7c96\u87e4\u092f\u893b\u9143\u5ca4\u8c66\ud311\u85b0\u9129\u813c\u056a\ub066\u7889\u7889\u7889\u7889\u7889\u7889\u7889' from t1 WHERE id = 1");
+            conn.commit();
+            fail();
+        } catch (SQLException e) {
+            assertEquals(SQLExceptionCode.DATA_EXCEEDS_MAX_CAPACITY.getErrorCode(), e.getErrorCode());
+        }
+    }
 
     @Test
     public void testParallelUpsertSelect() throws Exception {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/faa2cd04/phoenix-core/src/main/java/org/apache/phoenix/compile/UpsertCompiler.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/UpsertCompiler.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/UpsertCompiler.java
index 85517a1..8837445 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/UpsertCompiler.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/UpsertCompiler.java
@@ -203,8 +203,8 @@ public class UpsertCompiler {
                     Integer scale = rsScale == 0 ? null : rsScale;
                     // We are guaranteed that the two column will have compatible types,
                     // as we checked that before.
-                    if (!column.getDataType().isSizeCompatible(ptr, value, column.getDataType(), precision, scale,
-                            column.getMaxLength(), column.getScale())) { throw new SQLExceptionInfo.Builder(
+                    if (!column.getDataType().isSizeCompatible(ptr, value, column.getDataType(), SortOrder.getDefault(), precision,
+                            scale, column.getMaxLength(), column.getScale())) { throw new SQLExceptionInfo.Builder(
                             SQLExceptionCode.DATA_EXCEEDS_MAX_CAPACITY).setColumnName(column.getName().getString())
                             .setMessage("value=" + column.getDataType().toStringLiteral(ptr, null)).build()
                             .buildException(); }
@@ -1001,8 +1001,8 @@ public class UpsertCompiler {
                                         + constantExpression.toString() + " in column " + column);
                         }
                         if (!column.getDataType().isSizeCompatible(ptr, value, constantExpression.getDataType(),
-                                constantExpression.getMaxLength(), constantExpression.getScale(), 
-                                column.getMaxLength(), column.getScale())) { 
+                                constantExpression.getSortOrder(), constantExpression.getMaxLength(), 
+                                constantExpression.getScale(), column.getMaxLength(), column.getScale())) { 
                             throw new SQLExceptionInfo.Builder(
                                 SQLExceptionCode.DATA_EXCEEDS_MAX_CAPACITY).setColumnName(column.getName().getString())
                                 .setMessage("value=" + constantExpression.toString()).build().buildException();

http://git-wip-us.apache.org/repos/asf/phoenix/blob/faa2cd04/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/UngroupedAggregateRegionObserver.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/UngroupedAggregateRegionObserver.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/UngroupedAggregateRegionObserver.java
index 10d21d3..9ee0054 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/UngroupedAggregateRegionObserver.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/UngroupedAggregateRegionObserver.java
@@ -86,7 +86,6 @@ import org.apache.phoenix.index.PhoenixIndexCodec;
 import org.apache.phoenix.join.HashJoinInfo;
 import org.apache.phoenix.query.QueryConstants;
 import org.apache.phoenix.query.QueryServicesOptions;
-import org.apache.phoenix.schema.ConstraintViolationException;
 import org.apache.phoenix.schema.PColumn;
 import org.apache.phoenix.schema.PRow;
 import org.apache.phoenix.schema.PTable;
@@ -109,7 +108,6 @@ import org.apache.phoenix.util.IndexUtil;
 import org.apache.phoenix.util.KeyValueUtil;
 import org.apache.phoenix.util.LogUtil;
 import org.apache.phoenix.util.ScanUtil;
-import org.apache.phoenix.util.SchemaUtil;
 import org.apache.phoenix.util.ServerUtil;
 import org.apache.phoenix.util.StringUtil;
 import org.apache.phoenix.util.TimeKeeper;
@@ -403,205 +401,199 @@ public class UngroupedAggregateRegionObserver extends BaseScannerRegionObserver
                     if (!results.isEmpty()) {
                         rowCount++;
                         result.setKeyValues(results);
-                        try {
-                            if (isDescRowKeyOrderUpgrade) {
-                                Arrays.fill(values, null);
-                                Cell firstKV = results.get(0);
-                                RowKeySchema schema = projectedTable.getRowKeySchema();
-                                int maxOffset = schema.iterator(firstKV.getRowArray(), firstKV.getRowOffset() + offset, firstKV.getRowLength(), ptr);
-                                for (int i = 0; i < schema.getFieldCount(); i++) {
-                                    Boolean hasValue = schema.next(ptr, i, maxOffset);
-                                    if (hasValue == null) {
-                                        break;
+                        if (isDescRowKeyOrderUpgrade) {
+                            Arrays.fill(values, null);
+                            Cell firstKV = results.get(0);
+                            RowKeySchema schema = projectedTable.getRowKeySchema();
+                            int maxOffset = schema.iterator(firstKV.getRowArray(), firstKV.getRowOffset() + offset, firstKV.getRowLength(), ptr);
+                            for (int i = 0; i < schema.getFieldCount(); i++) {
+                                Boolean hasValue = schema.next(ptr, i, maxOffset);
+                                if (hasValue == null) {
+                                    break;
+                                }
+                                Field field = schema.getField(i);
+                                if (field.getSortOrder() == SortOrder.DESC) {
+                                    // Special case for re-writing DESC ARRAY, as the actual byte value needs to change in this case
+                                    if (field.getDataType().isArrayType()) {
+                                        field.getDataType().coerceBytes(ptr, null, field.getDataType(),
+                                            field.getMaxLength(), field.getScale(), field.getSortOrder(), 
+                                            field.getMaxLength(), field.getScale(), field.getSortOrder(), true); // force to use correct separator byte
                                     }
-                                    Field field = schema.getField(i);
-                                    if (field.getSortOrder() == SortOrder.DESC) {
-                                        // Special case for re-writing DESC ARRAY, as the actual byte value needs to change in this case
-                                        if (field.getDataType().isArrayType()) {
-                                            field.getDataType().coerceBytes(ptr, null, field.getDataType(),
-                                                field.getMaxLength(), field.getScale(), field.getSortOrder(), 
-                                                field.getMaxLength(), field.getScale(), field.getSortOrder(), true); // force to use correct separator byte
-                                        }
-                                        // Special case for re-writing DESC CHAR or DESC BINARY, to force the re-writing of trailing space characters
-                                        else if (field.getDataType() == PChar.INSTANCE || field.getDataType() == PBinary.INSTANCE) {
-                                            int len = ptr.getLength();
-                                            while (len > 0 && ptr.get()[ptr.getOffset() + len - 1] == StringUtil.SPACE_UTF8) {
-                                                len--;
-                                            }
-                                            ptr.set(ptr.get(), ptr.getOffset(), len);
-                                            // Special case for re-writing DESC FLOAT and DOUBLE, as they're not inverted like they should be (PHOENIX-2171)
-                                        } else if (field.getDataType() == PFloat.INSTANCE || field.getDataType() == PDouble.INSTANCE) {
-                                            byte[] invertedBytes = SortOrder.invert(ptr.get(), ptr.getOffset(), ptr.getLength());
-                                            ptr.set(invertedBytes);
-                                        }
-                                    } else if (field.getDataType() == PBinary.INSTANCE) {
-                                        // Remove trailing space characters so that the setValues call below will replace them
-                                        // with the correct zero byte character. Note this is somewhat dangerous as these
-                                        // could be legit, but I don't know what the alternative is.
+                                    // Special case for re-writing DESC CHAR or DESC BINARY, to force the re-writing of trailing space characters
+                                    else if (field.getDataType() == PChar.INSTANCE || field.getDataType() == PBinary.INSTANCE) {
                                         int len = ptr.getLength();
                                         while (len > 0 && ptr.get()[ptr.getOffset() + len - 1] == StringUtil.SPACE_UTF8) {
                                             len--;
                                         }
-                                        ptr.set(ptr.get(), ptr.getOffset(), len);                                        
+                                        ptr.set(ptr.get(), ptr.getOffset(), len);
+                                        // Special case for re-writing DESC FLOAT and DOUBLE, as they're not inverted like they should be (PHOENIX-2171)
+                                    } else if (field.getDataType() == PFloat.INSTANCE || field.getDataType() == PDouble.INSTANCE) {
+                                        byte[] invertedBytes = SortOrder.invert(ptr.get(), ptr.getOffset(), ptr.getLength());
+                                        ptr.set(invertedBytes);
                                     }
-                                    values[i] = ptr.copyBytes();
-                                }
-                                writeToTable.newKey(ptr, values);
-                                if (Bytes.compareTo(
-                                    firstKV.getRowArray(), firstKV.getRowOffset() + offset, firstKV.getRowLength(), 
-                                    ptr.get(),ptr.getOffset() + offset,ptr.getLength()) == 0) {
-                                    continue;
-                                }
-                                byte[] newRow = ByteUtil.copyKeyBytesIfNecessary(ptr);
-                                if (offset > 0) { // for local indexes (prepend region start key)
-                                    byte[] newRowWithOffset = new byte[offset + newRow.length];
-                                    System.arraycopy(firstKV.getRowArray(), firstKV.getRowOffset(), newRowWithOffset, 0, offset);;
-                                    System.arraycopy(newRow, 0, newRowWithOffset, offset, newRow.length);
-                                    newRow = newRowWithOffset;
+                                } else if (field.getDataType() == PBinary.INSTANCE) {
+                                    // Remove trailing space characters so that the setValues call below will replace them
+                                    // with the correct zero byte character. Note this is somewhat dangerous as these
+                                    // could be legit, but I don't know what the alternative is.
+                                    int len = ptr.getLength();
+                                    while (len > 0 && ptr.get()[ptr.getOffset() + len - 1] == StringUtil.SPACE_UTF8) {
+                                        len--;
+                                    }
+                                    ptr.set(ptr.get(), ptr.getOffset(), len);                                        
                                 }
-                                byte[] oldRow = Bytes.copy(firstKV.getRowArray(), firstKV.getRowOffset(), firstKV.getRowLength());
-                                for (Cell cell : results) {
-                                    // Copy existing cell but with new row key
-                                    Cell newCell = new KeyValue(newRow, 0, newRow.length,
+                                values[i] = ptr.copyBytes();
+                            }
+                            writeToTable.newKey(ptr, values);
+                            if (Bytes.compareTo(
+                                firstKV.getRowArray(), firstKV.getRowOffset() + offset, firstKV.getRowLength(), 
+                                ptr.get(),ptr.getOffset() + offset,ptr.getLength()) == 0) {
+                                continue;
+                            }
+                            byte[] newRow = ByteUtil.copyKeyBytesIfNecessary(ptr);
+                            if (offset > 0) { // for local indexes (prepend region start key)
+                                byte[] newRowWithOffset = new byte[offset + newRow.length];
+                                System.arraycopy(firstKV.getRowArray(), firstKV.getRowOffset(), newRowWithOffset, 0, offset);;
+                                System.arraycopy(newRow, 0, newRowWithOffset, offset, newRow.length);
+                                newRow = newRowWithOffset;
+                            }
+                            byte[] oldRow = Bytes.copy(firstKV.getRowArray(), firstKV.getRowOffset(), firstKV.getRowLength());
+                            for (Cell cell : results) {
+                                // Copy existing cell but with new row key
+                                Cell newCell = new KeyValue(newRow, 0, newRow.length,
+                                    cell.getFamilyArray(), cell.getFamilyOffset(), cell.getFamilyLength(),
+                                    cell.getQualifierArray(), cell.getQualifierOffset(), cell.getQualifierLength(),
+                                    cell.getTimestamp(), KeyValue.Type.codeToType(cell.getTypeByte()),
+                                    cell.getValueArray(), cell.getValueOffset(), cell.getValueLength());
+                                switch (KeyValue.Type.codeToType(cell.getTypeByte())) {
+                                case Put:
+                                    // If Put, point delete old Put
+                                    Delete del = new Delete(oldRow);
+                                    del.addDeleteMarker(new KeyValue(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength(),
                                         cell.getFamilyArray(), cell.getFamilyOffset(), cell.getFamilyLength(),
-                                        cell.getQualifierArray(), cell.getQualifierOffset(), cell.getQualifierLength(),
-                                        cell.getTimestamp(), KeyValue.Type.codeToType(cell.getTypeByte()),
-                                        cell.getValueArray(), cell.getValueOffset(), cell.getValueLength());
-                                    switch (KeyValue.Type.codeToType(cell.getTypeByte())) {
-                                    case Put:
-                                        // If Put, point delete old Put
-                                        Delete del = new Delete(oldRow);
-                                        del.addDeleteMarker(new KeyValue(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength(),
-                                            cell.getFamilyArray(), cell.getFamilyOffset(), cell.getFamilyLength(),
-                                            cell.getQualifierArray(), cell.getQualifierOffset(),
-                                            cell.getQualifierLength(), cell.getTimestamp(), KeyValue.Type.Delete,
-                                            ByteUtil.EMPTY_BYTE_ARRAY, 0, 0));
-                                        mutations.add(del);
+                                        cell.getQualifierArray(), cell.getQualifierOffset(),
+                                        cell.getQualifierLength(), cell.getTimestamp(), KeyValue.Type.Delete,
+                                        ByteUtil.EMPTY_BYTE_ARRAY, 0, 0));
+                                    mutations.add(del);
 
-                                        Put put = new Put(newRow);
-                                        put.add(newCell);
-                                        mutations.add(put);
-                                        break;
-                                    case Delete:
-                                    case DeleteColumn:
-                                    case DeleteFamily:
-                                    case DeleteFamilyVersion:
-                                        Delete delete = new Delete(newRow);
-                                        delete.addDeleteMarker(newCell);
-                                        mutations.add(delete);
-                                        break;
-                                    }
+                                    Put put = new Put(newRow);
+                                    put.add(newCell);
+                                    mutations.add(put);
+                                    break;
+                                case Delete:
+                                case DeleteColumn:
+                                case DeleteFamily:
+                                case DeleteFamilyVersion:
+                                    Delete delete = new Delete(newRow);
+                                    delete.addDeleteMarker(newCell);
+                                    mutations.add(delete);
+                                    break;
                                 }
-                            } else if (buildLocalIndex) {
-                                for (IndexMaintainer maintainer : indexMaintainers) {
-                                    if (!results.isEmpty()) {
-                                        result.getKey(ptr);
-                                        ValueGetter valueGetter =
-                                                maintainer.createGetterFromKeyValues(
-                                                    ImmutableBytesPtr.copyBytesIfNecessary(ptr),
-                                                    results);
-                                        Put put = maintainer.buildUpdateMutation(kvBuilder,
-                                            valueGetter, ptr, results.get(0).getTimestamp(),
-                                            env.getRegion().getRegionInfo().getStartKey(),
-                                            env.getRegion().getRegionInfo().getEndKey());
-                                        indexMutations.add(put);
-                                    }
+                            }
+                        } else if (buildLocalIndex) {
+                            for (IndexMaintainer maintainer : indexMaintainers) {
+                                if (!results.isEmpty()) {
+                                    result.getKey(ptr);
+                                    ValueGetter valueGetter =
+                                            maintainer.createGetterFromKeyValues(
+                                                ImmutableBytesPtr.copyBytesIfNecessary(ptr),
+                                                results);
+                                    Put put = maintainer.buildUpdateMutation(kvBuilder,
+                                        valueGetter, ptr, results.get(0).getTimestamp(),
+                                        env.getRegion().getRegionInfo().getStartKey(),
+                                        env.getRegion().getRegionInfo().getEndKey());
+                                    indexMutations.add(put);
                                 }
-                                result.setKeyValues(results);
-                            } else if (isDelete) {
-                                // FIXME: the version of the Delete constructor without the lock
-                                // args was introduced in 0.94.4, thus if we try to use it here
-                                // we can no longer use the 0.94.2 version of the client.
-                                Cell firstKV = results.get(0);
-                                Delete delete = new Delete(firstKV.getRowArray(),
-                                    firstKV.getRowOffset(), firstKV.getRowLength(),ts);
-                                mutations.add(delete);
-                                // force tephra to ignore this deletes
-                                delete.setAttribute(TxConstants.TX_ROLLBACK_ATTRIBUTE_KEY, new byte[0]);
-                            } else if (isUpsert) {
-                                Arrays.fill(values, null);
-                                int i = 0;
-                                List<PColumn> projectedColumns = projectedTable.getColumns();
-                                for (; i < projectedTable.getPKColumns().size(); i++) {
-                                    Expression expression = selectExpressions.get(i);
-                                    if (expression.evaluate(result, ptr)) {
-                                        values[i] = ptr.copyBytes();
-                                        // If SortOrder from expression in SELECT doesn't match the
-                                        // column being projected into then invert the bits.
-                                        if (expression.getSortOrder() !=
-                                                projectedColumns.get(i).getSortOrder()) {
-                                            SortOrder.invert(values[i], 0, values[i], 0,
-                                                values[i].length);
-                                        }
+                            }
+                            result.setKeyValues(results);
+                        } else if (isDelete) {
+                            // FIXME: the version of the Delete constructor without the lock
+                            // args was introduced in 0.94.4, thus if we try to use it here
+                            // we can no longer use the 0.94.2 version of the client.
+                            Cell firstKV = results.get(0);
+                            Delete delete = new Delete(firstKV.getRowArray(),
+                                firstKV.getRowOffset(), firstKV.getRowLength(),ts);
+                            mutations.add(delete);
+                            // force tephra to ignore this deletes
+                            delete.setAttribute(TxConstants.TX_ROLLBACK_ATTRIBUTE_KEY, new byte[0]);
+                        } else if (isUpsert) {
+                            Arrays.fill(values, null);
+                            int i = 0;
+                            List<PColumn> projectedColumns = projectedTable.getColumns();
+                            for (; i < projectedTable.getPKColumns().size(); i++) {
+                                Expression expression = selectExpressions.get(i);
+                                if (expression.evaluate(result, ptr)) {
+                                    values[i] = ptr.copyBytes();
+                                    // If SortOrder from expression in SELECT doesn't match the
+                                    // column being projected into then invert the bits.
+                                    if (expression.getSortOrder() !=
+                                            projectedColumns.get(i).getSortOrder()) {
+                                        SortOrder.invert(values[i], 0, values[i], 0,
+                                            values[i].length);
                                     }
                                 }
-                                projectedTable.newKey(ptr, values);
-                                PRow row = projectedTable.newRow(kvBuilder, ts, ptr, false);
-                                for (; i < projectedColumns.size(); i++) {
-                                    Expression expression = selectExpressions.get(i);
-                                    if (expression.evaluate(result, ptr)) {
-                                        PColumn column = projectedColumns.get(i);
-                                        Object value = expression.getDataType()
-                                                .toObject(ptr, column.getSortOrder());
-                                        // We are guaranteed that the two column will have the
-                                        // same type.
-                                        if (!column.getDataType().isSizeCompatible(ptr, value,
-                                            column.getDataType(), expression.getMaxLength(),
-                                            expression.getScale(), column.getMaxLength(),
-                                            column.getScale())) {
-                                            throw new DataExceedsCapacityException(
-                                                column.getDataType(), column.getMaxLength(),
-                                                column.getScale());
-                                        }
-                                        column.getDataType().coerceBytes(ptr, value,
-                                            expression.getDataType(), expression.getMaxLength(),
-                                            expression.getScale(), expression.getSortOrder(), 
-                                            column.getMaxLength(), column.getScale(),
-                                            column.getSortOrder(), projectedTable.rowKeyOrderOptimizable());
-                                        byte[] bytes = ByteUtil.copyKeyBytesIfNecessary(ptr);
-                                        row.setValue(column, bytes);
+                            }
+                            projectedTable.newKey(ptr, values);
+                            PRow row = projectedTable.newRow(kvBuilder, ts, ptr, false);
+                            for (; i < projectedColumns.size(); i++) {
+                                Expression expression = selectExpressions.get(i);
+                                if (expression.evaluate(result, ptr)) {
+                                    PColumn column = projectedColumns.get(i);
+                                    if (!column.getDataType().isSizeCompatible(ptr, null,
+                                        expression.getDataType(), expression.getSortOrder(),
+                                        expression.getMaxLength(), expression.getScale(),
+                                        column.getMaxLength(), column.getScale())) {
+                                        throw new DataExceedsCapacityException(
+                                            column.getDataType(), column.getMaxLength(),
+                                            column.getScale(), column.getName().getString(), ptr);
                                     }
-                                }
-                                for (Mutation mutation : row.toRowMutations()) {
-                                    mutations.add(mutation);
-                                }
-                                for (i = 0; i < selectExpressions.size(); i++) {
-                                    selectExpressions.get(i).reset();
-                                }
-                            } else if (deleteCF != null && deleteCQ != null) {
-                                // No need to search for delete column, since we project only it
-                                // if no empty key value is being set
-                                if (emptyCF == null ||
-                                        result.getValue(deleteCF, deleteCQ) != null) {
-                                    Delete delete = new Delete(results.get(0).getRowArray(),
-                                        results.get(0).getRowOffset(),
-                                        results.get(0).getRowLength());
-                                    delete.deleteColumns(deleteCF,  deleteCQ, ts);
-                                    // force tephra to ignore this deletes
-                                    delete.setAttribute(TxConstants.TX_ROLLBACK_ATTRIBUTE_KEY, new byte[0]);
-                                    mutations.add(delete);
+                                    column.getDataType().coerceBytes(ptr, null,
+                                        expression.getDataType(), expression.getMaxLength(),
+                                        expression.getScale(), expression.getSortOrder(), 
+                                        column.getMaxLength(), column.getScale(),
+                                        column.getSortOrder(), projectedTable.rowKeyOrderOptimizable());
+                                    byte[] bytes = ByteUtil.copyKeyBytesIfNecessary(ptr);
+                                    row.setValue(column, bytes);
                                 }
                             }
-                            if (emptyCF != null) {
-                                /*
-                                 * If we've specified an emptyCF, then we need to insert an empty
-                                 * key value "retroactively" for any key value that is visible at
-                                 * the timestamp that the DDL was issued. Key values that are not
-                                 * visible at this timestamp will not ever be projected up to
-                                 * scans past this timestamp, so don't need to be considered.
-                                 * We insert one empty key value per row per timestamp.
-                                 */
-                                Set<Long> timeStamps =
-                                        Sets.newHashSetWithExpectedSize(results.size());
-                                for (Cell kv : results) {
-                                    long kvts = kv.getTimestamp();
-                                    if (!timeStamps.contains(kvts)) {
-                                        Put put = new Put(kv.getRowArray(), kv.getRowOffset(),
-                                            kv.getRowLength());
-                                        put.add(emptyCF, QueryConstants.EMPTY_COLUMN_BYTES, kvts,
-                                            ByteUtil.EMPTY_BYTE_ARRAY);
-                                        mutations.add(put);
-                                    }
+                            for (Mutation mutation : row.toRowMutations()) {
+                                mutations.add(mutation);
+                            }
+                            for (i = 0; i < selectExpressions.size(); i++) {
+                                selectExpressions.get(i).reset();
+                            }
+                        } else if (deleteCF != null && deleteCQ != null) {
+                            // No need to search for delete column, since we project only it
+                            // if no empty key value is being set
+                            if (emptyCF == null ||
+                                    result.getValue(deleteCF, deleteCQ) != null) {
+                                Delete delete = new Delete(results.get(0).getRowArray(),
+                                    results.get(0).getRowOffset(),
+                                    results.get(0).getRowLength());
+                                delete.deleteColumns(deleteCF,  deleteCQ, ts);
+                                // force tephra to ignore this deletes
+                                delete.setAttribute(TxConstants.TX_ROLLBACK_ATTRIBUTE_KEY, new byte[0]);
+                                mutations.add(delete);
+                            }
+                        }
+                        if (emptyCF != null) {
+                            /*
+                             * If we've specified an emptyCF, then we need to insert an empty
+                             * key value "retroactively" for any key value that is visible at
+                             * the timestamp that the DDL was issued. Key values that are not
+                             * visible at this timestamp will not ever be projected up to
+                             * scans past this timestamp, so don't need to be considered.
+                             * We insert one empty key value per row per timestamp.
+                             */
+                            Set<Long> timeStamps =
+                                    Sets.newHashSetWithExpectedSize(results.size());
+                            for (Cell kv : results) {
+                                long kvts = kv.getTimestamp();
+                                if (!timeStamps.contains(kvts)) {
+                                    Put put = new Put(kv.getRowArray(), kv.getRowOffset(),
+                                        kv.getRowLength());
+                                    put.add(emptyCF, QueryConstants.EMPTY_COLUMN_BYTES, kvts,
+                                        ByteUtil.EMPTY_BYTE_ARRAY);
+                                    mutations.add(put);
                                 }
                             }
                             // Commit in batches based on UPSERT_BATCH_SIZE_ATTRIB in config
@@ -617,13 +609,6 @@ public class UngroupedAggregateRegionObserver extends BaseScannerRegionObserver
                                 commitBatch(region, indexMutations, null, blockingMemStoreSize, null, txState);
                                 indexMutations.clear();
                             }
-                        } catch (ConstraintViolationException e) {
-                            // Log and ignore in count
-                            logger.error(LogUtil.addCustomAnnotations("Failed to create row in " +
-                                    region.getRegionInfo().getRegionNameAsString() + " with values " +
-                                    SchemaUtil.toString(values),
-                                    ScanUtil.getCustomAnnotations(scan)), e);
-                            continue;
                         }
                         aggregators.aggregate(rowAggregators, result);
                         hasAny = true;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/faa2cd04/phoenix-core/src/main/java/org/apache/phoenix/exception/DataExceedsCapacityException.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/exception/DataExceedsCapacityException.java b/phoenix-core/src/main/java/org/apache/phoenix/exception/DataExceedsCapacityException.java
index 0ee81a0..a12c8a0 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/exception/DataExceedsCapacityException.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/exception/DataExceedsCapacityException.java
@@ -17,8 +17,10 @@
  */
 package org.apache.phoenix.exception;
 
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.phoenix.schema.IllegalDataException;
 import org.apache.phoenix.schema.types.PDataType;
+import org.apache.phoenix.util.SchemaUtil;
 
 
 public class DataExceedsCapacityException extends IllegalDataException {
@@ -29,12 +31,16 @@ public class DataExceedsCapacityException extends IllegalDataException {
                 SQLExceptionCode.DATA_EXCEEDS_MAX_CAPACITY).setMessage(message).build().buildException());
     }
     
+    public DataExceedsCapacityException(PDataType type, Integer precision, Integer scale, String columnName, ImmutableBytesWritable value) {
+        super(new SQLExceptionInfo.Builder(SQLExceptionCode.DATA_EXCEEDS_MAX_CAPACITY)
+            .setMessage((columnName == null ? "" : columnName + " ") + getTypeDisplayString(type, precision, scale, value))
+            .build().buildException());
+    }
     public DataExceedsCapacityException(PDataType type, Integer precision, Integer scale) {
-        super(new SQLExceptionInfo.Builder(SQLExceptionCode.DATA_EXCEEDS_MAX_CAPACITY).setMessage(getTypeDisplayString(type, precision, scale))
-                .build().buildException());
+        this(type, precision, scale, null, null);
     }
 
-    private static String getTypeDisplayString(PDataType type, Integer precision, Integer scale) {
-        return type.toString() + "(" + precision + (scale == null ? "" : ("," + scale + ")"));
+    private static String getTypeDisplayString(PDataType type, Integer precision, Integer scale, ImmutableBytesWritable value) {
+        return type.toString() + "(" + precision + (scale == null ? "" : ("," + scale)) + ")" + (value == null || value.getLength() == 0 ? "" : (" value="+SchemaUtil.toString(type, value)));
     }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/faa2cd04/phoenix-core/src/main/java/org/apache/phoenix/exception/SQLExceptionInfo.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/exception/SQLExceptionInfo.java b/phoenix-core/src/main/java/org/apache/phoenix/exception/SQLExceptionInfo.java
index 50dffde..1c3694d 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/exception/SQLExceptionInfo.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/exception/SQLExceptionInfo.java
@@ -120,9 +120,14 @@ public class SQLExceptionInfo {
 
     @Override
     public String toString() {
-        StringBuilder builder = new StringBuilder(code.toString());
+        String baseMessage = code.toString();
+        StringBuilder builder = new StringBuilder(baseMessage);
         if (message != null) {
-            builder.append(" ").append(message);
+            if (message.startsWith(baseMessage)) {
+                builder.append(message.substring(baseMessage.length()));
+            } else {
+                builder.append(" ").append(message);
+            }
         }
         if (functionName != null) {
             builder.append(" ").append(FUNCTION_NAME).append("=").append(functionName);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/faa2cd04/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ArrayConcatFunction.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ArrayConcatFunction.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ArrayConcatFunction.java
index 77790b9..85655c6 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ArrayConcatFunction.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ArrayConcatFunction.java
@@ -22,6 +22,7 @@ import java.util.List;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.phoenix.expression.Expression;
 import org.apache.phoenix.parse.FunctionParseNode;
+import org.apache.phoenix.schema.SortOrder;
 import org.apache.phoenix.schema.TypeMismatchException;
 import org.apache.phoenix.schema.tuple.Tuple;
 import org.apache.phoenix.schema.types.PArrayDataType;
@@ -52,16 +53,18 @@ public class ArrayConcatFunction extends ArrayModifierFunction {
         }
         boolean isLHSRowKeyOrderOptimized = PArrayDataType.isRowKeyOrderOptimized(getLHSExpr().getDataType(), getLHSExpr().getSortOrder(), ptr);
 
+        SortOrder sortOrder = getRHSExpr().getSortOrder();
         int actualLengthOfArray1 = Math.abs(PArrayDataType.getArrayLength(ptr, getLHSBaseType(), getLHSExpr().getMaxLength()));
         int lengthArray1 = ptr.getLength();
         int offsetArray1 = ptr.getOffset();
         byte[] array1Bytes = ptr.get();
         if (!getRHSExpr().evaluate(tuple, ptr)|| ptr.getLength() == 0){
+            sortOrder = getLHSExpr().getSortOrder();
             ptr.set(array1Bytes, offsetArray1, lengthArray1);
             return true;
         }
 
-        checkSizeCompatibility(ptr, getLHSExpr(), getLHSExpr().getDataType(), getRHSExpr(),getRHSExpr().getDataType());
+        checkSizeCompatibility(ptr, sortOrder, getLHSExpr(), getLHSExpr().getDataType(), getRHSExpr(),getRHSExpr().getDataType());
 
         // FIXME: calling version of coerceBytes that takes into account the separator used by LHS
         // If the RHS does not have the same separator, it'll be coerced to use it. It's unclear

http://git-wip-us.apache.org/repos/asf/phoenix/blob/faa2cd04/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ArrayModifierFunction.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ArrayModifierFunction.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ArrayModifierFunction.java
index 9bd7372..bcf2a5a 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ArrayModifierFunction.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ArrayModifierFunction.java
@@ -104,7 +104,7 @@ public abstract class ArrayModifierFunction extends ScalarFunction {
 
         otherExpr.evaluate(tuple, ptr);
 
-        checkSizeCompatibility(ptr, arrayExpr, baseDataType, otherExpr, otherExpressionType);
+        checkSizeCompatibility(ptr, otherExpr.getSortOrder(), arrayExpr, baseDataType, otherExpr, otherExpressionType);
         coerceBytes(ptr, arrayExpr, baseDataType, otherExpr, otherExpressionType);
         return modifierFunction(ptr, length, offset, arrayBytes, baseDataType, arrayLength, getMaxLength(),
                 arrayExpr);
@@ -117,11 +117,11 @@ public abstract class ArrayModifierFunction extends ScalarFunction {
         return false;
     }
 
-    protected void checkSizeCompatibility(ImmutableBytesWritable ptr, Expression arrayExpr,
+    protected void checkSizeCompatibility(ImmutableBytesWritable ptr, SortOrder sortOrder, Expression arrayExpr,
                                           PDataType baseDataType, Expression otherExpr, PDataType otherExpressionType) {
         if (!baseDataType.isSizeCompatible(ptr, null, otherExpressionType,
-                otherExpr.getMaxLength(), otherExpr.getScale(), arrayExpr.getMaxLength(),
-                arrayExpr.getScale())) {
+                sortOrder, otherExpr.getMaxLength(), otherExpr.getScale(),
+                arrayExpr.getMaxLength(), arrayExpr.getScale())) {
             throw new DataExceedsCapacityException("Values are not size compatible");
         }
     }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/faa2cd04/phoenix-core/src/main/java/org/apache/phoenix/index/PhoenixIndexBuilder.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/index/PhoenixIndexBuilder.java b/phoenix-core/src/main/java/org/apache/phoenix/index/PhoenixIndexBuilder.java
index d6adc71..ac1e2e4 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/index/PhoenixIndexBuilder.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/index/PhoenixIndexBuilder.java
@@ -223,8 +223,8 @@ public class PhoenixIndexBuilder extends NonTxIndexBuilder {
                     // We are guaranteed that the two column will have the
                     // same type.
                     if (!column.getDataType().isSizeCompatible(ptr, value, column.getDataType(),
-                            expression.getMaxLength(), expression.getScale(), column.getMaxLength(),
-                            column.getScale())) {
+                            expression.getSortOrder(), expression.getMaxLength(), expression.getScale(),
+                            column.getMaxLength(), column.getScale())) {
                         throw new DataExceedsCapacityException(column.getDataType(), column.getMaxLength(),
                             column.getScale());
                     }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/faa2cd04/phoenix-core/src/main/java/org/apache/phoenix/parse/ColumnDef.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/ColumnDef.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/ColumnDef.java
index 4b148dd..0be7c16 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/parse/ColumnDef.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/ColumnDef.java
@@ -281,8 +281,8 @@ public class ColumnDef {
             throw e;
         }
         if (!targetType.isSizeCompatible(ptr, defaultValue.getValue(), sourceType, 
-                defaultValue.getMaxLength(), defaultValue.getScale(), 
-                this.getMaxLength(), this.getScale())) {
+                sortOrder, defaultValue.getMaxLength(), 
+                defaultValue.getScale(), this.getMaxLength(), this.getScale())) {
             throw new SQLExceptionInfo.Builder(
                     SQLExceptionCode.DATA_EXCEEDS_MAX_CAPACITY).setColumnName(this.getColumnDefName().getColumnName())
                     .setMessage("DEFAULT " + this.getExpression()).build()

http://git-wip-us.apache.org/repos/asf/phoenix/blob/faa2cd04/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableImpl.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableImpl.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableImpl.java
index 627740b..98a0b99 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableImpl.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableImpl.java
@@ -66,7 +66,6 @@ import org.apache.phoenix.util.ByteUtil;
 import org.apache.phoenix.util.PhoenixRuntime;
 import org.apache.phoenix.util.SchemaUtil;
 import org.apache.phoenix.util.SizedUtil;
-import org.apache.phoenix.util.StringUtil;
 import org.apache.phoenix.util.TrustedByteArrayOutputStream;
 import org.apache.tephra.TxConstants;
 
@@ -671,19 +670,14 @@ public class PTableImpl implements PTable {
                     throw new ConstraintViolationException(name.getString() + "." + column.getName().getString() + " may not be null");
                 }
                 Integer	maxLength = column.getMaxLength();
-                if (maxLength != null && type.isFixedWidth() && byteValue.length < maxLength) {
-                    if (rowKeyOrderOptimizable()) {
-                        key.set(byteValue);
-                        type.pad(key, maxLength, sortOrder);
-                        byteValue = ByteUtil.copyKeyBytesIfNecessary(key);
-                    } else {
-                        // TODO: remove this incorrect code and move StringUtil.padChar() to TestUtil
-                        // once we require tables to have been upgraded
-                        byteValue = StringUtil.padChar(byteValue, maxLength);
-                    }
-                } else if (maxLength != null && !type.isArrayType() && byteValue.length > maxLength) {
-                    throw new DataExceedsCapacityException(name.getString() + "." + column.getName().getString() + " may not exceed " + maxLength + " bytes (" + SchemaUtil.toString(type, byteValue) + ")");
+                Integer scale = column.getScale();
+                key.set(byteValue);
+                if (!type.isSizeCompatible(key, null, type, sortOrder, null, null, maxLength, scale)) {
+                    throw new DataExceedsCapacityException(name.getString() + "." + column.getName().getString() + " may not exceed " + maxLength + " (" + SchemaUtil.toString(type, byteValue) + ")");
                 }
+                key.set(byteValue);
+                type.pad(key, maxLength, sortOrder);
+                byteValue = ByteUtil.copyKeyBytesIfNecessary(key);
                 os.write(byteValue, 0, byteValue.length);
             }
             // Need trailing byte for DESC columns
@@ -853,11 +847,14 @@ public class PTableImpl implements PTable {
             byte[] qualifier = column.getName().getBytes();
             PDataType<?> type = column.getDataType();
             // Check null, since some types have no byte representation for null
+            if (byteValue == null) {
+                byteValue = ByteUtil.EMPTY_BYTE_ARRAY;
+            }
             boolean isNull = type.isNull(byteValue);
             if (isNull && !column.isNullable()) {
-                throw new ConstraintViolationException(name.getString() + "." + column.getName().getString() + " may not be null");
-            } else if (isNull && PTableImpl.this.isImmutableRows()
-                    && column.getExpressionStr() == null) {
+                throw new ConstraintViolationException(name.getString() + "." + column.getName().getString() + 
+                        " may not be null");
+            } else if (isNull && PTableImpl.this.isImmutableRows() && column.getExpressionStr() == null) {
                 // Store nulls for immutable tables otherwise default value would be used
                 removeIfPresent(setValues, family, qualifier);
                 removeIfPresent(unsetValues, family, qualifier);
@@ -869,16 +866,16 @@ public class PTableImpl implements PTable {
                 deleteQuietly(unsetValues, kvBuilder, kvBuilder.buildDeleteColumns(keyPtr, column
                             .getFamilyName().getBytesPtr(), column.getName().getBytesPtr(), ts));
             } else {
-                ImmutableBytesWritable ptr = new ImmutableBytesWritable(byteValue == null ?
-                        HConstants.EMPTY_BYTE_ARRAY : byteValue);
+                ImmutableBytesWritable ptr = new ImmutableBytesWritable(byteValue);
                 Integer	maxLength = column.getMaxLength();
-            	if (!isNull && type.isFixedWidth() && maxLength != null) {
-    				if (ptr.getLength() < maxLength) {
-                        type.pad(ptr, maxLength, column.getSortOrder());
-                    } else if (ptr.getLength() > maxLength) {
-                        throw new DataExceedsCapacityException(name.getString() + "." + column.getName().getString() + " may not exceed " + maxLength + " bytes (" + type.toObject(byteValue) + ")");
-                    }
-            	}
+                Integer scale = column.getScale();
+                SortOrder sortOrder = column.getSortOrder();
+                if (!type.isSizeCompatible(ptr, null, type, sortOrder, null, null, maxLength, scale)) {
+                    throw new DataExceedsCapacityException(name.getString() + "." + column.getName().getString() + 
+                            " may not exceed " + maxLength + " (" + SchemaUtil.toString(type, byteValue) + ")");
+                }
+                ptr.set(byteValue);
+                type.pad(ptr, maxLength, sortOrder);
                 removeIfPresent(unsetValues, family, qualifier);
                 addQuietly(setValues, kvBuilder, kvBuilder.buildPut(keyPtr,
                         column.getFamilyName().getBytesPtr(), column.getName().getBytesPtr(),

http://git-wip-us.apache.org/repos/asf/phoenix/blob/faa2cd04/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PArrayDataType.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PArrayDataType.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PArrayDataType.java
index eb1a7ff..1d2cfb2 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PArrayDataType.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PArrayDataType.java
@@ -256,15 +256,18 @@ public abstract class PArrayDataType<T> extends PDataType<T> {
     }
 
     @Override
-    public boolean isSizeCompatible(ImmutableBytesWritable ptr, Object value, PDataType srcType, Integer maxLength,
-            Integer scale, Integer desiredMaxLength, Integer desiredScale) {
+    public boolean isSizeCompatible(ImmutableBytesWritable ptr, Object value, PDataType srcType, SortOrder sortOrder,
+            Integer maxLength, Integer scale, Integer desiredMaxLength, Integer desiredScale) {
         if (value == null) return true;
         PhoenixArray pArr = (PhoenixArray)value;
         PDataType baseType = PDataType.fromTypeId(srcType.getSqlType() - PDataType.ARRAY_TYPE_BASE);
+        // Since we only have a value and no byte[], use an empty length byte[] as otherwise
+        // isSizeCompatible will attempt to interpret the array ptr as a ptr to an element.
+        ImmutableBytesWritable elementPtr = new ImmutableBytesWritable(ByteUtil.EMPTY_BYTE_ARRAY);
         for (int i = 0; i < pArr.numElements; i++) {
             Object val = pArr.getElement(i);
-            if (!baseType.isSizeCompatible(ptr, val, baseType, srcType.getMaxLength(val), scale, desiredMaxLength,
-                    desiredScale)) { return false; }
+            if (!baseType.isSizeCompatible(elementPtr, val, baseType, sortOrder, srcType.getMaxLength(val), scale,
+                    desiredMaxLength, desiredScale)) { return false; }
         }
         return true;
     }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/faa2cd04/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PBinary.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PBinary.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PBinary.java
index 7b4aa38..43906f0 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PBinary.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PBinary.java
@@ -30,26 +30,26 @@ import org.apache.phoenix.schema.SortOrder;
 
 public class PBinary extends PBinaryBase {
 
-  public static final PBinary INSTANCE = new PBinary();
+    public static final PBinary INSTANCE = new PBinary();
 
-  private PBinary() {
-    super("BINARY", Types.BINARY, byte[].class, null, 23);
-  }
+    private PBinary() {
+        super("BINARY", Types.BINARY, byte[].class, null, 23);
+    }
 
-  @Override
-  public void coerceBytes(ImmutableBytesWritable ptr, Object o, PDataType actualType, Integer actualMaxLength,
-          Integer actualScale, SortOrder actualModifier, Integer desiredMaxLength, Integer desiredScale,
-          SortOrder expectedModifier) {
-    PVarbinary.INSTANCE.coerceBytes(ptr, o, actualType, actualMaxLength, actualScale, actualModifier, desiredMaxLength, desiredScale, expectedModifier);
-    if (null != desiredMaxLength && null != expectedModifier) {
-      pad(ptr, desiredMaxLength, expectedModifier);
+    @Override
+    public void coerceBytes(ImmutableBytesWritable ptr, Object o, PDataType actualType, Integer actualMaxLength,
+            Integer actualScale, SortOrder actualModifier, Integer desiredMaxLength, Integer desiredScale,
+            SortOrder expectedModifier) {
+        PVarbinary.INSTANCE.coerceBytes(ptr, o, actualType, actualMaxLength, actualScale, actualModifier, desiredMaxLength, desiredScale, expectedModifier);
+        if (null != desiredMaxLength && null != expectedModifier) {
+            pad(ptr, desiredMaxLength, expectedModifier);
+        }
     }
-  }
 
-  @Override
-  public byte[] pad(byte[] b, Integer maxLength, SortOrder sortOrder) {
-      if (b == null || b.length >= maxLength) {
-          return b;
+    @Override
+    public byte[] pad(byte[] b, Integer maxLength, SortOrder sortOrder) {
+        if (b == null || b.length >= maxLength) {
+            return b;
         }
         byte[] newBytes = new byte[maxLength];
         System.arraycopy(b, 0, newBytes, 0, b.length);
@@ -57,164 +57,152 @@ public class PBinary extends PBinaryBase {
             Arrays.fill(newBytes, b.length, maxLength, QueryConstants.DESC_SEPARATOR_BYTE);
         }
         return newBytes;
-  }
-  
-  @Override
-  public void pad(ImmutableBytesWritable ptr, Integer maxLength, SortOrder sortOrder) {
-    if (ptr.getLength() >= maxLength) {
-      return;
-    }
-    byte[] newBytes = new byte[maxLength];
-    System.arraycopy(ptr.get(), ptr.getOffset(), newBytes, 0, ptr.getLength());
-    if (sortOrder == SortOrder.DESC) {
-        Arrays.fill(newBytes, ptr.getLength(), maxLength, QueryConstants.DESC_SEPARATOR_BYTE);
-    }
-    ptr.set(newBytes);
-  }
-
-  @Override
-  public Object pad(Object object, Integer maxLength) {
-    byte[] b = (byte[]) object;
-    int length = (b == null ? 0 : b.length);
-    if (length == maxLength) {
-      return object;
-    }
-    if (length > maxLength) {
-      throw new DataExceedsCapacityException(this, maxLength, null);
-    }
-    byte[] newBytes = new byte[maxLength];
-    System.arraycopy(b, 0, newBytes, 0, length);
-
-    return newBytes;
-  }
-
-  @Override
-  public byte[] toBytes(Object object) { // Delegate to VARBINARY
-    if (object == null) {
-      throw newIllegalDataException(this + " may not be null");
-    }
-    return PVarbinary.INSTANCE.toBytes(object);
-  }
-
-  @Override
-  public int toBytes(Object object, byte[] bytes, int offset) {
-    if (object == null) {
-      throw newIllegalDataException(this + " may not be null");
-    }
-    return PVarbinary.INSTANCE.toBytes(object, bytes, offset);
-
-  }
-
-  @Override
-  public byte[] toBytes(Object object, SortOrder sortOrder) {
-    byte[] bytes = toBytes(object);
-    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 (!actualType.isCoercibleTo(this)) {
-      throwConstraintViolationException(actualType, this);
-    }
-    return PVarbinary.INSTANCE.toObject(bytes, offset, length, actualType, sortOrder);
-  }
-
-  @Override
-  public Object toObject(Object object, PDataType actualType) {
-    return actualType.toBytes(object);
-  }
-
-  @Override
-  public boolean isFixedWidth() {
-    return true;
-  }
-
-  @Override
-  public int estimateByteSize(Object o) {
-    byte[] value = (byte[]) o;
-    return value == null ? 1 : value.length;
-  }
-
-  @Override
-  public boolean isCoercibleTo(PDataType targetType) {
-    return equalsAny(targetType, this, PVarbinary.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(PVarbinary.INSTANCE) && ((String) value).length() != ptr.getLength()) ||
-            (maxLength != null && desiredMaxLength != null && maxLength > desiredMaxLength))) {
-      return false;
-    }
-    return true;
-  }
-
-  @Override
-  public Integer estimateByteSizeFromLength(Integer length) {
-    return length;
-  }
-
-  @Override
-  public Integer getByteSize() {
-    return null;
-  }
-
-  @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, PVarbinary.INSTANCE, PBinary.INSTANCE)) {
-      return Bytes.compareTo((byte[]) lhs, (byte[]) rhs);
-    } else {
-      byte[] rhsBytes = rhsType.toBytes(rhs);
-      return Bytes.compareTo((byte[]) lhs, rhsBytes);
-    }
-  }
-
-  @Override
-  public Integer getMaxLength(Object o) {
-    if (o == null) {
-      return null;
-    }
-    byte[] value = (byte[]) o;
-    return value.length;
-  }
-
-  @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 offset, int length, Format formatter) {
-    if (length == 1) {
-      return Integer.toString(0xFF & b[offset]);
-    }
-    return PVarbinary.INSTANCE.toStringLiteral(b, offset, length, formatter);
-  }
-
-  @Override
-  public String toStringLiteral(Object o, Format formatter) {
-    return toStringLiteral((byte[])o, 0, ((byte[]) o).length, formatter);
-  }
-
-  @Override
-  public Object getSampleValue(Integer maxLength, Integer arrayLength) {
-    return PVarbinary.INSTANCE.getSampleValue(maxLength, arrayLength);
-  }
+    }
+
+    @Override
+    public void pad(ImmutableBytesWritable ptr, Integer maxLength, SortOrder sortOrder) {
+        if (ptr.getLength() >= maxLength) {
+            return;
+        }
+        byte[] newBytes = new byte[maxLength];
+        System.arraycopy(ptr.get(), ptr.getOffset(), newBytes, 0, ptr.getLength());
+        if (sortOrder == SortOrder.DESC) {
+            Arrays.fill(newBytes, ptr.getLength(), maxLength, QueryConstants.DESC_SEPARATOR_BYTE);
+        }
+        ptr.set(newBytes);
+    }
+
+    @Override
+    public Object pad(Object object, Integer maxLength) {
+        byte[] b = (byte[]) object;
+        int length = (b == null ? 0 : b.length);
+        if (length == maxLength) {
+            return object;
+        }
+        if (length > maxLength) {
+            throw new DataExceedsCapacityException(this, maxLength, null);
+        }
+        byte[] newBytes = new byte[maxLength];
+        System.arraycopy(b, 0, newBytes, 0, length);
+
+        return newBytes;
+    }
+
+    @Override
+    public byte[] toBytes(Object object) { // Delegate to VARBINARY
+        if (object == null) {
+            throw newIllegalDataException(this + " may not be null");
+        }
+        return PVarbinary.INSTANCE.toBytes(object);
+    }
+
+    @Override
+    public int toBytes(Object object, byte[] bytes, int offset) {
+        if (object == null) {
+            throw newIllegalDataException(this + " may not be null");
+        }
+        return PVarbinary.INSTANCE.toBytes(object, bytes, offset);
+
+    }
+
+    @Override
+    public byte[] toBytes(Object object, SortOrder sortOrder) {
+        byte[] bytes = toBytes(object);
+        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 (!actualType.isCoercibleTo(this)) {
+            throwConstraintViolationException(actualType, this);
+        }
+        return PVarbinary.INSTANCE.toObject(bytes, offset, length, actualType, sortOrder);
+    }
+
+    @Override
+    public Object toObject(Object object, PDataType actualType) {
+        return actualType.toBytes(object);
+    }
+
+    @Override
+    public boolean isFixedWidth() {
+        return true;
+    }
+
+    @Override
+    public int estimateByteSize(Object o) {
+        byte[] value = (byte[]) o;
+        return value == null ? 1 : value.length;
+    }
+
+    @Override
+    public boolean isCoercibleTo(PDataType targetType) {
+        return equalsAny(targetType, this, PVarbinary.INSTANCE);
+    }
+
+    @Override
+    public Integer estimateByteSizeFromLength(Integer length) {
+        return length;
+    }
+
+    @Override
+    public Integer getByteSize() {
+        return null;
+    }
+
+    @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, PVarbinary.INSTANCE, PBinary.INSTANCE)) {
+            return Bytes.compareTo((byte[]) lhs, (byte[]) rhs);
+        } else {
+            byte[] rhsBytes = rhsType.toBytes(rhs);
+            return Bytes.compareTo((byte[]) lhs, rhsBytes);
+        }
+    }
+
+    @Override
+    public Integer getMaxLength(Object o) {
+        if (o == null) {
+            return null;
+        }
+        byte[] value = (byte[]) o;
+        return value.length;
+    }
+
+    @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 offset, int length, Format formatter) {
+        if (length == 1) {
+            return Integer.toString(0xFF & b[offset]);
+        }
+        return PVarbinary.INSTANCE.toStringLiteral(b, offset, length, formatter);
+    }
+
+    @Override
+    public String toStringLiteral(Object o, Format formatter) {
+        return toStringLiteral((byte[])o, 0, ((byte[]) o).length, formatter);
+    }
+
+    @Override
+    public Object getSampleValue(Integer maxLength, Integer arrayLength) {
+        return PVarbinary.INSTANCE.getSampleValue(maxLength, arrayLength);
+    }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/faa2cd04/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PBinaryBase.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PBinaryBase.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PBinaryBase.java
index 0ad4ce1..562875d 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PBinaryBase.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PBinaryBase.java
@@ -95,4 +95,21 @@ public abstract class PBinaryBase extends PDataType<byte[]> {
         PInteger.INSTANCE.getCodec().encodeInt(length, bytes, 0);
         outPtr.set(bytes);
     }
+
+    @Override
+    public boolean isSizeCompatible(ImmutableBytesWritable ptr, Object value, PDataType srcType,
+        SortOrder sortOrder, Integer maxLength, Integer scale, Integer desiredMaxLength, Integer desiredScale) {
+        if (ptr.getLength() != 0 && desiredMaxLength != null) {
+            if (maxLength == null) { // If not specified, compute
+                if (value != null && srcType instanceof PBinaryBase) { // Use value if provided
+                    maxLength = ((byte[])value).length;
+                } else { // Else use ptr, coercing (which is likely a noop)
+                    this.coerceBytes(ptr, value, srcType, maxLength, scale, sortOrder, desiredMaxLength, desiredScale, sortOrder, true);
+                    maxLength = ptr.getLength();
+                }
+            }
+            return maxLength <= desiredMaxLength;
+        }
+        return true;
+    }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/faa2cd04/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PChar.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PChar.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PChar.java
index 2853bc4..fa97992 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PChar.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PChar.java
@@ -153,8 +153,19 @@ public class PChar extends PDataType<String> {
 
     @Override
     public boolean isSizeCompatible(ImmutableBytesWritable ptr, Object value, PDataType srcType,
-        Integer maxLength, Integer scale, Integer desiredMaxLength, Integer desiredScale) {
-      return PVarchar.INSTANCE.isSizeCompatible(ptr, value, srcType, maxLength, scale, desiredMaxLength, desiredScale);
+        SortOrder sortOrder, Integer maxLength, Integer scale, Integer desiredMaxLength, Integer desiredScale) {
+        if (ptr.getLength() != 0 && desiredMaxLength != null) {
+            if (maxLength == null) {
+                if (value != null && srcType == INSTANCE) { // Use value if provided
+                    maxLength = ((String)value).length();
+                } else {
+                    this.coerceBytes(ptr, value, srcType, maxLength, scale, sortOrder, desiredMaxLength, desiredScale, sortOrder, true);
+                    maxLength = ptr.getLength(); // Only single byte characters
+                }
+            }
+            return maxLength <= desiredMaxLength;
+        }
+        return true;
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/phoenix/blob/faa2cd04/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PDataType.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PDataType.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PDataType.java
index 5d611e9..58018ac 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PDataType.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PDataType.java
@@ -747,14 +747,15 @@ public abstract class PDataType<T> implements DataType<T>, Comparable<PDataType<
      * @param ptr bytes pointer for the value
      * @param value object representation of the value. May be null in which case ptr will be used
      * @param srcType the type of the value
+     * @param sortOrder the sort order of the value
      * @param maxLength the max length of the source value or null if not applicable
      * @param scale the scale of the source value or null if not applicable
      * @param desiredMaxLength the desired max length for the value to be coerced
      * @param desiredScale the desired scale for the value to be coerced 
      * @return true if the value may be coerced without losing precision and false otherwise.
      */
-    public boolean isSizeCompatible(ImmutableBytesWritable ptr, Object value, PDataType srcType, Integer maxLength,
-            Integer scale, Integer desiredMaxLength, Integer desiredScale) {
+    public boolean isSizeCompatible(ImmutableBytesWritable ptr, Object value, PDataType srcType, SortOrder sortOrder,
+            Integer maxLength, Integer scale, Integer desiredMaxLength, Integer desiredScale) {
         return true;
     }
 


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

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

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

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

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

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