You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@phoenix.apache.org by ch...@apache.org on 2019/05/02 20:32:31 UTC

[phoenix] branch master updated: PHOENIX-5262 Wrong Result on Salted table with some Variable Length PKs

This is an automated email from the ASF dual-hosted git repository.

chinmayskulkarni pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/phoenix.git


The following commit(s) were added to refs/heads/master by this push:
     new 4eec41f  PHOENIX-5262 Wrong Result on Salted table with some Variable Length PKs
4eec41f is described below

commit 4eec41f3f2b04865b6d59ebd3fbd3aa1e0a0fd80
Author: Daniel <da...@salesforce.com>
AuthorDate: Fri Apr 26 18:13:49 2019 -0700

    PHOENIX-5262 Wrong Result on Salted table with some Variable Length PKs
---
 .../salted/SaltedTableVarLengthRowKeyIT.java       |  82 +++
 .../java/org/apache/phoenix/util/ScanUtil.java     |   4 +-
 .../apache/phoenix/compile/WhereCompilerTest.java  |   5 +-
 .../java/org/apache/phoenix/util/ScanUtilTest.java | 571 +++++++++++----------
 4 files changed, 382 insertions(+), 280 deletions(-)

diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/salted/SaltedTableVarLengthRowKeyIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/salted/SaltedTableVarLengthRowKeyIT.java
index fa43876..85d518d 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/salted/SaltedTableVarLengthRowKeyIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/salted/SaltedTableVarLengthRowKeyIT.java
@@ -19,16 +19,20 @@
 package org.apache.phoenix.end2end.salted;
 
 import static org.apache.phoenix.util.TestUtil.TEST_PROPERTIES;
+import static org.junit.Assert.assertArrayEquals;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 
+import java.sql.Array;
 import java.sql.Connection;
 import java.sql.DriverManager;
 import java.sql.PreparedStatement;
 import java.sql.ResultSet;
+import java.util.Arrays;
 import java.util.Properties;
 
+import org.apache.commons.lang.ArrayUtils;
 import org.apache.phoenix.end2end.ParallelStatsDisabledIT;
 import org.apache.phoenix.util.PropertiesUtil;
 import org.junit.Test;
@@ -87,4 +91,82 @@ public class SaltedTableVarLengthRowKeyIT extends ParallelStatsDisabledIT {
             conn.close();
         }
     }
