You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@phoenix.apache.org by el...@apache.org on 2015/08/13 20:01:21 UTC

phoenix git commit: PHOENIX-1673 Allow TenantId to be of any integral data type

Repository: phoenix
Updated Branches:
  refs/heads/master b28686fdb -> 5c048430e


PHOENIX-1673 Allow TenantId to be of any integral data type


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

Branch: refs/heads/master
Commit: 5c048430ed39ce23f95609607c1bd845401900a9
Parents: b28686f
Author: Jeffrey Lyons <Je...@d2l.com>
Authored: Thu Jul 9 10:07:40 2015 -0400
Committer: Eli Levine <el...@apache.org>
Committed: Thu Aug 13 11:01:00 2015 -0700

----------------------------------------------------------------------
 .../end2end/BaseTenantSpecificViewIndexIT.java  |  33 ++-
 .../apache/phoenix/end2end/CreateTableIT.java   |  25 ++
 .../apache/phoenix/end2end/TenantIdTypeIT.java  | 226 ++++++++++++++
 .../end2end/TenantSpecificTablesDDLIT.java      |  12 -
 .../end2end/TenantSpecificViewIndexIT.java      |   5 +
 .../apache/phoenix/compile/DeleteCompiler.java  |   3 +-
 .../apache/phoenix/compile/UpsertCompiler.java  |   3 +-
 .../apache/phoenix/compile/WhereOptimizer.java  |  11 +-
 .../phoenix/exception/SQLExceptionCode.java     |   3 +-
 .../apache/phoenix/execute/BaseQueryPlan.java   |  14 +-
 .../apache/phoenix/execute/MutationState.java   |  23 +-
 .../apache/phoenix/schema/MetaDataClient.java   |   4 +-
 .../java/org/apache/phoenix/util/ScanUtil.java  |  35 ++-
 .../util/TenantIdByteConversionTest.java        | 294 +++++++++++++++++++
 14 files changed, 631 insertions(+), 60 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/5c048430/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseTenantSpecificViewIndexIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseTenantSpecificViewIndexIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseTenantSpecificViewIndexIT.java
