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 2014/08/04 20:54:24 UTC

[1/2] PHOENIX-1140 Incorrect padding for row key columns (Samarth Jain)

Repository: phoenix
Updated Branches:
  refs/heads/3.0 ddf970a38 -> e574d1ea8


http://git-wip-us.apache.org/repos/asf/phoenix/blob/e574d1ea/phoenix-core/src/test/java/org/apache/phoenix/util/ScanUtilTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/util/ScanUtilTest.java b/phoenix-core/src/test/java/org/apache/phoenix/util/ScanUtilTest.java
index 8fa5314..b1246b8 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/util/ScanUtilTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/util/ScanUtilTest.java
@@ -128,7 +128,7 @@ public class ScanUtilTest {
                     PDataType.CHAR.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("1"), true),},{
                     PDataType.CHAR.getKeyRange(Bytes.toBytes("A"), true, Bytes.toBytes("A"), true),}},
                 new int[] {1,1,1},
-                ByteUtil.fillKey(PDataType.VARCHAR.toBytes("a1A"), 3),
+                PDataType.CHAR.toBytes("a1A"),
                 Bound.LOWER
                 ));
         // 2, Lower bound, all range keys, all inclusive.
@@ -138,7 +138,7 @@ public class ScanUtilTest {
                     PDataType.CHAR.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("2"), true),},{
                     PDataType.CHAR.getKeyRange(Bytes.toBytes("A"), true, Bytes.toBytes("B"), true),}},
                 new int[] {1,1,1},
-                ByteUtil.fillKey(PDataType.VARCHAR.toBytes("a1A"), 3),
+                PDataType.CHAR.toBytes("a1A"),
                 Bound.LOWER
                 ));
         // 3, Lower bound, mixed single and range keys, all inclusive.
@@ -148,7 +148,7 @@ public class ScanUtilTest {
                     PDataType.CHAR.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("2"), true),},{
                     PDataType.CHAR.getKeyRange(Bytes.toBytes("A"), true, Bytes.toBytes("A"), true),}},
                 new int[] {1,1,1},
-                ByteUtil.fillKey(PDataType.VARCHAR.toBytes("a1A"), 3),
+                PDataType.CHAR.toBytes("a1A"),
                 Bound.LOWER
                 ));
         // 4, Lower bound, all range key, all exclusive on lower bound.
@@ -158,7 +158,7 @@ public class ScanUtilTest {
                     PDataType.CHAR.getKeyRange(Bytes.toBytes("1"), false, Bytes.toBytes("2"), true),},{
                     PDataType.CHAR.getKeyRange(Bytes.toBytes("A"), false, Bytes.toBytes("B"), true),}},
                 new int[] {1,1,1},
-                ByteUtil.fillKey(PDataType.VARCHAR.toBytes("b2B"), 3),
+                PDataType.CHAR.toBytes("b2B"),
                 Bound.LOWER
                 ));
         // 5, Lower bound, all range key, some exclusive.
@@ -168,7 +168,7 @@ public class ScanUtilTest {
                     PDataType.CHAR.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("2"), true),},{
                     PDataType.CHAR.getKeyRange(Bytes.toBytes("A"), false, Bytes.toBytes("B"), true),}},
                 new int[] {1,1,1},
-                ByteUtil.fillKey(PDataType.VARCHAR.toBytes("b1B"), 3),
+                PDataType.CHAR.toBytes("b1B"),
                 Bound.LOWER
                 ));
         // 6, Lower bound, mixed single and range key, mixed inclusive and exclusive.
@@ -178,7 +178,7 @@ public class ScanUtilTest {
                     PDataType.CHAR.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("2"), true),},{
                     PDataType.CHAR.getKeyRange(Bytes.toBytes("A"), false, Bytes.toBytes("B"), true),}},
                 new int[] {1,1,1},
-                ByteUtil.fillKey(PDataType.VARCHAR.toBytes("a1B"), 3),
+                PDataType.CHAR.toBytes("a1B"),
                 Bound.LOWER
                 ));
         // 7, Lower bound, unbound key in the middle, fixed length.
@@ -188,7 +188,7 @@ public class ScanUtilTest {
                         KeyRange.EVERYTHING_RANGE,},{
                     PDataType.CHAR.getKeyRange(Bytes.toBytes("A"), false, Bytes.toBytes("B"), true),}},
                 new int[] {1,1,1},
-                ByteUtil.fillKey(PDataType.VARCHAR.toBytes("a"), 1),
+                PDataType.CHAR.toBytes("a"),
                 Bound.LOWER
                 ));
         // 8, Lower bound, unbound key in the middle, variable length.
@@ -197,7 +197,7 @@ public class ScanUtilTest {
                         PDataType.CHAR.getKeyRange(Bytes.toBytes("a"), true, Bytes.toBytes("a"), true),},{
                             KeyRange.EVERYTHING_RANGE,}},
                     new int[] {1,1},
-                    ByteUtil.concat(PDataType.VARCHAR.toBytes("a")),
+                    PDataType.CHAR.toBytes("a"),
                     Bound.LOWER
                     ));
         // 9, Lower bound, unbound key at end, variable length.
@@ -207,7 +207,7 @@ public class ScanUtilTest {
                         KeyRange.EVERYTHING_RANGE,},{
                     PDataType.CHAR.getKeyRange(Bytes.toBytes("A"), true, Bytes.toBytes("B"), true),}},
                 new int[] {1,1,1},
-                ByteUtil.concat(PDataType.VARCHAR.toBytes("a")),
+                PDataType.CHAR.toBytes("a"),
                 Bound.LOWER
                 ));
         // 10, Upper bound, all single keys, all inclusive, increment at end.
@@ -217,7 +217,7 @@ public class ScanUtilTest {
                     PDataType.CHAR.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("1"), true),},{
                     PDataType.CHAR.getKeyRange(Bytes.toBytes("A"), true, Bytes.toBytes("A"), true),}},
                 new int[] {1,1,1},
-                ByteUtil.fillKey(PDataType.VARCHAR.toBytes("a1B"), 3),
+                PDataType.CHAR.toBytes("a1B"),
                 Bound.UPPER
                 ));
         // 11, Upper bound, all range keys, all inclusive, increment at end.
@@ -227,7 +227,7 @@ public class ScanUtilTest {
                     PDataType.CHAR.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("2"), true),},{
                     PDataType.CHAR.getKeyRange(Bytes.toBytes("A"), true, Bytes.toBytes("B"), true),}},
                 new int[] {1,1,1},
-                ByteUtil.fillKey(PDataType.VARCHAR.toBytes("b2C"), 3),
+                PDataType.CHAR.toBytes("b2C"),
                 Bound.UPPER
                 ));
         // 12, Upper bound, all range keys, all exclusive, no increment at end.
@@ -237,7 +237,7 @@ public class ScanUtilTest {
                     PDataType.CHAR.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("2"), false),},{
                     PDataType.CHAR.getKeyRange(Bytes.toBytes("A"), true, Bytes.toBytes("B"), false),}},
                 new int[] {1,1,1},
-                ByteUtil.fillKey(PDataType.VARCHAR.toBytes("b2B"), 3),
+                PDataType.CHAR.toBytes("b2B"),
                 Bound.UPPER
                 ));
         // 13, Upper bound, single inclusive, range inclusive, increment at end.
@@ -246,7 +246,7 @@ public class ScanUtilTest {
                     PDataType.CHAR.getKeyRange(Bytes.toBytes("a"), true, Bytes.toBytes("a"), true),},{
                     PDataType.CHAR.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("2"), true),}},
                 new int[] {1,1},
-                ByteUtil.fillKey(PDataType.VARCHAR.toBytes("a3"), 2),
+                PDataType.CHAR.toBytes("a3"),
                 Bound.UPPER
                 ));
         // 14, Upper bound, range exclusive, single inclusive, increment at end.
@@ -255,7 +255,7 @@ public class ScanUtilTest {
                     PDataType.CHAR.getKeyRange(Bytes.toBytes("a"), true, Bytes.toBytes("b"), false),},{
                     PDataType.CHAR.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("1"), true),}},
                 new int[] {1,1},
-                ByteUtil.fillKey(PDataType.VARCHAR.toBytes("b2"), 2),
+                PDataType.CHAR.toBytes("b2"),
                 Bound.UPPER
                 ));
         // 15, Upper bound, range inclusive, single inclusive, increment at end.
@@ -264,7 +264,7 @@ public class ScanUtilTest {
                     PDataType.CHAR.getKeyRange(Bytes.toBytes("a"), true, Bytes.toBytes("b"), true),},{
                     PDataType.CHAR.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("1"), true),}},
                 new int[] {1,1},
-                ByteUtil.fillKey(PDataType.VARCHAR.toBytes("b2"), 2),
+                PDataType.CHAR.toBytes("b2"),
                 Bound.UPPER
                 ));
         // 16, Upper bound, single inclusive, range exclusive, no increment at end.
@@ -273,7 +273,7 @@ public class ScanUtilTest {
                     PDataType.CHAR.getKeyRange(Bytes.toBytes("a"), true, Bytes.toBytes("a"), true),},{
                     PDataType.CHAR.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("2"), false),}},
                 new int[] {1,1},
-                ByteUtil.fillKey(PDataType.VARCHAR.toBytes("a2"), 2),
+                PDataType.CHAR.toBytes("a2"),
                 Bound.UPPER
                 ));
         // 17, Upper bound, unbound key, fixed length;
@@ -282,7 +282,7 @@ public class ScanUtilTest {
                     PDataType.CHAR.getKeyRange(Bytes.toBytes("a"), true, Bytes.toBytes("a"), true),},{
                         KeyRange.EVERYTHING_RANGE,}},
                 new int[] {1,1},
-                ByteUtil.fillKey(PDataType.VARCHAR.toBytes("b"), 1),
+                PDataType.CHAR.toBytes("b"),
                 Bound.UPPER
                 ));
         // 18, Upper bound, unbound key, variable length;
@@ -291,7 +291,7 @@ public class ScanUtilTest {
                     PDataType.CHAR.getKeyRange(Bytes.toBytes("a"), true, Bytes.toBytes("a"), true),},{
                     KeyRange.EVERYTHING_RANGE,}},
                 new int[] {1,1},
-                ByteUtil.concat(PDataType.VARCHAR.toBytes("b")),
+                PDataType.CHAR.toBytes("b"),
                 Bound.UPPER
                 ));
         // 19, Upper bound, keys wrapped around when incrementing.


[2/2] git commit: PHOENIX-1140 Incorrect padding for row key columns (Samarth Jain)

Posted by ja...@apache.org.
PHOENIX-1140 Incorrect padding for row key columns (Samarth Jain)


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

Branch: refs/heads/3.0
Commit: e574d1ea888d8d2e29ed787b8ed8edf54905c68b
Parents: ddf970a
Author: James Taylor <jt...@salesforce.com>
Authored: Mon Aug 4 11:57:50 2014 -0700
Committer: James Taylor <jt...@salesforce.com>
Committed: Mon Aug 4 11:57:50 2014 -0700