+
+    @Test
+    public void testSaltedVarbinaryUpperBoundQuery() throws Exception {
+        String tableName = generateUniqueName();
+        String ddl = "CREATE TABLE " + tableName +
+                " ( k VARBINARY PRIMARY KEY, a INTEGER ) SALT_BUCKETS = 3";
+        String dml = "UPSERT INTO " + tableName + " values (?, ?)";
+        String sql2 = "SELECT * FROM " + tableName + " WHERE k = ?";
+
+        try (Connection conn = DriverManager.getConnection(getUrl())) {
+            conn.createStatement().execute(ddl);
+            PreparedStatement stmt = conn.prepareStatement(dml);
+            stmt.setInt(2, 1);
+
+            stmt.setBytes(1, new byte[] { 5 });
+            stmt.executeUpdate();
+            stmt.setBytes(1, new byte[] { 5, 0 });
+            stmt.executeUpdate();
+            stmt.setBytes(1, new byte[] { 5, 1 });
+            stmt.executeUpdate();
+            stmt.close();
+            conn.commit();
+
+            stmt = conn.prepareStatement(sql2);
+            stmt.setBytes(1, new byte[] { 5 });
+            ResultSet rs = stmt.executeQuery();
+
+            assertTrue(rs.next());
+            assertArrayEquals(new byte[] {5},rs.getBytes(1));
+            assertEquals(1,rs.getInt(2));
+            assertFalse(rs.next());
+            stmt.close();
+        }
+    }
+
+    @Test
+    public void testSaltedArrayTypeUpperBoundQuery() throws Exception {
+        String tableName = generateUniqueName();
+        String ddl = "CREATE TABLE " + tableName +
+                " ( k TINYINT ARRAY[10] PRIMARY KEY, a INTEGER ) SALT_BUCKETS = 3";
+        String dml = "UPSERT INTO " + tableName + " values (?, ?)";
+        String sql2 = "SELECT * FROM " + tableName + " WHERE k = ?";
+
+        try (Connection conn = DriverManager.getConnection(getUrl())) {
+            conn.createStatement().execute(ddl);
+            PreparedStatement stmt = conn.prepareStatement(dml);
+            stmt.setInt(2, 1);
+
+            Byte[] byteArray1 = ArrayUtils.toObject(new byte[] {5});
+            Byte[] byteArray2 = ArrayUtils.toObject(new byte[] {5, -128});
+            Byte[] byteArray3 = ArrayUtils.toObject(new byte[] {5, -127});
+
+
+            Array array1 = conn.createArrayOf("TINYINT", byteArray1);
+            Array array2 = conn.createArrayOf("TINYINT", byteArray2);
+            Array array3 = conn.createArrayOf("TINYINT", byteArray3);
+
+            stmt.setArray(1,array1);
+            stmt.executeUpdate();
+            stmt.setArray(1,array2);
+            stmt.executeUpdate();
+            stmt.setArray(1,array3);
+            stmt.executeUpdate();
+            stmt.close();
+            conn.commit();
+
+            stmt = conn.prepareStatement(sql2);
+            stmt.setArray(1, array1);
+            ResultSet rs = stmt.executeQuery();
+
+            assertTrue(rs.next());
+            byte[] resultByteArray = (byte[])(rs.getArray(1).getArray());
+            assertArrayEquals(new byte[]{5},resultByteArray);
+            assertEquals(1,rs.getInt(2));
+            assertFalse(rs.next());
+            stmt.close();
+        }
+    }
 }
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 b7bb099..131d98b 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
@@ -448,7 +448,9 @@ public class ScanUtil {
                 key[offset++] = sepByte;
                 // 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
-                lastInclusiveUpperSingleKey &= i < schema.getMaxFields()-1;
+                // To test if this is the last pk column we need to consider the span of this slot
+                // and the field index to see if this slot considers the last column
+                lastInclusiveUpperSingleKey &= (fieldIndex + slotSpan[i]) < schema.getMaxFields()-1;
             }
             if (exclusiveUpper) {
                 // Cannot include anything else on the key, as otherwise
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 ee23a0b..b84ead0 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
@@ -183,7 +183,7 @@ public class WhereCompilerTest extends BaseConnectionlessQueryTest {
     @Test
     public void testSingleVariableFullPkSalted() throws SQLException {
         PhoenixConnection pconn = DriverManager.getConnection(getUrl(), PropertiesUtil.deepCopy(TEST_PROPERTIES)).unwrap(PhoenixConnection.class);
-        pconn.createStatement().execute("CREATE TABLE t (k varchar primary key, v varchar) SALT_BUCKETS=20");
+        pconn.createStatement().execute("CREATE TABLE t (k varchar(10) primary key, v varchar) SALT_BUCKETS=20");
         String query = "select * from t where k='a'";
         PhoenixPreparedStatement pstmt = newPreparedStatement(pconn, query);
         QueryPlan plan = pstmt.optimizeQuery();
@@ -194,7 +194,8 @@ public class WhereCompilerTest extends BaseConnectionlessQueryTest {
         PVarchar.INSTANCE.toBytes("a", key, 1);
         key[0] = SaltingUtil.getSaltingByte(key, 1, 1, 20);
         byte[] expectedStartKey = key;
-        byte[] expectedEndKey = ByteUtil.nextKey(ByteUtil.concat(key, QueryConstants.SEPARATOR_BYTE_ARRAY));
+        //lexicographically this is the next PK
+        byte[] expectedEndKey = ByteUtil.concat(key,new byte[]{0});
         byte[] startKey = scan.getStartRow();
         byte[] stopKey = scan.getStopRow();
         assertTrue(Bytes.compareTo(expectedStartKey, startKey) == 0);
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 6cdc800..6310640 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
@@ -30,10 +30,13 @@ import org.apache.phoenix.schema.PDatum;
 import org.apache.phoenix.schema.RowKeySchema;
 import org.apache.phoenix.schema.RowKeySchema.RowKeySchemaBuilder;
 import org.apache.phoenix.schema.SortOrder;
+import org.apache.phoenix.schema.types.PBinary;
 import org.apache.phoenix.schema.types.PChar;
 import org.apache.phoenix.schema.types.PDataType;
+import org.apache.phoenix.schema.types.PVarbinary;
 import org.apache.phoenix.schema.types.PVarchar;
 import org.junit.Test;
+import org.junit.experimental.runners.Enclosed;
 import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
 import org.junit.runners.Parameterized.Parameters;
@@ -45,291 +48,305 @@ import com.google.common.collect.Lists;
 /**
  * Test the SetKey method in ScanUtil.
  */
-@RunWith(Parameterized.class)
+@RunWith(Enclosed.class)
 public class ScanUtilTest {
 
-    private final List<List<KeyRange>> slots;
-    private final byte[] expectedKey;
-    private final RowKeySchema schema;
-    private final Bound bound;
-
-    public ScanUtilTest(List<List<KeyRange>> slots, int[] widths, byte[] expectedKey, Bound bound) throws Exception {
-        RowKeySchemaBuilder builder = new RowKeySchemaBuilder(widths.length);
-        for (final int width : widths) {
-            if (width > 0) {
-                builder.addField(new PDatum() {
-                    @Override
-                    public boolean isNullable() {
-                        return false;
-                    }
-                    @Override
-                    public PDataType getDataType() {
-                        return PChar.INSTANCE;
-                    }
-                    @Override
-                    public Integer getMaxLength() {
-                        return width;
-                    }
-                    @Override
-                    public Integer getScale() {
-                        return null;
-                    }
-                    @Override
-                    public SortOrder getSortOrder() {
-                        return SortOrder.getDefault();
-                    }
-                }, false, SortOrder.getDefault());
-            } else {
-                builder.addField(new PDatum() {
-                    @Override
-                    public boolean isNullable() {
-                        return false;
-                    }
-                    @Override
-                    public PDataType getDataType() {
-                        return PVarchar.INSTANCE;
-                    }
-                    @Override
-                    public Integer getMaxLength() {
-                        return null;
-                    }
-                    @Override
-                    public Integer getScale() {
-                        return null;
-                    }
-                    @Override
-                    public SortOrder getSortOrder() {
-                        return SortOrder.getDefault();
-                    }
-                }, false, SortOrder.getDefault());
+    @RunWith(Parameterized.class)
+    public static class ParameterizedScanUtilTest {
+        private final List<List<KeyRange>> slots;
+        private final byte[] expectedKey;
+        private final RowKeySchema schema;
+        private final Bound bound;
+
+        public ParameterizedScanUtilTest(List<List<KeyRange>> slots, int[] widths, byte[] expectedKey, Bound bound)
+                throws Exception {
+            RowKeySchemaBuilder builder = new RowKeySchemaBuilder(widths.length);
+            for (final int width : widths) {
+                if (width > 0) {
+                    builder.addField(new PDatum() {
+                        @Override public boolean isNullable() {
+                            return false;
+                        }
+
+                        @Override public PDataType getDataType() {
+                            return PChar.INSTANCE;
+                        }
+
+                        @Override public Integer getMaxLength() {
+                            return width;
+                        }
+
+                        @Override public Integer getScale() {
+                            return null;
+                        }
+
+                        @Override public SortOrder getSortOrder() {
+                            return SortOrder.getDefault();
+                        }
+                    }, false, SortOrder.getDefault());
+                } else {
+                    builder.addField(new PDatum() {
+                        @Override public boolean isNullable() {
+                            return false;
+                        }
+
+                        @Override public PDataType getDataType() {
+                            return PVarchar.INSTANCE;
+                        }
+
+                        @Override public Integer getMaxLength() {
+                            return null;
+                        }
+
+                        @Override public Integer getScale() {
+                            return null;
+                        }
+
+                        @Override public SortOrder getSortOrder() {
+                            return SortOrder.getDefault();
+                        }
+                    }, false, SortOrder.getDefault());
+                }
             }
+            this.schema = builder.build();
+            this.slots = slots;
+            this.expectedKey = expectedKey;
+            this.bound = bound;
         }
-        this.schema = builder.build();
-        this.slots = slots;
-        this.expectedKey = expectedKey;
-        this.bound = bound;
-    }
 
-    @Test
-    public void test() {
-        byte[] key = new byte[1024];
-        int[] position = new int[slots.size()];
-        int offset = ScanUtil.setKey(schema, slots, ScanUtil.getDefaultSlotSpans(slots.size()), position, bound, key, 0, 0, slots.size());
-        byte[] actualKey = new byte[offset];
-        System.arraycopy(key, 0, actualKey, 0, offset);
-        assertArrayEquals(expectedKey, actualKey);
-    }
+        @Test
+        public void test() {
+            byte[] key = new byte[1024];
+            int[] position = new int[slots.size()];
+            int
+                    offset =
+                    ScanUtil.setKey(schema, slots, ScanUtil.getDefaultSlotSpans(slots.size()), position,
+                            bound, key, 0, 0, slots.size());
+            byte[] actualKey = new byte[offset];
+            System.arraycopy(key, 0, actualKey, 0, offset);
+            assertArrayEquals(expectedKey, actualKey);
+        }
 
-    @Parameters(name="{0} {1} {2} {3} {4}")
-    public static Collection<Object> data() {
-        List<Object> testCases = Lists.newArrayList();
-        // 1, Lower bound, all single keys, all inclusive.
-        testCases.addAll(
-            foreach(new KeyRange[][]{{
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("a"), true, Bytes.toBytes("a"), true),},{
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("1"), true),},{
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("A"), true, Bytes.toBytes("A"), true),}},
-                new int[] {1,1,1},
-                PChar.INSTANCE.toBytes("a1A"),
-                Bound.LOWER
-                ));
-        // 2, Lower bound, all range keys, all inclusive.
-        testCases.addAll(
-            foreach(new KeyRange[][]{{
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("a"), true, Bytes.toBytes("b"), true),},{
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("2"), true),},{
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("A"), true, Bytes.toBytes("B"), true),}},
-                new int[] {1,1,1},
-                PChar.INSTANCE.toBytes("a1A"),
-                Bound.LOWER
-                ));
-        // 3, Lower bound, mixed single and range keys, all inclusive.
-        testCases.addAll(
-            foreach(new KeyRange[][]{{
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("a"), true, Bytes.toBytes("a"), true),},{
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("2"), true),},{
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("A"), true, Bytes.toBytes("A"), true),}},
-                new int[] {1,1,1},
-                PChar.INSTANCE.toBytes("a1A"),
-                Bound.LOWER
-                ));
-        // 4, Lower bound, all range key, all exclusive on lower bound.
-        testCases.addAll(
-            foreach(new KeyRange[][]{{
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("a"), false, Bytes.toBytes("b"), true),},{
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("1"), false, Bytes.toBytes("2"), true),},{
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("A"), false, Bytes.toBytes("B"), true),}},
-                new int[] {1,1,1},
-                PChar.INSTANCE.toBytes("b2B"),
-                Bound.LOWER
-                ));
-        // 5, Lower bound, all range key, some exclusive.
-        testCases.addAll(
-            foreach(new KeyRange[][]{{
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("a"), false, Bytes.toBytes("b"), true),},{
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("2"), true),},{
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("A"), false, Bytes.toBytes("B"), true),}},
-                new int[] {1,1,1},
-                PChar.INSTANCE.toBytes("b1B"),
-                Bound.LOWER
-                ));
-        // 6, Lower bound, mixed single and range key, mixed inclusive and exclusive.
-        testCases.addAll(
-            foreach(new KeyRange[][]{{
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("a"), true, Bytes.toBytes("a"), true),},{
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("2"), true),},{
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("A"), false, Bytes.toBytes("B"), true),}},
-                new int[] {1,1,1},
-                PChar.INSTANCE.toBytes("a1B"),
-                Bound.LOWER
-                ));
-        // 7, Lower bound, unbound key in the middle, fixed length.
-        testCases.addAll(
-            foreach(new KeyRange[][]{{
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("a"), true, Bytes.toBytes("a"), true),},{
-                        KeyRange.EVERYTHING_RANGE,},{
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("A"), false, Bytes.toBytes("B"), true),}},
-                new int[] {1,1,1},
-                PChar.INSTANCE.toBytes("a"),
-                Bound.LOWER
-                ));
-        // 8, Lower bound, unbound key in the middle, variable length.
-        testCases.addAll(
-                foreach(new KeyRange[][]{{
-                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("a"), true, Bytes.toBytes("a"), true),},{
-                            KeyRange.EVERYTHING_RANGE,}},
-                    new int[] {1,1},
-                    PChar.INSTANCE.toBytes("a"),
-                    Bound.LOWER
-                    ));
-        // 9, Lower bound, unbound key at end, variable length.
-        testCases.addAll(
-            foreach(new KeyRange[][]{{
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("a"), true, Bytes.toBytes("a"), true),},{
-                        KeyRange.EVERYTHING_RANGE,},{
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("A"), true, Bytes.toBytes("B"), true),}},
-                new int[] {1,1,1},
-                PChar.INSTANCE.toBytes("a"),
-                Bound.LOWER
-                ));
-        // 10, Upper bound, all single keys, all inclusive, increment at end.
-        testCases.addAll(
-            foreach(new KeyRange[][]{{
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("a"), true, Bytes.toBytes("a"), true),},{
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("1"), true),},{
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("A"), true, Bytes.toBytes("A"), true),}},
-                new int[] {1,1,1},
-                PChar.INSTANCE.toBytes("a1B"),
-                Bound.UPPER
-                ));
-        // 11, Upper bound, all range keys, all inclusive, increment at end.
-        testCases.addAll(
-            foreach(new KeyRange[][]{{
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("a"), true, Bytes.toBytes("b"), true),},{
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("2"), true),},{
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("A"), true, Bytes.toBytes("B"), true),}},
-                new int[] {1,1,1},
-                PChar.INSTANCE.toBytes("b2C"),
-                Bound.UPPER
-                ));
-        // 12, Upper bound, all range keys, all exclusive, no increment at end.
-        testCases.addAll(
-            foreach(new KeyRange[][]{{
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("a"), true, Bytes.toBytes("b"), false),},{
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("2"), false),},{
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("A"), true, Bytes.toBytes("B"), false),}},
-                new int[] {1,1,1},
-                PChar.INSTANCE.toBytes("b"),
-                Bound.UPPER
-                ));
-        // 13, Upper bound, single inclusive, range inclusive, increment at end.
-        testCases.addAll(
-            foreach(new KeyRange[][]{{
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("a"), true, Bytes.toBytes("a"), true),},{
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("2"), true),}},
-                new int[] {1,1},
-                PChar.INSTANCE.toBytes("a3"),
-                Bound.UPPER
-                ));
-        // 14, Upper bound, range exclusive, single inclusive, increment at end.
-        testCases.addAll(
-            foreach(new KeyRange[][]{{
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("a"), true, Bytes.toBytes("b"), false),},{
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("1"), true),}},
-                new int[] {1,1},
-                PChar.INSTANCE.toBytes("b"),
-                Bound.UPPER
-                ));
-        // 15, Upper bound, range inclusive, single inclusive, increment at end.
-        testCases.addAll(
-            foreach(new KeyRange[][]{{
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("a"), true, Bytes.toBytes("b"), true),},{
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("1"), true),}},
-                new int[] {1,1},
-                PChar.INSTANCE.toBytes("b2"),
-                Bound.UPPER
-                ));
-        // 16, Upper bound, single inclusive, range exclusive, no increment at end.
-        testCases.addAll(
-            foreach(new KeyRange[][]{{
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("a"), true, Bytes.toBytes("a"), true),},{
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("2"), false),}},
-                new int[] {1,1},
-                PChar.INSTANCE.toBytes("a2"),
-                Bound.UPPER
-                ));
-        // 17, Upper bound, unbound key, fixed length;
-        testCases.addAll(
-            foreach(new KeyRange[][]{{
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("a"), true, Bytes.toBytes("a"), true),},{
-                        KeyRange.EVERYTHING_RANGE,}},
-                new int[] {1,1},
-                PChar.INSTANCE.toBytes("b"),
-                Bound.UPPER
-                ));
-        // 18, Upper bound, unbound key, variable length;
-        testCases.addAll(
-            foreach(new KeyRange[][]{{
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("a"), true, Bytes.toBytes("a"), true),},{
-                    KeyRange.EVERYTHING_RANGE,}},
-                new int[] {1,1},
-                PChar.INSTANCE.toBytes("b"),
-                Bound.UPPER
-                ));
-        // 19, Upper bound, keys wrapped around when incrementing.
-        testCases.addAll(
-            foreach(new KeyRange[][]{{
-                PChar.INSTANCE.getKeyRange(new byte[] {-1}, true, new byte[] {-1}, true)},{
-                PChar.INSTANCE.getKeyRange(new byte[] {-1}, true, new byte[] {-1}, true)}},
-                new int[] {1, 1},
-                ByteUtil.EMPTY_BYTE_ARRAY,
-                Bound.UPPER
-                ));
-        // 20, Variable length
-        testCases.addAll(
-            foreach(new KeyRange[][]{{
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("a"), true, Bytes.toBytes("a"), true),},{
-                    PVarchar.INSTANCE.getKeyRange(Bytes.toBytes("A"), true, Bytes.toBytes("B"), true),}},
-                new int[] {1,0},
-                ByteUtil.nextKey(ByteUtil.concat(PVarchar.INSTANCE.toBytes("aB"), QueryConstants.SEPARATOR_BYTE_ARRAY)),
-                Bound.UPPER
-                ));
-        return testCases;
-    }
+        @Parameters(name = "{0} {1} {2} {3} {4}")
+        public static Collection<Object> data() {
+            List<Object> testCases = Lists.newArrayList();
+            // 1, Lower bound, all single keys, all inclusive.
+            testCases.addAll(foreach(new KeyRange[][] {
+                            { PChar.INSTANCE.getKeyRange(Bytes.toBytes("a"), true, Bytes.toBytes("a"), true), },
+                            { PChar.INSTANCE.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("1"), true), },
+                            { PChar.INSTANCE.getKeyRange(Bytes.toBytes("A"), true, Bytes.toBytes("A"),
+                                    true), } }, new int[] { 1, 1, 1 }, PChar.INSTANCE.toBytes("a1A"),
+                    Bound.LOWER));
+            // 2, Lower bound, all range keys, all inclusive.
+            testCases.addAll(foreach(new KeyRange[][] {
+                            { PChar.INSTANCE.getKeyRange(Bytes.toBytes("a"), true, Bytes.toBytes("b"), true), },
+                            { PChar.INSTANCE.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("2"), true), },
+                            { PChar.INSTANCE.getKeyRange(Bytes.toBytes("A"), true, Bytes.toBytes("B"),
+                                    true), } }, new int[] { 1, 1, 1 }, PChar.INSTANCE.toBytes("a1A"),
+                    Bound.LOWER));
+            // 3, Lower bound, mixed single and range keys, all inclusive.
+            testCases.addAll(foreach(new KeyRange[][] {
+                            { PChar.INSTANCE.getKeyRange(Bytes.toBytes("a"), true, Bytes.toBytes("a"), true), },
+                            { PChar.INSTANCE.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("2"), true), },
+                            { PChar.INSTANCE.getKeyRange(Bytes.toBytes("A"), true, Bytes.toBytes("A"),
+                                    true), } }, new int[] { 1, 1, 1 }, PChar.INSTANCE.toBytes("a1A"),
+                    Bound.LOWER));
+            // 4, Lower bound, all range key, all exclusive on lower bound.
+            testCases.addAll(foreach(new KeyRange[][] {
+                            { PChar.INSTANCE.getKeyRange(Bytes.toBytes("a"), false, Bytes.toBytes("b"), true), },
+                            { PChar.INSTANCE.getKeyRange(Bytes.toBytes("1"), false, Bytes.toBytes("2"), true), },
+                            { PChar.INSTANCE.getKeyRange(Bytes.toBytes("A"), false, Bytes.toBytes("B"),
+                                    true), } }, new int[] { 1, 1, 1 }, PChar.INSTANCE.toBytes("b2B"),
+                    Bound.LOWER));
+            // 5, Lower bound, all range key, some exclusive.
+            testCases.addAll(foreach(new KeyRange[][] {
+                            { PChar.INSTANCE.getKeyRange(Bytes.toBytes("a"), false, Bytes.toBytes("b"), true), },
+                            { PChar.INSTANCE.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("2"), true), },
+                            { PChar.INSTANCE.getKeyRange(Bytes.toBytes("A"), false, Bytes.toBytes("B"),
+                                    true), } }, new int[] { 1, 1, 1 }, PChar.INSTANCE.toBytes("b1B"),
+                    Bound.LOWER));
+            // 6, Lower bound, mixed single and range key, mixed inclusive and exclusive.
+            testCases.addAll(foreach(new KeyRange[][] {
+                            { PChar.INSTANCE.getKeyRange(Bytes.toBytes("a"), true, Bytes.toBytes("a"), true), },
+                            { PChar.INSTANCE.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("2"), true), },
+                            { PChar.INSTANCE.getKeyRange(Bytes.toBytes("A"), false, Bytes.toBytes("B"),
+                                    true), } }, new int[] { 1, 1, 1 }, PChar.INSTANCE.toBytes("a1B"),
+                    Bound.LOWER));
+            // 7, Lower bound, unbound key in the middle, fixed length.
+            testCases.addAll(foreach(new KeyRange[][] {
+                            { PChar.INSTANCE.getKeyRange(Bytes.toBytes("a"), true, Bytes.toBytes("a"), true), },
+                            { KeyRange.EVERYTHING_RANGE, },
+                            { PChar.INSTANCE.getKeyRange(Bytes.toBytes("A"), false, Bytes.toBytes("B"),
+                                    true), } }, new int[] { 1, 1, 1 }, PChar.INSTANCE.toBytes("a"),
+                    Bound.LOWER));
+            // 8, Lower bound, unbound key in the middle, variable length.
+            testCases.addAll(foreach(new KeyRange[][] {
+                            { PChar.INSTANCE.getKeyRange(Bytes.toBytes("a"), true, Bytes.toBytes("a"), true), },
+                            { KeyRange.EVERYTHING_RANGE, } }, new int[] { 1, 1 }, PChar.INSTANCE.toBytes("a"),
+                    Bound.LOWER));
+            // 9, Lower bound, unbound key at end, variable length.
+            testCases.addAll(foreach(new KeyRange[][] {
+                            { PChar.INSTANCE.getKeyRange(Bytes.toBytes("a"), true, Bytes.toBytes("a"), true), },
+                            { KeyRange.EVERYTHING_RANGE, },
+                            { PChar.INSTANCE.getKeyRange(Bytes.toBytes("A"), true, Bytes.toBytes("B"),
+                                    true), } }, new int[] { 1, 1, 1 }, PChar.INSTANCE.toBytes("a"),
+                    Bound.LOWER));
+            // 10, Upper bound, all single keys, all inclusive, increment at end.
+            testCases.addAll(foreach(new KeyRange[][] {
+                            { PChar.INSTANCE.getKeyRange(Bytes.toBytes("a"), true, Bytes.toBytes("a"), true), },
+                            { PChar.INSTANCE.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("1"), true), },
+                            { PChar.INSTANCE.getKeyRange(Bytes.toBytes("A"), true, Bytes.toBytes("A"),
+                                    true), } }, new int[] { 1, 1, 1 }, PChar.INSTANCE.toBytes("a1B"),
+                    Bound.UPPER));
+            // 11, Upper bound, all range keys, all inclusive, increment at end.
+            testCases.addAll(foreach(new KeyRange[][] {
+                            { PChar.INSTANCE.getKeyRange(Bytes.toBytes("a"), true, Bytes.toBytes("b"), true), },
+                            { PChar.INSTANCE.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("2"), true), },
+                            { PChar.INSTANCE.getKeyRange(Bytes.toBytes("A"), true, Bytes.toBytes("B"),
+                                    true), } }, new int[] { 1, 1, 1 }, PChar.INSTANCE.toBytes("b2C"),
+                    Bound.UPPER));
+            // 12, Upper bound, all range keys, all exclusive, no increment at end.
+            testCases.addAll(foreach(new KeyRange[][] {
+                            { PChar.INSTANCE.getKeyRange(Bytes.toBytes("a"), true, Bytes.toBytes("b"), false), },
+                            { PChar.INSTANCE.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("2"), false), },
+                            { PChar.INSTANCE.getKeyRange(Bytes.toBytes("A"), true, Bytes.toBytes("B"),
+                                    false), } }, new int[] { 1, 1, 1 }, PChar.INSTANCE.toBytes("b"),
+                    Bound.UPPER));
+            // 13, Upper bound, single inclusive, range inclusive, increment at end.
+            testCases.addAll(foreach(new KeyRange[][] {
+                    { PChar.INSTANCE.getKeyRange(Bytes.toBytes("a"), true, Bytes.toBytes("a"), true), },
+                    { PChar.INSTANCE.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("2"),
+                            true), } }, new int[] { 1, 1 }, PChar.INSTANCE.toBytes("a3"), Bound.UPPER));
+            // 14, Upper bound, range exclusive, single inclusive, increment at end.
+            testCases.addAll(foreach(new KeyRange[][] {
+                    { PChar.INSTANCE.getKeyRange(Bytes.toBytes("a"), true, Bytes.toBytes("b"), false), },
+                    { PChar.INSTANCE.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("1"),
+                            true), } }, new int[] { 1, 1 }, PChar.INSTANCE.toBytes("b"), Bound.UPPER));
+            // 15, Upper bound, range inclusive, single inclusive, increment at end.
+            testCases.addAll(foreach(new KeyRange[][] {
+                    { PChar.INSTANCE.getKeyRange(Bytes.toBytes("a"), true, Bytes.toBytes("b"), true), },
+                    { PChar.INSTANCE.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("1"),
+                            true), } }, new int[] { 1, 1 }, PChar.INSTANCE.toBytes("b2"), Bound.UPPER));
+            // 16, Upper bound, single inclusive, range exclusive, no increment at end.
+            testCases.addAll(foreach(new KeyRange[][] {
+                            { PChar.INSTANCE.getKeyRange(Bytes.toBytes("a"), true, Bytes.toBytes("a"), true), },
+                            { PChar.INSTANCE.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("2"),
+                                    false), } }, new int[] { 1, 1 }, PChar.INSTANCE.toBytes("a2"),
+                    Bound.UPPER));
+            // 17, Upper bound, unbound key, fixed length;
+            testCases.addAll(foreach(new KeyRange[][] {
+                            { PChar.INSTANCE.getKeyRange(Bytes.toBytes("a"), true, Bytes.toBytes("a"), true), },
+                            { KeyRange.EVERYTHING_RANGE, } }, new int[] { 1, 1 }, PChar.INSTANCE.toBytes("b"),
+                    Bound.UPPER));
+            // 18, Upper bound, unbound key, variable length;
+            testCases.addAll(foreach(new KeyRange[][] {
+                            { PChar.INSTANCE.getKeyRange(Bytes.toBytes("a"), true, Bytes.toBytes("a"), true), },
+                            { KeyRange.EVERYTHING_RANGE, } }, new int[] { 1, 1 }, PChar.INSTANCE.toBytes("b"),
+                    Bound.UPPER));
+            // 19, Upper bound, keys wrapped around when incrementing.
+            testCases.addAll(foreach(new KeyRange[][] {
+                            { PChar.INSTANCE.getKeyRange(new byte[] { -1 }, true, new byte[] { -1 }, true) },
+                            { PChar.INSTANCE.getKeyRange(new byte[] { -1 }, true, new byte[] { -1 }, true) } },
+                    new int[] { 1, 1 }, ByteUtil.EMPTY_BYTE_ARRAY, Bound.UPPER));
+            // 20, Variable length
+            testCases.addAll(foreach(new KeyRange[][] {
+                            { PChar.INSTANCE.getKeyRange(Bytes.toBytes("a"), true, Bytes.toBytes("a"), true), },
+                            { PVarchar.INSTANCE.getKeyRange(Bytes.toBytes("A"), true, Bytes.toBytes("B"),
+                                    true), } }, new int[] { 1, 0 },
+                    ByteUtil.nextKey(ByteUtil.concat(PVarchar.INSTANCE.toBytes("aB"), QueryConstants.SEPARATOR_BYTE_ARRAY)),
+                    Bound.UPPER));
+            return testCases;
+        }
+
+        private static Collection<?> foreach(KeyRange[][] ranges, int[] widths, byte[] expectedKey,
+                Bound bound) {
+            List<List<KeyRange>> slots = Lists.transform(Lists.newArrayList(ranges), ARRAY_TO_LIST);
+            List<Object> ret = Lists.newArrayList();
+            ret.add(new Object[] { slots, widths, expectedKey, bound });
+            return ret;
+        }
 