index 93fc222..b450643 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseTenantSpecificViewIndexIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseTenantSpecificViewIndexIT.java
@@ -40,6 +40,7 @@ public class BaseTenantSpecificViewIndexIT extends BaseHBaseManagedTimeIT {
     
     public static final String TENANT1_ID = "tenant1";
     public static final String TENANT2_ID = "tenant2";
+    public static final String NON_STRING_TENANT_ID = "1234";
     
     protected Set<Pair<String, String>> tenantViewsToDelete = newHashSet();
     
@@ -48,7 +49,7 @@ public class BaseTenantSpecificViewIndexIT extends BaseHBaseManagedTimeIT {
     }
     
     protected void testUpdatableView(Integer saltBuckets, boolean localIndex) throws Exception {
-        createBaseTable("t", saltBuckets);
+        createBaseTable("t", saltBuckets, true);
         Connection conn = createTenantConnection(TENANT1_ID);
         try {
             createAndPopulateTenantView(conn, TENANT1_ID, "t", "");
@@ -58,13 +59,25 @@ public class BaseTenantSpecificViewIndexIT extends BaseHBaseManagedTimeIT {
             try { conn.close();} catch (Exception ignored) {}
         }
     }
+
+    protected void testUpdatableViewNonString(Integer saltBuckets, boolean localIndex) throws Exception {
+        createBaseTable("t", saltBuckets, false);
+        Connection conn = createTenantConnection(NON_STRING_TENANT_ID);
+        try {
+            createAndPopulateTenantView(conn, NON_STRING_TENANT_ID, "t", "");
+            createAndVerifyIndexNonStringTenantId(conn, NON_STRING_TENANT_ID, "");
+            verifyViewData(conn, "");
+        } finally {
+            try { conn.close();} catch (Exception ignored) {}
+        }
+    }
     
     protected void testUpdatableViewsWithSameNameDifferentTenants(Integer saltBuckets) throws Exception {
         testUpdatableViewsWithSameNameDifferentTenants(saltBuckets, false);
     }
 
     protected void testUpdatableViewsWithSameNameDifferentTenants(Integer saltBuckets, boolean localIndex) throws Exception {
-        createBaseTable("t", saltBuckets);
+        createBaseTable("t", saltBuckets, true);
         Connection conn1 = createTenantConnection(TENANT1_ID);
         Connection conn2 = createTenantConnection(TENANT2_ID);
         try {
@@ -86,9 +99,10 @@ public class BaseTenantSpecificViewIndexIT extends BaseHBaseManagedTimeIT {
         }
     }
     
-    private void createBaseTable(String tableName, Integer saltBuckets) throws SQLException {
+    private void createBaseTable(String tableName, Integer saltBuckets, boolean hasStringTenantId) throws SQLException {
         Connection conn = DriverManager.getConnection(getUrl());
-        String ddl = "CREATE TABLE " + tableName + " (t_id VARCHAR NOT NULL,\n" +
+        String tenantIdType = hasStringTenantId ? "VARCHAR" : "BIGINT";
+        String ddl = "CREATE TABLE " + tableName + " (t_id " + tenantIdType + " NOT NULL,\n" +
                 "k1 INTEGER NOT NULL,\n" +
                 "k2 INTEGER NOT NULL,\n" +
                 "v1 VARCHAR,\n" +
@@ -135,6 +149,17 @@ public class BaseTenantSpecificViewIndexIT extends BaseHBaseManagedTimeIT {
             assertEquals(expected, QueryUtil.getExplainPlan(rs));
         }
     }
+
+    private void createAndVerifyIndexNonStringTenantId(Connection conn, String tenantId, String valuePrefix) throws SQLException {
+        conn.createStatement().execute("CREATE LOCAL INDEX i ON v(v2)");
+        conn.createStatement().execute("UPSERT INTO v(k2,v1,v2) VALUES (-1, 'blah', 'superblah')"); // sanity check that we can upsert after index is there
+        conn.commit();
+        ResultSet rs = conn.createStatement().executeQuery("EXPLAIN SELECT k1, k2, v2 FROM v WHERE v2='" + valuePrefix + "v2-1'");
+        assertEquals(
+                "CLIENT PARALLEL 1-WAY RANGE SCAN OVER _LOCAL_IDX_T [" + tenantId + ",-32768,'" + valuePrefix + "v2-1']\n"
+                        + "    SERVER FILTER BY FIRST KEY ONLY\n"
+                        + "CLIENT MERGE SORT", QueryUtil.getExplainPlan(rs));
+    }
     
     private Connection createTenantConnection(String tenantId) throws SQLException {
         Properties props = new Properties();

http://git-wip-us.apache.org/repos/asf/phoenix/blob/5c048430/phoenix-core/src/it/java/org/apache/phoenix/end2end/CreateTableIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/CreateTableIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/CreateTableIT.java
index 31abd9b..7c4576c 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/CreateTableIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/CreateTableIT.java
@@ -105,6 +105,31 @@ public class CreateTableIT extends BaseClientManagedTimeIT {
         conn = DriverManager.getConnection(getUrl(), props);
         conn.createStatement().execute("DROP TABLE m_interface_job");
     }
+
+    @Test
+    public void testCreateMultiTenantTable() throws Exception {
+        long ts = nextTimestamp();
+        Properties props = new Properties();
+        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts));
+        Connection conn = DriverManager.getConnection(getUrl(), props);
+        String ddl = "CREATE TABLE m_multi_tenant_test(                TenantId UNSIGNED_INT NOT NULL ,\n" +
+                "                Id UNSIGNED_INT NOT NULL ,\n" +
+                "                val VARCHAR ,\n" +
+                "                CONSTRAINT pk PRIMARY KEY(TenantId, Id) \n" +
+                "                ) MULTI_TENANT=true";
+        conn.createStatement().execute(ddl);
+        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 10));
+        conn = DriverManager.getConnection(getUrl(), props);
+        try {
+            conn.createStatement().execute(ddl);
+            fail();
+        } catch (TableAlreadyExistsException e) {
+            // expected
+        }
+        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 20));
+        conn = DriverManager.getConnection(getUrl(), props);
+        conn.createStatement().execute("DROP TABLE m_multi_tenant_test");
+    }
     
     /**
      * Test that when the ddl only has PK cols, ttl is set.

http://git-wip-us.apache.org/repos/asf/phoenix/blob/5c048430/phoenix-core/src/it/java/org/apache/phoenix/end2end/TenantIdTypeIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/TenantIdTypeIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/TenantIdTypeIT.java
new file mode 100644
index 0000000..f28436e
--- /dev/null
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/TenantIdTypeIT.java
@@ -0,0 +1,226 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.phoenix.end2end;
+
+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.*;
+import java.util.Properties;
+import java.util.Collection;
+import java.util.List;
+import com.google.common.collect.Lists;
+
+import org.apache.phoenix.exception.SQLExceptionCode;
+import org.apache.phoenix.schema.SequenceNotFoundException;
+import org.apache.phoenix.schema.TableAlreadyExistsException;
+import org.apache.phoenix.util.PhoenixRuntime;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameters;
+
+@RunWith(Parameterized.class)
+public class TenantIdTypeIT extends BaseHBaseManagedTimeIT {
+
+    private Connection regularConnection(String url) throws SQLException {
+        Properties props = new Properties();
+        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(nextTimestamp()));
+        return DriverManager.getConnection(url, props);
+    }
+
+    private Connection tenantConnection(String url) throws SQLException {
+        Properties props = new Properties();
+        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(nextTimestamp()));
+        String tenantIdProperty = this.tenantId.replaceAll("\'", "");
+        props.setProperty(PhoenixRuntime.TENANT_ID_ATTRIB, tenantIdProperty);
+        return DriverManager.getConnection(url, props);
+    }
+
+    private Connection inconvertibleConnection(String url) throws SQLException {
+        Properties props = new Properties();
+        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(nextTimestamp()));
+        String tenantIdProperty = "ABigOlString";
+        props.setProperty(PhoenixRuntime.TENANT_ID_ATTRIB, tenantIdProperty);
+        return DriverManager.getConnection(url, props);
+    }
+
+    private final String ddl;
+    private final String dataType;
+    private final String tenantId;
+    private final String otherTenantId;
+    private final String table;
+    private final String view;
+    private final String sequence;
+
+    public TenantIdTypeIT(String dataType, String tenantId, String otherTenantId) {
+        this.dataType = dataType;
+        this.tenantId = tenantId;
+        this.otherTenantId = otherTenantId;
+        String tbl = "foo" + dataType;
+        if(tbl.contains("(")){
+            tbl = tbl.substring(0, tbl.indexOf("("));
+        }
+        this.table = tbl;
+        this.view = tbl + "view";
+        this.sequence = tbl + "sequence";
+        this.ddl = "create table " + table + " (" + "tid "+ dataType + " NOT NULL," + "id INTEGER NOT NULL, \n"
+                + "val VARCHAR " + "CONSTRAINT pk PRIMARY KEY(tid, id)) \n"
+                + "MULTI_TENANT=true";
+    }
+
+    @Parameters
+    public static Collection<Object[]> data() {
+        List<Object[]> testCases = Lists.newArrayList();
+        testCases.add(new Object[] { "INTEGER", "2147483647", "2147483646" });
+        testCases.add(new Object[] { "UNSIGNED_INT", "2147483647", "2147483646" });
+        testCases.add(new Object[] { "BIGINT", "9223372036854775807", "9223372036854775806" });
+        testCases.add(new Object[] { "UNSIGNED_LONG", "9223372036854775807", "9223372036854775806" });
+        testCases.add(new Object[] { "TINYINT", "127", "126" });
+        testCases.add(new Object[] { "UNSIGNED_TINYINT", "85", "84" });
+        testCases.add(new Object[] { "SMALLINT", "32767", "32766" });
+        testCases.add(new Object[] { "UNSIGNED_SMALLINT", "32767", "32766" });
+        testCases.add(new Object[] { "FLOAT", "3.4028234", "3.4028232" });
+        testCases.add(new Object[] { "UNSIGNED_FLOAT", "3.4028234", "3.4028232" });
+        testCases.add(new Object[] { "DOUBLE", "1.7976931348623157", "1.7976931348623156" });
+        testCases.add(new Object[] { "UNSIGNED_DOUBLE", "1.7976931348623157", "1.7976931348623156" });
+        testCases.add(new Object[] { "DECIMAL", "3.402823466", "3.402823465" });
+        testCases.add(new Object[] { "VARCHAR", "\'NameOfTenant\'", "\'Nemesis\'" });
+        testCases.add(new Object[] { "CHAR(10)", "\'1234567890\'", "\'Nemesis\'" });
+
+        return testCases;
+    }
+
+    @Test
+    public void testMultiTenantTables() throws Exception {
+        //Verify we can create the table
+        try (Connection conn = regularConnection(getUrl())) {
+            conn.setAutoCommit(true);
+            conn.createStatement().execute(ddl);
+
+            try {
+                conn.createStatement().execute(ddl);
+                fail("Table with " + dataType + " tenantId not created correctly");
+            } catch (TableAlreadyExistsException e) {
+                // expected
+            }
+        }
+
+        //Insert test data
+        try (Connection conn = regularConnection(getUrl())) {
+            conn.setAutoCommit(true);
+            String query = "upsert into " + table +
+                    " values (" + tenantId + ", 1 , 'valid')";
+
+            conn.createStatement().execute("upsert into " + table +
+                    " values (" + tenantId + ", 1 , 'valid')");
+            conn.createStatement().execute("upsert into " + table +
+                    " values (" + otherTenantId + ", 2 , 'invalid')");
+        }
+
+        //Make sure access is properly restricted and add some tenant-specific schema
+        try (Connection conn = tenantConnection(getUrl())) {
+            conn.setAutoCommit(true);
+            ResultSet rs = conn.createStatement().executeQuery("select * from " + table);
+            assertTrue("Expected 1 row in result set", rs.next());
+            assertEquals("valid", rs.getString(2));
+            assertFalse("Expected 1 row in result set", rs.next());
+
+            try {
+                conn.createStatement()
+                        .executeQuery("select * from " + table + " where tenantId = 2");
+                fail("TenantId column not hidden on multi-tenant connection");
+            } catch (SQLException ex) {
+                assertEquals(SQLExceptionCode.COLUMN_NOT_FOUND.getErrorCode(), ex.getErrorCode());
+            }
+
+            conn.createStatement().execute("create view " + view +
+                    " as select * from " + table);
+
+            conn.createStatement().execute("create sequence " + sequence + " start with 100");
+        }
+
+        //Try inserting data to the view
+        try (Connection conn = tenantConnection(getUrl())) {
+            conn.setAutoCommit(true);
+            conn.createStatement().execute("upsert into " + view +
+                    " values ( next value for " + sequence + ", 'valid')");
+        }
+
+        //Try reading data from the view
+        try (Connection conn = tenantConnection(getUrl())) {
+            ResultSet rs = conn.createStatement().executeQuery("select * from " + view);
+            assertTrue("Expected 2 rows in result set", rs.next());
+            assertEquals("valid", rs.getString(2));
+            assertTrue("Expected 2 rows in result set", rs.next());
+            assertEquals("valid", rs.getString(2));
+            assertFalse("Expected 2 rows in result set", rs.next());
+        }
+
+        //Make sure the tenant-specific schema is specific to that tenant
+        try (Connection conn = regularConnection(getUrl())) {
+            try {
+                conn.createStatement().execute("upsert into " + table +
+                        " values (" + tenantId + ", next value for " + sequence + ", 'valid')");
+                fail();
+            } catch (SequenceNotFoundException ex) {}
+
+            try {
+                ResultSet rs = conn.createStatement().executeQuery("select * from " + view);
+                fail();
+            } catch (SQLException ex) {
+                assertEquals(SQLExceptionCode.TABLE_UNDEFINED.getErrorCode(), ex.getErrorCode());
+            }
+
+        }
+
+        if(dataType != "VARCHAR" && dataType != "CHAR(10)") {
+            //Try setting up an invalid tenant-specific view
+            try (Connection conn = inconvertibleConnection(getUrl())) {
+                conn.setAutoCommit(true);
+                conn.createStatement().execute("create view " + view +
+                        " as select * from " + table);
+            }
+
+            //Try inserting data to the invalid tenant-specific view
+            try (Connection conn = inconvertibleConnection(getUrl())) {
+                conn.setAutoCommit(true);
+                try {
+                    conn.createStatement().execute("upsert into " + view +
+                            " values ( 3 , 'invalid')");
+                    fail();
+                } catch (SQLException ex) {
+                    assertEquals(SQLExceptionCode.TENANTID_IS_OF_WRONG_TYPE.getErrorCode(), ex.getErrorCode());
+                }
+            }
+
+            //Try reading data from the invalid tenant-specific view
+            try (Connection conn = inconvertibleConnection(getUrl())) {
+                try {
+                    ResultSet rs = conn.createStatement().executeQuery("select * from " + view);
+                    fail();
+                } catch (SQLException ex) {
+                    assertEquals(SQLExceptionCode.TENANTID_IS_OF_WRONG_TYPE.getErrorCode(), ex.getErrorCode());
+                }
+            }
+        }
+
+    }
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/5c048430/phoenix-core/src/it/java/org/apache/phoenix/end2end/TenantSpecificTablesDDLIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/TenantSpecificTablesDDLIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/TenantSpecificTablesDDLIT.java
index 05b36c3..327627d 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/TenantSpecificTablesDDLIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/TenantSpecificTablesDDLIT.java
@@ -217,18 +217,6 @@ public class TenantSpecificTablesDDLIT extends BaseTenantSpecificTablesIT {
     }
     
     @Test
-    public void testBaseTableWrongFormatWithNoTenantTypeId() throws Exception {
-        // tenantId column of wrong type
-        try {
-            createTestTable(getUrl(), "CREATE TABLE BASE_TABLE5 (TENANT_ID INTEGER NOT NULL, ID VARCHAR, A INTEGER CONSTRAINT PK PRIMARY KEY (TENANT_ID, ID)) MULTI_TENANT=true", null, nextTimestamp());
-            fail();
-        }
-        catch (SQLException expected) {
-            assertEquals(SQLExceptionCode.INSUFFICIENT_MULTI_TENANT_COLUMNS.getErrorCode(), expected.getErrorCode());
-        }
-    }
-    
-    @Test
     public void testAddDropColumn() throws Exception {
         Properties props = new Properties();
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(nextTimestamp()));

http://git-wip-us.apache.org/repos/asf/phoenix/blob/5c048430/phoenix-core/src/it/java/org/apache/phoenix/end2end/TenantSpecificViewIndexIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/TenantSpecificViewIndexIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/TenantSpecificViewIndexIT.java
index e02a473..fc9489d 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/TenantSpecificViewIndexIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/TenantSpecificViewIndexIT.java
@@ -47,6 +47,11 @@ public class TenantSpecificViewIndexIT extends BaseTenantSpecificViewIndexIT {
     }
 
     @Test
+    public void testUpdatableViewLocalIndexNonStringTenantId() throws Exception {
+        testUpdatableViewNonString(null, true);
+    }
+
+    @Test
     public void testUpdatableViewsWithSameNameDifferentTenants() throws Exception {
         testUpdatableViewsWithSameNameDifferentTenants(null);
     }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/5c048430/phoenix-core/src/main/java/org/apache/phoenix/compile/DeleteCompiler.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/DeleteCompiler.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/DeleteCompiler.java
index a28f614..ebbfd9c 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/DeleteCompiler.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/DeleteCompiler.java
@@ -101,8 +101,7 @@ public class DeleteCompiler {
         PName tenantId = connection.getTenantId();
         byte[] tenantIdBytes = null;
         if (tenantId != null) {
-            tenantId = ScanUtil.padTenantIdIfNecessary(table.getRowKeySchema(), table.getBucketNum() != null, tenantId);
-            tenantIdBytes = tenantId.getBytes();
+            tenantIdBytes = ScanUtil.getTenantIdBytes(table.getRowKeySchema(), table.getBucketNum() != null, tenantId);
         }
         final boolean isAutoCommit = connection.getAutoCommit();
         ConnectionQueryServices services = connection.getQueryServices();

http://git-wip-us.apache.org/repos/asf/phoenix/blob/5c048430/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 e12f5a4..08529cc 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
@@ -722,8 +722,7 @@ public class UpsertCompiler {
         final byte[][] values = new byte[nValuesToSet][];
         if (isTenantSpecific) {
             PName tenantId = connection.getTenantId();
-            tenantId = ScanUtil.padTenantIdIfNecessary(table.getRowKeySchema(), table.getBucketNum() != null, tenantId);
-            values[nodeIndex++] = connection.getTenantId().getBytes();
+            values[nodeIndex++] = ScanUtil.getTenantIdBytes(table.getRowKeySchema(), table.getBucketNum() != null, tenantId);
         }
         if (isSharedViewIndex) {
             values[nodeIndex++] = MetaDataUtil.getViewIndexIdDataType().toBytes(table.getViewIndexId());

http://git-wip-us.apache.org/repos/asf/phoenix/blob/5c048430/phoenix-core/src/main/java/org/apache/phoenix/compile/WhereOptimizer.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/WhereOptimizer.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/WhereOptimizer.java
index 601eee1..c575c2e 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/WhereOptimizer.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/WhereOptimizer.java
@@ -99,21 +99,23 @@ public class WhereOptimizer {
      * @param whereClause the where clause expression
      * @return the new where clause with the key expressions removed
      */