----------------------------------------------------------------------
 .../org/apache/phoenix/end2end/QueryMoreIT.java |  19 ++-
 .../org/apache/phoenix/end2end/QueryPlanIT.java |   5 +-
 .../phoenix/end2end/RowValueConstructorIT.java  | 136 +++++++++++++++
 .../phoenix/end2end/VariableLengthPKIT.java     |  10 +-
 .../org/apache/phoenix/compile/ScanRanges.java  |  12 +-
 .../apache/phoenix/compile/WhereOptimizer.java  |  21 ++-
 .../phoenix/expression/CaseExpression.java      |   6 -
 .../phoenix/expression/InListExpression.java    |  29 +---
 .../expression/function/PrefixFunction.java     |  35 ++--
 .../expression/function/RTrimFunction.java      |  27 ++-
 .../expression/function/ScalarFunction.java     |   9 +
 .../java/org/apache/phoenix/query/KeyRange.java |  29 ----
 .../org/apache/phoenix/schema/PDataType.java    |  25 +++
 .../org/apache/phoenix/schema/PTableImpl.java   |   9 +-
 .../java/org/apache/phoenix/util/ByteUtil.java  |  19 ++-
 .../java/org/apache/phoenix/util/ScanUtil.java  |  16 +-
 .../phoenix/compile/QueryCompilerTest.java      |   8 +-
 .../compile/StatementHintsCompilationTest.java  |   2 +-
 .../phoenix/compile/WhereCompilerTest.java      |  10 +-
 .../phoenix/compile/WhereOptimizerTest.java     | 166 +++++++++++++++----
 .../org/apache/phoenix/util/ScanUtilTest.java   |  36 ++--
 21 files changed, 450 insertions(+), 179 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/e574d1ea/phoenix-core/src/it/java/org/apache/phoenix/end2end/QueryMoreIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/QueryMoreIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/QueryMoreIT.java
index 50b3d47..5173b0e 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/QueryMoreIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/QueryMoreIT.java
@@ -17,7 +17,6 @@
  */
 package org.apache.phoenix.end2end;
 
-import static org.apache.commons.lang.StringUtils.rightPad;
 import static org.junit.Assert.assertEquals;
 
 import java.sql.Connection;