-    private static Collection<?> foreach(KeyRange[][] ranges, int[] widths, byte[] expectedKey,
-            Bound bound) {
-        List<List<KeyRange>> slots = Lists.transform(Lists.newArrayList(ranges), ARRAY_TO_LIST);
-        List<Object> ret = Lists.newArrayList();
-        ret.add(new Object[] {slots, widths, expectedKey, bound});
-        return ret;
+        private static final Function<KeyRange[], List<KeyRange>> ARRAY_TO_LIST = new Function<KeyRange[], List<KeyRange>>() {
+            @Override public List<KeyRange> apply(KeyRange[] input) {
+                return Lists.newArrayList(input);
+            }
+        };
     }
 
-    private static final Function<KeyRange[], List<KeyRange>> ARRAY_TO_LIST = 
-            new Function<KeyRange[], List<KeyRange>>() {
-                @Override 
-                public List<KeyRange> apply(KeyRange[] input) {
-                    return Lists.newArrayList(input);
+    public static class NonParameterizedScanUtilTest {
+
+        @Test
+        public void testSlotsSaltedVarbinaryPk() {
+            byte[] key = new byte[1024];
+
+            RowKeySchemaBuilder builder = new RowKeySchemaBuilder(2);
+
+            builder.addField(new PDatum() {
+                @Override
+                public boolean isNullable() {
+                    return false;
+                }
+
+                @Override
+                public PDataType getDataType() {
+                    return PBinary.INSTANCE;
+                }
+
+                @Override
+                public Integer getMaxLength() {
+                    return 1;
+                }
+
+                @Override
+                public Integer getScale() {
+                    return null;
+                }
+
+                @Override
+                public SortOrder getSortOrder() {
+                    return SortOrder.getDefault();
+                }
+            }, false, SortOrder.getDefault());
+
+            builder.addField(new PDatum() {
+                @Override
+                public boolean isNullable() {
+                    return false;
+                }
+
+                @Override
+                public PDataType getDataType() {
+                    return PVarbinary.INSTANCE;
                 }
-    };
+
+                @Override
+                public Integer getMaxLength() {
+                    return 60;
+                }
+
+                @Override
+                public Integer getScale() {
+                    return null;
+                }
+
+                @Override
+                public SortOrder getSortOrder() {
+                    return SortOrder.getDefault();
+                }
+            }, false, SortOrder.getDefault());
+
+            List<KeyRange> ranges = Lists.newArrayList(KeyRange.getKeyRange(new byte[] { 0, 5 }));
+            List<List<KeyRange>> pkKeyRanges = Lists.newArrayList();
+            pkKeyRanges.add(ranges);
+
+            // For this case slots the salt bucket and key are one span
+            int[] slotSpans = new int[] { 1 };
+
+            int offset = ScanUtil.setKey(builder.build(), pkKeyRanges, slotSpans, new int[] { 0 }, Bound.UPPER, key, 0,
+                    0, slotSpans.length);
+            byte[] actualKey = new byte[offset];
+            System.arraycopy(key, 0, actualKey, 0, offset);
+            assertArrayEquals(new byte[] { 0, 5, 0 }, actualKey);
+        }
+    }
 }