-    public static Expression pushKeyExpressionsToScan(StatementContext context, FilterableStatement statement, Expression whereClause) {
+    public static Expression pushKeyExpressionsToScan(StatementContext context, FilterableStatement statement, Expression whereClause)
+            throws SQLException{
         return pushKeyExpressionsToScan(context, statement, whereClause, null);
     }
 
     // For testing so that the extractedNodes can be verified
     public static Expression pushKeyExpressionsToScan(StatementContext context, FilterableStatement statement,
-            Expression whereClause, Set<Expression> extractNodes) {
+            Expression whereClause, Set<Expression> extractNodes) throws SQLException {
         PName tenantId = context.getConnection().getTenantId();
+        byte[] tenantIdBytes = null;
         PTable table = context.getCurrentTable().getTable();
     	Integer nBuckets = table.getBucketNum();
     	boolean isSalted = nBuckets != null;
     	RowKeySchema schema = table.getRowKeySchema();
     	boolean isMultiTenant = tenantId != null && table.isMultiTenant();
     	if (isMultiTenant) {
-    		tenantId = ScanUtil.padTenantIdIfNecessary(schema, isSalted, tenantId);
+            tenantIdBytes = ScanUtil.getTenantIdBytes(schema, isSalted, tenantId);
     	}
 
         if (whereClause == null && (tenantId == null || !table.isMultiTenant()) && table.getViewIndexId() == null) {
@@ -167,7 +169,7 @@ public class WhereOptimizer {
         boolean hasViewIndex = table.getViewIndexId() != null;
         if (hasMinMaxRange) {
             int minMaxRangeSize = (isSalted ? SaltingUtil.NUM_SALTING_BYTES : 0)
-                    + (isMultiTenant ? tenantId.getBytes().length + 1 : 0) 
+                    + (isMultiTenant ? tenantIdBytes.length + 1 : 0)
                     + (hasViewIndex ? MetaDataUtil.getViewIndexIdDataType().getByteSize() : 0);
             minMaxRangePrefix = new byte[minMaxRangeSize];
         }
@@ -188,7 +190,6 @@ public class WhereOptimizer {
         
         // Add tenant data isolation for tenant-specific tables
         if (isMultiTenant) {
-            byte[] tenantIdBytes = tenantId.getBytes();
             KeyRange tenantIdKeyRange = KeyRange.getKeyRange(tenantIdBytes);
             cnf.add(singletonList(tenantIdKeyRange));
             if (hasMinMaxRange) {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/5c048430/phoenix-core/src/main/java/org/apache/phoenix/exception/SQLExceptionCode.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/exception/SQLExceptionCode.java b/phoenix-core/src/main/java/org/apache/phoenix/exception/SQLExceptionCode.java
index acc3c86..59c8e68 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/exception/SQLExceptionCode.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/exception/SQLExceptionCode.java
@@ -235,7 +235,8 @@ public enum SQLExceptionCode {
             
     CANNOT_CREATE_TENANT_SPECIFIC_TABLE(1030, "42Y89", "Cannot create table for tenant-specific connection"),
     DEFAULT_COLUMN_FAMILY_ONLY_ON_CREATE_TABLE(1034, "42Y93", "Default column family may only be specified when creating a table."),
-    INSUFFICIENT_MULTI_TENANT_COLUMNS(1040, "42Y96", "A MULTI_TENANT table must have two or more PK columns with the first column being NOT NULL and of type VARCHAR or CHAR."),
+    INSUFFICIENT_MULTI_TENANT_COLUMNS(1040, "42Y96", "A MULTI_TENANT table must have two or more PK columns with the first column being NOT NULL."),
+    TENANTID_IS_OF_WRONG_TYPE(1041, "42Y97", "The TenantId could not be converted to correct format for this table."),
     VIEW_WHERE_IS_CONSTANT(1045, "43A02", "WHERE clause in VIEW should not evaluate to a constant."),
     CANNOT_UPDATE_VIEW_COLUMN(1046, "43A03", "Column updated in VIEW may not differ from value specified in WHERE clause."),
     TOO_MANY_INDEXES(1047, "43A04", "Too many indexes have already been created on the physical table."),

http://git-wip-us.apache.org/repos/asf/phoenix/blob/5c048430/phoenix-core/src/main/java/org/apache/phoenix/execute/BaseQueryPlan.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/execute/BaseQueryPlan.java b/phoenix-core/src/main/java/org/apache/phoenix/execute/BaseQueryPlan.java
index 11da1c9..af5b25a 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/execute/BaseQueryPlan.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/execute/BaseQueryPlan.java
@@ -204,8 +204,18 @@ public abstract class BaseQueryPlan implements QueryPlan {
         } else {
             ScanUtil.setTimeRange(scan, context.getScanTimeRange());
         }
-        
-        ScanUtil.setTenantId(scan, connection.getTenantId() == null ? null : connection.getTenantId().getBytes());
+        byte[] tenantIdBytes;
+        if( table.isMultiTenant() == true ) {
+            tenantIdBytes = connection.getTenantId() == null ? null :
+                    ScanUtil.getTenantIdBytes(
+                            table.getRowKeySchema(),
+                            table.getBucketNum()!=null,
+                            connection.getTenantId());
+        } else {
+            tenantIdBytes = connection.getTenantId() == null ? null : connection.getTenantId().getBytes();
+        }
+
+        ScanUtil.setTenantId(scan, tenantIdBytes);
         String customAnnotations = LogUtil.customAnnotationsToString(connection);
         ScanUtil.setCustomAnnotations(scan, customAnnotations == null ? null : customAnnotations.getBytes());
         // Set local index related scan attributes. 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/5c048430/phoenix-core/src/main/java/org/apache/phoenix/execute/MutationState.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/execute/MutationState.java b/phoenix-core/src/main/java/org/apache/phoenix/execute/MutationState.java
index af3bcf3..b723381 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/execute/MutationState.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/execute/MutationState.java
@@ -49,20 +49,9 @@ import org.apache.phoenix.monitoring.MutationMetricQueue.MutationMetric;
 import org.apache.phoenix.monitoring.MutationMetricQueue.NoOpMutationMetricsQueue;
 import org.apache.phoenix.monitoring.ReadMetricQueue;
 import org.apache.phoenix.query.QueryConstants;
-import org.apache.phoenix.schema.IllegalDataException;
-import org.apache.phoenix.schema.MetaDataClient;
-import org.apache.phoenix.schema.PColumn;
-import org.apache.phoenix.schema.PRow;
-import org.apache.phoenix.schema.PTable;
-import org.apache.phoenix.schema.PTableType;
-import org.apache.phoenix.schema.TableRef;
+import org.apache.phoenix.schema.*;
 import org.apache.phoenix.trace.util.Tracing;
-import org.apache.phoenix.util.ByteUtil;
-import org.apache.phoenix.util.IndexUtil;
-import org.apache.phoenix.util.LogUtil;
-import org.apache.phoenix.util.PhoenixRuntime;
-import org.apache.phoenix.util.SQLCloseable;
-import org.apache.phoenix.util.ServerUtil;
+import org.apache.phoenix.util.*;
 import org.apache.htrace.Span;
 import org.apache.htrace.TraceScope;
 import org.slf4j.Logger;
@@ -370,7 +359,7 @@ public class MutationState implements SQLCloseable {
     @SuppressWarnings("deprecation")
     public void commit() throws SQLException {
         int i = 0;
-        byte[] tenantId = connection.getTenantId() == null ? null : connection.getTenantId().getBytes();
+        PName tenantId = connection.getTenantId();
         long[] serverTimeStamps = validate();
         Iterator<Map.Entry<TableRef, Map<ImmutableBytesPtr,RowMutationState>>> iterator = this.mutations.entrySet().iterator();
         // add tracing for this operation
@@ -424,7 +413,11 @@ public class MutationState implements SQLCloseable {
                             // or set the index metadata directly on the Mutation
                             for (Mutation mutation : mutations) {
                                 if (tenantId != null) {
-                                    mutation.setAttribute(PhoenixRuntime.TENANT_ID_ATTRIB, tenantId);
+                                    byte[] tenantIdBytes = ScanUtil.getTenantIdBytes(
+                                        table.getRowKeySchema(),
+                                        table.getBucketNum()!=null,
+                                        tenantId);
+                                    mutation.setAttribute(PhoenixRuntime.TENANT_ID_ATTRIB, tenantIdBytes);
                                 }
                                 mutation.setAttribute(PhoenixIndexCodec.INDEX_UUID, uuidValue);
                                 if (attribValue != null) {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/5c048430/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java
index f74133a..7a7369b 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java
@@ -1438,7 +1438,7 @@ public class MetaDataClient {
             String tableName = tableNameNode.getTableName();
             String parentTableName = null;
             PName tenantId = connection.getTenantId();
-            String tenantIdStr = tenantId == null ? null : connection.getTenantId().getString();
+            String tenantIdStr = tenantId == null ? null : tenantId.getString();
             Long scn = connection.getSCN();
             long clientTimeStamp = scn == null ? HConstants.LATEST_TIMESTAMP : scn;
             boolean multiTenant = false;
@@ -2032,7 +2032,7 @@ public class MetaDataClient {
         // NOT NULL is a requirement, since otherwise the table key would conflict
         // potentially with the global table definition.
         PColumn tenantIdCol = iterator.next();
-        if (!tenantIdCol.getDataType().isCoercibleTo(PVarchar.INSTANCE) || tenantIdCol.isNullable()) {
+        if ( tenantIdCol.isNullable()) {
             throw new SQLExceptionInfo.Builder(INSUFFICIENT_MULTI_TENANT_COLUMNS).setSchemaName(schemaName).setTableName(tableName).build().buildException();
         }
     }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/5c048430/phoenix-core/src/main/java/org/apache/phoenix/util/ScanUtil.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/util/ScanUtil.java b/phoenix-core/src/main/java/org/apache/phoenix/util/ScanUtil.java
index ae073e2..2a73e25 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/util/ScanUtil.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/ScanUtil.java
@@ -46,14 +46,16 @@ import org.apache.phoenix.compile.ScanRanges;
 import org.apache.phoenix.compile.StatementContext;
 import org.apache.phoenix.coprocessor.BaseScannerRegionObserver;
 import org.apache.phoenix.coprocessor.MetaDataProtocol;
-import org.apache.phoenix.execute.DescVarLengthFastByteComparisons;
-import org.apache.phoenix.filter.BooleanExpressionFilter;
+import org.apache.phoenix.exception.SQLExceptionCode;
+import org.apache.phoenix.exception.SQLExceptionInfo;
+import org.apache.phoenix.execute.DescVarLengthFastByteComparisons;import org.apache.phoenix.filter.BooleanExpressionFilter;
 import org.apache.phoenix.filter.SkipScanFilter;
 import org.apache.phoenix.query.KeyRange;
 import org.apache.phoenix.query.KeyRange.Bound;
 import org.apache.phoenix.query.QueryConstants;
 import org.apache.phoenix.query.QueryServices;
 import org.apache.phoenix.query.QueryServicesOptions;
+import org.apache.phoenix.schema.IllegalDataException;
 import org.apache.phoenix.schema.PName;
 import org.apache.phoenix.schema.PNameFactory;
 import org.apache.phoenix.schema.RowKeySchema;
@@ -536,7 +538,7 @@ public class ScanUtil {
         while (schema.next(ptr, pos, maxOffset) != null) {
             pos++;
         }
-        Field field = schema.getField(pos-1);
+        Field field = schema.getField(pos - 1);
         if (!field.getDataType().isFixedWidth()) {
             byte[] newLowerRange = new byte[key.length + 1];
             System.arraycopy(key, 0, newLowerRange, 0, key.length);
@@ -688,21 +690,24 @@ public class ScanUtil {
         }
         return Bytes.compareTo(key, 0, nBytesToCheck, ZERO_BYTE_ARRAY, 0, nBytesToCheck) != 0;
     }
-    
-    public static PName padTenantIdIfNecessary(RowKeySchema schema, boolean isSalted, PName tenantId) {
+
+    public static byte[] getTenantIdBytes(RowKeySchema schema, boolean isSalted, PName tenantId)
+            throws SQLException {
         int pkPos = isSalted ? 1 : 0;
-        String tenantIdStr = tenantId.getString();
         Field field = schema.getField(pkPos);
         PDataType dataType = field.getDataType();
-        boolean isFixedWidth = dataType.isFixedWidth();
-        Integer maxLength = field.getMaxLength();
-        if (isFixedWidth && maxLength != null) {
-            if (tenantIdStr.length() < maxLength) {
-                tenantIdStr = (String)dataType.pad(tenantIdStr, maxLength);
-                return PNameFactory.newName(tenantIdStr);
-            }
-        }
-        return tenantId;
+        byte[] convertedValue;
+        try {
+            Object value = dataType.toObject(tenantId.getString());
+            convertedValue = dataType.toBytes(value);
+            ImmutableBytesWritable ptr = new ImmutableBytesWritable(convertedValue);
+            dataType.pad(ptr, field.getMaxLength(), field.getSortOrder());
+            convertedValue = ByteUtil.copyKeyBytesIfNecessary(ptr);
+        } catch(IllegalDataException ex) {
+            throw new SQLExceptionInfo.Builder(SQLExceptionCode.TENANTID_IS_OF_WRONG_TYPE)
+                    .build().buildException();
+        }
+        return convertedValue;
     }
 
     public static Iterator<Filter> getFilterIterator(Scan scan) {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/5c048430/phoenix-core/src/test/java/org/apache/phoenix/util/TenantIdByteConversionTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/util/TenantIdByteConversionTest.java b/phoenix-core/src/test/java/org/apache/phoenix/util/TenantIdByteConversionTest.java
new file mode 100644
index 0000000..4d433aa
--- /dev/null
+++ b/phoenix-core/src/test/java/org/apache/phoenix/util/TenantIdByteConversionTest.java
@@ -0,0 +1,294 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.phoenix.util;
+
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.fail;
+
+import java.sql.SQLException;
+import org.apache.hadoop.hbase.util.Base64;
+import java.util.Collection;
+import java.util.List;
+
+import org.apache.phoenix.schema.*;
+import org.apache.phoenix.schema.types.*;
+import org.apache.phoenix.schema.RowKeySchema.RowKeySchemaBuilder;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameters;
+
+import com.google.common.collect.Lists;
+import org.mockito.Mockito;
+
+/*Test the getTenantIdBytes method in ScanUtil*/
+@RunWith(Parameterized.class)
+public class TenantIdByteConversionTest {
+
+    private RowKeySchema schema;
+    private boolean isSalted;
+    private PName tenantId;
+    private byte[] expectedTenantIdBytes;
+
+
+    public TenantIdByteConversionTest(
+            RowKeySchema schema,
+            boolean isSalted,
+            PName tenantId,
+            byte[] expectedTenantIdBytes ) {
+        this.schema = schema;
+        this.isSalted = isSalted;
+        this.tenantId = tenantId;
+        this.expectedTenantIdBytes = expectedTenantIdBytes;
+    }
+
+    @Test
+    public void test() {
+        try {
+            byte[] actualTenantIdBytes = ScanUtil.getTenantIdBytes(schema, isSalted, tenantId);
+            assertArrayEquals(expectedTenantIdBytes, actualTenantIdBytes);
+        } catch (SQLException ex) {
+            fail(ex.getMessage());
+        }
+    }
+
+    @Parameters
+    public static Collection<Object[]> data() {
+        List<Object[]> testCases = Lists.newArrayList();
+        // Varchar
+        testCases.add(new Object[] {
+                getDataSchema(PVarchar.INSTANCE, SortOrder.getDefault()),
+                false,
+                PNameFactory.newName("NameOfTenant"),
+                PVarchar.INSTANCE.toBytes("NameOfTenant")
+        });
+
+        // Char
+        testCases.add(new Object[] {
+                getDataSchema(PChar.INSTANCE, SortOrder.getDefault()),
+                false,
+                PNameFactory.newName("N"),
+                PChar.INSTANCE.toBytes(PChar.INSTANCE.toObject("N"))
+        });
+
+        //Int
+        testCases.add(new Object[] {
+                getDataSchema(PInteger.INSTANCE, SortOrder.getDefault()),
+                false,
+                PNameFactory.newName("2147483646"),
+                PInteger.INSTANCE.toBytes(PInteger.INSTANCE.toObject("2147483646"))
+        });
+
+        // UnsignedInt
+        testCases.add(new Object[] {
+                getDataSchema(PUnsignedInt.INSTANCE, SortOrder.getDefault()),
+                false,
+                PNameFactory.newName("2147483646"),
+                PUnsignedInt.INSTANCE.toBytes(PUnsignedInt.INSTANCE.toObject("2147483646"))
+        });
+
+        //BigInt
+        testCases.add(new Object[] {
+                getDataSchema(PLong.INSTANCE, SortOrder.getDefault()),
+                false,
+                PNameFactory.newName("9223372036854775806"),
+                PLong.INSTANCE.toBytes(PLong.INSTANCE.toObject("9223372036854775806"))
+        });
+
+        //UnsignedLong
+        testCases.add(new Object[] {
+                getDataSchema(PUnsignedLong.INSTANCE, SortOrder.getDefault()),
+                false,
+                PNameFactory.newName("9223372036854775806"),
+                PUnsignedLong.INSTANCE.toBytes(PUnsignedLong.INSTANCE.toObject("9223372036854775806"))
+        });
+
+        //TinyInt
+        testCases.add(new Object[] {
+                getDataSchema(PTinyint.INSTANCE, SortOrder.getDefault()),
+                false,
+                PNameFactory.newName("126"),
+                PTinyint.INSTANCE.toBytes(PTinyint.INSTANCE.toObject("126"))
+        });
+
+        //UnsignedTinyInt
+        testCases.add(new Object[] {
+                getDataSchema(PUnsignedTinyint.INSTANCE, SortOrder.getDefault()),
+                false,
+                PNameFactory.newName("126"),
+                PUnsignedTinyint.INSTANCE.toBytes(PUnsignedTinyint.INSTANCE.toObject("126"))
+        });
+
+        //SmallInt
+        testCases.add(new Object[] {
+                getDataSchema(PSmallint.INSTANCE, SortOrder.getDefault()),
+                false,
+                PNameFactory.newName("32766"),
+                PSmallint.INSTANCE.toBytes(PSmallint.INSTANCE.toObject("32766"))
+        });
+
+        //UnsignedSmallInt
+        testCases.add(new Object[] {
+                getDataSchema(PUnsignedSmallint.INSTANCE, SortOrder.getDefault()),
+                false,
+                PNameFactory.newName("32766"),
+                PUnsignedSmallint.INSTANCE.toBytes(PUnsignedSmallint.INSTANCE.toObject("32766"))
+        });
+
+        //Float
+        testCases.add(new Object[] {
+                getDataSchema(PFloat.INSTANCE, SortOrder.getDefault()),
+                false,
+                PNameFactory.newName("3.402823466"),
+                PFloat.INSTANCE.toBytes(PFloat.INSTANCE.toObject("3.402823466"))
+        });
+
+        //UnsignedFloat
+        testCases.add(new Object[] {
+                getDataSchema(PUnsignedFloat.INSTANCE, SortOrder.getDefault()),
+                false,
+                PNameFactory.newName("3.402823466"),
+                PUnsignedFloat.INSTANCE.toBytes(PUnsignedFloat.INSTANCE.toObject("3.402823466"))
+        });
+
+        //Double
+        testCases.add(new Object[] {
+                getDataSchema(PDouble.INSTANCE, SortOrder.getDefault()),
+                false,
+                PNameFactory.newName("1.7976931348623158"),
+                PDouble.INSTANCE.toBytes(PDouble.INSTANCE.toObject("1.7976931348623158"))
+        });
+
+        //UnsignedDouble
+        testCases.add(new Object[] {
+                getDataSchema(PUnsignedDouble.INSTANCE, SortOrder.getDefault()),
+                false,
+                PNameFactory.newName("1.7976931348623158"),
+                PUnsignedDouble.INSTANCE.toBytes(PUnsignedDouble.INSTANCE.toObject("1.7976931348623158"))
+        });
+
+        //UnsignedDecimal
+        testCases.add(new Object[] {
+                getDataSchema(PDecimal.INSTANCE, SortOrder.getDefault()),
+                false,
+                PNameFactory.newName("3.402823466"),
+                PDecimal.INSTANCE.toBytes(PDecimal.INSTANCE.toObject("3.402823466"))
+        });
+
+        //Boolean
+        testCases.add(new Object[] {
+                getDataSchema(PBoolean.INSTANCE, SortOrder.getDefault()),
+                false,
+                PNameFactory.newName("true"),
+                PBoolean.INSTANCE.toBytes(PBoolean.INSTANCE.toObject("true"))
+        });
+
+        //Binary
+        byte[] bytes = new byte[] {0, 1, 2, 3};
+        String byteString = new String( Base64.encodeBytes(bytes) );
+        testCases.add(new Object[] {
+                getDataSchema(PBinary.INSTANCE, SortOrder.getDefault()),
+                false,
+                PNameFactory.newName(byteString),
+                PBinary.INSTANCE.toBytes(PBinary.INSTANCE.toObject(byteString))
+        });
+
+        //Descending TenantId
+        testCases.add(new Object[] {
+                getDataSchema(PUnsignedInt.INSTANCE, SortOrder.DESC),
+                false,
+                PNameFactory.newName("2147483646"),
+                PUnsignedInt.INSTANCE.toBytes(PUnsignedInt.INSTANCE.toObject("2147483646"))
+        });
+
+        return testCases;
+    }
+
+    public static RowKeySchema getDataSchema (final PDataType data, final SortOrder sortOrder) {
+        RowKeySchemaBuilder builder = new RowKeySchemaBuilder(3);
+
+        builder.addField(new PDatum() {
+            @Override public boolean isNullable() {
+                return false;
+            }
+
+            @Override public PDataType getDataType() {
+                return data;
+            }
+
+            @Override public Integer getMaxLength() {
+                return 1;
+            }
+
+            @Override public Integer getScale() {
+                return null;
+            }
+
+            @Override public SortOrder getSortOrder() {
+                return sortOrder;
+            }
+        }, false, sortOrder);
+
+        builder.addField(new PDatum() {
+            @Override public boolean isNullable() {
+                return false;
+            }
+
+            @Override public PDataType getDataType() {
+                return PUnsignedInt.INSTANCE;
+            }
+
+            @Override public Integer getMaxLength() {
+                return 3;
+            }
+
+            @Override public Integer getScale() {
+                return null;
+            }
+
+            @Override public SortOrder getSortOrder() {
+                return sortOrder;
+            }
+        }, false, sortOrder);
+
+        builder.addField(new PDatum() {
+            @Override public boolean isNullable() {
+                return true;
+            }
+
+            @Override public PDataType getDataType() {
+                return PVarchar.INSTANCE;
+            }
+
+            @Override public Integer getMaxLength() {
+                return 3;
+            }
+
+            @Override public Integer getScale() {
+                return null;
+            }
+
+            @Override public SortOrder getSortOrder() {
+                return sortOrder;
+            }
+        }, false, sortOrder);
+
+        return builder.build();
+    }
+}