@@ -95,15 +94,20 @@ public class QueryMoreIT extends BaseHBaseManagedTimeIT {
         conn.createStatement().execute(baseDataTableDDL);
         conn.close();
         
-        //upsert rows in the history table.
+        //upsert rows in the data table.
         Map<String, List<String>> historyIdsPerTenant = createHistoryTableRows(dataTableName, tenantIds, numRowsPerTenant);
         
-        //create sequence. Use the sequence to upsert select records in cursor table.
         String tenantId = tenantIds[0];
         String cursorQueryId = "00TcursrqueryId";
         String tenantViewName = dataTableMultiTenant ? ("HISTORY_TABLE" + "_" + tenantId) : null;
         assertEquals(numRowsPerTenant, upsertSelectRecordsInCursorTableForTenant(dataTableName, dataTableMultiTenant, tenantId, tenantViewName, cursorQueryId));
         
+        Connection conn2 = DriverManager.getConnection(getUrl());
+        ResultSet rs = conn2.createStatement().executeQuery("SELECT count(*) from " + cursorTableName);
+        rs.next();
+        assertEquals(numRowsPerTenant, rs.getInt(1));
+        conn2.close();
+        
         int startOrder = 0;
         int endOrder = 5;
         int numRecordsThatShouldBeRetrieved = 5;
@@ -132,9 +136,10 @@ public class QueryMoreIT extends BaseHBaseManagedTimeIT {
                 List<String> parentIds = new ArrayList<String>();
                 for (int i = 0; i < numRowsPerTenant; i++) {
                     stmt.setString(1, tenantIds[j]);
-                    stmt.setString(2, rightPad("parentId", 15, 'p'));
+                    String parentId = "parentId" + i;
+                    stmt.setString(2, parentId);
                     stmt.setDate(3, new Date(100));
-                    String historyId = rightPad("history" + i, 15, 'h'); 
+                    String historyId = "historyId" + i; 
                     stmt.setString(4, historyId);
                     stmt.setString(5, "datatype");
                     stmt.setString(6, "oldval");
@@ -154,6 +159,8 @@ public class QueryMoreIT extends BaseHBaseManagedTimeIT {
     private int upsertSelectRecordsInCursorTableForTenant(String baseTableName, boolean dataTableMultiTenant, String tenantId, String tenantViewName, String cursorQueryId) throws Exception {
         String sequenceName = "\"" + tenantId + "_SEQ\"";
         Connection conn = dataTableMultiTenant ? getTenantSpecificConnection(tenantId) : DriverManager.getConnection(getUrl());
+        
+        // Create a sequence. This sequence is used to fill cursor_order column for each row inserted in the cursor table.
         conn.createStatement().execute("CREATE SEQUENCE " + sequenceName + " CACHE " + Long.MAX_VALUE);
         conn.setAutoCommit(true);
         if (dataTableMultiTenant) {
@@ -162,6 +169,8 @@ public class QueryMoreIT extends BaseHBaseManagedTimeIT {
         try {
             String tableName = dataTableMultiTenant ? tenantViewName : baseTableName;
             String tenantIdFilter = dataTableMultiTenant ? "" : " WHERE TENANT_ID = ? ";
+            
+            // Using dynamic columns, we can use the same cursor table for storing primary keys for all the tables.  
             String upsertSelectDML = "UPSERT INTO CURSOR_TABLE " +
                                      "(TENANT_ID, QUERY_ID, CURSOR_ORDER, PARENT_ID CHAR(15), CREATED_DATE DATE, ENTITY_HISTORY_ID CHAR(15)) " + 
                                      "SELECT ?, ?, NEXT VALUE FOR " + sequenceName + ", PARENT_ID, CREATED_DATE, ENTITY_HISTORY_ID " +

http://git-wip-us.apache.org/repos/asf/phoenix/blob/e574d1ea/phoenix-core/src/it/java/org/apache/phoenix/end2end/QueryPlanIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/QueryPlanIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/QueryPlanIT.java
index 0f40d90..60409b2 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/QueryPlanIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/QueryPlanIT.java
@@ -108,14 +108,13 @@ public class QueryPlanIT extends BaseHBaseManagedTimeIT {
                 "    SERVER FILTER BY FIRST KEY ONLY\n" + 
                 "    SERVER AGGREGATE INTO SINGLE ROW",
 
-                // TODO: review: why does this change with parallelized non aggregate queries?
                 "SELECT count(*) FROM atable WHERE organization_id='000000000000001' AND SUBSTR(entity_id,1,3) > '002' AND SUBSTR(entity_id,1,3) <= '003'",
-                "CLIENT PARALLEL 1-WAY RANGE SCAN OVER ATABLE ['000000000000001','003'] - ['000000000000001','004']\n" + 
+                "CLIENT PARALLEL 1-WAY RANGE SCAN OVER ATABLE ['000000000000001','003            '] - ['000000000000001','004            ']\n" + 
                 "    SERVER FILTER BY FIRST KEY ONLY\n" + 
                 "    SERVER AGGREGATE INTO SINGLE ROW",
 
                 "SELECT a_string FROM atable WHERE organization_id='000000000000001' AND SUBSTR(entity_id,1,3) > '002' AND SUBSTR(entity_id,1,3) <= '003'",
-                "CLIENT PARALLEL 1-WAY RANGE SCAN OVER ATABLE ['000000000000001','003'] - ['000000000000001','004']",
+                "CLIENT PARALLEL 1-WAY RANGE SCAN OVER ATABLE ['000000000000001','003            '] - ['000000000000001','004            ']",
 
                 "SELECT count(1) FROM atable GROUP BY a_string",
                 "CLIENT PARALLEL 4-WAY FULL SCAN OVER ATABLE\n" +

http://git-wip-us.apache.org/repos/asf/phoenix/blob/e574d1ea/phoenix-core/src/it/java/org/apache/phoenix/end2end/RowValueConstructorIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/RowValueConstructorIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/RowValueConstructorIT.java
index bdae151..041725c 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/RowValueConstructorIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/RowValueConstructorIT.java
@@ -1091,5 +1091,141 @@ public class RowValueConstructorIT extends BaseClientManagedTimeIT {
         assertEquals(5, rs.getInt(2));
         conn.close();
     }
+    
+    @Test 
+    public void testInListOfRVCColumnValuesSmallerLengthThanSchema() throws Exception {
+        String tenantId = "ABC";
+        String tableDDL = "CREATE TABLE t (tenantId char(15) NOT NULL, pk2 char(15) NOT NULL, pk3 INTEGER NOT NULL, c1 INTEGER constraint pk primary key (tenantId,pk2,pk3))";
+        createTestTable(getUrl(), tableDDL, null, nextTimestamp());
+
+        Connection conn = nextConnection(getUrl());
+        conn.createStatement().executeUpdate("upsert into t (tenantId, pk2, pk3, c1) values ('ABC', 'hel1', 1, 1)");
+        conn.createStatement().executeUpdate("upsert into t (tenantId, pk2, pk3, c1) values ('ABC', 'hel2', 2, 2)");
+        conn.createStatement().executeUpdate("upsert into t (tenantId, pk2, pk3, c1) values ('ABC', 'hel3', 3, 3)");
+        conn.createStatement().executeUpdate("upsert into t (tenantId, pk2, pk3, c1) values ('ABC', 'hel4', 4, 4)");
+        conn.createStatement().executeUpdate("upsert into t (tenantId, pk2, pk3, c1) values ('ABC', 'hel5', 5, 5)");
+        conn.commit();
+        conn.close();
+
+        conn = nextConnection(getUrl());
+        //order by needed on the query to make the order of rows returned deterministic.
+        PreparedStatement stmt = conn.prepareStatement("select pk2, pk3 from t WHERE (tenantId, pk2, pk3) IN ((?, ?, ?), (?, ?, ?)) ORDER BY PK2");
+        stmt.setString(1, tenantId);
+        stmt.setString(2, "hel3");
+        stmt.setInt(3, 3);
+        stmt.setString(4, tenantId);
+        stmt.setString(5, "hel5");
+        stmt.setInt(6, 5);
+
+        ResultSet rs = stmt.executeQuery();
+        assertTrue(rs.next());
+        assertEquals("hel3", rs.getString(1));
+        assertEquals(3, rs.getInt(2));
+        assertTrue(rs.next());
+        assertEquals("hel5", rs.getString(1));
+        assertEquals(5, rs.getInt(2));
+        conn.close();
+    }
+    
+    @Test
+    public void testRVCWithColumnValuesOfSmallerLengthThanSchema() throws Exception {
+        testRVCWithComparisonOps(true);
+    }
+    
+    @Test
+    public void testRVCWithColumnValuesEqualToLengthInSchema() throws Exception {
+        testRVCWithComparisonOps(false);
+    }
+    
+    private void testRVCWithComparisonOps(boolean columnValueLengthSmaller) throws Exception {
+        String tenantId = "ABC";
+        String tableDDLFormat = "CREATE TABLE t (tenantId char(%s) NOT NULL, pk2 char(%s) NOT NULL, pk3 INTEGER NOT NULL, c1 INTEGER constraint pk primary key (tenantId,pk2,pk3))";
+        String tableDDL;
+        if (columnValueLengthSmaller) {
+            tableDDL = String.format(tableDDLFormat, 15, 15);
+        } else {
+            tableDDL = String.format(tableDDLFormat, 3, 5);
+        }
+        createTestTable(getUrl(), tableDDL, null, nextTimestamp());
+
+        Connection conn = nextConnection(getUrl());
+        conn.createStatement().executeUpdate("upsert into t (tenantId, pk2, pk3, c1) values ('ABC', 'helo1', 1, 1)");
+        conn.createStatement().executeUpdate("upsert into t (tenantId, pk2, pk3, c1) values ('ABC', 'helo2', 2, 2)");
+        conn.createStatement().executeUpdate("upsert into t (tenantId, pk2, pk3, c1) values ('ABC', 'helo3', 3, 3)");
+        conn.createStatement().executeUpdate("upsert into t (tenantId, pk2, pk3, c1) values ('ABC', 'helo4', 4, 4)");
+        conn.createStatement().executeUpdate("upsert into t (tenantId, pk2, pk3, c1) values ('ABC', 'helo5', 5, 5)");
+        conn.commit();
+        conn.close();
 
+        conn = nextConnection(getUrl());
+        
+        // >
+        PreparedStatement stmt = conn.prepareStatement("select pk2, pk3 from t WHERE (tenantId, pk2, pk3) > (?, ?, ?)");
+        stmt.setString(1, tenantId);
+        stmt.setString(2, "helo3");
+        stmt.setInt(3, 3);
+        
+        ResultSet rs = stmt.executeQuery();
+        assertTrue(rs.next());
+        assertEquals("helo4", rs.getString(1));
+        assertEquals(4, rs.getInt(2));
+        assertTrue(rs.next());
+        assertEquals("helo5", rs.getString(1));
+        assertEquals(5, rs.getInt(2));
+        assertFalse(rs.next());
+        
+        // >=
+        stmt = conn.prepareStatement("select pk2, pk3 from t WHERE (tenantId, pk2, pk3) >= (?, ?, ?)");
+        stmt.setString(1, tenantId);
+        stmt.setString(2, "helo4");
+        stmt.setInt(3, 4);
+        
+        rs = stmt.executeQuery();
+        assertTrue(rs.next());
+        assertEquals("helo4", rs.getString(1));
+        assertEquals(4, rs.getInt(2));
+        assertTrue(rs.next());
+        assertEquals("helo5", rs.getString(1));
+        assertEquals(5, rs.getInt(2));
+        assertFalse(rs.next());
+        
+        // <
+        stmt = conn.prepareStatement("select pk2, pk3 from t WHERE (tenantId, pk2, pk3) < (?, ?, ?)");
+        stmt.setString(1, tenantId);
+        stmt.setString(2, "helo2");
+        stmt.setInt(3, 2);
+        
+        rs = stmt.executeQuery();
+        assertTrue(rs.next());
+        assertEquals("helo1", rs.getString(1));
+        assertEquals(1, rs.getInt(2));
+        assertFalse(rs.next());
+        
+        // <=
+        stmt = conn.prepareStatement("select pk2, pk3 from t WHERE (tenantId, pk2, pk3) <= (?, ?, ?)");
+        stmt.setString(1, tenantId);
+        stmt.setString(2, "helo2");
+        stmt.setInt(3, 2);
+        rs = stmt.executeQuery(); 
+        
+        assertTrue(rs.next());
+        assertEquals("helo1", rs.getString(1));
+        assertEquals(1, rs.getInt(2));
+        assertTrue(rs.next());
+        assertEquals("helo2", rs.getString(1));
+        assertEquals(2, rs.getInt(2));
+        assertFalse(rs.next());
+        
+        // =
+        stmt = conn.prepareStatement("select pk2, pk3 from t WHERE (tenantId, pk2, pk3) = (?, ?, ?)");
+        stmt.setString(1, tenantId);
+        stmt.setString(2, "helo4");
+        stmt.setInt(3, 4);
+        
+        rs = stmt.executeQuery();
+        assertTrue(rs.next());
+        assertEquals("helo4", rs.getString(1));
+        assertEquals(4, rs.getInt(2));
+        assertFalse(rs.next());
+    }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/e574d1ea/phoenix-core/src/it/java/org/apache/phoenix/end2end/VariableLengthPKIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/VariableLengthPKIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/VariableLengthPKIT.java
index 460d417..5fe71a8 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/VariableLengthPKIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/VariableLengthPKIT.java
@@ -1451,13 +1451,21 @@ public class VariableLengthPKIT extends BaseClientManagedTimeIT {
             "SELECT CASE WHEN 'a' IN (null,'b') THEN '1' ELSE '2' END FROM BTABLE LIMIT 1",
             "SELECT CASE WHEN NOT 'a' IN ('c','b') THEN '1' ELSE '2' END FROM BTABLE LIMIT 1",
             "SELECT CASE WHEN 1 IN ('foo',2,1) THEN '1' ELSE '2' END FROM BTABLE LIMIT 1",
+            "SELECT CASE WHEN NOT null IN ('c','b') THEN '1' ELSE '2' END FROM BTABLE LIMIT 1",
+            "SELECT CASE WHEN NOT null IN (null,'c','b') THEN '1' ELSE '2' END FROM BTABLE LIMIT 1",
+            "SELECT CASE WHEN null IN (null,'c','b') THEN '1' ELSE '2' END FROM BTABLE LIMIT 1",
+            "SELECT CASE WHEN 'a' IN (null,1) THEN '1' ELSE '2' END FROM BTABLE LIMIT 1",
         };
         String result[] = {
             "1",
-            "2",
+            "1",
             "2",
             "1",
             "1",
+            "2",
+            "2",
+            "2",
+            "2"
         };
         assertEquals(query.length,result.length);
         String url = getUrl() + ";" + PhoenixRuntime.CURRENT_SCN_ATTRIB + "=" + (ts + 5); // Run query at timestamp 5

http://git-wip-us.apache.org/repos/asf/phoenix/blob/e574d1ea/phoenix-core/src/main/java/org/apache/phoenix/compile/ScanRanges.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/ScanRanges.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/ScanRanges.java
index ddff6da..21ccff2 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/ScanRanges.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/ScanRanges.java
@@ -64,8 +64,14 @@ public class ScanRanges {
                 keyRanges.add(KeyRange.getKeyRange(key));
             }
             ranges = Collections.singletonList(keyRanges);
-            schema = SchemaUtil.VAR_BINARY_SCHEMA;
-            slotSpan = ScanUtil.SINGLE_COLUMN_SLOT_SPAN;
+            if (keys.size() > 1) {
+                schema = SchemaUtil.VAR_BINARY_SCHEMA;
+                slotSpan = ScanUtil.SINGLE_COLUMN_SLOT_SPAN;
+            } else {
+                // Keep original schema and don't use skip scan as it's not necessary
+                // when there's a single key.
+                slotSpan = new int[] {schema.getMaxFields()-1};
+            }
         } else if (nBuckets != null) {
             List<List<KeyRange>> saltedRanges = Lists.newArrayListWithExpectedSize(ranges.size());
             saltedRanges.add(SaltingUtil.generateAllSaltingRanges(nBuckets));
@@ -134,7 +140,7 @@ public class ScanRanges {
         }
         boolean hasRangeKey = false, useSkipScan = false;
         for (List<KeyRange> orRanges : ranges) {
-            useSkipScan |= orRanges.size() > 1 | hasRangeKey;
+            useSkipScan |= (orRanges.size() > 1 || hasRangeKey);
             if (useSkipScan) {
                 return true;
             }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/e574d1ea/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 b6b9090..9c20610 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
@@ -61,6 +61,7 @@ import org.apache.phoenix.schema.tuple.Tuple;
 import org.apache.phoenix.util.ByteUtil;
 import org.apache.phoenix.util.MetaDataUtil;
 import org.apache.phoenix.util.SchemaUtil;
+import org.apache.phoenix.util.StringUtil;
 
 import com.google.common.base.Preconditions;
 import com.google.common.collect.Iterators;
@@ -830,18 +831,22 @@ public class WhereOptimizer {
             // and the key length doesn't match the column length, the expression can
             // never be true.
             // An zero length byte literal is null which can never be compared against as true
-            Integer childNodeFixedLength = node.getChildren().get(0).getMaxLength();
+            Expression firstChild = node.getChildren().get(0);
+            Integer childNodeFixedLength = firstChild.getDataType().isFixedWidth() ? firstChild.getMaxLength() : null;
             if (childNodeFixedLength != null && key.length > childNodeFixedLength) {
                 return DEGENERATE_KEY_PARTS;
             }
             // TODO: is there a case where we'd need to go through the childPart to calculate the key range?
             PColumn column = childSlot.getKeyPart().getColumn();
             PDataType type = column.getDataType();
-            KeyRange keyRange = type.getKeyRange(key, true, ByteUtil.nextKey(key), false);
+            byte[] lowerRange = key;
+            byte[] upperRange = ByteUtil.nextKey(key);
             Integer columnFixedLength = column.getMaxLength();
-            if (columnFixedLength != null) {
-                keyRange = keyRange.fill(columnFixedLength);
+            if (type.isFixedWidth() && columnFixedLength != null) {
+                lowerRange = StringUtil.padChar(lowerRange, columnFixedLength);
+                upperRange = StringUtil.padChar(upperRange, columnFixedLength);
             }
+            KeyRange keyRange = type.getKeyRange(lowerRange, true, upperRange, false);
             // Only extract LIKE expression if pattern ends with a wildcard and everything else was extracted
             return newKeyParts(childSlot, node.endsWithOnlyWildcard() ? node : null, keyRange);
         }
@@ -1081,15 +1086,16 @@ public class WhereOptimizer {
             public KeyRange getKeyRange(CompareOp op, Expression rhs) {
                 ImmutableBytesWritable ptr = new ImmutableBytesWritable();
                 rhs.evaluate(null, ptr);
-                byte[] key = ByteUtil.copyKeyBytesIfNecessary(ptr);
                 // If the column is fixed width, fill is up to it's byte size
                 PDataType type = getColumn().getDataType();
                 if (type.isFixedWidth()) {
                     Integer length = getColumn().getMaxLength();
                     if (length != null) {
-                        key = ByteUtil.fillKey(key, length);
+                        // Go through type to pad as the fill character depends on the type.
+                        type.pad(ptr, length);
                     }
                 }
+                byte[] key = ByteUtil.copyKeyBytesIfNecessary(ptr);
                 return ByteUtil.getKeyRange(key, op, type);
             }
 
@@ -1216,7 +1222,8 @@ public class WhereOptimizer {
                                     // use to compute the next key when we evaluate the RHS row value constructor
                                     // below.  We could create a new childPart with a delegate column that returns
                                     // null for getByteSize().
-                                    if (lhs.getDataType().isFixedWidth() && lhs.getMaxLength() != null && key.length != lhs.getMaxLength()) {
+                                    if (lhs.getDataType().isFixedWidth() && lhs.getMaxLength() != null && key.length > lhs.getMaxLength()) {
+                                        // Don't use PDataType.pad(), as this only grows the value, while this is shrinking it.
                                         key = Arrays.copyOf(key, lhs.getMaxLength());
                                     }
                                     ptr.set(key);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/e574d1ea/phoenix-core/src/main/java/org/apache/phoenix/expression/CaseExpression.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/CaseExpression.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/CaseExpression.java
index 41ad029..c3a0c0b 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/expression/CaseExpression.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/CaseExpression.java
@@ -124,12 +124,6 @@ public class CaseExpression extends BaseCompoundExpression {
         return returnType;
     }
 
-//    @Override TODO: do we need this?
-//    public Integer getMaxLength() {
-//        return children.get(0).getMaxLength();
-//    }
-
-    
     @Override
     public void reset() {
         foundIndex = false;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/e574d1ea/phoenix-core/src/main/java/org/apache/phoenix/expression/InListExpression.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/InListExpression.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/InListExpression.java
index 011da55..c171153 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/expression/InListExpression.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/InListExpression.java
@@ -53,9 +53,7 @@ public class InListExpression extends BaseSingleExpression {
     private ImmutableBytesPtr minValue;
     private ImmutableBytesPtr maxValue;
     private int valuesByteLength;
-    private boolean containsNull;
     private int fixedWidth = -1;
-    private ImmutableBytesPtr value = new ImmutableBytesPtr();
     private List<Expression> keyExpressions; // client side only
 
     public static Expression create (List<Expression> children, boolean isNegate, ImmutableBytesWritable ptr) throws SQLException {
@@ -113,9 +111,7 @@ public class InListExpression extends BaseSingleExpression {
             ImmutableBytesPtr ptr = new ImmutableBytesPtr();
             Expression child = keyExpressions.get(i);
             child.evaluate(null, ptr);
-            if (ptr.getLength() == 0) {
-                containsNull = true;
-            } else {
+            if (ptr.getLength() > 0) { // filter null as it has no impact
                 if (values.add(ptr)) {
                     int length = ptr.getLength();
                     if (fixedWidth == -1) {
@@ -150,13 +146,12 @@ public class InListExpression extends BaseSingleExpression {
         if (!getChild().evaluate(tuple, ptr)) {
             return false;
         }
-        value.set(ptr);
-        if (values.contains(value)) {
-            ptr.set(PDataType.TRUE_BYTES);
+        if (ptr.getLength() == 0) { // null IN (...) is always null
+            ptr.set(ByteUtil.EMPTY_BYTE_ARRAY);
             return true;
         }
-        if (containsNull) { // If any null value and value not found
-            ptr.set(ByteUtil.EMPTY_BYTE_ARRAY);
+        if (values.contains(ptr)) {
+            ptr.set(PDataType.TRUE_BYTES);
             return true;
         }
         ptr.set(PDataType.FALSE_BYTES);
@@ -167,7 +162,6 @@ public class InListExpression extends BaseSingleExpression {
     public int hashCode() {
         final int prime = 31;
         int result = 1;
-        result = prime * result + (containsNull ? 1231 : 1237);
         result = prime * result + values.hashCode();
         return result;
     }
@@ -178,7 +172,6 @@ public class InListExpression extends BaseSingleExpression {
         if (obj == null) return false;
         if (getClass() != obj.getClass()) return false;
         InListExpression other = (InListExpression)obj;
-        if (containsNull != other.containsNull) return false;
         if (!values.equals(other.values)) return false;
         return true;
     }
@@ -188,11 +181,6 @@ public class InListExpression extends BaseSingleExpression {
         return PDataType.BOOLEAN;
     }
 
-    @Override
-    public boolean isNullable() {
-        return super.isNullable() || containsNull;
-    }
-
     private int readValue(DataInput input, byte[] valuesBytes, int offset, ImmutableBytesPtr ptr) throws IOException {
         int valueLen = fixedWidth == -1 ? WritableUtils.readVInt(input) : fixedWidth;
         values.add(new ImmutableBytesPtr(valuesBytes,offset,valueLen));
@@ -202,7 +190,7 @@ public class InListExpression extends BaseSingleExpression {
     @Override
     public void readFields(DataInput input) throws IOException {
         super.readFields(input);
-        containsNull = input.readBoolean();
+        input.readBoolean(); // Unused, but left for b/w compat. TODO: remove in next major release
         fixedWidth = WritableUtils.readVInt(input);
         byte[] valuesBytes = Bytes.readByteArray(input);
         valuesByteLength = valuesBytes.length;
@@ -229,7 +217,7 @@ public class InListExpression extends BaseSingleExpression {
     @Override
     public void write(DataOutput output) throws IOException {
         super.write(output);
-        output.writeBoolean(containsNull);
+        output.writeBoolean(false); // Unused, but left for b/w compat. TODO: remove in next major release
         WritableUtils.writeVInt(output, fixedWidth);
         WritableUtils.writeVInt(output, valuesByteLength);
         for (ImmutableBytesPtr ptr : values) {
@@ -271,9 +259,6 @@ public class InListExpression extends BaseSingleExpression {
         Expression firstChild = children.get(0);
         PDataType type = firstChild.getDataType();
         StringBuilder buf = new StringBuilder(firstChild + " IN (");
-        if (containsNull) {
-            buf.append("null,");
-        }
         for (ImmutableBytesPtr value : values) {
             if (firstChild.getSortOrder() != null) {
                 type.coerceBytes(value, type, firstChild.getSortOrder(), SortOrder.getDefault());

http://git-wip-us.apache.org/repos/asf/phoenix/blob/e574d1ea/phoenix-core/src/main/java/org/apache/phoenix/expression/function/PrefixFunction.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/PrefixFunction.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/PrefixFunction.java
index 156f860..40d2467 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/PrefixFunction.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/PrefixFunction.java
@@ -22,13 +22,13 @@ import java.util.Collections;
 import java.util.List;
 
 import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
-import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.phoenix.compile.KeyPart;
 import org.apache.phoenix.expression.Expression;
 import org.apache.phoenix.query.KeyRange;
 import org.apache.phoenix.schema.PColumn;
 import org.apache.phoenix.schema.PDataType;
 import org.apache.phoenix.util.ByteUtil;
+import org.apache.phoenix.util.StringUtil;
 
 abstract public class PrefixFunction extends ScalarFunction {
     public PrefixFunction() {
@@ -47,13 +47,6 @@ abstract public class PrefixFunction extends ScalarFunction {
         return false;
     }
 
-    private static byte[] evaluateExpression(Expression rhs) {
-        ImmutableBytesWritable ptr = new ImmutableBytesWritable();
-        rhs.evaluate(null, ptr);
-        byte[] key = ByteUtil.copyKeyBytesIfNecessary(ptr);
-        return key;
-    }
-    
     @Override
     public KeyPart newKeyPart(final KeyPart childPart) {
         return new KeyPart() {
@@ -71,27 +64,35 @@ abstract public class PrefixFunction extends ScalarFunction {
 
             @Override
             public KeyRange getKeyRange(CompareOp op, Expression rhs) {
-                byte[] key;
-                KeyRange range;
+                byte[] lowerRange = KeyRange.UNBOUND;
+                byte[] upperRange = KeyRange.UNBOUND;
+                boolean lowerInclusive = true;
                 PDataType type = getColumn().getDataType();
                 switch (op) {
                 case EQUAL:
-                    key = evaluateExpression(rhs);
-                    range = type.getKeyRange(key, true, ByteUtil.nextKey(key), false);
+                    lowerRange = evaluateExpression(rhs);
+                    upperRange = ByteUtil.nextKey(lowerRange);
                     break;
                 case GREATER:
-                    key = evaluateExpression(rhs);
-                    range = type.getKeyRange(ByteUtil.nextKey(key), true, KeyRange.UNBOUND, false);
+                    lowerRange = ByteUtil.nextKey(evaluateExpression(rhs));
                     break;
                 case LESS_OR_EQUAL:
-                    key = evaluateExpression(rhs);
-                    range = type.getKeyRange(KeyRange.UNBOUND, false, ByteUtil.nextKey(key), false);
+                    upperRange = ByteUtil.nextKey(evaluateExpression(rhs));
+                    lowerInclusive = false;
                     break;
                 default:
                     return childPart.getKeyRange(op, rhs);
                 }
                 Integer length = getColumn().getMaxLength();
-                return length == null || !type.isFixedWidth() ? range : range.fill(length);
+                if (type.isFixedWidth() && length != null) {
+                    if (lowerRange != KeyRange.UNBOUND) {
+                        lowerRange = StringUtil.padChar(lowerRange, length);
+                    }
+                    if (upperRange != KeyRange.UNBOUND) {
+                        upperRange = StringUtil.padChar(upperRange, length);
+                    }
+                }
+                return KeyRange.getKeyRange(lowerRange, lowerInclusive, upperRange, false);
             }
         };
     }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/e574d1ea/phoenix-core/src/main/java/org/apache/phoenix/expression/function/RTrimFunction.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/RTrimFunction.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/RTrimFunction.java
index 1a46334..a19a1bf 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/RTrimFunction.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/RTrimFunction.java
@@ -104,24 +104,33 @@ public class RTrimFunction extends ScalarFunction {
         return new KeyPart() {
             @Override
             public KeyRange getKeyRange(CompareOp op, Expression rhs) {
-                ImmutableBytesWritable ptr = new ImmutableBytesWritable();
-                rhs.evaluate(null, ptr);
-                byte[] key = ByteUtil.copyKeyBytesIfNecessary(ptr);
+                byte[] lowerRange = KeyRange.UNBOUND;
+                byte[] upperRange = KeyRange.UNBOUND;
+                boolean lowerInclusive = true;
+                
                 PDataType type = getColumn().getDataType();
-                KeyRange range;
                 switch (op) {
                 case EQUAL:
-                    range = type.getKeyRange(key, true, ByteUtil.nextKey(ByteUtil.concat(key, new byte[] {StringUtil.SPACE_UTF8})), false);
+                    lowerRange = evaluateExpression(rhs);
+                    upperRange = ByteUtil.nextKey(ByteUtil.concat(lowerRange, new byte[] {StringUtil.SPACE_UTF8}));
                     break;
                 case LESS_OR_EQUAL:
-                    range = type.getKeyRange(KeyRange.UNBOUND, false, ByteUtil.nextKey(ByteUtil.concat(key, new byte[] {StringUtil.SPACE_UTF8})), false);
+                    lowerInclusive = false;
+                    upperRange = ByteUtil.nextKey(ByteUtil.concat(evaluateExpression(rhs), new byte[] {StringUtil.SPACE_UTF8}));
                     break;
                 default:
-                    range = childPart.getKeyRange(op, rhs);
-                    break;
+                    return childPart.getKeyRange(op, rhs);
                 }
                 Integer length = getColumn().getMaxLength();
-                return length == null || !type.isFixedWidth() ? range : range.fill(length);
+                if (type.isFixedWidth() && length != null) {
+                    if (lowerRange != KeyRange.UNBOUND) {
+                        lowerRange = StringUtil.padChar(lowerRange, length);
+                    }
+                    if (upperRange != KeyRange.UNBOUND) {
+                        upperRange = StringUtil.padChar(upperRange, length);
+                    }
+                }
+                return KeyRange.getKeyRange(lowerRange, lowerInclusive, upperRange, false);
             }
 
             @Override

http://git-wip-us.apache.org/repos/asf/phoenix/blob/e574d1ea/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ScalarFunction.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ScalarFunction.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ScalarFunction.java
index 7f81d96..37b8816 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ScalarFunction.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ScalarFunction.java
@@ -19,9 +19,11 @@ package org.apache.phoenix.expression.function;
 
 import java.util.List;
 
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.phoenix.compile.KeyPart;
 import org.apache.phoenix.expression.Expression;
 import org.apache.phoenix.expression.visitor.ExpressionVisitor;
+import org.apache.phoenix.util.ByteUtil;
 
 
 public abstract class ScalarFunction extends FunctionExpression {
@@ -34,6 +36,13 @@ public abstract class ScalarFunction extends FunctionExpression {
         super(children);
     }
     
+    protected static byte[] evaluateExpression(Expression rhs) {
+        ImmutableBytesWritable ptr = new ImmutableBytesWritable();
+        rhs.evaluate(null, ptr);
+        byte[] key = ByteUtil.copyKeyBytesIfNecessary(ptr);
+        return key;
+    }
+    
     @Override
     public final <T> T accept(ExpressionVisitor<T> visitor) {
         List<T> l = acceptChildren(visitor, visitor.visitEnter(this));

http://git-wip-us.apache.org/repos/asf/phoenix/blob/e574d1ea/phoenix-core/src/main/java/org/apache/phoenix/query/KeyRange.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/KeyRange.java b/phoenix-core/src/main/java/org/apache/phoenix/query/KeyRange.java
index 13e034d..68f786a 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/query/KeyRange.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/KeyRange.java
@@ -528,35 +528,6 @@ public class KeyRange implements Writable {
         return tmp2;
     }
     
-    /**
-     * Fill both upper and lower range of keyRange to keyLength bytes.
-     * If the upper bound is inclusive, it must be filled such that an
-     * intersection with a longer key would still match if the shorter
-     * length matches.  For example: (*,00C] intersected with [00Caaa,00Caaa]
-     * should still return [00Caaa,00Caaa] since the 00C matches and is
-     * inclusive.
-     * @param keyLength
-     * @return the newly filled KeyRange
-     */
-    public KeyRange fill(int keyLength) {
-        byte[] lowerRange = this.getLowerRange();
-        byte[] newLowerRange = lowerRange;
-        if (!this.lowerUnbound()) {
-            // If lower range is inclusive, fill with 0x00 since conceptually these bytes are included in the range
-            newLowerRange = ByteUtil.fillKey(lowerRange, keyLength);
-        }
-        byte[] upperRange = this.getUpperRange();
-        byte[] newUpperRange = upperRange;
-        if (!this.upperUnbound()) {
-            // If upper range is inclusive, fill with 0xFF since conceptually these bytes are included in the range
-            newUpperRange = ByteUtil.fillKey(upperRange, keyLength);
-        }
-        if (newLowerRange != lowerRange || newUpperRange != upperRange) {
-            return KeyRange.getKeyRange(newLowerRange, this.isLowerInclusive(), newUpperRange, this.isUpperInclusive());
-        }
-        return this;
-    }
-    
     public KeyRange invert() {
         byte[] lower = this.getLowerRange();
         if (!this.lowerUnbound()) {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/e574d1ea/phoenix-core/src/main/java/org/apache/phoenix/schema/PDataType.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/PDataType.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/PDataType.java
index c2fcbf0..a915b2f 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/PDataType.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/PDataType.java
@@ -26,6 +26,7 @@ import java.sql.Time;
 import java.sql.Timestamp;
 import java.sql.Types;
 import java.text.Format;
+import java.util.Arrays;
 import java.util.Map;
 
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
@@ -180,6 +181,17 @@ public enum PDataType {
      */
     CHAR("CHAR", Types.CHAR, String.class, null) { // Delegate to VARCHAR
         @Override
+        public void pad(ImmutableBytesWritable ptr, Integer maxLength) {
+            if (ptr.getLength() >= maxLength) {
+                return;
+            }
+            byte[] newBytes = new byte[maxLength];
+            System.arraycopy(ptr.get(), ptr.getOffset(), newBytes, 0, ptr.getLength());
+            Arrays.fill(newBytes, ptr.getLength(), maxLength, StringUtil.SPACE_UTF8);
+            ptr.set(newBytes);
+        }
+
+        @Override
         public Object pad(Object object, Integer maxLength) {
             String s = (String) object;
             if (s == null) {
@@ -3124,6 +3136,16 @@ public enum PDataType {
     },
     BINARY("BINARY", Types.BINARY, byte[].class, null) {
         @Override
+        public void pad(ImmutableBytesWritable ptr, Integer maxLength) {
+            if (ptr.getLength() >= maxLength) {
+                return;
+            }
+            byte[] newBytes = new byte[maxLength];
+            System.arraycopy(ptr.get(), ptr.getOffset(), newBytes, 0, ptr.getLength());
+            ptr.set(newBytes);
+        }
+        
+        @Override
         public Object pad(Object object, Integer maxLength) {
             byte[] b = (byte[]) object;
             if (b == null) {
@@ -7136,4 +7158,7 @@ public enum PDataType {
         return object;
     }
     
+    public void pad(ImmutableBytesWritable ptr, Integer maxLength) {
+    }
+    
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/e574d1ea/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 9fd6982..e348a4c 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
@@ -621,18 +621,19 @@ public class PTableImpl implements PTable {
                 deleteQuietly(unsetValues, kvBuilder, kvBuilder.buildDeleteColumns(keyPtr, column
                         .getFamilyName().getBytesPtr(), column.getName().getBytesPtr(), ts));
             } else {
+                ImmutableBytesWritable ptr = new ImmutableBytesWritable(byteValue);
             	Integer	maxLength = column.getMaxLength();
             	if (type.isFixedWidth() && maxLength != null) {
-    				if (byteValue.length <= maxLength) { 
-                        byteValue = StringUtil.padChar(byteValue, maxLength);
-                    } else if (byteValue.length > maxLength) {
+    				if (ptr.getLength() <= maxLength) {
+                        type.pad(ptr, maxLength);
+                    } else if (ptr.getLength() > maxLength) {
                         throw new ConstraintViolationException(name.getString() + "." + column.getName().getString() + " may not exceed " + maxLength + " bytes (" + type.toObject(byteValue) + ")");
                     }
             	}
                 removeIfPresent(unsetValues, family, qualifier);
                 addQuietly(setValues, kvBuilder, kvBuilder.buildPut(keyPtr, column.getFamilyName()
                         .getBytesPtr(),
-                        column.getName().getBytesPtr(), ts, new ImmutableBytesPtr(byteValue)));
+                        column.getName().getBytesPtr(), ts, ptr));
             }
         }
         

http://git-wip-us.apache.org/repos/asf/phoenix/blob/e574d1ea/phoenix-core/src/main/java/org/apache/phoenix/util/ByteUtil.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/util/ByteUtil.java b/phoenix-core/src/main/java/org/apache/phoenix/util/ByteUtil.java
index fd66d8a..82be510 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/util/ByteUtil.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/ByteUtil.java
@@ -462,8 +462,7 @@ public class ByteUtil {
     }
 
     /**
-     * Expand the key to length bytes using the fillByte to fill the
-     * bytes beyond the current key length.
+     * Expand the key to length bytes using a null byte.
      */
     public static byte[] fillKey(byte[] key, int length) {
         if(key.length > length) {
@@ -478,6 +477,22 @@ public class ByteUtil {
     }
 
     /**
+     * Expand the key to length bytes using the fillByte to fill the
+     * bytes beyond the current key length.
+     */
+    public static void nullPad(ImmutableBytesWritable ptr, int length) {
+        if(ptr.getLength() > length) {
+            throw new IllegalStateException();
+        }
+        if (ptr.getLength() == length) {
+            return;
+        }
+        byte[] newBound = new byte[length];
+        System.arraycopy(ptr.get(), ptr.getOffset(), newBound, 0, ptr.getLength());
+        ptr.set(newBound);
+    }
+
+    /**
      * Get the size in bytes of the UTF-8 encoded CharSequence
      * @param sequence the CharSequence
      */

http://git-wip-us.apache.org/repos/asf/phoenix/blob/e574d1ea/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 b42212f..5375169 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
@@ -216,7 +216,7 @@ public class ScanUtil {
         for (int i = 0; i < position.length; i++) {
             position[i] = bound == Bound.LOWER ? 0 : slots.get(i).size()-1;
             KeyRange range = slots.get(i).get(position[i]);
-            maxLength += range.getRange(bound).length + (schema.getField(i).getDataType().isFixedWidth() ? 0 : 1);
+            maxLength += range.getRange(bound).length + (schema.getField(i + slotSpan[i]).getDataType().isFixedWidth() ? 0 : 1);
         }
         byte[] key = new byte[maxLength];
         int length = setKey(schema, slots, slotSpan, position, bound, key, 0, 0, position.length);
@@ -259,13 +259,14 @@ public class ScanUtil {
         // but the index for the field it represents in the schema
         // should be incremented by 1 + value in the current slotSpan index
         // slotSpan stores the number of columns beyond one that the range spans
-        for (int i = slotStartIndex; i < slotEndIndex; i++) {
+        int i = slotStartIndex, fieldIndex = slotStartIndex;
+        for (i = slotStartIndex; i < slotEndIndex; i++) {
             // Build up the key by appending the bound of each key range
             // from the current position of each slot. 
             KeyRange range = slots.get(i).get(position[i]);
             // Use last slot in a multi-span column to determine if fixed width
-            boolean isFixedWidth = schema.getField(schemaStartIndex + slotSpan[i]).getDataType().isFixedWidth();
-            schemaStartIndex += slotSpan[i] + 1;
+            boolean isFixedWidth = schema.getField(fieldIndex + slotSpan[i]).getDataType().isFixedWidth();
+            fieldIndex += slotSpan[i] + 1;
             /*
              * If the current slot is unbound then stop if:
              * 1) setting the upper bound. There's no value in
@@ -302,7 +303,7 @@ public class ScanUtil {
             lastInclusiveUpperSingleKey = range.isSingleKey() && inclusiveUpper;
             anyInclusiveUpperRangeKey |= !range.isSingleKey() && inclusiveUpper;
             
-            if (!isFixedWidth && ( i < schema.getMaxFields()-1 || inclusiveUpper || exclusiveLower)) {
+            if (!isFixedWidth && ( fieldIndex < schema.getMaxFields() || inclusiveUpper || exclusiveLower)) {
                 key[offset++] = QueryConstants.SEPARATOR_BYTE;
                 // Set lastInclusiveUpperSingleKey back to false if this is the last pk column
                 // as we don't want to increment the null byte in this case
@@ -336,10 +337,11 @@ public class ScanUtil {
         // after the table has data, in which case there won't be a separator
         // byte.
         if (bound == Bound.LOWER) {
-            while (schemaStartIndex > 0 && offset > byteOffset && 
-                    !schema.getField(--schemaStartIndex).getDataType().isFixedWidth() && 
+            while (--i >= schemaStartIndex && offset > byteOffset && 
+                    !schema.getField(--fieldIndex).getDataType().isFixedWidth() && 
                     key[offset-1] == QueryConstants.SEPARATOR_BYTE) {
                 offset--;
+                fieldIndex -= slotSpan[i];
             }
         }
         return offset - byteOffset;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/e574d1ea/phoenix-core/src/test/java/org/apache/phoenix/compile/QueryCompilerTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/compile/QueryCompilerTest.java b/phoenix-core/src/test/java/org/apache/phoenix/compile/QueryCompilerTest.java
index 4745c44..07564d4 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/compile/QueryCompilerTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/compile/QueryCompilerTest.java
@@ -854,7 +854,7 @@ public class QueryCompilerTest extends BaseConnectionlessQueryTest {
         List<Object> binds = Collections.emptyList();
         Scan scan = compileQuery(query, binds);
         assertArrayEquals(ByteUtil.concat(Bytes.toBytes("abc")), scan.getStartRow());
-        assertArrayEquals(ByteUtil.concat(ByteUtil.nextKey(Bytes.toBytes("abc")),QueryConstants.SEPARATOR_BYTE_ARRAY), scan.getStopRow());
+        assertArrayEquals(ByteUtil.concat(ByteUtil.nextKey(Bytes.toBytes("abc")), QueryConstants.SEPARATOR_BYTE_ARRAY),scan.getStopRow());
         assertTrue(scan.getFilter() != null);
 
         query = "SELECT host FROM ptsdb WHERE regexp_substr(inst, '[a-zA-Z]+', 0) = 'abc'";
@@ -962,8 +962,8 @@ public class QueryCompilerTest extends BaseConnectionlessQueryTest {
         String query = "SELECT inst FROM ptsdb WHERE substr(inst, 0, 3) = 'abc'";
         List<Object> binds = Collections.emptyList();
         Scan scan = compileQuery(query, binds);
-        assertArrayEquals(ByteUtil.concat(Bytes.toBytes("abc")), scan.getStartRow());
-        assertArrayEquals(ByteUtil.concat(Bytes.toBytes("abd"),QueryConstants.SEPARATOR_BYTE_ARRAY), scan.getStopRow());
+        assertArrayEquals(Bytes.toBytes("abc"), scan.getStartRow());
+        assertArrayEquals(ByteUtil.concat(ByteUtil.nextKey(Bytes.toBytes("abc")), QueryConstants.SEPARATOR_BYTE_ARRAY), scan.getStopRow());
         assertTrue(scan.getFilter() == null); // Extracted.
     }
 
@@ -973,7 +973,7 @@ public class QueryCompilerTest extends BaseConnectionlessQueryTest {
         List<Object> binds = Collections.emptyList();
         Scan scan = compileQuery(query, binds);
         assertArrayEquals(ByteUtil.concat(Bytes.toBytes("abc")), scan.getStartRow());
-        assertArrayEquals(ByteUtil.concat(ByteUtil.nextKey(Bytes.toBytes("abc ")),QueryConstants.SEPARATOR_BYTE_ARRAY), scan.getStopRow());
+        assertArrayEquals(ByteUtil.concat(ByteUtil.nextKey(Bytes.toBytes("abc ")), QueryConstants.SEPARATOR_BYTE_ARRAY), scan.getStopRow());
         assertNotNull(scan.getFilter());
     }
     

http://git-wip-us.apache.org/repos/asf/phoenix/blob/e574d1ea/phoenix-core/src/test/java/org/apache/phoenix/compile/StatementHintsCompilationTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/compile/StatementHintsCompilationTest.java b/phoenix-core/src/test/java/org/apache/phoenix/compile/StatementHintsCompilationTest.java
index 55885e1..d651d14 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/compile/StatementHintsCompilationTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/compile/StatementHintsCompilationTest.java
@@ -98,7 +98,7 @@ public class StatementHintsCompilationTest extends BaseConnectionlessQueryTest {
         Connection conn = DriverManager.getConnection(getUrl());
         conn.createStatement().execute("create table eh (organization_id char(15) not null,parent_id char(15) not null, created_date date not null, entity_history_id char(15) not null constraint pk primary key (organization_id, parent_id, created_date, entity_history_id))");
         ResultSet rs = conn.createStatement().executeQuery("explain select /*+ RANGE_SCAN */ ORGANIZATION_ID, PARENT_ID, CREATED_DATE, ENTITY_HISTORY_ID from eh where ORGANIZATION_ID='111111111111111' and SUBSTR(PARENT_ID, 1, 3) = 'foo' and CREATED_DATE >= TO_DATE ('2012-11-01 00:00:00') and CREATED_DATE < TO_DATE ('2012-11-30 00:00:00') order by ORGANIZATION_ID, PARENT_ID, CREATED_DATE DESC, ENTITY_HISTORY_ID limit 100");
-        assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER EH ['111111111111111','foo','2012-11-01 00:00:00.000'] - ['111111111111111','fop','2012-11-30 00:00:00.000']\n" + 
+        assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER EH ['111111111111111','foo            ','2012-11-01 00:00:00.000'] - ['111111111111111','fop            ','2012-11-30 00:00:00.000']\n" + 
                 "    SERVER FILTER BY (CREATED_DATE >= '2012-11-01 00:00:00.000' AND CREATED_DATE < '2012-11-30 00:00:00.000')\n" + 
                 "    SERVER TOP 100 ROWS SORTED BY [ORGANIZATION_ID, PARENT_ID, CREATED_DATE DESC, ENTITY_HISTORY_ID]\n" + 
                 "CLIENT MERGE SORT",QueryUtil.getExplainPlan(rs));

http://git-wip-us.apache.org/repos/asf/phoenix/blob/e574d1ea/phoenix-core/src/test/java/org/apache/phoenix/compile/WhereCompilerTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/compile/WhereCompilerTest.java b/phoenix-core/src/test/java/org/apache/phoenix/compile/WhereCompilerTest.java
index b3b8ac0..3c0a952 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/compile/WhereCompilerTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/compile/WhereCompilerTest.java
@@ -107,11 +107,11 @@ public class WhereCompilerTest extends BaseConnectionlessQueryTest {
         PDataType.LONG.toBytes(1L, key, 1);
         key[0] = SaltingUtil.getSaltingByte(key, 1, PDataType.LONG.getByteSize(), 20);
         byte[] expectedStartKey = key;
-        byte[] expectedEndKey = ByteUtil.concat(key, QueryConstants.SEPARATOR_BYTE_ARRAY);
+        byte[] expectedEndKey = ByteUtil.nextKey(key);
         byte[] startKey = scan.getStartRow();
         byte[] stopKey = scan.getStopRow();
-        assertTrue(Bytes.compareTo(expectedStartKey, startKey) == 0);
-        assertTrue(Bytes.compareTo(expectedEndKey, stopKey) == 0);
+        assertArrayEquals(expectedStartKey, startKey);
+        assertArrayEquals(expectedEndKey, stopKey);
     }
 
     @Test
@@ -128,7 +128,7 @@ public class WhereCompilerTest extends BaseConnectionlessQueryTest {
         PDataType.VARCHAR.toBytes("a", key, 1);
         key[0] = SaltingUtil.getSaltingByte(key, 1, 1, 20);
         byte[] expectedStartKey = key;
-        byte[] expectedEndKey = ByteUtil.concat(key, QueryConstants.SEPARATOR_BYTE_ARRAY);
+        byte[] expectedEndKey = ByteUtil.nextKey(ByteUtil.concat(key, QueryConstants.SEPARATOR_BYTE_ARRAY));
         byte[] startKey = scan.getStartRow();
         byte[] stopKey = scan.getStopRow();
         assertTrue(Bytes.compareTo(expectedStartKey, startKey) == 0);
@@ -389,7 +389,7 @@ public class WhereCompilerTest extends BaseConnectionlessQueryTest {
         Scan scan = plan.getContext().getScan();
         byte[] expectedStartRow = ByteUtil.concat(Bytes.toBytes(tenantId), StringUtil.padChar(Bytes.toBytes(keyPrefix), 15));
         assertArrayEquals(expectedStartRow,scan.getStartRow());
-        assertArrayEquals(ByteUtil.concat(expectedStartRow,QueryConstants.SEPARATOR_BYTE_ARRAY),scan.getStopRow());
+        assertArrayEquals(ByteUtil.nextKey(expectedStartRow),scan.getStopRow());
     }
 
     @Test

http://git-wip-us.apache.org/repos/asf/phoenix/blob/e574d1ea/phoenix-core/src/test/java/org/apache/phoenix/compile/WhereOptimizerTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/compile/WhereOptimizerTest.java b/phoenix-core/src/test/java/org/apache/phoenix/compile/WhereOptimizerTest.java
index 321eb6e..2bfe381 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/compile/WhereOptimizerTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/compile/WhereOptimizerTest.java
@@ -37,6 +37,7 @@ import java.sql.DriverManager;
 import java.sql.SQLException;
 import java.util.Arrays;
 import java.util.Collections;
+import java.util.Iterator;
 import java.util.List;
 import java.util.Properties;
 
@@ -58,6 +59,7 @@ import org.apache.phoenix.util.ByteUtil;
 import org.apache.phoenix.util.DateUtil;
 import org.apache.phoenix.util.PhoenixRuntime;
 import org.apache.phoenix.util.PropertiesUtil;
+import org.apache.phoenix.util.StringUtil;
 import org.apache.phoenix.util.TestUtil;
 import org.junit.Test;
 
@@ -98,6 +100,32 @@ public class WhereOptimizerTest extends BaseConnectionlessQueryTest {
     }
 
     @Test
+    public void testSingleCharPaddedKeyExpression() throws SQLException {
+        String tenantId = "1";
+        String query = "select * from atable where organization_id='" + tenantId + "'";
+        Scan scan = compileStatement(query).getScan();
+
+        assertNull(scan.getFilter());
+        byte[] key = StringUtil.padChar(PDataType.CHAR.toBytes(tenantId), 15);
+        assertArrayEquals(key, scan.getStartRow());
+        assertArrayEquals(ByteUtil.nextKey(key), scan.getStopRow());
+    }
+
+    @Test
+    public void testSingleBinaryPaddedKeyExpression() throws SQLException {
+        Connection conn = DriverManager.getConnection(getUrl(), PropertiesUtil.deepCopy(TEST_PROPERTIES));
+        conn.createStatement().execute("create table bintable (k BINARY(15) PRIMARY KEY)");
+        String tenantId = "1";
+        String query = "select * from bintable where k='" + tenantId + "'";
+        Scan scan = compileStatement(query).getScan();
+
+        assertNull(scan.getFilter());
+        byte[] key = ByteUtil.fillKey(PDataType.VARCHAR.toBytes(tenantId), 15);
+        assertArrayEquals(key, scan.getStartRow());
+        assertArrayEquals(ByteUtil.nextKey(key), scan.getStopRow());
+    }
+
+    @Test
     public void testReverseSingleKeyExpression() throws SQLException {
         String tenantId = "000000000000001";
         String query = "select * from atable where '" + tenantId + "' = organization_id";
@@ -165,9 +193,9 @@ public class WhereOptimizerTest extends BaseConnectionlessQueryTest {
         Scan scan = compileStatement(query).getScan();
 
         assertNull(scan.getFilter());
-        byte[] startRow = ByteUtil.concat(PDataType.VARCHAR.toBytes(tenantId),ByteUtil.fillKey(PDataType.VARCHAR.toBytes(keyPrefix),15));
+        byte[] startRow = ByteUtil.concat(PDataType.VARCHAR.toBytes(tenantId),StringUtil.padChar(PDataType.VARCHAR.toBytes(keyPrefix),15));
         assertArrayEquals(startRow, scan.getStartRow());
-        byte[] stopRow = ByteUtil.concat(PDataType.VARCHAR.toBytes(tenantId),ByteUtil.fillKey(ByteUtil.nextKey(PDataType.VARCHAR.toBytes(keyPrefix)), 15));
+        byte[] stopRow = ByteUtil.concat(PDataType.VARCHAR.toBytes(tenantId),StringUtil.padChar(ByteUtil.nextKey(PDataType.VARCHAR.toBytes(keyPrefix)), 15));
         assertArrayEquals(stopRow, scan.getStopRow());
     }
 
@@ -180,9 +208,9 @@ public class WhereOptimizerTest extends BaseConnectionlessQueryTest {
         Scan scan = compileStatement(query, binds).getScan();
 
         assertNull(scan.getFilter());
-        byte[] startRow = ByteUtil.concat(PDataType.VARCHAR.toBytes(tenantId),ByteUtil.fillKey(PDataType.VARCHAR.toBytes(keyPrefix),15));
+        byte[] startRow = ByteUtil.concat(PDataType.VARCHAR.toBytes(tenantId),StringUtil.padChar(PDataType.VARCHAR.toBytes(keyPrefix),15));
         assertArrayEquals(startRow, scan.getStartRow());
-        byte[] stopRow = ByteUtil.concat(PDataType.VARCHAR.toBytes(tenantId),ByteUtil.fillKey(ByteUtil.nextKey(PDataType.VARCHAR.toBytes(keyPrefix)),15));
+        byte[] stopRow = ByteUtil.concat(PDataType.VARCHAR.toBytes(tenantId),StringUtil.padChar(ByteUtil.nextKey(PDataType.VARCHAR.toBytes(keyPrefix)),15));
         assertArrayEquals(stopRow, scan.getStopRow());
     }
 
@@ -369,7 +397,7 @@ public class WhereOptimizerTest extends BaseConnectionlessQueryTest {
         assertNull(scan.getFilter());
         byte[] startRow = ByteUtil.concat(PDataType.VARCHAR.toBytes(tenantId),PDataType.VARCHAR.toBytes(entityId));
         assertArrayEquals(startRow, scan.getStartRow());
-        assertArrayEquals(ByteUtil.concat(startRow, QueryConstants.SEPARATOR_BYTE_ARRAY), scan.getStopRow());
+        assertArrayEquals(ByteUtil.nextKey(startRow), scan.getStopRow());
     }
 
     @Test
@@ -380,14 +408,14 @@ public class WhereOptimizerTest extends BaseConnectionlessQueryTest {
         Scan scan = compileStatement(query).getScan();
         assertNotNull(scan.getFilter());
 
-        byte[] startRow = ByteUtil.concat(ByteUtil.fillKey(PDataType.VARCHAR.toBytes(tenantId.substring(0,3)),15),PDataType.VARCHAR.toBytes(entityId));
+        byte[] startRow = ByteUtil.concat(StringUtil.padChar(PDataType.VARCHAR.toBytes(tenantId.substring(0,3)),15),PDataType.VARCHAR.toBytes(entityId));
         assertArrayEquals(startRow, scan.getStartRow());
         // Even though the first slot is a non inclusive range, we need to do a next key
         // on the second slot because of the algorithm we use to seek to and terminate the
         // loop during skip scan. We could end up having a first slot just under the upper
         // limit of slot one and a value equal to the value in slot two and we need this to
         // be less than the upper range that would get formed.
-        byte[] stopRow = ByteUtil.concat(ByteUtil.fillKey(ByteUtil.nextKey(PDataType.VARCHAR.toBytes(tenantId.substring(0,3))),15),ByteUtil.nextKey(PDataType.VARCHAR.toBytes(entityId)));
+        byte[] stopRow = ByteUtil.concat(StringUtil.padChar(ByteUtil.nextKey(PDataType.VARCHAR.toBytes(tenantId.substring(0,3))),15),ByteUtil.nextKey(PDataType.VARCHAR.toBytes(entityId)));
         assertArrayEquals(stopRow, scan.getStopRow());
     }
 
@@ -412,9 +440,9 @@ public class WhereOptimizerTest extends BaseConnectionlessQueryTest {
         Scan scan = compileStatement(query).getScan();
         assertNull(scan.getFilter());
 
-        byte[] startRow = ByteUtil.concat(PDataType.VARCHAR.toBytes(tenantId),ByteUtil.fillKey(PDataType.VARCHAR.toBytes(keyPrefix1),15));
+        byte[] startRow = ByteUtil.concat(PDataType.CHAR.toBytes(tenantId),StringUtil.padChar(PDataType.CHAR.toBytes(keyPrefix1),15));
         assertArrayEquals(startRow, scan.getStartRow());
-        byte[] stopRow = ByteUtil.concat(PDataType.VARCHAR.toBytes(tenantId),ByteUtil.fillKey(PDataType.VARCHAR.toBytes(keyPrefix2),15));
+        byte[] stopRow = ByteUtil.concat(PDataType.CHAR.toBytes(tenantId),StringUtil.padChar(PDataType.CHAR.toBytes(keyPrefix2),15));
         assertArrayEquals(stopRow, scan.getStopRow());
     }
 
@@ -427,9 +455,9 @@ public class WhereOptimizerTest extends BaseConnectionlessQueryTest {
         Scan scan = compileStatement(query).getScan();
         assertNull(scan.getFilter());
 
-        byte[] startRow = ByteUtil.concat(PDataType.VARCHAR.toBytes(tenantId),ByteUtil.fillKey(PDataType.VARCHAR.toBytes(keyPrefix1),15));
+        byte[] startRow = ByteUtil.concat(PDataType.CHAR.toBytes(tenantId),StringUtil.padChar(PDataType.CHAR.toBytes(keyPrefix1),15));
         assertArrayEquals(startRow, scan.getStartRow());
-        byte[] stopRow = ByteUtil.concat(PDataType.VARCHAR.toBytes(tenantId),ByteUtil.fillKey(ByteUtil.nextKey(PDataType.VARCHAR.toBytes(keyPrefix2)),15));
+        byte[] stopRow = ByteUtil.concat(PDataType.CHAR.toBytes(tenantId),StringUtil.padChar(ByteUtil.nextKey(PDataType.CHAR.toBytes(keyPrefix2)),15));
         assertArrayEquals(stopRow, scan.getStopRow());
     }
 
@@ -442,9 +470,9 @@ public class WhereOptimizerTest extends BaseConnectionlessQueryTest {
         Scan scan = compileStatement(query).getScan();
         
         assertNull(scan.getFilter());
-        byte[] startRow = ByteUtil.concat(PDataType.VARCHAR.toBytes(tenantId),ByteUtil.fillKey(ByteUtil.nextKey(PDataType.VARCHAR.toBytes(keyPrefix1)),15));
+        byte[] startRow = ByteUtil.concat(PDataType.VARCHAR.toBytes(tenantId),StringUtil.padChar(ByteUtil.nextKey(PDataType.VARCHAR.toBytes(keyPrefix1)),15));
         assertArrayEquals(startRow, scan.getStartRow());
-        byte[] stopRow = ByteUtil.concat(PDataType.VARCHAR.toBytes(tenantId),ByteUtil.fillKey(ByteUtil.nextKey(PDataType.VARCHAR.toBytes(keyPrefix2)),15));
+        byte[] stopRow = ByteUtil.concat(PDataType.VARCHAR.toBytes(tenantId),StringUtil.padChar(ByteUtil.nextKey(PDataType.VARCHAR.toBytes(keyPrefix2)),15));
         assertArrayEquals(stopRow, scan.getStopRow());
     }
 
@@ -470,7 +498,7 @@ public class WhereOptimizerTest extends BaseConnectionlessQueryTest {
         byte[] startRow = ByteUtil.concat(PDataType.VARCHAR.toBytes(tenantId),PDataType.VARCHAR.toBytes(entityId));
         assertArrayEquals(startRow, scan.getStartRow());
         byte[] stopRow = ByteUtil.concat(PDataType.VARCHAR.toBytes(tenantId),PDataType.VARCHAR.toBytes(entityId));
-        assertArrayEquals(ByteUtil.concat(stopRow, QueryConstants.SEPARATOR_BYTE_ARRAY), scan.getStopRow());
+        assertArrayEquals(ByteUtil.nextKey(stopRow), scan.getStopRow());
     }
 
     @Test
@@ -492,9 +520,9 @@ public class WhereOptimizerTest extends BaseConnectionlessQueryTest {
         Scan scan = compileStatement(query).getScan();
         assertNull(scan.getFilter());
 
-        byte[] startRow = PDataType.VARCHAR.toBytes(tenantId);
+        byte[] startRow = PDataType.CHAR.toBytes(tenantId);
         assertArrayEquals(startRow, scan.getStartRow());
-        byte[] stopRow = ByteUtil.concat(PDataType.VARCHAR.toBytes(tenantId),PDataType.VARCHAR.toBytes(keyPrefix1),new byte[entityId.length() - keyPrefix1.length()]);
+        byte[] stopRow = ByteUtil.concat(PDataType.CHAR.toBytes(tenantId),StringUtil.padChar(PDataType.CHAR.toBytes(keyPrefix1),entityId.length()));
         assertArrayEquals(stopRow, scan.getStopRow());
     }
 
@@ -510,7 +538,7 @@ public class WhereOptimizerTest extends BaseConnectionlessQueryTest {
         byte[] startRow = ByteUtil.concat(PDataType.VARCHAR.toBytes(tenantId),PDataType.VARCHAR.toBytes(entityId));
         assertArrayEquals(startRow, scan.getStartRow());
         byte[] stopRow = ByteUtil.concat(PDataType.VARCHAR.toBytes(tenantId),PDataType.VARCHAR.toBytes(entityId));
-        assertArrayEquals(ByteUtil.concat(stopRow, QueryConstants.SEPARATOR_BYTE_ARRAY), scan.getStopRow());
+        assertArrayEquals(ByteUtil.nextKey(stopRow), scan.getStopRow());
     }
 
     @Test
@@ -522,9 +550,9 @@ public class WhereOptimizerTest extends BaseConnectionlessQueryTest {
         Scan scan = compileStatement(query).getScan();
 
         assertNull(scan.getFilter());
-        byte[] startRow = ByteUtil.concat(PDataType.VARCHAR.toBytes(tenantId),ByteUtil.fillKey(PDataType.VARCHAR.toBytes(keyPrefix1),15)); // extra byte is due to implicit internal padding
+        byte[] startRow = ByteUtil.concat(PDataType.CHAR.toBytes(tenantId),StringUtil.padChar(PDataType.CHAR.toBytes(keyPrefix1),15)); // extra byte is due to implicit internal padding
         assertArrayEquals(startRow, scan.getStartRow());
-        byte[] stopRow = ByteUtil.concat(PDataType.VARCHAR.toBytes(tenantId),ByteUtil.fillKey(ByteUtil.nextKey(PDataType.VARCHAR.toBytes(keyPrefix2)),15));
+        byte[] stopRow = ByteUtil.concat(PDataType.CHAR.toBytes(tenantId),StringUtil.padChar(ByteUtil.nextKey(PDataType.CHAR.toBytes(keyPrefix2)),15));
         assertArrayEquals(stopRow, scan.getStopRow());
     }
 
@@ -635,9 +663,9 @@ public class WhereOptimizerTest extends BaseConnectionlessQueryTest {
         Scan scan = context.getScan();
 
         assertNull(scan.getFilter());
-        byte[] startRow = ByteUtil.concat(PDataType.VARCHAR.toBytes(tenantId),ByteUtil.fillKey(PDataType.VARCHAR.toBytes(keyPrefix),15));
+        byte[] startRow = ByteUtil.concat(PDataType.VARCHAR.toBytes(tenantId),StringUtil.padChar(PDataType.VARCHAR.toBytes(keyPrefix),15));
         assertArrayEquals(startRow, scan.getStartRow());
-        byte[] stopRow = ByteUtil.concat(PDataType.VARCHAR.toBytes(tenantId),ByteUtil.fillKey(ByteUtil.nextKey(PDataType.VARCHAR.toBytes(keyPrefix)),15));
+        byte[] stopRow = ByteUtil.concat(PDataType.VARCHAR.toBytes(tenantId),StringUtil.padChar(ByteUtil.nextKey(PDataType.VARCHAR.toBytes(keyPrefix)),15));
         assertArrayEquals(stopRow, scan.getStopRow());
     }
 
@@ -651,9 +679,9 @@ public class WhereOptimizerTest extends BaseConnectionlessQueryTest {
         Scan scan = context.getScan();
 
         assertNull(scan.getFilter());
-        byte[] startRow = ByteUtil.concat(PDataType.VARCHAR.toBytes(tenantId),ByteUtil.fillKey(PDataType.VARCHAR.toBytes(keyPrefix),15));
+        byte[] startRow = ByteUtil.concat(PDataType.VARCHAR.toBytes(tenantId),StringUtil.padChar(PDataType.VARCHAR.toBytes(keyPrefix),15));
         assertArrayEquals(startRow, scan.getStartRow());
-        byte[] stopRow = ByteUtil.concat(PDataType.VARCHAR.toBytes(tenantId),ByteUtil.fillKey(ByteUtil.nextKey(PDataType.VARCHAR.toBytes(keyPrefix)),15));
+        byte[] stopRow = ByteUtil.concat(PDataType.VARCHAR.toBytes(tenantId),StringUtil.padChar(ByteUtil.nextKey(PDataType.VARCHAR.toBytes(keyPrefix)),15));
         assertArrayEquals(stopRow, scan.getStopRow());
     }
 
@@ -686,9 +714,9 @@ public class WhereOptimizerTest extends BaseConnectionlessQueryTest {
                     likeArg)),
                 filter);
 
-        byte[] startRow = ByteUtil.concat(PDataType.VARCHAR.toBytes(tenantId),ByteUtil.fillKey(PDataType.VARCHAR.toBytes(keyPrefix),15));
+        byte[] startRow = ByteUtil.concat(PDataType.VARCHAR.toBytes(tenantId),StringUtil.padChar(PDataType.VARCHAR.toBytes(keyPrefix),15));
         assertArrayEquals(startRow, scan.getStartRow());
-        byte[] stopRow = ByteUtil.concat(PDataType.VARCHAR.toBytes(tenantId),ByteUtil.fillKey(ByteUtil.nextKey(PDataType.VARCHAR.toBytes(keyPrefix)),15));
+        byte[] stopRow = ByteUtil.concat(PDataType.VARCHAR.toBytes(tenantId),StringUtil.padChar(ByteUtil.nextKey(PDataType.VARCHAR.toBytes(keyPrefix)),15));
         assertArrayEquals(stopRow, scan.getStopRow());
     }
 
@@ -710,9 +738,9 @@ public class WhereOptimizerTest extends BaseConnectionlessQueryTest {
                     likeArg)),
                 filter);
 
-        byte[] startRow = ByteUtil.concat(PDataType.VARCHAR.toBytes(tenantId),ByteUtil.fillKey(PDataType.VARCHAR.toBytes(keyPrefix),15));
+        byte[] startRow = ByteUtil.concat(PDataType.VARCHAR.toBytes(tenantId),StringUtil.padChar(PDataType.VARCHAR.toBytes(keyPrefix),15));
         assertArrayEquals(startRow, scan.getStartRow());
-        byte[] stopRow = ByteUtil.concat(PDataType.VARCHAR.toBytes(tenantId),ByteUtil.fillKey(ByteUtil.nextKey(PDataType.VARCHAR.toBytes(keyPrefix)),15));
+        byte[] stopRow = ByteUtil.concat(PDataType.VARCHAR.toBytes(tenantId),StringUtil.padChar(ByteUtil.nextKey(PDataType.VARCHAR.toBytes(keyPrefix)),15));
         assertArrayEquals(stopRow, scan.getStopRow());
     }
 
@@ -734,8 +762,8 @@ public class WhereOptimizerTest extends BaseConnectionlessQueryTest {
                     likeArg)),
                 filter);
 
-        byte[] startRow = ByteUtil.concat(PDataType.VARCHAR.toBytes(tenantId),ByteUtil.fillKey(PDataType.VARCHAR.toBytes(keyPrefix),15));
-        byte[] stopRow = ByteUtil.concat(PDataType.VARCHAR.toBytes(tenantId),ByteUtil.fillKey(ByteUtil.nextKey(PDataType.VARCHAR.toBytes(keyPrefix)),15));
+        byte[] startRow = ByteUtil.concat(PDataType.VARCHAR.toBytes(tenantId),StringUtil.padChar(PDataType.VARCHAR.toBytes(keyPrefix),15));
+        byte[] stopRow = ByteUtil.concat(PDataType.VARCHAR.toBytes(tenantId),StringUtil.padChar(ByteUtil.nextKey(PDataType.VARCHAR.toBytes(keyPrefix)),15));
         assertArrayEquals(startRow, scan.getStartRow());
         assertArrayEquals(stopRow, scan.getStopRow());
     }
@@ -1064,11 +1092,11 @@ public class WhereOptimizerTest extends BaseConnectionlessQueryTest {
         assertEquals(1,ranges.size());
         List<List<KeyRange>> expectedRanges = Collections.singletonList(Arrays.asList(
                 PDataType.CHAR.getKeyRange(
-                        ByteUtil.fillKey(PDataType.CHAR.toBytes("00D"),15), true, 
-                        ByteUtil.fillKey(ByteUtil.nextKey(PDataType.CHAR.toBytes("00D")),15), false), 
+                        StringUtil.padChar(PDataType.CHAR.toBytes("00D"),15), true, 
+                        StringUtil.padChar(ByteUtil.nextKey(PDataType.CHAR.toBytes("00D")),15), false), 
                 PDataType.CHAR.getKeyRange(
-                        ByteUtil.fillKey(PDataType.CHAR.toBytes("foo"),15), true, 
-                        ByteUtil.fillKey(ByteUtil.nextKey(PDataType.CHAR.toBytes("foo")),15), false)));
+                        StringUtil.padChar(PDataType.CHAR.toBytes("foo"),15), true, 
+                        StringUtil.padChar(ByteUtil.nextKey(PDataType.CHAR.toBytes("foo")),15), false)));
         assertEquals(expectedRanges, ranges);
     }
     
@@ -1125,9 +1153,9 @@ public class WhereOptimizerTest extends BaseConnectionlessQueryTest {
         assertNotNull(filter);
         assertTrue(filter instanceof RowKeyComparisonFilter);
 
-        byte[] expectedStartRow = ByteUtil.concat(PDataType.VARCHAR.toBytes(tenantId), ByteUtil.fillKey(PDataType.VARCHAR.toBytes(keyPrefix),15), PDataType.DATE.toBytes(startTime));
+        byte[] expectedStartRow = ByteUtil.concat(PDataType.VARCHAR.toBytes(tenantId), StringUtil.padChar(PDataType.VARCHAR.toBytes(keyPrefix),15), PDataType.DATE.toBytes(startTime));
         assertArrayEquals(expectedStartRow, scan.getStartRow());
-        byte[] expectedStopRow = ByteUtil.concat(PDataType.VARCHAR.toBytes(tenantId), ByteUtil.fillKey(ByteUtil.nextKey(PDataType.VARCHAR.toBytes(keyPrefix)),15), PDataType.DATE.toBytes(stopTime));
+        byte[] expectedStopRow = ByteUtil.concat(PDataType.VARCHAR.toBytes(tenantId), StringUtil.padChar(ByteUtil.nextKey(PDataType.VARCHAR.toBytes(keyPrefix)),15), PDataType.DATE.toBytes(stopTime));
         assertArrayEquals(expectedStopRow, scan.getStopRow());
     }
 
@@ -1252,7 +1280,7 @@ public class WhereOptimizerTest extends BaseConnectionlessQueryTest {
         Filter filter = scan.getFilter();
         assertNull(filter);
         byte[] expectedStartRow = ByteUtil.concat(PDataType.VARCHAR.toBytes(lowerTenantId), PDataType.VARCHAR.toBytes(lowerParentId), PDataType.DATE.toBytes(lowerCreatedDate));
-        byte[] expectedStopRow = ByteUtil.nextKey(ByteUtil.concat(ByteUtil.fillKey(PDataType.VARCHAR.toBytes("7"),15), ByteUtil.fillKey(PDataType.VARCHAR.toBytes("7"), 15)));
+        byte[] expectedStopRow = ByteUtil.nextKey(ByteUtil.concat(StringUtil.padChar(PDataType.VARCHAR.toBytes("7"),15), StringUtil.padChar(PDataType.VARCHAR.toBytes("7"), 15)));
         assertArrayEquals(expectedStartRow, scan.getStartRow());
         assertArrayEquals(expectedStopRow, scan.getStopRow());
     }
@@ -1626,6 +1654,72 @@ public class WhereOptimizerTest extends BaseConnectionlessQueryTest {
         assertEquals(SkipScanFilter.class, filter.getClass());
     }
     
+    @Test
+    public void testRVCWithCompareOpsForRowKeyColumnValuesSmallerThanSchema() throws SQLException {
+        String orgId = "0000005";
+        String entityId = "011";
+        String orgId2 = "000005";
+        String entityId2 = "11";
+        
+        // CASE 1: >=
+        String query = "select * from atable where (organization_id, entity_id) >= (?,?)";
+        List<Object> binds = Arrays.<Object>asList(orgId, entityId);
+        StatementContext context = compileStatement(query, binds);
+        Scan scan = context.getScan();
+        Filter filter = scan.getFilter();
+        assertNull(filter);
+        assertArrayEquals(ByteUtil.concat(StringUtil.padChar(PDataType.CHAR.toBytes(orgId), 15), StringUtil.padChar(PDataType.CHAR.toBytes(entityId), 15)), scan.getStartRow());
+        assertArrayEquals(HConstants.EMPTY_END_ROW, scan.getStopRow());
+        
+        // CASE 2: >
+        query = "select * from atable where (organization_id, entity_id) > (?,?)";
+        binds = Arrays.<Object>asList(orgId, entityId);
+        context = compileStatement(query, binds);
+        scan = context.getScan();
+        filter = scan.getFilter();
+        assertNull(filter);
+        assertArrayEquals(ByteUtil.nextKey(ByteUtil.concat(StringUtil.padChar(PDataType.CHAR.toBytes(orgId), 15), StringUtil.padChar(PDataType.CHAR.toBytes(entityId), 15))), scan.getStartRow());
+        assertArrayEquals(HConstants.EMPTY_END_ROW, scan.getStopRow());
+        
+        // CASE 3: <=
+        query = "select * from atable where (organization_id, entity_id) <= (?,?)";
+        binds = Arrays.<Object>asList(orgId, entityId);
+        context = compileStatement(query, binds);
+        scan = context.getScan();
+        filter = scan.getFilter();
+        assertNull(filter);
+        assertArrayEquals(HConstants.EMPTY_END_ROW, scan.getStartRow());
+        assertArrayEquals(ByteUtil.nextKey(ByteUtil.concat(StringUtil.padChar(PDataType.CHAR.toBytes(orgId), 15), StringUtil.padChar(PDataType.CHAR.toBytes(entityId), 15))), scan.getStopRow());
+        
+        // CASE 4: <
+        query = "select * from atable where (organization_id, entity_id) < (?,?)";
+        binds = Arrays.<Object>asList(orgId, entityId);
+        context = compileStatement(query, binds);
+        scan = context.getScan();
+        filter = scan.getFilter();
+        assertNull(filter);
+        assertArrayEquals(HConstants.EMPTY_END_ROW, scan.getStartRow());
+        assertArrayEquals(ByteUtil.concat(StringUtil.padChar(PDataType.CHAR.toBytes(orgId), 15), StringUtil.padChar(PDataType.CHAR.toBytes(entityId), 15)), scan.getStopRow());
+        
+        // CASE 5: =
+        // For RVC, this will only occur if there's more than one key in the IN
+        query = "select * from atable where (organization_id, entity_id) IN ((?,?),(?,?))";
+        binds = Arrays.<Object>asList(orgId, entityId, orgId2, entityId2);
+        context = compileStatement(query, binds);
+        scan = context.getScan();
+        filter = scan.getFilter();
+        assertTrue(filter instanceof SkipScanFilter);
+        ScanRanges scanRanges = context.getScanRanges();
+        assertEquals(2,scanRanges.getPointLookupCount());
+        Iterator<KeyRange> iterator = scanRanges.getPointLookupKeyIterator();
+        KeyRange k1 = iterator.next();
+        assertTrue(k1.isSingleKey());
+        assertArrayEquals(ByteUtil.concat(StringUtil.padChar(PDataType.CHAR.toBytes(orgId), 15), StringUtil.padChar(PDataType.CHAR.toBytes(entityId), 15)), k1.getLowerRange());
+        KeyRange k2 = iterator.next();
+        assertTrue(k2.isSingleKey());
+        assertArrayEquals(ByteUtil.concat(StringUtil.padChar(PDataType.CHAR.toBytes(orgId2), 15), StringUtil.padChar(PDataType.CHAR.toBytes(entityId2), 15)), k2.getLowerRange());
+    }
+    
     private static StatementContext compileStatementTenantSpecific(String tenantId, String query, List<Object> binds) throws Exception {
     	PhoenixConnection pconn = getTenantSpecificConnection("tenantId").unwrap(PhoenixConnection.class);
         PhoenixPreparedStatement pstmt = new PhoenixPreparedStatement(pconn, query);