You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@phoenix.apache.org by st...@apache.org on 2023/04/24 05:04:39 UTC

[phoenix] branch master updated: PHOENIX-6916 Cannot handle ranges where start is a prefix of end for desc columns

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

stoty 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 9a50b02ca6 PHOENIX-6916 Cannot handle ranges where start is a prefix of end for desc columns
9a50b02ca6 is described below

commit 9a50b02ca6ce5fcca7f1b7975bd22d605ad84dc0
Author: Istvan Toth <st...@apache.org>
AuthorDate: Sat Mar 25 05:20:15 2023 +0100

    PHOENIX-6916 Cannot handle ranges where start is a prefix of end for desc columns
    
    make KeyRange aware of normal/inverted ranges
---
 .../java/org/apache/phoenix/end2end/KeyOnlyIT.java |  99 ++-
 .../phoenix/end2end/ParallelIteratorsIT.java       |   3 +-
 .../org/apache/phoenix/compile/WhereOptimizer.java |  18 +-
 .../coprocessor/GlobalIndexRegionScanner.java      |   6 +-
 .../coprocessor/IndexRepairRegionScanner.java      |   3 +-
 .../phoenix/coprocessor/IndexerRegionScanner.java  |   3 +-
 .../phoenix/coprocessor/MetaDataEndpointImpl.java  |   6 +-
 .../coprocessor/UncoveredIndexRegionScanner.java   |   3 +-
 .../execute/PhoenixTxIndexMutationGenerator.java   |   4 +-
 .../expression/function/RoundDateExpression.java   |  19 +-
 .../phoenix/hbase/index/IndexRegionObserver.java   |   3 +-
 .../hbase/index/covered/data/CachedLocalTable.java |  10 +-
 .../java/org/apache/phoenix/query/KeyRange.java    |  67 +-
 .../org/apache/phoenix/schema/SaltingUtil.java     |   2 +-
 .../org/apache/phoenix/schema/types/PDataType.java |  23 +-
 .../apache/phoenix/schema/types/PTimestamp.java    |  48 +-
 .../java/org/apache/phoenix/util/ByteUtil.java     |  18 +-
 .../java/org/apache/phoenix/util/ScanUtil.java     |   2 +-
 .../apache/phoenix/compile/QueryCompilerTest.java  |  54 ++
 .../phoenix/compile/SaltedScanRangesTest.java      |  26 +-
 .../org/apache/phoenix/compile/ScanRangesTest.java | 276 +++----
 .../apache/phoenix/compile/WhereCompilerTest.java  |   2 +-
 .../apache/phoenix/compile/WhereOptimizerTest.java |   8 +-
 .../filter/SkipScanFilterIntersectTest.java        | 848 +++++++++++----------
 .../apache/phoenix/filter/SkipScanFilterTest.java  | 126 +--
 .../apache/phoenix/query/KeyRangeCoalesceTest.java |  57 +-
 .../phoenix/query/KeyRangeIntersectTest.java       |  35 +-
 .../org/apache/phoenix/query/KeyRangeMoreTest.java |  13 +-
 .../apache/phoenix/query/KeyRangeUnionTest.java    |  31 +-
 .../phoenix/query/ParallelIteratorsSplitTest.java  |   8 +-
 .../apache/phoenix/schema/types/PDataTypeTest.java |   4 +-
 .../java/org/apache/phoenix/util/ScanUtilTest.java |  92 +--
 32 files changed, 1082 insertions(+), 835 deletions(-)

diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/KeyOnlyIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/KeyOnlyIT.java
index 70542663cb..e942e3dfea 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/KeyOnlyIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/KeyOnlyIT.java
@@ -29,12 +29,15 @@ import java.sql.DriverManager;
 import java.sql.PreparedStatement;
 import java.sql.ResultSet;
 import java.sql.SQLException;
+import java.sql.Statement;
 import java.util.List;
 import java.util.Properties;
 
 import org.apache.phoenix.compile.ExplainPlan;
 import org.apache.phoenix.compile.ExplainPlanAttributes;
+import org.apache.phoenix.compile.QueryPlan;
 import org.apache.phoenix.jdbc.PhoenixPreparedStatement;
+import org.apache.phoenix.jdbc.PhoenixStatement;
 import org.apache.phoenix.query.KeyRange;
 import org.apache.phoenix.util.PropertiesUtil;
 import org.apache.phoenix.util.TestUtil;
@@ -137,7 +140,7 @@ public class KeyOnlyIT extends ParallelStatsEnabledIT {
     @Test
     public void testQueryWithLimitAndStats() throws Exception {
         Properties props = PropertiesUtil.deepCopy(TestUtil.TEST_PROPERTIES);
-        Connection conn = DriverManager.getConnection(getUrl(), props);        
+        Connection conn = DriverManager.getConnection(getUrl(), props);
         initTableValues(conn, 100);
         analyzeTable(conn, tableName);
         
@@ -162,7 +165,99 @@ public class KeyOnlyIT extends ParallelStatsEnabledIT {
         assertEquals(1, explainPlanAttributes.getServerRowLimit().intValue());
         assertEquals(1, explainPlanAttributes.getClientRowLimit().intValue());
     }
-    
+
+    @Test
+    public void testDescKeys() throws SQLException {
+        Properties props = PropertiesUtil.deepCopy(TestUtil.TEST_PROPERTIES);
+        Connection conn = DriverManager.getConnection(getUrl(), props);
+        String tableName = generateUniqueName();
+        try (Statement stmt = conn.createStatement()) {
+            stmt.execute("create table " + tableName + " (k varchar primary key desc)");
+            stmt.execute("upsert into " + tableName + " values ('a')");
+            stmt.execute("upsert into " + tableName + " values ('aa')");
+            stmt.execute("upsert into " + tableName + " values ('aaa')");
+            stmt.execute("upsert into " + tableName + " values ('aaab')");
+            conn.commit();
+
+            ResultSet rs = stmt.executeQuery("select * from " + tableName + " where k > 'a'");
+            assertTrue(rs.next());
+            assertEquals("aaab", rs.getString(1));
+            assertTrue(rs.next());
+            assertEquals("aaa", rs.getString(1));
+            assertTrue(rs.next());
+            assertEquals("aa", rs.getString(1));
+            assertFalse(rs.next());
+
+            rs = stmt.executeQuery("select * from " + tableName + " where k >= 'a'");
+            assertTrue(rs.next());
+            assertEquals("aaab", rs.getString(1));
+            assertTrue(rs.next());
+            assertEquals("aaa", rs.getString(1));
+            assertTrue(rs.next());
+            assertEquals("aa", rs.getString(1));
+            assertTrue(rs.next());
+            assertEquals("a", rs.getString(1));
+            assertFalse(rs.next());
+
+            rs = stmt.executeQuery("select * from " + tableName + " where k >= 'aaa'");
+            assertTrue(rs.next());
+            assertEquals("aaab", rs.getString(1));
+            assertTrue(rs.next());
+            assertEquals("aaa", rs.getString(1));
+            assertFalse(rs.next());
+
+            rs = stmt.executeQuery("select * from " + tableName + " where k > 'aaa'");
+            assertTrue(rs.next());
+            assertEquals("aaab", rs.getString(1));
+            assertFalse(rs.next());
+
+            rs = stmt.executeQuery("select * from " + tableName + " where k < 'a'");
+            assertFalse(rs.next());
+
+            rs = stmt.executeQuery("select * from " + tableName + " where k <= 'a'");
+            assertTrue(rs.next());
+            assertEquals("a", rs.getString(1));
+            assertFalse(rs.next());
+
+            rs = stmt.executeQuery("select * from " + tableName + " where k < 'aaa'");
+            assertTrue(rs.next());
+            assertEquals("aa", rs.getString(1));
+            assertTrue(rs.next());
+            assertEquals("a", rs.getString(1));
+            assertFalse(rs.next());
+
+            rs = stmt.executeQuery("select * from " + tableName + " where k <= 'aaa'");
+            assertTrue(rs.next());
+            assertEquals("aaa", rs.getString(1));
+            assertTrue(rs.next());
+            assertEquals("aa", rs.getString(1));
+            assertTrue(rs.next());
+            assertEquals("a", rs.getString(1));
+            assertFalse(rs.next());
+
+            rs = stmt.executeQuery("select * from " + tableName + " where k > 'a' and k < 'aaa'");
+            assertTrue(rs.next());
+            assertEquals("aa", rs.getString(1));
+            assertFalse(rs.next());
+
+            rs = stmt.executeQuery("select * from " + tableName + " where k > 'a' and k <= 'aaa'");
+            assertTrue(rs.next());
+            assertEquals("aaa", rs.getString(1));
+            assertTrue(rs.next());
+            assertEquals("aa", rs.getString(1));
+            assertFalse(rs.next());
+
+            rs = stmt.executeQuery("select * from " + tableName + " where k >= 'a' and k <= 'aaa'");
+            assertTrue(rs.next());
+            assertEquals("aaa", rs.getString(1));
+            assertTrue(rs.next());
+            assertEquals("aa", rs.getString(1));
+            assertTrue(rs.next());
+            assertEquals("a", rs.getString(1));
+            assertFalse(rs.next());
+        }
+    }
+
     private void initTableValues(Connection conn) throws Exception {
         PreparedStatement stmt = conn.prepareStatement(
             "upsert into " +
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ParallelIteratorsIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ParallelIteratorsIT.java
index 8ba9332c4f..93e3847b9a 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ParallelIteratorsIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ParallelIteratorsIT.java
@@ -41,6 +41,7 @@ import org.apache.phoenix.compile.QueryPlan;
 import org.apache.phoenix.coprocessor.BaseScannerRegionObserver;
 import org.apache.phoenix.jdbc.PhoenixStatement;
 import org.apache.phoenix.query.KeyRange;
+import org.apache.phoenix.schema.SortOrder;
 import org.apache.phoenix.schema.types.PChar;
 import org.apache.phoenix.util.TestUtil;
 import org.junit.Before;
@@ -214,7 +215,7 @@ public class ParallelIteratorsIT extends ParallelStatsEnabledIT {
     }
 
     private static KeyRange newKeyRange(byte[] lowerRange, byte[] upperRange) {
-        return PChar.INSTANCE.getKeyRange(lowerRange, true, upperRange, false);
+        return PChar.INSTANCE.getKeyRange(lowerRange, true, upperRange, false, SortOrder.ASC);
     }
     
     private void createTable (Connection conn, byte[][] splits) throws SQLException {
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 4d946888cd..94b1972c9d 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
@@ -97,8 +97,11 @@ import edu.umd.cs.findbugs.annotations.NonNull;
  * @since 0.1
  */
 public class WhereOptimizer {
-    private static final List<KeyRange> EVERYTHING_RANGES = Collections.<KeyRange>singletonList(KeyRange.EVERYTHING_RANGE);
-    private static final List<KeyRange> SALT_PLACEHOLDER = Collections.singletonList(PChar.INSTANCE.getKeyRange(QueryConstants.SEPARATOR_BYTE_ARRAY));
+    private static final List<KeyRange> EVERYTHING_RANGES =
+            Collections.<KeyRange> singletonList(KeyRange.EVERYTHING_RANGE);
+    private static final List<KeyRange> SALT_PLACEHOLDER =
+            Collections.singletonList(
+                PChar.INSTANCE.getKeyRange(QueryConstants.SEPARATOR_BYTE_ARRAY, SortOrder.ASC));
 
     private WhereOptimizer() {
     }
@@ -1714,7 +1717,7 @@ public class WhereOptimizer {
                 lowerRange = Arrays.copyOf(lowerRange, lowerRange.length+1);
                 lowerRange[lowerRange.length-1] = QueryConstants.SEPARATOR_BYTE;
             }
-            KeyRange range = type.getKeyRange(lowerRange, true, upperRange, false);
+            KeyRange range = type.getKeyRange(lowerRange, true, upperRange, false, SortOrder.ASC);
             if (column.getSortOrder() == SortOrder.DESC) {
                 range = range.invert();
             }
@@ -1769,8 +1772,9 @@ public class WhereOptimizer {
             // we represent NULL the same way for ASC and DESC
             if (isFixedWidth) { // if column can't be null
                 return node.isNegate() ? null :
-                    newKeyParts(childSlot, node, type.getKeyRange(new byte[SchemaUtil.getFixedByteSize(column)], true,
-                                                                  KeyRange.UNBOUND, true));
+                        newKeyParts(childSlot, node,
+                            type.getKeyRange(new byte[SchemaUtil.getFixedByteSize(column)], true,
+                                KeyRange.UNBOUND, true, SortOrder.ASC));
             } else {
                 KeyRange keyRange = node.isNegate() ? KeyRange.IS_NOT_NULL_RANGE : KeyRange.IS_NULL_RANGE;
                 return newKeyParts(childSlot, node, keyRange);
@@ -1942,7 +1946,7 @@ public class WhereOptimizer {
                     }
                 }
                 byte[] key = ByteUtil.copyKeyBytesIfNecessary(ptr);
-                KeyRange range = ByteUtil.getKeyRange(key, rhs.getSortOrder().transform(op)/*op*/, type);
+                KeyRange range = ByteUtil.getKeyRange(key, rhs.getSortOrder(), op, type);
                 // Constants will have been inverted, so we invert them back here so that
                 // RVC comparisons work correctly (see PHOENIX-3383).
                 if (rhs.getSortOrder() == SortOrder.DESC) {
@@ -2148,7 +2152,7 @@ public class WhereOptimizer {
                     return null;
                 }
                 byte[] key = ByteUtil.copyKeyBytesIfNecessary(ptr);
-                KeyRange range = ByteUtil.getKeyRange(key, /*rvc.getChildren().get(rhs.getChildren().size()-1).getSortOrder().transform(op)*/op, PVarbinary.INSTANCE);
+                KeyRange range = ByteUtil.getKeyRange(key, SortOrder.ASC, /*rvc.getChildren().get(rhs.getChildren().size()-1).getSortOrder().transform(op)*/op, PVarbinary.INSTANCE);
                 return range;
             }
 
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/GlobalIndexRegionScanner.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/GlobalIndexRegionScanner.java
index 83f4b65339..4eebee0457 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/GlobalIndexRegionScanner.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/GlobalIndexRegionScanner.java
@@ -62,6 +62,7 @@ import org.apache.phoenix.mapreduce.index.IndexVerificationOutputRepository;
 import org.apache.phoenix.mapreduce.index.IndexVerificationResultRepository;
 import org.apache.phoenix.query.KeyRange;
 import org.apache.phoenix.query.QueryServicesOptions;
+import org.apache.phoenix.schema.SortOrder;
 import org.apache.phoenix.schema.transform.TransformMaintainer;
 import org.apache.phoenix.schema.types.PVarbinary;
 import org.apache.phoenix.thirdparty.com.google.common.annotations.VisibleForTesting;
@@ -1069,7 +1070,7 @@ public abstract class GlobalIndexRegionScanner extends BaseRegionScanner {
     protected Scan prepareIndexScan(Map<byte[], List<Mutation>> indexMutationMap) throws IOException {
         List<KeyRange> keys = new ArrayList<>(indexMutationMap.size());
         for (byte[] indexKey : indexMutationMap.keySet()) {
-            keys.add(PVarbinary.INSTANCE.getKeyRange(indexKey));
+            keys.add(PVarbinary.INSTANCE.getKeyRange(indexKey, SortOrder.ASC));
         }
 
         ScanRanges scanRanges = ScanRanges.createPointLookup(keys);
@@ -1431,7 +1432,8 @@ public abstract class GlobalIndexRegionScanner extends BaseRegionScanner {
                             row.clear();
                             continue;
                         }
-                        keys.add(PVarbinary.INSTANCE.getKeyRange(CellUtil.cloneRow(row.get(0))));
+                        keys.add(PVarbinary.INSTANCE.getKeyRange(CellUtil.cloneRow(row.get(0)),
+                            SortOrder.ASC));
                         rowCount++;
                     }
                     row.clear();
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/IndexRepairRegionScanner.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/IndexRepairRegionScanner.java
index 22c63703b7..a8147fb6b4 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/IndexRepairRegionScanner.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/IndexRepairRegionScanner.java
@@ -58,6 +58,7 @@ import org.apache.phoenix.hbase.index.parallel.TaskBatch;
 import org.apache.phoenix.hbase.index.util.ImmutableBytesPtr;
 import org.apache.phoenix.mapreduce.index.IndexTool;
 import org.apache.phoenix.query.KeyRange;
+import org.apache.phoenix.schema.SortOrder;
 import org.apache.phoenix.schema.types.PLong;
 import org.apache.phoenix.schema.types.PVarbinary;
 import org.apache.phoenix.util.PhoenixKeyValueUtil;
@@ -143,7 +144,7 @@ public class IndexRepairRegionScanner extends GlobalIndexRegionScanner {
         Map<byte[], List<Mutation>> expectedIndexMutationMap = Maps.newTreeMap(Bytes.BYTES_COMPARATOR);
         List<KeyRange> keys = new ArrayList<>(dataRowKeys.size());
         for (byte[] indexKey: dataRowKeys) {
-            keys.add(PVarbinary.INSTANCE.getKeyRange(indexKey));
+            keys.add(PVarbinary.INSTANCE.getKeyRange(indexKey, SortOrder.ASC));
         }
         ScanRanges scanRanges = ScanRanges.createPointLookup(keys);
         Scan dataScan = new Scan();
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/IndexerRegionScanner.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/IndexerRegionScanner.java
index e8a5133023..839ddb2e22 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/IndexerRegionScanner.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/IndexerRegionScanner.java
@@ -71,6 +71,7 @@ import org.apache.phoenix.index.PhoenixIndexCodec;
 import org.apache.phoenix.mapreduce.index.IndexTool;
 import org.apache.phoenix.mapreduce.index.IndexVerificationResultRepository;
 import org.apache.phoenix.query.KeyRange;
+import org.apache.phoenix.schema.SortOrder;
 import org.apache.phoenix.schema.types.PLong;
 import org.apache.phoenix.schema.types.PVarbinary;
 import org.apache.phoenix.util.IndexUtil;
@@ -288,7 +289,7 @@ public class IndexerRegionScanner extends GlobalIndexRegionScanner {
         verificationPhaseResultList.add(perTaskVerificationPhaseResult);
 
         for (Map.Entry<byte[], Put> entry: indexKeyToDataPutMap.entrySet()) {
-            keys.add(PVarbinary.INSTANCE.getKeyRange(entry.getKey()));
+            keys.add(PVarbinary.INSTANCE.getKeyRange(entry.getKey(), SortOrder.ASC));
             perTaskDataKeyToDataPutMap.put(entry.getValue().getRow(), entry.getValue());
             if (keys.size() == rowCountPerTask) {
                 addVerifyTask(tasks, keys, perTaskDataKeyToDataPutMap, perTaskVerificationPhaseResult);
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java
index b101f533b4..4ea7b245ce 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java
@@ -756,7 +756,8 @@ TABLE_FAMILY_BYTES, TABLE_SEQ_NUM_BYTES);
         for (byte[] key : keys) {
             byte[] stopKey = ByteUtil.concat(key, QueryConstants.SEPARATOR_BYTE_ARRAY);
             ByteUtil.nextKey(stopKey, stopKey.length);
-            keyRanges.add(PVarbinary.INSTANCE.getKeyRange(key, true, stopKey, false));
+            keyRanges
+                    .add(PVarbinary.INSTANCE.getKeyRange(key, true, stopKey, false, SortOrder.ASC));
         }
         Scan scan = new Scan();
         scan.setTimeRange(MIN_TABLE_TIMESTAMP, clientTimeStamp);
@@ -792,7 +793,8 @@ TABLE_FAMILY_BYTES, TABLE_SEQ_NUM_BYTES);
         for (byte[] key : keys) {
             byte[] stopKey = ByteUtil.concat(key, QueryConstants.SEPARATOR_BYTE_ARRAY);
             ByteUtil.nextKey(stopKey, stopKey.length);
-            keyRanges.add(PVarbinary.INSTANCE.getKeyRange(key, true, stopKey, false));
+            keyRanges
+                    .add(PVarbinary.INSTANCE.getKeyRange(key, true, stopKey, false, SortOrder.ASC));
         }
         Scan scan = new Scan();
         if (clientTimeStamp != HConstants.LATEST_TIMESTAMP
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/UncoveredIndexRegionScanner.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/UncoveredIndexRegionScanner.java
index d1784d471f..8b1436ee10 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/UncoveredIndexRegionScanner.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/UncoveredIndexRegionScanner.java
@@ -40,6 +40,7 @@ import org.apache.phoenix.query.KeyRange;
 import org.apache.phoenix.query.QueryConstants;
 import org.apache.phoenix.query.QueryServices;
 import org.apache.phoenix.query.QueryServicesOptions;
+import org.apache.phoenix.schema.SortOrder;
 import org.apache.phoenix.schema.tuple.ResultTuple;
 import org.apache.phoenix.schema.types.PVarbinary;
 import org.apache.phoenix.thirdparty.com.google.common.collect.Maps;
@@ -171,7 +172,7 @@ public abstract class UncoveredIndexRegionScanner extends BaseRegionScanner {
     protected Scan prepareDataTableScan(Collection<byte[]> dataRowKeys) throws IOException {
         List<KeyRange> keys = new ArrayList<>(dataRowKeys.size());
         for (byte[] dataRowKey : dataRowKeys) {
-            keys.add(PVarbinary.INSTANCE.getKeyRange(dataRowKey));
+            keys.add(PVarbinary.INSTANCE.getKeyRange(dataRowKey, SortOrder.ASC));
         }
         ScanRanges scanRanges = ScanRanges.createPointLookup(keys);
         Scan dataScan = new Scan(dataTableScan);
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/execute/PhoenixTxIndexMutationGenerator.java b/phoenix-core/src/main/java/org/apache/phoenix/execute/PhoenixTxIndexMutationGenerator.java
index d5b9faedb3..d40bf44d97 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/execute/PhoenixTxIndexMutationGenerator.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/execute/PhoenixTxIndexMutationGenerator.java
@@ -65,6 +65,7 @@ import org.apache.phoenix.jdbc.PhoenixConnection;
 import org.apache.phoenix.query.ConnectionQueryServices;
 import org.apache.phoenix.query.KeyRange;
 import org.apache.phoenix.schema.PTable;
+import org.apache.phoenix.schema.SortOrder;
 import org.apache.phoenix.schema.types.PVarbinary;
 import org.apache.phoenix.transaction.PhoenixTransactionContext;
 import org.apache.phoenix.transaction.PhoenixTransactionContext.PhoenixVisibilityLevel;
@@ -173,7 +174,8 @@ public class PhoenixTxIndexMutationGenerator {
         if (!findPriorValueMutations.isEmpty()) {
             List<KeyRange> keys = Lists.newArrayListWithExpectedSize(mutations.size());
             for (ImmutableBytesPtr ptr : findPriorValueMutations.keySet()) {
-                keys.add(PVarbinary.INSTANCE.getKeyRange(ptr.copyBytesIfNecessary()));
+                keys.add(
+                    PVarbinary.INSTANCE.getKeyRange(ptr.copyBytesIfNecessary(), SortOrder.ASC));
             }
             Scan scan = new Scan();
             // Project all mutable columns
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/RoundDateExpression.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/RoundDateExpression.java
index 4a88995584..6b8271a111 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/RoundDateExpression.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/RoundDateExpression.java
@@ -278,6 +278,7 @@ public class RoundDateExpression extends ScalarFunction {
                 long value = codec.decodeLong(key, 0, SortOrder.getDefault());
                 byte[] lowerKey = new byte[type.getByteSize()];
                 byte[] upperKey = new byte[type.getByteSize()];
+                SortOrder order = this.getColumn().getSortOrder();
                 KeyRange range;
                 switch (op) {
                 case EQUAL:
@@ -291,7 +292,7 @@ public class RoundDateExpression extends ScalarFunction {
                     }
                     codec.encodeLong(rangeLower(value), lowerKey, 0);
                     codec.encodeLong(rangeUpper(value), upperKey, 0);
-                    range = type.getKeyRange(lowerKey, true, upperKey, true);
+                    range = type.getKeyRange(lowerKey, true, upperKey, true, order);
                     break;
                     // a simple number example (with half up rounding):
                     // round(x) = 10 ==> [9.5, 10.5)
@@ -302,41 +303,41 @@ public class RoundDateExpression extends ScalarFunction {
                 case GREATER:
                     if (value == roundTime(value)) {
                         codec.encodeLong(rangeUpper(value), lowerKey, 0);
-                        range = type.getKeyRange(lowerKey, false, KeyRange.UNBOUND, false);
+                        range = type.getKeyRange(lowerKey, false, KeyRange.UNBOUND, false, order);
                         break;
                     }
                     //fallthrough intended
                 case GREATER_OR_EQUAL:
                     codec.encodeLong(rangeLower(value), lowerKey, 0);
-                    range = type.getKeyRange(lowerKey, true, KeyRange.UNBOUND, false);
+                    range = type.getKeyRange(lowerKey, true, KeyRange.UNBOUND, false, order);
                     if (value <= roundTime(value)) {
                         //always true for ceil
                         codec.encodeLong(rangeLower(value), lowerKey, 0);
-                        range = type.getKeyRange(lowerKey, true, KeyRange.UNBOUND, false);
+                        range = type.getKeyRange(lowerKey, true, KeyRange.UNBOUND, false, order);
                     } else {
                         //always true for floor, except when exact
                         codec.encodeLong(rangeUpper(value), lowerKey, 0);
-                        range = type.getKeyRange(lowerKey, false, KeyRange.UNBOUND, false);
+                        range = type.getKeyRange(lowerKey, false, KeyRange.UNBOUND, false, order);
                     }
                     break;
                 case LESS:
                     if (value == roundTime(value)) {
                         codec.encodeLong(rangeLower(value), upperKey, 0);
-                        range = type.getKeyRange(KeyRange.UNBOUND, false, upperKey, false);
+                        range = type.getKeyRange(KeyRange.UNBOUND, false, upperKey, false, order);
                         break;
                     }
                     //fallthrough intended
                 case LESS_OR_EQUAL:
                     codec.encodeLong(rangeUpper(value), upperKey, 0);
-                    range = type.getKeyRange(KeyRange.UNBOUND, false, upperKey, true);
+                    range = type.getKeyRange(KeyRange.UNBOUND, false, upperKey, true, order);
                     if (value >= roundTime(value)) {
                         //always true for floor
                         codec.encodeLong(rangeUpper(value), upperKey, 0);
-                        range = type.getKeyRange(KeyRange.UNBOUND, false, upperKey, true);
+                        range = type.getKeyRange(KeyRange.UNBOUND, false, upperKey, true, order);
                     } else {
                         //always true for ceil, except when exact
                         codec.encodeLong(rangeLower(value), upperKey, 0);
-                        range = type.getKeyRange(KeyRange.UNBOUND, false, upperKey, false);
+                        range = type.getKeyRange(KeyRange.UNBOUND, false, upperKey, false, order);
                     }
                     break;
                 default:
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/IndexRegionObserver.java b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/IndexRegionObserver.java
index 54222d4a75..4738f63042 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/IndexRegionObserver.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/IndexRegionObserver.java
@@ -112,6 +112,7 @@ import org.apache.phoenix.schema.PRow;
 import org.apache.phoenix.schema.PTable;
 import org.apache.phoenix.schema.PTableImpl;
 import org.apache.phoenix.schema.PTableType;
+import org.apache.phoenix.schema.SortOrder;
 import org.apache.phoenix.schema.tuple.MultiKeyValueTuple;
 import org.apache.phoenix.schema.transform.TransformMaintainer;
 import org.apache.phoenix.schema.types.PVarbinary;
@@ -864,7 +865,7 @@ public class IndexRegionObserver implements RegionCoprocessor, RegionObserver {
                 }
             }
             else {
-                keys.add(PVarbinary.INSTANCE.getKeyRange(rowKeyPtr.get()));
+                keys.add(PVarbinary.INSTANCE.getKeyRange(rowKeyPtr.get(), SortOrder.ASC));
             }
         }
         if (keys.isEmpty()) {
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/covered/data/CachedLocalTable.java b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/covered/data/CachedLocalTable.java
index 5edc59acb0..eeab9f4264 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/covered/data/CachedLocalTable.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/covered/data/CachedLocalTable.java
@@ -41,6 +41,7 @@ import org.apache.phoenix.hbase.index.util.IndexManagementUtil;
 import org.apache.phoenix.index.IndexMaintainer;
 import org.apache.phoenix.index.PhoenixIndexMetaData;
 import org.apache.phoenix.query.KeyRange;
+import org.apache.phoenix.schema.SortOrder;
 import org.apache.phoenix.schema.types.PVarbinary;
 
 import org.apache.phoenix.thirdparty.com.google.common.annotations.VisibleForTesting;
@@ -124,11 +125,12 @@ public class CachedLocalTable implements LocalHBaseState {
             Collection<? extends Mutation> dataTableMutationsWithSameRowKeyAndTimestamp,
             PhoenixIndexMetaData indexMetaData,
             Region region) throws IOException {
-        Set<KeyRange> keys = new HashSet<KeyRange>(dataTableMutationsWithSameRowKeyAndTimestamp.size());
+        Set<KeyRange> keys =
+                new HashSet<KeyRange>(dataTableMutationsWithSameRowKeyAndTimestamp.size());
         for (Mutation mutation : dataTableMutationsWithSameRowKeyAndTimestamp) {
-          if (indexMetaData.requiresPriorRowState(mutation)) {
-            keys.add(PVarbinary.INSTANCE.getKeyRange(mutation.getRow()));
-          }
+            if (indexMetaData.requiresPriorRowState(mutation)) {
+                keys.add(PVarbinary.INSTANCE.getKeyRange(mutation.getRow(), SortOrder.ASC));
+            }
         }
         if (keys.isEmpty()) {
             return new CachedLocalTable(Collections.emptyMap(), region);
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 e154480743..4bde3cf07c 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
@@ -34,11 +34,10 @@ import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.WritableUtils;
 import org.apache.phoenix.execute.DescVarLengthFastByteComparisons;
 import org.apache.phoenix.schema.SortOrder;
-import org.apache.phoenix.util.ByteUtil;
-import org.apache.phoenix.util.ScanUtil.BytesComparator;
-
 import org.apache.phoenix.thirdparty.com.google.common.base.Function;
 import org.apache.phoenix.thirdparty.com.google.common.collect.Lists;
+import org.apache.phoenix.util.ByteUtil;
+import org.apache.phoenix.util.ScanUtil.BytesComparator;
 
 import edu.umd.cs.findbugs.annotations.NonNull;
 
@@ -59,28 +58,28 @@ public class KeyRange implements Writable {
      * is what we use for upper/lower bound), we create this range using the private constructor rather than
      * going through the static creation method (where this would not be possible).
      */
-    public static final KeyRange IS_NULL_RANGE = new KeyRange(NULL_BOUND, true, NULL_BOUND, true);
+    public static final KeyRange IS_NULL_RANGE = new KeyRange(NULL_BOUND, true, NULL_BOUND, true, false);
     /**
      * KeyRange for non null variable length values. Since we need to represent this using an empty byte array (which
      * is what we use for upper/lower bound), we create this range using the private constructor rather than going
      * through the static creation method (where this would not be possible).
      */
-    public static final KeyRange IS_NOT_NULL_RANGE = new KeyRange(ByteUtil.nextKey(QueryConstants.SEPARATOR_BYTE_ARRAY), true, UNBOUND, false);
+    public static final KeyRange IS_NOT_NULL_RANGE = new KeyRange(ByteUtil.nextKey(QueryConstants.SEPARATOR_BYTE_ARRAY), true, UNBOUND, false, false);
     
     /**
      * KeyRange for an empty key range
      */
-    public static final KeyRange EMPTY_RANGE = new KeyRange(DEGENERATE_KEY, false, DEGENERATE_KEY, false);
+    public static final KeyRange EMPTY_RANGE = new KeyRange(DEGENERATE_KEY, false, DEGENERATE_KEY, false, false);
     
     /**
      * KeyRange that contains all values
      */
-    public static final KeyRange EVERYTHING_RANGE = new KeyRange(UNBOUND, false, UNBOUND, false);
+    public static final KeyRange EVERYTHING_RANGE = new KeyRange(UNBOUND, false, UNBOUND, false, false);
     
     public static final Function<byte[], KeyRange> POINT = new Function<byte[], KeyRange>() {
         @Override 
         public KeyRange apply(byte[] input) {
-            return new KeyRange(input, true, input, true);
+            return new KeyRange(input, true, input, true, false);
         }
     };
     public static final Comparator<KeyRange> COMPARATOR = new Comparator<KeyRange>() {
@@ -137,11 +136,12 @@ public class KeyRange implements Writable {
         }
     };
 
-    private byte[] lowerRange;
-    private boolean lowerInclusive;
-    private byte[] upperRange;
-    private boolean upperInclusive;
-    private boolean isSingleKey;
+    protected byte[] lowerRange;
+    protected boolean lowerInclusive;
+    protected byte[] upperRange;
+    protected boolean upperInclusive;
+    protected boolean isSingleKey;
+    protected boolean inverted = false;
 
     public static KeyRange getKeyRange(byte[] point) {
         return getKeyRange(point, true, point, true);
@@ -153,6 +153,12 @@ public class KeyRange implements Writable {
 
     private static KeyRange getSingleton(byte[] lowerRange, boolean lowerInclusive,
             byte[] upperRange, boolean upperInclusive) {
+        return getSingleton(lowerRange, lowerInclusive,
+            upperRange, upperInclusive, false);
+    }
+
+    private static KeyRange getSingleton(byte[] lowerRange, boolean lowerInclusive,
+            byte[] upperRange, boolean upperInclusive, boolean inverted) {
         if (lowerRange == null || upperRange == null) {
             return EMPTY_RANGE;
         }
@@ -161,18 +167,32 @@ public class KeyRange implements Writable {
             // than an unbound RANGE.
             return lowerInclusive && upperInclusive ? IS_NULL_RANGE : EVERYTHING_RANGE;
         }
-        if ( ( lowerRange.length != 0 || lowerRange == NULL_BOUND ) && ( upperRange.length != 0 || upperRange == NULL_BOUND ) ) {
-            int cmp = Bytes.compareTo(lowerRange, upperRange);
+        if ((lowerRange.length != 0 || lowerRange == NULL_BOUND)
+                && (upperRange.length != 0 || upperRange == NULL_BOUND)) {
+            int cmp;
+            if (inverted) {
+                // Allow illegal ranges to be defined. These will be fixed during processing.
+                cmp =
+                        Bytes.compareTo(SortOrder.invert(upperRange, 0, upperRange.length),
+                            SortOrder.invert(lowerRange, 0, lowerRange.length));
+            } else {
+                cmp = Bytes.compareTo(lowerRange, upperRange);
+            }
             if (cmp > 0 || (cmp == 0 && !(lowerInclusive && upperInclusive))) {
                 return EMPTY_RANGE;
             }
         }
         return null;
     }
-    
+
     public static KeyRange getKeyRange(byte[] lowerRange, boolean lowerInclusive,
             byte[] upperRange, boolean upperInclusive) {
-        KeyRange range = getSingleton(lowerRange, lowerInclusive, upperRange, upperInclusive);
+        return getKeyRange(lowerRange, lowerInclusive,
+            upperRange, upperInclusive, false);
+    }
+    public static KeyRange getKeyRange(byte[] lowerRange, boolean lowerInclusive,
+            byte[] upperRange, boolean upperInclusive, boolean inverted) {
+        KeyRange range = getSingleton(lowerRange, lowerInclusive, upperRange, upperInclusive, inverted);
         if (range != null) {
             return range;
         }
@@ -190,7 +210,7 @@ public class KeyRange implements Writable {
         }
 
         return new KeyRange(lowerRange, unboundLower ? false : lowerInclusive,
-                upperRange, unboundUpper ? false : upperInclusive);
+                upperRange, unboundUpper ? false : upperInclusive, inverted);
     }
 
     public static KeyRange read(DataInput input) throws IOException {
@@ -205,7 +225,7 @@ public class KeyRange implements Writable {
         return range;
     }
     
-    private KeyRange() {
+    protected KeyRange() {
         this.lowerRange = DEGENERATE_KEY;
         this.lowerInclusive = false;
         this.upperRange = DEGENERATE_KEY;
@@ -213,11 +233,12 @@ public class KeyRange implements Writable {
         this.isSingleKey = false;
     }
     
-    private KeyRange(byte[] lowerRange, boolean lowerInclusive, byte[] upperRange, boolean upperInclusive) {
+    protected KeyRange(byte[] lowerRange, boolean lowerInclusive, byte[] upperRange, boolean upperInclusive, boolean inverted) {
         this.lowerRange = lowerRange;
         this.lowerInclusive = lowerInclusive;
         this.upperRange = upperRange;
         this.upperInclusive = upperInclusive;
+        this.inverted = inverted;
         init();
     }
     
@@ -439,7 +460,8 @@ public class KeyRange implements Writable {
                 && newUpperRange == upperRange && newUpperInclusive == upperInclusive) {
             return this;
         }
-        return getKeyRange(newLowerRange, newLowerInclusive, newUpperRange, newUpperInclusive);
+        return getKeyRange(newLowerRange, newLowerInclusive, newUpperRange, newUpperInclusive,
+            this.inverted && range.inverted);
     }
 
     public static boolean isDegenerate(byte[] lowerRange, byte[] upperRange) {
@@ -616,6 +638,7 @@ public class KeyRange implements Writable {
         return result;
     }
 
+    // The range generated here is possibly invalid
     public KeyRange invert() {
         // these special ranges do not get inverted because we
         // represent NULL in the same way for ASC and DESC.
@@ -635,7 +658,7 @@ public class KeyRange implements Writable {
                 upperBound = SortOrder.invert(upperBound, 0, upperBound.length);
             }
         }
-        return KeyRange.getKeyRange(upperBound, this.isUpperInclusive(), lowerBound, this.isLowerInclusive());
+        return KeyRange.getKeyRange(upperBound, this.isUpperInclusive(), lowerBound, this.isLowerInclusive(), !this.inverted);
     }
 
     @Override
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/SaltingUtil.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/SaltingUtil.java
index 613e3cae4d..60633877d4 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/SaltingUtil.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/SaltingUtil.java
@@ -50,7 +50,7 @@ public class SaltingUtil {
         for (int i=0; i<bucketNum; i++) {
             byte[] saltByte = new byte[] {(byte) i};
             allRanges.add(SALTING_COLUMN.getDataType().getKeyRange(
-                    saltByte, true, saltByte, true));
+                    saltByte, true, saltByte, true, SortOrder.ASC));
         }
         return allRanges;
     }
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PDataType.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PDataType.java
index cfa8a56141..c3071ba50d 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PDataType.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PDataType.java
@@ -1096,8 +1096,8 @@ public abstract class PDataType<T> implements DataType<T>, Comparable<PDataType<
         return this.sqlType;
     }
 
-    public KeyRange getKeyRange(byte[] point) {
-        return getKeyRange(point, true, point, true);
+    public KeyRange getKeyRange(byte[] point, SortOrder order) {
+        return getKeyRange(point, true, point, true, order);
     }
 
     public final String toStringLiteral(ImmutableBytesWritable ptr, Format formatter) {
@@ -1146,14 +1146,16 @@ public abstract class PDataType<T> implements DataType<T>, Comparable<PDataType<
         return actualType.getArrayFactory().newArray(actualType, elements);
     }
 
-    public KeyRange getKeyRange(byte[] lowerRange, boolean lowerInclusive, byte[] upperRange, boolean upperInclusive) {
+    public KeyRange getKeyRange(byte[] lowerRange, boolean lowerInclusive, byte[] upperRange,
+            boolean upperInclusive, SortOrder order) {
         /*
-         * Force lower bound to be inclusive for fixed width keys because it makes comparisons less expensive when you
-         * can count on one bound or the other being inclusive. Comparing two fixed width exclusive bounds against each
-         * other is inherently more expensive, because you need to take into account if the bigger key is equal to the
-         * next key after the smaller key. For example: (A-B] compared against [A-B) An exclusive lower bound A is
-         * bigger than an exclusive upper bound B. Forcing a fixed width exclusive lower bound key to be inclusive
-         * prevents us from having to do this extra logic in the compare function.
+         * Force lower bound to be inclusive for fixed width keys because it makes comparisons less
+         * expensive when you can count on one bound or the other being inclusive. Comparing two
+         * fixed width exclusive bounds against each other is inherently more expensive, because you
+         * need to take into account if the bigger key is equal to the next key after the smaller
+         * key. For example: (A-B] compared against [A-B) An exclusive lower bound A is bigger than
+         * an exclusive upper bound B. Forcing a fixed width exclusive lower bound key to be
+         * inclusive prevents us from having to do this extra logic in the compare function.
          */
         if (lowerRange != KeyRange.UNBOUND && !lowerInclusive && isFixedWidth()) {
             lowerRange = ByteUtil.nextKey(lowerRange);
@@ -1162,7 +1164,8 @@ public abstract class PDataType<T> implements DataType<T>, Comparable<PDataType<
             }
             lowerInclusive = true;
         }
-        return KeyRange.getKeyRange(lowerRange, lowerInclusive, upperRange, upperInclusive);
+        return KeyRange.getKeyRange(lowerRange, lowerInclusive, upperRange, upperInclusive,
+            order == SortOrder.DESC);
     }
 
     //TODO this could be improved by some lookup tables instead of iterating over all types
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PTimestamp.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PTimestamp.java
index 4d26807a9e..53149ec22b 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PTimestamp.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PTimestamp.java
@@ -293,39 +293,47 @@ public class PTimestamp extends PDataType<Timestamp> {
     }
 
     /**
-     * With timestamp, because our last 4 bytes store a value from [0 - 1000000), we need
-     * to detect when the boundary is crossed if we increment to the nextKey.
+     * With timestamp, because our last 4 bytes store a value from [0 - 1000000), we need to detect
+     * when the boundary is crossed if we increment to the nextKey.
      */
     @Override
-    public KeyRange getKeyRange(byte[] lowerRange, boolean lowerInclusive, byte[] upperRange, boolean upperInclusive) {
+    public KeyRange getKeyRange(byte[] lowerRange, boolean lowerInclusive, byte[] upperRange,
+            boolean upperInclusive, SortOrder sortOrder) {
         /*
-         * Force lower bound to be inclusive for fixed width keys because it makes comparisons less expensive when you
-         * can count on one bound or the other being inclusive. Comparing two fixed width exclusive bounds against each
-         * other is inherently more expensive, because you need to take into account if the bigger key is equal to the
-         * next key after the smaller key. For example: (A-B] compared against [A-B) An exclusive lower bound A is
-         * bigger than an exclusive upper bound B. Forcing a fixed width exclusive lower bound key to be inclusive
-         * prevents us from having to do this extra logic in the compare function.
-         * 
+         * Force lower bound to be inclusive for fixed width keys because it makes comparisons less
+         * expensive when you can count on one bound or the other being inclusive. Comparing two
+         * fixed width exclusive bounds against each other is inherently more expensive, because you
+         * need to take into account if the bigger key is equal to the next key after the smaller
+         * key. For example: (A-B] compared against [A-B) An exclusive lower bound A is bigger than
+         * an exclusive upper bound B. Forcing a fixed width exclusive lower bound key to be
+         * inclusive prevents us from having to do this extra logic in the compare function.
          */
         if (lowerRange != KeyRange.UNBOUND && !lowerInclusive && isFixedWidth()) {
             if (lowerRange.length != MAX_TIMESTAMP_BYTES) {
-                throw new IllegalDataException("Unexpected size of " + lowerRange.length + " for " + this);
+                throw new IllegalDataException(
+                        "Unexpected size of " + lowerRange.length + " for " + this);
             }
-            // Infer sortOrder based on most significant byte
-            SortOrder sortOrder = lowerRange[Bytes.SIZEOF_LONG] < 0 ? SortOrder.DESC : SortOrder.ASC;
-            int nanos = PUnsignedInt.INSTANCE.getCodec().decodeInt(lowerRange, Bytes.SIZEOF_LONG, sortOrder);
-            if ((sortOrder == SortOrder.DESC && nanos == 0) || (sortOrder == SortOrder.ASC && nanos == MAX_NANOS_VALUE_EXCLUSIVE-1)) {
-                // With timestamp, because our last 4 bytes store a value from [0 - 1000000), we need
+
+            int nanos =
+                    PUnsignedInt.INSTANCE.getCodec().decodeInt(lowerRange, Bytes.SIZEOF_LONG,
+                        sortOrder);
+            if ((sortOrder == SortOrder.DESC && nanos == 0)
+                    || (sortOrder == SortOrder.ASC && nanos == MAX_NANOS_VALUE_EXCLUSIVE - 1)) {
+                // With timestamp, because our last 4 bytes store a value from [0 - 1000000), we
+                // need
                 // to detect when the boundary is crossed with our nextKey
                 byte[] newLowerRange = new byte[MAX_TIMESTAMP_BYTES];
                 if (sortOrder == SortOrder.DESC) {
                     // Set nanos part as inverted 999999 as it needs to be the max nano value
                     // The millisecond part is moving to the previous value below
                     System.arraycopy(lowerRange, 0, newLowerRange, 0, Bytes.SIZEOF_LONG);
-                    PUnsignedInt.INSTANCE.getCodec().encodeInt(MAX_NANOS_VALUE_EXCLUSIVE-1, newLowerRange, Bytes.SIZEOF_LONG);
-                    SortOrder.invert(newLowerRange, Bytes.SIZEOF_LONG, newLowerRange, Bytes.SIZEOF_LONG, Bytes.SIZEOF_INT);
+                    PUnsignedInt.INSTANCE.getCodec().encodeInt(MAX_NANOS_VALUE_EXCLUSIVE - 1,
+                        newLowerRange, Bytes.SIZEOF_LONG);
+                    SortOrder.invert(newLowerRange, Bytes.SIZEOF_LONG, newLowerRange,
+                        Bytes.SIZEOF_LONG, Bytes.SIZEOF_INT);
                 } else {
-                    // Leave nanos part as zero as the millisecond part is rolling over to the next value
+                    // Leave nanos part as zero as the millisecond part is rolling over to the next
+                    // value
                     System.arraycopy(lowerRange, 0, newLowerRange, 0, Bytes.SIZEOF_LONG);
                 }
                 // Increment millisecond part, but leave nanos alone
@@ -337,7 +345,7 @@ public class PTimestamp extends PDataType<Timestamp> {
                 return KeyRange.getKeyRange(lowerRange, true, upperRange, upperInclusive);
             }
         }
-        return super.getKeyRange(lowerRange, lowerInclusive, upperRange, upperInclusive);
+        return super.getKeyRange(lowerRange, lowerInclusive, upperRange, upperInclusive, sortOrder);
     }
 
 }
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 e616519670..a1bb50807e 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
@@ -588,24 +588,26 @@ public class ByteUtil {
         }
         return ptr.copyBytes();
     }
-    
-    public static KeyRange getKeyRange(byte[] key, CompareOperator op, PDataType type) {
+
+    public static KeyRange getKeyRange(byte[] key, SortOrder order, CompareOperator op,
+            PDataType type) {
+        op = order.transform(op);
         switch (op) {
         case EQUAL:
-            return type.getKeyRange(key, true, key, true);
+            return type.getKeyRange(key, true, key, true, order);
         case GREATER:
-            return type.getKeyRange(key, false, KeyRange.UNBOUND, false);
+            return type.getKeyRange(key, false, KeyRange.UNBOUND, false, order);
         case GREATER_OR_EQUAL:
-            return type.getKeyRange(key, true, KeyRange.UNBOUND, false);
+            return type.getKeyRange(key, true, KeyRange.UNBOUND, false, order);
         case LESS:
-            return type.getKeyRange(KeyRange.UNBOUND, false, key, false);
+            return type.getKeyRange(KeyRange.UNBOUND, false, key, false, order);
         case LESS_OR_EQUAL:
-            return type.getKeyRange(KeyRange.UNBOUND, false, key, true);
+            return type.getKeyRange(KeyRange.UNBOUND, false, key, true, order);
         default:
             throw new IllegalArgumentException("Unknown operator " + op);
         }
     }
-    
+
     public static boolean contains(Collection<byte[]> keys, byte[] key) {
         for (byte[] k : keys) {
             if (Arrays.equals(k, key)) { return true; }
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 4136388035..107a24866e 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
@@ -618,7 +618,7 @@ public class ScanUtil {
     public static ScanRanges newScanRanges(List<? extends Mutation> mutations) throws SQLException {
         List<KeyRange> keys = Lists.newArrayListWithExpectedSize(mutations.size());
         for (Mutation m : mutations) {
-            keys.add(PVarbinary.INSTANCE.getKeyRange(m.getRow()));
+            keys.add(PVarbinary.INSTANCE.getKeyRange(m.getRow(), SortOrder.ASC));
         }
         ScanRanges keyRanges = ScanRanges.createPointLookup(keys);
         return keyRanges;
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 3955059718..1c30618c7e 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
@@ -6927,4 +6927,58 @@ public class QueryCompilerTest extends BaseConnectionlessQueryTest {
             conn.close();
         }
     }
+
+    @Test
+    public void testReverseIndexRangeBugPhoenix6916() throws Exception {
+        String tableName = generateUniqueName();
+        String indexName = generateUniqueName();
+        try (Connection conn = DriverManager.getConnection(getUrl());
+                Statement stmt = conn.createStatement()) {
+            stmt.execute("create table " + tableName + " (id varchar primary key, ts timestamp)");
+            stmt.execute("create index " + indexName + " on " + tableName + "(ts desc)");
+
+            String query =
+                    "select id, ts from " + tableName
+                            + " where ts >= TIMESTAMP '2023-02-23 13:30:00'  and ts < TIMESTAMP '2023-02-23 13:40:00'";
+            ResultSet rs = stmt.executeQuery("EXPLAIN " + query);
+            String explainPlan = QueryUtil.getExplainPlan(rs);
+            assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + indexName
+                    + " [~1,677,159,600,000] - [~1,677,159,000,000]\n    SERVER FILTER BY FIRST KEY ONLY",
+                explainPlan);
+        }
+    }
+
+    @Test
+    public void testReverseVarLengthRange6916() throws Exception {
+        String tableName = generateUniqueName();
+        try (Connection conn = DriverManager.getConnection(getUrl());
+                Statement stmt = conn.createStatement()) {
+
+            stmt.execute("create table " + tableName + " (k varchar primary key desc)");
+
+            // Explain doesn't display open/closed ranges
+            String explainExpected =
+                    "CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + tableName
+                            + " [~'aaa'] - [~'a']\n    SERVER FILTER BY FIRST KEY ONLY";
+
+            String openQry = "select * from " + tableName + " where k > 'a' and k<'aaa'";
+            Scan openScan =
+                    getOptimizedQueryPlan(openQry, Collections.emptyList()).getContext().getScan();
+            assertEquals("\\x9E\\x9E\\x9F\\x00", Bytes.toStringBinary(openScan.getStartRow()));
+            assertEquals("\\x9E\\xFF", Bytes.toStringBinary(openScan.getStopRow()));
+            ResultSet rs = stmt.executeQuery("EXPLAIN " + openQry);
+            String explainPlan = QueryUtil.getExplainPlan(rs);
+            assertEquals(explainExpected, explainPlan);
+
+            String closedQry = "select * from " + tableName + " where k >= 'a' and k <= 'aaa'";
+            Scan closedScan =
+                    getOptimizedQueryPlan(closedQry, Collections.emptyList()).getContext()
+                            .getScan();
+            assertEquals("\\x9E\\x9E\\x9E\\xFF", Bytes.toStringBinary(closedScan.getStartRow()));
+            assertEquals("\\x9F\\x00", Bytes.toStringBinary(closedScan.getStopRow()));
+            rs = stmt.executeQuery("EXPLAIN " + closedQry);
+            explainPlan = QueryUtil.getExplainPlan(rs);
+            assertEquals(explainExpected, explainPlan);
+        }
+    }
 }
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/compile/SaltedScanRangesTest.java b/phoenix-core/src/test/java/org/apache/phoenix/compile/SaltedScanRangesTest.java
index ac5f48922f..03bab724f2 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/compile/SaltedScanRangesTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/compile/SaltedScanRangesTest.java
@@ -85,7 +85,7 @@ public class SaltedScanRangesTest {
         List<Object> testCases = Lists.newArrayList();
         testCases.addAll(
                 foreach(new KeyRange[][]{{
-                        PVarchar.INSTANCE.getKeyRange(Bytes.toBytes("c"), true, Bytes.toBytes("e"), false),
+                        PVarchar.INSTANCE.getKeyRange(Bytes.toBytes("c"), true, Bytes.toBytes("e"), false, SortOrder.ASC),
                     }},
                     new int[] {0},
                     KeyRange.getKeyRange(KeyRange.UNBOUND, new byte[]{1}),
@@ -93,7 +93,7 @@ public class SaltedScanRangesTest {
                     true));
         testCases.addAll(
                 foreach(new KeyRange[][]{{
-                        PVarchar.INSTANCE.getKeyRange(Bytes.toBytes("c"), true, Bytes.toBytes("e"), false),
+                        PVarchar.INSTANCE.getKeyRange(Bytes.toBytes("c"), true, Bytes.toBytes("e"), false, SortOrder.ASC),
                     }},
                     new int[] {0},
                     KeyRange.getKeyRange(new byte[]{1},new byte[]{2}),
@@ -101,7 +101,7 @@ public class SaltedScanRangesTest {
                     true));
         testCases.addAll(
                 foreach(new KeyRange[][]{{
-                        PVarchar.INSTANCE.getKeyRange(Bytes.toBytes("c"), true, Bytes.toBytes("e"), false),
+                        PVarchar.INSTANCE.getKeyRange(Bytes.toBytes("c"), true, Bytes.toBytes("e"), false, SortOrder.ASC),
                     }},
                     new int[] {0},
                     KeyRange.getKeyRange(new byte[]{2},KeyRange.UNBOUND),
@@ -109,7 +109,7 @@ public class SaltedScanRangesTest {
                     true));
         testCases.addAll(
                 foreach(new KeyRange[][]{{
-                        PVarchar.INSTANCE.getKeyRange(Bytes.toBytes("c"), true, Bytes.toBytes("e"), false),
+                        PVarchar.INSTANCE.getKeyRange(Bytes.toBytes("c"), true, Bytes.toBytes("e"), false, SortOrder.ASC),
                     }},
                     new int[] {0},
                     KeyRange.getKeyRange(new byte[]{1},ByteUtil.concat(new byte[]{1}, Bytes.toBytes("c"))),
@@ -117,7 +117,7 @@ public class SaltedScanRangesTest {
                     false));
         testCases.addAll(
                 foreach(new KeyRange[][]{{
-                        PVarchar.INSTANCE.getKeyRange(Bytes.toBytes("c"), true, Bytes.toBytes("e"), false),
+                        PVarchar.INSTANCE.getKeyRange(Bytes.toBytes("c"), true, Bytes.toBytes("e"), false, SortOrder.ASC),
                     }},
                     new int[] {0},
                     KeyRange.getKeyRange(ByteUtil.concat(new byte[]{1}, Bytes.toBytes("e")), new byte[]{2}),
@@ -125,7 +125,7 @@ public class SaltedScanRangesTest {
                     false));
         testCases.addAll(
                 foreach(new KeyRange[][]{{
-                        PVarchar.INSTANCE.getKeyRange(Bytes.toBytes("c"), true, Bytes.toBytes("e"), false),
+                        PVarchar.INSTANCE.getKeyRange(Bytes.toBytes("c"), true, Bytes.toBytes("e"), false, SortOrder.ASC),
                     }},
                     new int[] {0},
                     KeyRange.getKeyRange(ByteUtil.concat(new byte[]{1}, Bytes.toBytes("d")), new byte[]{2}),
@@ -133,9 +133,9 @@ public class SaltedScanRangesTest {
                     true));
         testCases.addAll(
                 foreach(new KeyRange[][]{{
-                        PVarchar.INSTANCE.getKeyRange(Bytes.toBytes("c"), true, Bytes.toBytes("e"), false),
-                        PVarchar.INSTANCE.getKeyRange(Bytes.toBytes("h"), true, Bytes.toBytes("i"), false),
-                        PVarchar.INSTANCE.getKeyRange(Bytes.toBytes("m"), true, Bytes.toBytes("p"), false),
+                        PVarchar.INSTANCE.getKeyRange(Bytes.toBytes("c"), true, Bytes.toBytes("e"), false, SortOrder.ASC),
+                        PVarchar.INSTANCE.getKeyRange(Bytes.toBytes("h"), true, Bytes.toBytes("i"), false, SortOrder.ASC),
+                        PVarchar.INSTANCE.getKeyRange(Bytes.toBytes("m"), true, Bytes.toBytes("p"), false, SortOrder.ASC),
                     }},
                     new int[] {0},
                     KeyRange.getKeyRange(ByteUtil.concat(new byte[]{1}, Bytes.toBytes("f")), ByteUtil.concat(new byte[]{1}, Bytes.toBytes("g"))),
@@ -143,9 +143,9 @@ public class SaltedScanRangesTest {
                     true));
         testCases.addAll(
                 foreach(new KeyRange[][]{{
-                        PVarchar.INSTANCE.getKeyRange(Bytes.toBytes("c"), true, Bytes.toBytes("e"), false),
-                        PVarchar.INSTANCE.getKeyRange(Bytes.toBytes("h"), true, Bytes.toBytes("i"), false),
-                        PVarchar.INSTANCE.getKeyRange(Bytes.toBytes("m"), true, Bytes.toBytes("p"), false),
+                        PVarchar.INSTANCE.getKeyRange(Bytes.toBytes("c"), true, Bytes.toBytes("e"), false, SortOrder.ASC),
+                        PVarchar.INSTANCE.getKeyRange(Bytes.toBytes("h"), true, Bytes.toBytes("i"), false, SortOrder.ASC),
+                        PVarchar.INSTANCE.getKeyRange(Bytes.toBytes("m"), true, Bytes.toBytes("p"), false, SortOrder.ASC),
                     }},
                     new int[] {0},
                     KeyRange.getKeyRange(ByteUtil.concat(new byte[]{1}, Bytes.toBytes("f")), ByteUtil.concat(new byte[]{1}, Bytes.toBytes("g"))),
@@ -153,7 +153,7 @@ public class SaltedScanRangesTest {
                     false));
         testCases.addAll(
                 foreach(new KeyRange[][]{{
-                        PVarchar.INSTANCE.getKeyRange(Bytes.toBytes("c"), true, KeyRange.UNBOUND, false),
+                        PVarchar.INSTANCE.getKeyRange(Bytes.toBytes("c"), true, KeyRange.UNBOUND, false, SortOrder.ASC),
                     }},
                     new int[] {1},
                     KeyRange.getKeyRange(new byte[]{1,0},new byte[]{2,0}),
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/compile/ScanRangesTest.java b/phoenix-core/src/test/java/org/apache/phoenix/compile/ScanRangesTest.java
index 433f03291a..a8d2b689a6 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/compile/ScanRangesTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/compile/ScanRangesTest.java
@@ -83,383 +83,383 @@ public class ScanRangesTest {
         // must be added at end
         testCases.addAll(
                 foreach(new KeyRange[][]{{
-                        PVarchar.INSTANCE.getKeyRange(Bytes.toBytes("b"), false, Bytes.toBytes("c"), true),
+                        PVarchar.INSTANCE.getKeyRange(Bytes.toBytes("b"), false, Bytes.toBytes("c"), true, SortOrder.ASC),
                     }},
                     new int[] {0},
-                    PVarchar.INSTANCE.getKeyRange(Bytes.toBytes("ba"), true, Bytes.toBytes("bb"), true),
+                    PVarchar.INSTANCE.getKeyRange(Bytes.toBytes("ba"), true, Bytes.toBytes("bb"), true, SortOrder.ASC),
                     true));
         // KeyRange covers the first scan range.
         testCases.addAll(
                 foreach(new KeyRange[][]{{
-                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("b"), true, Bytes.toBytes("b"), true),
+                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("b"), true, Bytes.toBytes("b"), true, SortOrder.ASC),
                         },{
-                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("1"), true),
+                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("1"), true, SortOrder.ASC),
                         },{
-                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("B"), true, Bytes.toBytes("D"), true),
+                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("B"), true, Bytes.toBytes("D"), true, SortOrder.ASC),
                     }},
                     new int[] {1,1,1},
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("a9Z"), true, Bytes.toBytes("c0A"), true),
+                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("a9Z"), true, Bytes.toBytes("c0A"), true, SortOrder.ASC),
                     true));
         // KeyRange that requires a fixed width exclusive lower bound to be bumped up
         // and made inclusive. Otherwise, the comparison thinks its bigger than it really is.
         testCases.addAll(
                 foreach(new KeyRange[][]{{
-                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("b"), true, Bytes.toBytes("b"), true),
+                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("b"), true, Bytes.toBytes("b"), true, SortOrder.ASC),
                         },{
-                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("1"), true),
+                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("1"), true, SortOrder.ASC),
                         },{
-                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("A"), false, Bytes.toBytes("B"), true),
+                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("A"), false, Bytes.toBytes("B"), true, SortOrder.ASC),
                     }},
                     new int[] {1,1,1},
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("b1A"), true, Bytes.toBytes("b1A"), true),
+                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("b1A"), true, Bytes.toBytes("b1A"), true, SortOrder.ASC),
                     false));
         testCases.addAll(
                 foreach(new KeyRange[][]{{
-                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("b"), true, Bytes.toBytes("b"), true),
+                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("b"), true, Bytes.toBytes("b"), true, SortOrder.ASC),
                         },{
-                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("1"), true),
+                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("1"), true, SortOrder.ASC),
                         },{
-                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("B"), true, Bytes.toBytes("D"), true),
+                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("B"), true, Bytes.toBytes("D"), true, SortOrder.ASC),
                     }},
                     new int[] {1,1,1},
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("b0A"), true, Bytes.toBytes("b1C"), true),
+                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("b0A"), true, Bytes.toBytes("b1C"), true, SortOrder.ASC),
                     true));
         // KeyRange intersect with the first scan range on range's upper end.
         testCases.addAll(
                 foreach(new KeyRange[][]{{
-                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("b"), true, Bytes.toBytes("b"), true),
+                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("b"), true, Bytes.toBytes("b"), true, SortOrder.ASC),
                         },{
-                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("1"), true),
+                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("1"), true, SortOrder.ASC),
                         },{
-                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("B"), true, Bytes.toBytes("B"), true),
+                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("B"), true, Bytes.toBytes("B"), true, SortOrder.ASC),
                     }},
                     new int[] {1,1,1},
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("b0A"), true, Bytes.toBytes("b1B"), true),
+                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("b0A"), true, Bytes.toBytes("b1B"), true, SortOrder.ASC),
                     true));
          // ScanRanges is everything.
         testCases.addAll(
                 foreach(ScanRanges.EVERYTHING, 
                     null,
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("1"), true),
+                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("1"), true, SortOrder.ASC),
                     true));
         // ScanRanges is nothing.
         testCases.addAll(
                 foreach(ScanRanges.NOTHING,
                     null,
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("1"), true),
+                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("1"), true, SortOrder.ASC),
                     false));
         // KeyRange below the first scan range.
         testCases.addAll(
                 foreach(new KeyRange[][]{{
-                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("b"), true, Bytes.toBytes("b"), true),
+                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("b"), true, Bytes.toBytes("b"), true, SortOrder.ASC),
                     }},
                     new int[] {1},
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("a"), true, Bytes.toBytes("a"), true),
+                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("a"), true, Bytes.toBytes("a"), true, SortOrder.ASC),
                     false));
         testCases.addAll(
                 foreach(new KeyRange[][]{{
-                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("b"), true, Bytes.toBytes("b"), true),
+                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("b"), true, Bytes.toBytes("b"), true, SortOrder.ASC),
                         },{
-                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("2"), true, Bytes.toBytes("2"), true),
+                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("2"), true, Bytes.toBytes("2"), true, SortOrder.ASC),
                         },{
-                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("C"), true, Bytes.toBytes("C"), true),
+                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("C"), true, Bytes.toBytes("C"), true, SortOrder.ASC),
                     }},
                     new int[] {1,1,1},
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("b0Y"), true, Bytes.toBytes("b0Z"), true),
+                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("b0Y"), true, Bytes.toBytes("b0Z"), true, SortOrder.ASC),
                     false));
         testCases.addAll(
                 foreach(new KeyRange[][]{{
-                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("b"), true, Bytes.toBytes("b"), true),
+                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("b"), true, Bytes.toBytes("b"), true, SortOrder.ASC),
                         },{
-                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("2"), true, Bytes.toBytes("2"), true),
+                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("2"), true, Bytes.toBytes("2"), true, SortOrder.ASC),
                         },{
-                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("C"), true, Bytes.toBytes("C"), true),
+                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("C"), true, Bytes.toBytes("C"), true, SortOrder.ASC),
                     }},
                     new int[] {1,1,1},
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("b0A"), true, Bytes.toBytes("b2A"), true),
+                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("b0A"), true, Bytes.toBytes("b2A"), true, SortOrder.ASC),
                     false));
         testCases.addAll(
                 foreach(new KeyRange[][]{{
-                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("b"), true, Bytes.toBytes("b"), true),
+                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("b"), true, Bytes.toBytes("b"), true, SortOrder.ASC),
                         },{
-                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("1"), true),
+                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("1"), true, SortOrder.ASC),
                         },{
-                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("B"), true, Bytes.toBytes("B"), true),
+                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("B"), true, Bytes.toBytes("B"), true, SortOrder.ASC),
                     }},
                     new int[] {1,1,1},
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("b1A"), true, Bytes.toBytes("b1B"), false),
+                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("b1A"), true, Bytes.toBytes("b1B"), false, SortOrder.ASC),
                     false));
         testCases.addAll(
                 foreach(new KeyRange[][]{{
-                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("a"), true, Bytes.toBytes("a"), true),
+                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("a"), true, Bytes.toBytes("a"), true, SortOrder.ASC),
                         },{
-                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("1"), true),
+                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("1"), true, SortOrder.ASC),
                         },{
-                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("B"), true, Bytes.toBytes("E"), true),
+                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("B"), true, Bytes.toBytes("E"), true, SortOrder.ASC),
                     }},
                     new int[] {1,1,1},
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("a0Z"), false, Bytes.toBytes("a1A"), false),
+                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("a0Z"), false, Bytes.toBytes("a1A"), false, SortOrder.ASC),
                     false));
         testCases.addAll(
                 foreach(new KeyRange[][]{{
-                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("b"), true, Bytes.toBytes("c"), true),
+                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("b"), true, Bytes.toBytes("c"), true, SortOrder.ASC),
                         },{
-                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("2"), true),
+                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("2"), true, SortOrder.ASC),
                         },{
-                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("B"), true, Bytes.toBytes("C"), true),
+                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("B"), true, Bytes.toBytes("C"), true, SortOrder.ASC),
                     }},
                     new int[] {1,1,1},
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("a1A"), true, Bytes.toBytes("b1B"), false),
+                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("a1A"), true, Bytes.toBytes("b1B"), false, SortOrder.ASC),
                     false));
         // KeyRange intersects with the first scan range on range's lower end.
         testCases.addAll(
                 foreach(new KeyRange[][]{{
-                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("b"), true, Bytes.toBytes("b"), true),
+                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("b"), true, Bytes.toBytes("b"), true, SortOrder.ASC),
                         },{
-                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("1"), true),
+                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("1"), true, SortOrder.ASC),
                         },{
-                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("B"), true, Bytes.toBytes("D"), true),
+                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("B"), true, Bytes.toBytes("D"), true, SortOrder.ASC),
                     }},
                     new int[] {1,1,1},
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("b1C"), true, Bytes.toBytes("b2E"), true),
+                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("b1C"), true, Bytes.toBytes("b2E"), true, SortOrder.ASC),
                     true));
         testCases.addAll(
                 foreach(new KeyRange[][]{{
-                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("b"), true, Bytes.toBytes("b"), true),
+                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("b"), true, Bytes.toBytes("b"), true, SortOrder.ASC),
                         },{
-                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("1"), true),
+                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("1"), true, SortOrder.ASC),
                         },{
-                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("B"), true, Bytes.toBytes("D"), true),
+                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("B"), true, Bytes.toBytes("D"), true, SortOrder.ASC),
                     }},
                     new int[] {1,1,1},
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("b1D"), true, Bytes.toBytes("b2E"), true),
+                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("b1D"), true, Bytes.toBytes("b2E"), true, SortOrder.ASC),
                     true));
         // KeyRange above the first scan range, no intersect.
         testCases.addAll(
                 foreach(new KeyRange[][]{{
-                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("b"), true, Bytes.toBytes("b"), true),
+                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("b"), true, Bytes.toBytes("b"), true, SortOrder.ASC),
                         },{
-                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("1"), true),
+                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("1"), true, SortOrder.ASC),
                         },{
-                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("B"), true, Bytes.toBytes("D"), true),
-                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("G"), true, Bytes.toBytes("H"), true),
+                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("B"), true, Bytes.toBytes("D"), true, SortOrder.ASC),
+                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("G"), true, Bytes.toBytes("H"), true, SortOrder.ASC),
                     }},
                     new int[] {1,1,1},
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("b1E"), true, Bytes.toBytes("b1F"), true),
+                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("b1E"), true, Bytes.toBytes("b1F"), true, SortOrder.ASC),
                     false));
         testCases.addAll(
                 foreach(new KeyRange[][]{{
-                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("a"), true, Bytes.toBytes("a"), true),
-                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("b"), true, Bytes.toBytes("b"), true),
+                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("a"), true, Bytes.toBytes("a"), true, SortOrder.ASC),
+                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("b"), true, Bytes.toBytes("b"), true, SortOrder.ASC),
                         },{
-                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("1"), true),
-                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("2"), true, Bytes.toBytes("2"), true),
+                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("1"), true, SortOrder.ASC),
+                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("2"), true, Bytes.toBytes("2"), true, SortOrder.ASC),
                         },{
-                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("A"), true, Bytes.toBytes("A"), true),
-                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("C"), true, Bytes.toBytes("D"), true),
-                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("G"), true, Bytes.toBytes("G"), true),
+                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("A"), true, Bytes.toBytes("A"), true, SortOrder.ASC),
+                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("C"), true, Bytes.toBytes("D"), true, SortOrder.ASC),
+                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("G"), true, Bytes.toBytes("G"), true, SortOrder.ASC),
                     }},
                     new int[] {1,1,1},
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("a1I"), true, Bytes.toBytes("a2A"), false),
+                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("a1I"), true, Bytes.toBytes("a2A"), false, SortOrder.ASC),
                     false));
         // KeyRange above the first scan range, with intersects.
         testCases.addAll(
                 foreach(new KeyRange[][]{{
-                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("b"), true, Bytes.toBytes("b"), true),
+                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("b"), true, Bytes.toBytes("b"), true, SortOrder.ASC),
                         },{
-                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("1"), true),
+                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("1"), true, SortOrder.ASC),
                         },{
-                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("B"), true, Bytes.toBytes("D"), true),
-                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("G"), true, Bytes.toBytes("I"), true),
+                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("B"), true, Bytes.toBytes("D"), true, SortOrder.ASC),
+                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("G"), true, Bytes.toBytes("I"), true, SortOrder.ASC),
                     }},
                     new int[] {1,1,1},
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("b1E"), true, Bytes.toBytes("b1H"), true),
+                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("b1E"), true, Bytes.toBytes("b1H"), true, SortOrder.ASC),
                     true));
         testCases.addAll(
                 foreach(new KeyRange[][]{{
-                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("b"), true, Bytes.toBytes("b"), true),
-                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("c"), true, Bytes.toBytes("c"), true),
-                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("d"), true, Bytes.toBytes("d"), true),
+                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("b"), true, Bytes.toBytes("b"), true, SortOrder.ASC),
+                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("c"), true, Bytes.toBytes("c"), true, SortOrder.ASC),
+                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("d"), true, Bytes.toBytes("d"), true, SortOrder.ASC),
                         },{
-                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("1"), true),
+                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("1"), true, SortOrder.ASC),
                         },{
-                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("B"), true, Bytes.toBytes("D"), true),
-                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("G"), true, Bytes.toBytes("I"), true),
+                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("B"), true, Bytes.toBytes("D"), true, SortOrder.ASC),
+                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("G"), true, Bytes.toBytes("I"), true, SortOrder.ASC),
                     }},
                     new int[] {1,1,1},
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("b00"), true, Bytes.toBytes("d00"), true),
+                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("b00"), true, Bytes.toBytes("d00"), true, SortOrder.ASC),
                     true));
         testCases.addAll(
                 foreach(new KeyRange[][]{{
-                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("b"), true, Bytes.toBytes("b"), true),
-                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("c"), true, Bytes.toBytes("c"), true),
-                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("d"), true, Bytes.toBytes("d"), true),
+                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("b"), true, Bytes.toBytes("b"), true, SortOrder.ASC),
+                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("c"), true, Bytes.toBytes("c"), true, SortOrder.ASC),
+                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("d"), true, Bytes.toBytes("d"), true, SortOrder.ASC),
                         },{
-                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("3"), true, Bytes.toBytes("4"), true),
+                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("3"), true, Bytes.toBytes("4"), true, SortOrder.ASC),
                         },{
-                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("B"), true, Bytes.toBytes("D"), true),
-                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("G"), true, Bytes.toBytes("I"), true),
+                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("B"), true, Bytes.toBytes("D"), true, SortOrder.ASC),
+                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("G"), true, Bytes.toBytes("I"), true, SortOrder.ASC),
                     }},
                     new int[] {1,1,1},
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("b20"), true, Bytes.toBytes("b50"), true),
+                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("b20"), true, Bytes.toBytes("b50"), true, SortOrder.ASC),
                     true));
         // KeyRange above the last scan range.
         testCases.addAll(
                 foreach(new KeyRange[][]{{
-                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("b"), true, Bytes.toBytes("b"), true),
+                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("b"), true, Bytes.toBytes("b"), true, SortOrder.ASC),
                         },{
-                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("1"), true),
+                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("1"), true, SortOrder.ASC),
                         },{
-                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("B"), true, Bytes.toBytes("B"), true),
+                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("B"), true, Bytes.toBytes("B"), true, SortOrder.ASC),
                     }},
                     new int[] {1,1,1},
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("b1B"), false, Bytes.toBytes("b2A"), true),
+                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("b1B"), false, Bytes.toBytes("b2A"), true, SortOrder.ASC),
                     false));
         testCases.addAll(
                 foreach(new KeyRange[][]{{
-                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("b"), true, Bytes.toBytes("b"), false),
+                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("b"), true, Bytes.toBytes("b"), false, SortOrder.ASC),
                         },{
-                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("1"), false),
+                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("1"), false, SortOrder.ASC),
                         },{
-                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("B"), true, Bytes.toBytes("B"), false),
+                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("B"), true, Bytes.toBytes("B"), false, SortOrder.ASC),
                     }},
                     new int[] {1,1,1},
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("b2A"), true, Bytes.toBytes("b2A"), true),
+                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("b2A"), true, Bytes.toBytes("b2A"), true, SortOrder.ASC),
                     false));
         testCases.addAll(
                 foreach(new KeyRange[][]{{
-                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("b"), true, Bytes.toBytes("b"), true),
+                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("b"), true, Bytes.toBytes("b"), true, SortOrder.ASC),
                         },{
-                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("1"), true),
+                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("1"), true, SortOrder.ASC),
                         },{
-                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("B"), true, Bytes.toBytes("B"), true),
+                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("B"), true, Bytes.toBytes("B"), true, SortOrder.ASC),
                     }},
                     new int[] {1,1,1},
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("c1A"), false, Bytes.toBytes("c9Z"), true),
+                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("c1A"), false, Bytes.toBytes("c9Z"), true, SortOrder.ASC),
                     false));
         // KeyRange contains unbound lower bound.
         testCases.addAll(
                 foreach(new KeyRange[][]{{
-                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("b"), true, Bytes.toBytes("b"), true),
+                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("b"), true, Bytes.toBytes("b"), true, SortOrder.ASC),
                         },{
-                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("1"), true),
+                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("1"), true, SortOrder.ASC),
                         },{
-                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("A"), true, Bytes.toBytes("B"), true),
+                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("A"), true, Bytes.toBytes("B"), true, SortOrder.ASC),
                     }},
                     new int[] {1,1,1}, 
-                    PChar.INSTANCE.getKeyRange(KeyRange.UNBOUND, false, Bytes.toBytes("a0Z"), true),
+                    PChar.INSTANCE.getKeyRange(KeyRange.UNBOUND, false, Bytes.toBytes("a0Z"), true, SortOrder.ASC),
                     false));
         testCases.addAll(
                 foreach(new KeyRange[][]{{
-                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("a"), true, Bytes.toBytes("a"), true),
+                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("a"), true, Bytes.toBytes("a"), true, SortOrder.ASC),
                         },{
-                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("1"), true),
+                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("1"), true, SortOrder.ASC),
                         },{
-                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("A"), true, Bytes.toBytes("B"), true),
+                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("A"), true, Bytes.toBytes("B"), true, SortOrder.ASC),
                     }},
                     new int[] {1,1,1}, 
-                    PChar.INSTANCE.getKeyRange(KeyRange.UNBOUND, false, Bytes.toBytes("a0Z"), true),
+                    PChar.INSTANCE.getKeyRange(KeyRange.UNBOUND, false, Bytes.toBytes("a0Z"), true, SortOrder.ASC),
                     false));
         testCases.addAll(
                 foreach(new KeyRange[][]{{
-                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("a"), true, Bytes.toBytes("a"), true),
+                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("a"), true, Bytes.toBytes("a"), true, SortOrder.ASC),
                         },{
-                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("1"), true),
+                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("1"), true, SortOrder.ASC),
                         },{
-                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("D"), true, Bytes.toBytes("E"), true),
+                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("D"), true, Bytes.toBytes("E"), true, SortOrder.ASC),
                     }},
                     new int[] {1,1,1}, 
-                    PChar.INSTANCE.getKeyRange(KeyRange.UNBOUND, false, Bytes.toBytes("a1C"), true),
+                    PChar.INSTANCE.getKeyRange(KeyRange.UNBOUND, false, Bytes.toBytes("a1C"), true, SortOrder.ASC),
                     false));
         testCases.addAll(
                 foreach(new KeyRange[][]{{
-                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("a"), true, Bytes.toBytes("a"), true),
+                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("a"), true, Bytes.toBytes("a"), true, SortOrder.ASC),
                         },{
-                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("1"), true),
+                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("1"), true, SortOrder.ASC),
                         },{
-                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("D"), true, Bytes.toBytes("E"), true),
+                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("D"), true, Bytes.toBytes("E"), true, SortOrder.ASC),
                     }},
                     new int[] {1,1,1}, 
-                    PChar.INSTANCE.getKeyRange(KeyRange.UNBOUND, false, Bytes.toBytes("a1D"), true),
+                    PChar.INSTANCE.getKeyRange(KeyRange.UNBOUND, false, Bytes.toBytes("a1D"), true, SortOrder.ASC),
                     true));
         testCases.addAll(
                 foreach(new KeyRange[][]{{
-                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("a"), true, Bytes.toBytes("a"), true),
+                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("a"), true, Bytes.toBytes("a"), true, SortOrder.ASC),
                         },{
-                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("1"), true),
+                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("1"), true, SortOrder.ASC),
                         },{
-                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("D"), true, Bytes.toBytes("E"), true),
+                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("D"), true, Bytes.toBytes("E"), true, SortOrder.ASC),
                     }},
                     new int[] {1,1,1}, 
-                    PChar.INSTANCE.getKeyRange(KeyRange.UNBOUND, false, Bytes.toBytes("a2D"), true),
+                    PChar.INSTANCE.getKeyRange(KeyRange.UNBOUND, false, Bytes.toBytes("a2D"), true, SortOrder.ASC),
                     true));
         // KeyRange contains unbound upper bound
         testCases.addAll(
                 foreach(new KeyRange[][]{{
-                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("a"), true, Bytes.toBytes("a"), true),
+                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("a"), true, Bytes.toBytes("a"), true, SortOrder.ASC),
                         },{
-                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("1"), true),
+                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("1"), true, SortOrder.ASC),
                         },{
-                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("A"), true, Bytes.toBytes("B"), true),
+                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("A"), true, Bytes.toBytes("B"), true, SortOrder.ASC),
                     }},
                     new int[] {1,1,1}, 
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("a0A"), true, KeyRange.UNBOUND, false),
+                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("a0A"), true, KeyRange.UNBOUND, false, SortOrder.ASC),
                     true));
         testCases.addAll(
                 foreach(new KeyRange[][]{{
-                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("a"), true, Bytes.toBytes("a"), true),
+                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("a"), true, Bytes.toBytes("a"), true, SortOrder.ASC),
                         },{
-                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("1"), true),
+                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("1"), true, SortOrder.ASC),
                         },{
-                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("A"), true, Bytes.toBytes("B"), true),
+                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("A"), true, Bytes.toBytes("B"), true, SortOrder.ASC),
                     }},
                     new int[] {1,1,1}, 
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("a1B"), true, KeyRange.UNBOUND, false),
+                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("a1B"), true, KeyRange.UNBOUND, false, SortOrder.ASC),
                     true));
         testCases.addAll(
                 foreach(new KeyRange[][]{{
-                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("a"), true, Bytes.toBytes("a"), true),
+                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("a"), true, Bytes.toBytes("a"), true, SortOrder.ASC),
                         },{
-                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("1"), true),
+                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("1"), true, SortOrder.ASC),
                         },{
-                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("A"), true, Bytes.toBytes("B"), true),
+                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("A"), true, Bytes.toBytes("B"), true, SortOrder.ASC),
                     }},
                     new int[] {1,1,1}, 
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("a1C"), true, KeyRange.UNBOUND, false),
+                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("a1C"), true, KeyRange.UNBOUND, false, SortOrder.ASC),
                     false));
         testCases.addAll(
                 foreach(new KeyRange[][]{{
-                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("a"), true, Bytes.toBytes("a"), true),
+                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("a"), true, Bytes.toBytes("a"), true, SortOrder.ASC),
                         },{
-                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("1"), true),
+                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("1"), true, SortOrder.ASC),
                         },{
-                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("A"), true, Bytes.toBytes("B"), true),
+                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("A"), true, Bytes.toBytes("B"), true, SortOrder.ASC),
                     }},
                     new int[] {1,1,1}, 
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("a3A"), true, KeyRange.UNBOUND, false),
+                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("a3A"), true, KeyRange.UNBOUND, false, SortOrder.ASC),
                     false));
         testCases.addAll(
                 foreach(new KeyRange[][]{{
-                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("a"), true, Bytes.toBytes("a"), true),
+                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("a"), true, Bytes.toBytes("a"), true, SortOrder.ASC),
                         },{
-                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("1"), true),
+                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("1"), true, SortOrder.ASC),
                         },{
-                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("A"), true, Bytes.toBytes("B"), true),
+                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("A"), true, Bytes.toBytes("B"), true, SortOrder.ASC),
                     }},
                     new int[] {1,1,1}, 
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("d0A"), true, KeyRange.UNBOUND, false),
+                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("d0A"), true, KeyRange.UNBOUND, false, SortOrder.ASC),
                     false));
         // KeyRange is unbound to unbound.
         testCases.addAll(
                 foreach(new KeyRange[][]{{
-                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("a"), true, Bytes.toBytes("a"), true),
+                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("a"), true, Bytes.toBytes("a"), true, SortOrder.ASC),
                         },{
-                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("1"), true),
+                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("1"), true, SortOrder.ASC),
                         },{
-                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("A"), true, Bytes.toBytes("B"), true),
+                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("A"), true, Bytes.toBytes("B"), true, SortOrder.ASC),
                     }},
                     new int[] {1,1,1}, 
-                    PChar.INSTANCE.getKeyRange(KeyRange.UNBOUND, false, KeyRange.UNBOUND, false),
+                    PChar.INSTANCE.getKeyRange(KeyRange.UNBOUND, false, KeyRange.UNBOUND, false, SortOrder.ASC),
                     true));
         return testCases;
     }
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 b816e3bc8b..76cc9b0545 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
@@ -764,7 +764,7 @@ public class WhereCompilerTest extends BaseConnectionlessQueryTest {
                         Bytes.toBytes(entityId1),
                         true,
                         Bytes.toBytes(entityId2),
-                        true))),
+                        true, SortOrder.ASC))),
                 plan.getTableRef().getTable().getRowKeySchema()),
             filter);
     }
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 d6eab0401b..e46386afac 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
@@ -1566,8 +1566,8 @@ public class WhereOptimizerTest extends BaseConnectionlessQueryTest {
         List<List<KeyRange>> ranges = scanRanges.getRanges();
         assertEquals(1,ranges.size());
         List<List<KeyRange>> expectedRanges = Collections.singletonList(Arrays.asList(
-                PChar.INSTANCE.getKeyRange(PChar.INSTANCE.toBytes(tenantId1), true, PChar.INSTANCE.toBytes(tenantId1), true),
-                PChar.INSTANCE.getKeyRange(PChar.INSTANCE.toBytes(tenantId2), true, PChar.INSTANCE.toBytes(tenantId2), true)));
+                PChar.INSTANCE.getKeyRange(PChar.INSTANCE.toBytes(tenantId1), true, PChar.INSTANCE.toBytes(tenantId1), true, SortOrder.ASC),
+                PChar.INSTANCE.getKeyRange(PChar.INSTANCE.toBytes(tenantId2), true, PChar.INSTANCE.toBytes(tenantId2), true, SortOrder.ASC)));
         assertEquals(expectedRanges, ranges);
         byte[] startRow = PVarchar.INSTANCE.toBytes(tenantId1);
         assertArrayEquals(startRow, scan.getStartRow());
@@ -1630,10 +1630,10 @@ public class WhereOptimizerTest extends BaseConnectionlessQueryTest {
         List<List<KeyRange>> expectedRanges = Collections.singletonList(Arrays.asList(
                 PChar.INSTANCE.getKeyRange(
                         StringUtil.padChar(PChar.INSTANCE.toBytes("00D"),15), true,
-                        StringUtil.padChar(ByteUtil.nextKey(PChar.INSTANCE.toBytes("00D")),15), false),
+                        StringUtil.padChar(ByteUtil.nextKey(PChar.INSTANCE.toBytes("00D")),15), false, SortOrder.ASC),
                 PChar.INSTANCE.getKeyRange(
                         StringUtil.padChar(PChar.INSTANCE.toBytes("foo"),15), true,
-                        StringUtil.padChar(ByteUtil.nextKey(PChar.INSTANCE.toBytes("foo")),15), false)));
+                        StringUtil.padChar(ByteUtil.nextKey(PChar.INSTANCE.toBytes("foo")),15), false, SortOrder.ASC)));
         assertEquals(expectedRanges, ranges);
     }
 
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/filter/SkipScanFilterIntersectTest.java b/phoenix-core/src/test/java/org/apache/phoenix/filter/SkipScanFilterIntersectTest.java
index 5935e1bb2a..d3597b78ed 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/filter/SkipScanFilterIntersectTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/filter/SkipScanFilterIntersectTest.java
@@ -89,442 +89,482 @@ public class SkipScanFilterIntersectTest {
         }
     }
 
-    @Parameters(name="{0} {4}")
+    @Parameters(name = "{0} {4}")
     public static synchronized Collection<Object> data() {
         List<Object> testCases = Lists.newArrayList();
         // Both ranges in second slot are required b/c first slot contains range and upper/lower
         // values differ in this slot position.
         testCases.addAll(foreach(
-                new KeyRange[][] {{
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("b"), true, Bytes.toBytes("e"), false),
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("j"), true, Bytes.toBytes("m"), false),
-                    }, {
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("1"), true),
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("2"), true, Bytes.toBytes("4"), true),
-                    }, {
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("B"), true, Bytes.toBytes("B"), true),
-                }},
-                new int[] {1,1,1},
-                Bytes.toBytes("j3A"),
-                Bytes.toBytes("k4C"),
-                new KeyRange[][] {{
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("j"), true, Bytes.toBytes("m"), false),
-                    }, {
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("1"), true),
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("2"), true, Bytes.toBytes("4"), true),
-                    }, {
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("B"), true, Bytes.toBytes("B"), true),
-                }}));
+            new KeyRange[][] {
+                    { PChar.INSTANCE.getKeyRange(Bytes.toBytes("b"), true, Bytes.toBytes("e"),
+                        false, SortOrder.ASC),
+                            PChar.INSTANCE.getKeyRange(Bytes.toBytes("j"), true, Bytes.toBytes("m"),
+                                false, SortOrder.ASC), },
+                    { PChar.INSTANCE.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("1"), true,
+                        SortOrder.ASC),
+                            PChar.INSTANCE.getKeyRange(Bytes.toBytes("2"), true, Bytes.toBytes("4"),
+                                true, SortOrder.ASC), },
+                    { PChar.INSTANCE.getKeyRange(Bytes.toBytes("B"), true, Bytes.toBytes("B"), true,
+                        SortOrder.ASC), } },
+            new int[] { 1, 1, 1 }, Bytes.toBytes("j3A"), Bytes.toBytes("k4C"),
+            new KeyRange[][] {
+                    { PChar.INSTANCE.getKeyRange(Bytes.toBytes("j"), true, Bytes.toBytes("m"),
+                        false, SortOrder.ASC), },
+                    { PChar.INSTANCE.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("1"), true,
+                        SortOrder.ASC),
+                            PChar.INSTANCE.getKeyRange(Bytes.toBytes("2"), true, Bytes.toBytes("4"),
+                                true, SortOrder.ASC), },
+                    { PChar.INSTANCE.getKeyRange(Bytes.toBytes("B"), true, Bytes.toBytes("B"), true,
+                        SortOrder.ASC), } }));
         // Only second range in second slot is required b/c though first slot contains range,
         // upper/lower values do not differ in this slot position.
         testCases.addAll(foreach(
-                new KeyRange[][] {{
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("b"), true, Bytes.toBytes("e"), false),
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("j"), true, Bytes.toBytes("m"), false),
-                    }, {
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("1"), true),
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("2"), true, Bytes.toBytes("4"), true),
-                    }, {
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("B"), true, Bytes.toBytes("B"), true),
-                }},
-                new int[] {1,1,1},
-                Bytes.toBytes("j3A"),
-                Bytes.toBytes("j4C"),
-                new KeyRange[][] {{
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("j"), true, Bytes.toBytes("m"), false),
-                    }, {
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("2"), true, Bytes.toBytes("4"), true),
-                    }, {
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("B"), true, Bytes.toBytes("B"), true),
-                }}));
+            new KeyRange[][] {
+                    { PChar.INSTANCE.getKeyRange(Bytes.toBytes("b"), true, Bytes.toBytes("e"),
+                        false, SortOrder.ASC),
+                            PChar.INSTANCE.getKeyRange(Bytes.toBytes("j"), true, Bytes.toBytes("m"),
+                                false, SortOrder.ASC), },
+                    { PChar.INSTANCE.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("1"), true,
+                        SortOrder.ASC),
+                            PChar.INSTANCE.getKeyRange(Bytes.toBytes("2"), true, Bytes.toBytes("4"),
+                                true, SortOrder.ASC), },
+                    { PChar.INSTANCE.getKeyRange(Bytes.toBytes("B"), true, Bytes.toBytes("B"), true,
+                        SortOrder.ASC), } },
+            new int[] { 1, 1, 1 }, Bytes.toBytes("j3A"), Bytes.toBytes("j4C"),
+            new KeyRange[][] {
+                    { PChar.INSTANCE.getKeyRange(Bytes.toBytes("j"), true, Bytes.toBytes("m"),
+                        false, SortOrder.ASC), },
+                    { PChar.INSTANCE.getKeyRange(Bytes.toBytes("2"), true, Bytes.toBytes("4"), true,
+                        SortOrder.ASC), },
+                    { PChar.INSTANCE.getKeyRange(Bytes.toBytes("B"), true, Bytes.toBytes("B"), true,
+                        SortOrder.ASC), } }));
         // Test case exercising repositioning multiple times (initially to slot #2 and then again
-        // to slot #4). Because there's a range for slot #4 and the lower/upper values are different,
+        // to slot #4). Because there's a range for slot #4 and the lower/upper values are
+        // different,
         // all slot #5 ranges are part of the intersection.
         testCases.addAll(foreach(
-                new KeyRange[][] {{
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("b"), true, Bytes.toBytes("b"), true),
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("d"), true, Bytes.toBytes("d"), true),
-                    }, {
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("j"), true, Bytes.toBytes("m"), false),
-                    }, {
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("C"), true, Bytes.toBytes("C"), true),
-                    }, {
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("m"), true, Bytes.toBytes("u"), false),
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("z"), true, Bytes.toBytes("z"), true),
-                    }, {
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("A"), true, Bytes.toBytes("A"), true),                        
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("D"), true, Bytes.toBytes("D"), true),
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("M"), true, Bytes.toBytes("M"), true),                        
-                    }
-                },
-                new int[] {1,1,1,1,1},
-                Bytes.toBytes("bkCpM"),
-                Bytes.toBytes("bkCtD"),
-                new KeyRange[][] {{
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("b"), true, Bytes.toBytes("b"), true),
-                    }, {
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("j"), true, Bytes.toBytes("m"), false),
-                    }, {
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("C"), true, Bytes.toBytes("C"), true),
-                    }, {
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("m"), true, Bytes.toBytes("u"), false),
-                    }, {
-                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("A"), true, Bytes.toBytes("A"), true),                        
-                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("D"), true, Bytes.toBytes("D"), true),
-                        PChar.INSTANCE.getKeyRange(Bytes.toBytes("M"), true, Bytes.toBytes("M"), true),                        
-                    }
-                }));
+            new KeyRange[][] {
+                    { PChar.INSTANCE.getKeyRange(Bytes.toBytes("b"), true, Bytes.toBytes("b"), true,
+                        SortOrder.ASC),
+                            PChar.INSTANCE.getKeyRange(Bytes.toBytes("d"), true, Bytes.toBytes("d"),
+                                true, SortOrder.ASC), },
+                    { PChar.INSTANCE.getKeyRange(Bytes.toBytes("j"), true, Bytes.toBytes("m"),
+                        false, SortOrder.ASC), },
+                    { PChar.INSTANCE.getKeyRange(Bytes.toBytes("C"), true, Bytes.toBytes("C"), true,
+                        SortOrder.ASC), },
+                    { PChar.INSTANCE.getKeyRange(Bytes.toBytes("m"), true, Bytes.toBytes("u"),
+                        false, SortOrder.ASC),
+                            PChar.INSTANCE.getKeyRange(Bytes.toBytes("z"), true, Bytes.toBytes("z"),
+                                true, SortOrder.ASC), },
+                    { PChar.INSTANCE.getKeyRange(Bytes.toBytes("A"), true, Bytes.toBytes("A"), true,
+                        SortOrder.ASC),
+                            PChar.INSTANCE.getKeyRange(Bytes.toBytes("D"), true, Bytes.toBytes("D"),
+                                true, SortOrder.ASC),
+                            PChar.INSTANCE.getKeyRange(Bytes.toBytes("M"), true, Bytes.toBytes("M"),
+                                true, SortOrder.ASC), } },
+            new int[] { 1, 1, 1, 1, 1 }, Bytes.toBytes("bkCpM"), Bytes.toBytes("bkCtD"),
+            new KeyRange[][] {
+                    { PChar.INSTANCE.getKeyRange(Bytes.toBytes("b"), true, Bytes.toBytes("b"), true,
+                        SortOrder.ASC), },
+                    { PChar.INSTANCE.getKeyRange(Bytes.toBytes("j"), true, Bytes.toBytes("m"),
+                        false, SortOrder.ASC), },
+                    { PChar.INSTANCE.getKeyRange(Bytes.toBytes("C"), true, Bytes.toBytes("C"), true,
+                        SortOrder.ASC), },
+                    { PChar.INSTANCE.getKeyRange(Bytes.toBytes("m"), true, Bytes.toBytes("u"),
+                        false, SortOrder.ASC), },
+                    { PChar.INSTANCE.getKeyRange(Bytes.toBytes("A"), true, Bytes.toBytes("A"), true,
+                        SortOrder.ASC),
+                            PChar.INSTANCE.getKeyRange(Bytes.toBytes("D"), true, Bytes.toBytes("D"),
+                                true, SortOrder.ASC),
+                            PChar.INSTANCE.getKeyRange(Bytes.toBytes("M"), true, Bytes.toBytes("M"),
+                                true, SortOrder.ASC), } }));
         // Single matching in the first 2 slots.
         testCases.addAll(foreach(
-                new KeyRange[][] {{
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("b"), true, Bytes.toBytes("b"), true),
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("c"), true, Bytes.toBytes("d"), true),
-                    }, {
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("1"), true),
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("2"), true, Bytes.toBytes("3"), true),
-                    }, {
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("B"), true, Bytes.toBytes("B"), true),
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("C"), true, Bytes.toBytes("D"), true),
-                }},
-                new int[] {1,1,1},
-                Bytes.toBytes("b1B"),
-                Bytes.toBytes("b1C"),
-                new KeyRange[][] {{
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("b"), true, Bytes.toBytes("b"), true),
-                    }, {
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("1"), true),
-                    }, {
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("B"), true, Bytes.toBytes("B"), true),
-                }}));
+            new KeyRange[][] {
+                    { PChar.INSTANCE.getKeyRange(Bytes.toBytes("b"), true, Bytes.toBytes("b"), true,
+                        SortOrder.ASC),
+                            PChar.INSTANCE.getKeyRange(Bytes.toBytes("c"), true, Bytes.toBytes("d"),
+                                true, SortOrder.ASC), },
+                    { PChar.INSTANCE.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("1"), true,
+                        SortOrder.ASC),
+                            PChar.INSTANCE.getKeyRange(Bytes.toBytes("2"), true, Bytes.toBytes("3"),
+                                true, SortOrder.ASC), },
+                    { PChar.INSTANCE.getKeyRange(Bytes.toBytes("B"), true, Bytes.toBytes("B"), true,
+                        SortOrder.ASC),
+                            PChar.INSTANCE.getKeyRange(Bytes.toBytes("C"), true, Bytes.toBytes("D"),
+                                true, SortOrder.ASC), } },
+            new int[] { 1, 1, 1 }, Bytes.toBytes("b1B"), Bytes.toBytes("b1C"),
+            new KeyRange[][] {
+                    { PChar.INSTANCE.getKeyRange(Bytes.toBytes("b"), true, Bytes.toBytes("b"), true,
+                        SortOrder.ASC), },
+                    { PChar.INSTANCE.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("1"), true,
+                        SortOrder.ASC), },
+                    { PChar.INSTANCE.getKeyRange(Bytes.toBytes("B"), true, Bytes.toBytes("B"), true,
+                        SortOrder.ASC), } }));
         // Single matching in the first slot.
         testCases.addAll(foreach(
-                new KeyRange[][] {{
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("b"), true, Bytes.toBytes("b"), true),
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("c"), true, Bytes.toBytes("e"), true),
-                    }, {
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("1"), true),
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("2"), true, Bytes.toBytes("4"), true),
-                    }, {
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("B"), true, Bytes.toBytes("B"), true),
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("C"), true, Bytes.toBytes("E"), true),
-                }},
-                new int[] {1,1,1},
-                Bytes.toBytes("b1Z"),
-                Bytes.toBytes("b3Z"),
-                new KeyRange[][] {{
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("b"), true, Bytes.toBytes("b"), true),
-                    }, {
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("2"), true, Bytes.toBytes("4"), true),
-                    }, {
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("B"), true, Bytes.toBytes("B"), true),
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("C"), true, Bytes.toBytes("E"), true),
-                }}));
+            new KeyRange[][] {
+                    { PChar.INSTANCE.getKeyRange(Bytes.toBytes("b"), true, Bytes.toBytes("b"), true,
+                        SortOrder.ASC),
+                            PChar.INSTANCE.getKeyRange(Bytes.toBytes("c"), true, Bytes.toBytes("e"),
+                                true, SortOrder.ASC), },
+                    { PChar.INSTANCE.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("1"), true,
+                        SortOrder.ASC),
+                            PChar.INSTANCE.getKeyRange(Bytes.toBytes("2"), true, Bytes.toBytes("4"),
+                                true, SortOrder.ASC), },
+                    { PChar.INSTANCE.getKeyRange(Bytes.toBytes("B"), true, Bytes.toBytes("B"), true,
+                        SortOrder.ASC),
+                            PChar.INSTANCE.getKeyRange(Bytes.toBytes("C"), true, Bytes.toBytes("E"),
+                                true, SortOrder.ASC), } },
+            new int[] { 1, 1, 1 }, Bytes.toBytes("b1Z"), Bytes.toBytes("b3Z"),
+            new KeyRange[][] {
+                    { PChar.INSTANCE.getKeyRange(Bytes.toBytes("b"), true, Bytes.toBytes("b"), true,
+                        SortOrder.ASC), },
+                    { PChar.INSTANCE.getKeyRange(Bytes.toBytes("2"), true, Bytes.toBytes("4"), true,
+                        SortOrder.ASC), },
+                    { PChar.INSTANCE.getKeyRange(Bytes.toBytes("B"), true, Bytes.toBytes("B"), true,
+                        SortOrder.ASC),
+                            PChar.INSTANCE.getKeyRange(Bytes.toBytes("C"), true, Bytes.toBytes("E"),
+                                true, SortOrder.ASC), } }));
         // No overlap
         testCases.addAll(foreach(
-                new KeyRange[][]{{
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("a"), true, Bytes.toBytes("a"), true),
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("b"), true, Bytes.toBytes("b"), true),
-                    },{
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("1"), true),
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("2"), true, Bytes.toBytes("2"), true),
-                    },{
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("A"), true, Bytes.toBytes("A"), true),
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("C"), true, Bytes.toBytes("D"), true),
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("G"), true, Bytes.toBytes("G"), true),
-                }},
-                new int[] {1,1,1}, 
-                Bytes.toBytes("a1I"), 
-                Bytes.toBytes("a2A"),
-                null));
+            new KeyRange[][] {
+                    { PChar.INSTANCE.getKeyRange(Bytes.toBytes("a"), true, Bytes.toBytes("a"), true,
+                        SortOrder.ASC),
+                            PChar.INSTANCE.getKeyRange(Bytes.toBytes("b"), true, Bytes.toBytes("b"),
+                                true, SortOrder.ASC), },
+                    { PChar.INSTANCE.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("1"), true,
+                        SortOrder.ASC),
+                            PChar.INSTANCE.getKeyRange(Bytes.toBytes("2"), true, Bytes.toBytes("2"),
+                                true, SortOrder.ASC), },
+                    { PChar.INSTANCE.getKeyRange(Bytes.toBytes("A"), true, Bytes.toBytes("A"), true,
+                        SortOrder.ASC),
+                            PChar.INSTANCE.getKeyRange(Bytes.toBytes("C"), true, Bytes.toBytes("D"),
+                                true, SortOrder.ASC),
+                            PChar.INSTANCE.getKeyRange(Bytes.toBytes("G"), true, Bytes.toBytes("G"),
+                                true, SortOrder.ASC), } },
+            new int[] { 1, 1, 1 }, Bytes.toBytes("a1I"), Bytes.toBytes("a2A"), null));
         testCases.addAll(foreach(
-                new KeyRange[][] {{
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("b"), true, Bytes.toBytes("b"), true),
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("c"), true, Bytes.toBytes("d"), true),
-                    }, {
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("1"), true),
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("2"), true, Bytes.toBytes("3"), true),
-                    }, {
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("B"), true, Bytes.toBytes("B"), true),
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("C"), true, Bytes.toBytes("D"), true),
-                }},
-                new int[] {1,1,1},
-                Bytes.toBytes("a0A"),
-                Bytes.toBytes("b1B"),
-                null));
+            new KeyRange[][] {
+                    { PChar.INSTANCE.getKeyRange(Bytes.toBytes("b"), true, Bytes.toBytes("b"), true,
+                        SortOrder.ASC),
+                            PChar.INSTANCE.getKeyRange(Bytes.toBytes("c"), true, Bytes.toBytes("d"),
+                                true, SortOrder.ASC), },
+                    { PChar.INSTANCE.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("1"), true,
+                        SortOrder.ASC),
+                            PChar.INSTANCE.getKeyRange(Bytes.toBytes("2"), true, Bytes.toBytes("3"),
+                                true, SortOrder.ASC), },
+                    { PChar.INSTANCE.getKeyRange(Bytes.toBytes("B"), true, Bytes.toBytes("B"), true,
+                        SortOrder.ASC),
+                            PChar.INSTANCE.getKeyRange(Bytes.toBytes("C"), true, Bytes.toBytes("D"),
+                                true, SortOrder.ASC), } },
+            new int[] { 1, 1, 1 }, Bytes.toBytes("a0A"), Bytes.toBytes("b1B"), null));
         testCases.addAll(foreach(
-                new KeyRange[][] {{
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("b"), true, Bytes.toBytes("b"), true),
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("c"), true, Bytes.toBytes("d"), true),
-                    }, {
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("1"), true),
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("2"), true, Bytes.toBytes("3"), true),
-                    }, {
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("B"), true, Bytes.toBytes("B"), true),
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("C"), true, Bytes.toBytes("D"), true),
-                }},
-                new int[] {1,1,1},
-                Bytes.toBytes("a0A"),
-                Bytes.toBytes("b1C"),
-                new KeyRange[][] {{
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("b"), true, Bytes.toBytes("b"), true),
-                    }, {
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("1"), true),
-                    }, {
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("B"), true, Bytes.toBytes("B"), true),
-                }}));
+            new KeyRange[][] {
+                    { PChar.INSTANCE.getKeyRange(Bytes.toBytes("b"), true, Bytes.toBytes("b"), true,
+                        SortOrder.ASC),
+                            PChar.INSTANCE.getKeyRange(Bytes.toBytes("c"), true, Bytes.toBytes("d"),
+                                true, SortOrder.ASC), },
+                    { PChar.INSTANCE.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("1"), true,
+                        SortOrder.ASC),
+                            PChar.INSTANCE.getKeyRange(Bytes.toBytes("2"), true, Bytes.toBytes("3"),
+                                true, SortOrder.ASC), },
+                    { PChar.INSTANCE.getKeyRange(Bytes.toBytes("B"), true, Bytes.toBytes("B"), true,
+                        SortOrder.ASC),
+                            PChar.INSTANCE.getKeyRange(Bytes.toBytes("C"), true, Bytes.toBytes("D"),
+                                true, SortOrder.ASC), } },
+            new int[] { 1, 1, 1 }, Bytes.toBytes("a0A"), Bytes.toBytes("b1C"),
+            new KeyRange[][] {
+                    { PChar.INSTANCE.getKeyRange(Bytes.toBytes("b"), true, Bytes.toBytes("b"), true,
+                        SortOrder.ASC), },
+                    { PChar.INSTANCE.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("1"), true,
+                        SortOrder.ASC), },
+                    { PChar.INSTANCE.getKeyRange(Bytes.toBytes("B"), true, Bytes.toBytes("B"), true,
+                        SortOrder.ASC), } }));
         testCases.addAll(foreach(
-                new KeyRange[][] {{
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("b"), true, Bytes.toBytes("b"), true),
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("c"), true, Bytes.toBytes("d"), true),
-                    }, {
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("1"), true),
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("2"), true, Bytes.toBytes("3"), true),
-                    }, {
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("B"), true, Bytes.toBytes("B"), true),
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("C"), true, Bytes.toBytes("D"), true),
-                }},
-                new int[] {1,1,1},
-                Bytes.toBytes("a0A"),
-                Bytes.toBytes("b1D"),
-                new KeyRange[][] {{
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("b"), true, Bytes.toBytes("b"), true),
-                    }, {
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("1"), true),
-                    }, {
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("B"), true, Bytes.toBytes("B"), true),
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("C"), true, Bytes.toBytes("D"), true),
-                }}));
+            new KeyRange[][] {
+                    { PChar.INSTANCE.getKeyRange(Bytes.toBytes("b"), true, Bytes.toBytes("b"), true,
+                        SortOrder.ASC),
+                            PChar.INSTANCE.getKeyRange(Bytes.toBytes("c"), true, Bytes.toBytes("d"),
+                                true, SortOrder.ASC), },
+                    { PChar.INSTANCE.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("1"), true,
+                        SortOrder.ASC),
+                            PChar.INSTANCE.getKeyRange(Bytes.toBytes("2"), true, Bytes.toBytes("3"),
+                                true, SortOrder.ASC), },
+                    { PChar.INSTANCE.getKeyRange(Bytes.toBytes("B"), true, Bytes.toBytes("B"), true,
+                        SortOrder.ASC),
+                            PChar.INSTANCE.getKeyRange(Bytes.toBytes("C"), true, Bytes.toBytes("D"),
+                                true, SortOrder.ASC), } },
+            new int[] { 1, 1, 1 }, Bytes.toBytes("a0A"), Bytes.toBytes("b1D"),
+            new KeyRange[][] {
+                    { PChar.INSTANCE.getKeyRange(Bytes.toBytes("b"), true, Bytes.toBytes("b"), true,
+                        SortOrder.ASC), },
+                    { PChar.INSTANCE.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("1"), true,
+                        SortOrder.ASC), },
+                    { PChar.INSTANCE.getKeyRange(Bytes.toBytes("B"), true, Bytes.toBytes("B"), true,
+                        SortOrder.ASC),
+                            PChar.INSTANCE.getKeyRange(Bytes.toBytes("C"), true, Bytes.toBytes("D"),
+                                true, SortOrder.ASC), } }));
         testCases.addAll(foreach(
-                new KeyRange[][] {{
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("b"), true, Bytes.toBytes("b"), true),
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("c"), true, Bytes.toBytes("d"), true),
-                    }, {
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("1"), true),
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("2"), true, Bytes.toBytes("3"), true),
-                    }, {
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("B"), true, Bytes.toBytes("B"), true),
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("C"), true, Bytes.toBytes("D"), true),
-                }},
-                new int[] {1,1,1},
-                Bytes.toBytes("a0A"),
-                Bytes.toBytes("b1D"),
-                new KeyRange[][] {{
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("b"), true, Bytes.toBytes("b"), true),
-                    }, {
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("1"), true),
-                    }, {
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("B"), true, Bytes.toBytes("B"), true),
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("C"), true, Bytes.toBytes("D"), true),
-                }}));
+            new KeyRange[][] {
+                    { PChar.INSTANCE.getKeyRange(Bytes.toBytes("b"), true, Bytes.toBytes("b"), true,
+                        SortOrder.ASC),
+                            PChar.INSTANCE.getKeyRange(Bytes.toBytes("c"), true, Bytes.toBytes("d"),
+                                true, SortOrder.ASC), },
+                    { PChar.INSTANCE.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("1"), true,
+                        SortOrder.ASC),
+                            PChar.INSTANCE.getKeyRange(Bytes.toBytes("2"), true, Bytes.toBytes("3"),
+                                true, SortOrder.ASC), },
+                    { PChar.INSTANCE.getKeyRange(Bytes.toBytes("B"), true, Bytes.toBytes("B"), true,
+                        SortOrder.ASC),
+                            PChar.INSTANCE.getKeyRange(Bytes.toBytes("C"), true, Bytes.toBytes("D"),
+                                true, SortOrder.ASC), } },
+            new int[] { 1, 1, 1 }, Bytes.toBytes("a0A"), Bytes.toBytes("b1D"),
+            new KeyRange[][] {
+                    { PChar.INSTANCE.getKeyRange(Bytes.toBytes("b"), true, Bytes.toBytes("b"), true,
+                        SortOrder.ASC), },
+                    { PChar.INSTANCE.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("1"), true,
+                        SortOrder.ASC), },
+                    { PChar.INSTANCE.getKeyRange(Bytes.toBytes("B"), true, Bytes.toBytes("B"), true,
+                        SortOrder.ASC),
+                            PChar.INSTANCE.getKeyRange(Bytes.toBytes("C"), true, Bytes.toBytes("D"),
+                                true, SortOrder.ASC), } }));
         testCases.addAll(foreach(
-                new KeyRange[][] {{
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("b"), true, Bytes.toBytes("b"), true),
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("c"), true, Bytes.toBytes("d"), true),
-                    }, {
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("1"), true),
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("2"), true, Bytes.toBytes("3"), true),
-                    }, {
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("B"), true, Bytes.toBytes("B"), true),
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("C"), true, Bytes.toBytes("D"), true),
-                }},
-                new int[] {1,1,1},
-                Bytes.toBytes("b1B"),
-                Bytes.toBytes("b1D"),
-                new KeyRange[][] {{
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("b"), true, Bytes.toBytes("b"), true),
-                    }, {
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("1"), true),
-                    }, {
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("B"), true, Bytes.toBytes("B"), true),
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("C"), true, Bytes.toBytes("D"), true),
-                }}));
+            new KeyRange[][] {
+                    { PChar.INSTANCE.getKeyRange(Bytes.toBytes("b"), true, Bytes.toBytes("b"), true,
+                        SortOrder.ASC),
+                            PChar.INSTANCE.getKeyRange(Bytes.toBytes("c"), true, Bytes.toBytes("d"),
+                                true, SortOrder.ASC), },
+                    { PChar.INSTANCE.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("1"), true,
+                        SortOrder.ASC),
+                            PChar.INSTANCE.getKeyRange(Bytes.toBytes("2"), true, Bytes.toBytes("3"),
+                                true, SortOrder.ASC), },
+                    { PChar.INSTANCE.getKeyRange(Bytes.toBytes("B"), true, Bytes.toBytes("B"), true,
+                        SortOrder.ASC),
+                            PChar.INSTANCE.getKeyRange(Bytes.toBytes("C"), true, Bytes.toBytes("D"),
+                                true, SortOrder.ASC), } },
+            new int[] { 1, 1, 1 }, Bytes.toBytes("b1B"), Bytes.toBytes("b1D"),
+            new KeyRange[][] {
+                    { PChar.INSTANCE.getKeyRange(Bytes.toBytes("b"), true, Bytes.toBytes("b"), true,
+                        SortOrder.ASC), },
+                    { PChar.INSTANCE.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("1"), true,
+                        SortOrder.ASC), },
+                    { PChar.INSTANCE.getKeyRange(Bytes.toBytes("B"), true, Bytes.toBytes("B"), true,
+                        SortOrder.ASC),
+                            PChar.INSTANCE.getKeyRange(Bytes.toBytes("C"), true, Bytes.toBytes("D"),
+                                true, SortOrder.ASC), } }));
         testCases.addAll(foreach(
-                new KeyRange[][] {{
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("b"), true, Bytes.toBytes("b"), true),
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("c"), true, Bytes.toBytes("d"), true),
-                    }, {
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("1"), true),
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("2"), true, Bytes.toBytes("3"), true),
-                    }, {
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("B"), true, Bytes.toBytes("B"), true),
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("C"), true, Bytes.toBytes("D"), true),
-                }},
-                new int[] {1,1,1},
-                Bytes.toBytes("a0A"),
-                Bytes.toBytes("b1F"),
-                new KeyRange[][] {{
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("b"), true, Bytes.toBytes("b"), true),
-                    }, {
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("1"), true),
-                    }, {
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("B"), true, Bytes.toBytes("B"), true),
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("C"), true, Bytes.toBytes("D"), true),
-                }}));
+            new KeyRange[][] {
+                    { PChar.INSTANCE.getKeyRange(Bytes.toBytes("b"), true, Bytes.toBytes("b"), true,
+                        SortOrder.ASC),
+                            PChar.INSTANCE.getKeyRange(Bytes.toBytes("c"), true, Bytes.toBytes("d"),
+                                true, SortOrder.ASC), },
+                    { PChar.INSTANCE.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("1"), true,
+                        SortOrder.ASC),
+                            PChar.INSTANCE.getKeyRange(Bytes.toBytes("2"), true, Bytes.toBytes("3"),
+                                true, SortOrder.ASC), },
+                    { PChar.INSTANCE.getKeyRange(Bytes.toBytes("B"), true, Bytes.toBytes("B"), true,
+                        SortOrder.ASC),
+                            PChar.INSTANCE.getKeyRange(Bytes.toBytes("C"), true, Bytes.toBytes("D"),
+                                true, SortOrder.ASC), } },
+            new int[] { 1, 1, 1 }, Bytes.toBytes("a0A"), Bytes.toBytes("b1F"),
+            new KeyRange[][] {
+                    { PChar.INSTANCE.getKeyRange(Bytes.toBytes("b"), true, Bytes.toBytes("b"), true,
+                        SortOrder.ASC), },
+                    { PChar.INSTANCE.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("1"), true,
+                        SortOrder.ASC), },
+                    { PChar.INSTANCE.getKeyRange(Bytes.toBytes("B"), true, Bytes.toBytes("B"), true,
+                        SortOrder.ASC),
+                            PChar.INSTANCE.getKeyRange(Bytes.toBytes("C"), true, Bytes.toBytes("D"),
+                                true, SortOrder.ASC), } }));
         testCases.addAll(foreach(
-                new KeyRange[][] {{
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("b"), true, Bytes.toBytes("b"), true),
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("c"), true, Bytes.toBytes("e"), true),
-                    }, {
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("1"), true),
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("2"), true, Bytes.toBytes("4"), true),
-                    }, {
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("B"), true, Bytes.toBytes("B"), true),
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("C"), true, Bytes.toBytes("E"), true),
-                }},
-                new int[] {1,1,1},
-                Bytes.toBytes("a0Z"),
-                Bytes.toBytes("b3Z"),
-                new KeyRange[][] {{
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("b"), true, Bytes.toBytes("b"), true),
-                    }, {
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("1"), true),
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("2"), true, Bytes.toBytes("4"), true),
-                    }, {
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("B"), true, Bytes.toBytes("B"), true),
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("C"), true, Bytes.toBytes("E"), true),
-                }}));
+            new KeyRange[][] {
+                    { PChar.INSTANCE.getKeyRange(Bytes.toBytes("b"), true, Bytes.toBytes("b"), true,
+                        SortOrder.ASC),
+                            PChar.INSTANCE.getKeyRange(Bytes.toBytes("c"), true, Bytes.toBytes("e"),
+                                true, SortOrder.ASC), },
+                    { PChar.INSTANCE.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("1"), true,
+                        SortOrder.ASC),
+                            PChar.INSTANCE.getKeyRange(Bytes.toBytes("2"), true, Bytes.toBytes("4"),
+                                true, SortOrder.ASC), },
+                    { PChar.INSTANCE.getKeyRange(Bytes.toBytes("B"), true, Bytes.toBytes("B"), true,
+                        SortOrder.ASC),
+                            PChar.INSTANCE.getKeyRange(Bytes.toBytes("C"), true, Bytes.toBytes("E"),
+                                true, SortOrder.ASC), } },
+            new int[] { 1, 1, 1 }, Bytes.toBytes("a0Z"), Bytes.toBytes("b3Z"),
+            new KeyRange[][] {
+                    { PChar.INSTANCE.getKeyRange(Bytes.toBytes("b"), true, Bytes.toBytes("b"), true,
+                        SortOrder.ASC), },
+                    { PChar.INSTANCE.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("1"), true,
+                        SortOrder.ASC),
+                            PChar.INSTANCE.getKeyRange(Bytes.toBytes("2"), true, Bytes.toBytes("4"),
+                                true, SortOrder.ASC), },
+                    { PChar.INSTANCE.getKeyRange(Bytes.toBytes("B"), true, Bytes.toBytes("B"), true,
+                        SortOrder.ASC),
+                            PChar.INSTANCE.getKeyRange(Bytes.toBytes("C"), true, Bytes.toBytes("E"),
+                                true, SortOrder.ASC), } }));
         testCases.addAll(foreach(
-                new KeyRange[][] {{
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("b"), true, Bytes.toBytes("b"), true),
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("c"), true, Bytes.toBytes("e"), true),
-                    }, {
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("1"), true),
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("2"), true, Bytes.toBytes("4"), true),
-                    }, {
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("B"), true, Bytes.toBytes("B"), true),
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("C"), true, Bytes.toBytes("E"), true),
-                }},
-                new int[] {1,1,1},
-                Bytes.toBytes("a0Z"),
-                Bytes.toBytes("b9Z"),
-                new KeyRange[][] {{
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("b"), true, Bytes.toBytes("b"), true),
-                    }, {
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("1"), true),
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("2"), true, Bytes.toBytes("4"), true),
-                    }, {
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("B"), true, Bytes.toBytes("B"), true),
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("C"), true, Bytes.toBytes("E"), true),
-                }}));
+            new KeyRange[][] {
+                    { PChar.INSTANCE.getKeyRange(Bytes.toBytes("b"), true, Bytes.toBytes("b"), true,
+                        SortOrder.ASC),
+                            PChar.INSTANCE.getKeyRange(Bytes.toBytes("c"), true, Bytes.toBytes("e"),
+                                true, SortOrder.ASC), },
+                    { PChar.INSTANCE.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("1"), true,
+                        SortOrder.ASC),
+                            PChar.INSTANCE.getKeyRange(Bytes.toBytes("2"), true, Bytes.toBytes("4"),
+                                true, SortOrder.ASC), },
+                    { PChar.INSTANCE.getKeyRange(Bytes.toBytes("B"), true, Bytes.toBytes("B"), true,
+                        SortOrder.ASC),
+                            PChar.INSTANCE.getKeyRange(Bytes.toBytes("C"), true, Bytes.toBytes("E"),
+                                true, SortOrder.ASC), } },
+            new int[] { 1, 1, 1 }, Bytes.toBytes("a0Z"), Bytes.toBytes("b9Z"),
+            new KeyRange[][] {
+                    { PChar.INSTANCE.getKeyRange(Bytes.toBytes("b"), true, Bytes.toBytes("b"), true,
+                        SortOrder.ASC), },
+                    { PChar.INSTANCE.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("1"), true,
+                        SortOrder.ASC),
+                            PChar.INSTANCE.getKeyRange(Bytes.toBytes("2"), true, Bytes.toBytes("4"),
+                                true, SortOrder.ASC), },
+                    { PChar.INSTANCE.getKeyRange(Bytes.toBytes("B"), true, Bytes.toBytes("B"), true,
+                        SortOrder.ASC),
+                            PChar.INSTANCE.getKeyRange(Bytes.toBytes("C"), true, Bytes.toBytes("E"),
+                                true, SortOrder.ASC), } }));
         // Multiple matching in all slot.
         testCases.addAll(foreach(
-                new KeyRange[][] {{
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("b"), true, Bytes.toBytes("b"), true),
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("c"), true, Bytes.toBytes("e"), true),
-                    }, {
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("1"), true),
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("2"), true, Bytes.toBytes("4"), true),
-                    }, {
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("B"), true, Bytes.toBytes("B"), true),
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("C"), true, Bytes.toBytes("E"), true),
-                }},
-                new int[] {1,1,1},
-                Bytes.toBytes("a0Z"),
-                Bytes.toBytes("c3Z"),
-                new KeyRange[][] {{
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("b"), true, Bytes.toBytes("b"), true),
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("c"), true, Bytes.toBytes("e"), true),
-                    }, {
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("1"), true),
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("2"), true, Bytes.toBytes("4"), true),
-                    }, {
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("B"), true, Bytes.toBytes("B"), true),
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("C"), true, Bytes.toBytes("E"), true),
-                }}));
+            new KeyRange[][] {
+                    { PChar.INSTANCE.getKeyRange(Bytes.toBytes("b"), true, Bytes.toBytes("b"), true,
+                        SortOrder.ASC),
+                            PChar.INSTANCE.getKeyRange(Bytes.toBytes("c"), true, Bytes.toBytes("e"),
+                                true, SortOrder.ASC), },
+                    { PChar.INSTANCE.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("1"), true,
+                        SortOrder.ASC),
+                            PChar.INSTANCE.getKeyRange(Bytes.toBytes("2"), true, Bytes.toBytes("4"),
+                                true, SortOrder.ASC), },
+                    { PChar.INSTANCE.getKeyRange(Bytes.toBytes("B"), true, Bytes.toBytes("B"), true,
+                        SortOrder.ASC),
+                            PChar.INSTANCE.getKeyRange(Bytes.toBytes("C"), true, Bytes.toBytes("E"),
+                                true, SortOrder.ASC), } },
+            new int[] { 1, 1, 1 }, Bytes.toBytes("a0Z"), Bytes.toBytes("c3Z"),
+            new KeyRange[][] {
+                    { PChar.INSTANCE.getKeyRange(Bytes.toBytes("b"), true, Bytes.toBytes("b"), true,
+                        SortOrder.ASC),
+                            PChar.INSTANCE.getKeyRange(Bytes.toBytes("c"), true, Bytes.toBytes("e"),
+                                true, SortOrder.ASC), },
+                    { PChar.INSTANCE.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("1"), true,
+                        SortOrder.ASC),
+                            PChar.INSTANCE.getKeyRange(Bytes.toBytes("2"), true, Bytes.toBytes("4"),
+                                true, SortOrder.ASC), },
+                    { PChar.INSTANCE.getKeyRange(Bytes.toBytes("B"), true, Bytes.toBytes("B"), true,
+                        SortOrder.ASC),
+                            PChar.INSTANCE.getKeyRange(Bytes.toBytes("C"), true, Bytes.toBytes("E"),
+                                true, SortOrder.ASC), } }));
         testCases.addAll(foreach(
-                new KeyRange[][] {{
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("b"), true, Bytes.toBytes("b"), true),
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("c"), true, Bytes.toBytes("e"), true),
-                    }, {
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("1"), true),
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("2"), true, Bytes.toBytes("4"), true),
-                    }, {
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("B"), true, Bytes.toBytes("B"), true),
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("C"), true, Bytes.toBytes("E"), true),
-                }},
-                new int[] {1,1,1},
-                Bytes.toBytes("a0A"),
-                Bytes.toBytes("f4F"),
-                new KeyRange[][] {{
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("b"), true, Bytes.toBytes("b"), true),
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("c"), true, Bytes.toBytes("e"), true),
-                    }, {
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("1"), true),
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("2"), true, Bytes.toBytes("4"), true),
-                    }, {
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("B"), true, Bytes.toBytes("B"), true),
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("C"), true, Bytes.toBytes("E"), true),
-                }}));
+            new KeyRange[][] {
+                    { PChar.INSTANCE.getKeyRange(Bytes.toBytes("b"), true, Bytes.toBytes("b"), true,
+                        SortOrder.ASC),
+                            PChar.INSTANCE.getKeyRange(Bytes.toBytes("c"), true, Bytes.toBytes("e"),
+                                true, SortOrder.ASC), },
+                    { PChar.INSTANCE.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("1"), true,
+                        SortOrder.ASC),
+                            PChar.INSTANCE.getKeyRange(Bytes.toBytes("2"), true, Bytes.toBytes("4"),
+                                true, SortOrder.ASC), },
+                    { PChar.INSTANCE.getKeyRange(Bytes.toBytes("B"), true, Bytes.toBytes("B"), true,
+                        SortOrder.ASC),
+                            PChar.INSTANCE.getKeyRange(Bytes.toBytes("C"), true, Bytes.toBytes("E"),
+                                true, SortOrder.ASC), } },
+            new int[] { 1, 1, 1 }, Bytes.toBytes("a0A"), Bytes.toBytes("f4F"),
+            new KeyRange[][] {
+                    { PChar.INSTANCE.getKeyRange(Bytes.toBytes("b"), true, Bytes.toBytes("b"), true,
+                        SortOrder.ASC),
+                            PChar.INSTANCE.getKeyRange(Bytes.toBytes("c"), true, Bytes.toBytes("e"),
+                                true, SortOrder.ASC), },
+                    { PChar.INSTANCE.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("1"), true,
+                        SortOrder.ASC),
+                            PChar.INSTANCE.getKeyRange(Bytes.toBytes("2"), true, Bytes.toBytes("4"),
+                                true, SortOrder.ASC), },
+                    { PChar.INSTANCE.getKeyRange(Bytes.toBytes("B"), true, Bytes.toBytes("B"), true,
+                        SortOrder.ASC),
+                            PChar.INSTANCE.getKeyRange(Bytes.toBytes("C"), true, Bytes.toBytes("E"),
+                                true, SortOrder.ASC), } }));
         // VARCHAR as the last column, various cases.
         testCases.addAll(foreach(
-                new KeyRange[][] {{
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("b"), true, Bytes.toBytes("b"), true),
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("c"), true, Bytes.toBytes("e"), true),
-                    }, {
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("1"), true),
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("2"), true, Bytes.toBytes("4"), true),
-                    }, {
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("B"), true, Bytes.toBytes("B"), true),
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("C"), true, Bytes.toBytes("E"), true),
-                }},
-                new int[] {1,1,-1},
-                Bytes.toBytes("d3AA"),
-                Bytes.toBytes("d4FF"),
-                new KeyRange[][] {{
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("c"), true, Bytes.toBytes("e"), true),
-                    }, {
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("2"), true, Bytes.toBytes("4"), true),
-                    }, {
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("B"), true, Bytes.toBytes("B"), true),
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("C"), true, Bytes.toBytes("E"), true),
-                }}));
+            new KeyRange[][] {
+                    { PChar.INSTANCE.getKeyRange(Bytes.toBytes("b"), true, Bytes.toBytes("b"), true,
+                        SortOrder.ASC),
+                            PChar.INSTANCE.getKeyRange(Bytes.toBytes("c"), true, Bytes.toBytes("e"),
+                                true, SortOrder.ASC), },
+                    { PChar.INSTANCE.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("1"), true,
+                        SortOrder.ASC),
+                            PChar.INSTANCE.getKeyRange(Bytes.toBytes("2"), true, Bytes.toBytes("4"),
+                                true, SortOrder.ASC), },
+                    { PChar.INSTANCE.getKeyRange(Bytes.toBytes("B"), true, Bytes.toBytes("B"), true,
+                        SortOrder.ASC),
+                            PChar.INSTANCE.getKeyRange(Bytes.toBytes("C"), true, Bytes.toBytes("E"),
+                                true, SortOrder.ASC), } },
+            new int[] { 1, 1, -1 }, Bytes.toBytes("d3AA"), Bytes.toBytes("d4FF"),
+            new KeyRange[][] {
+                    { PChar.INSTANCE.getKeyRange(Bytes.toBytes("c"), true, Bytes.toBytes("e"), true,
+                        SortOrder.ASC), },
+                    { PChar.INSTANCE.getKeyRange(Bytes.toBytes("2"), true, Bytes.toBytes("4"), true,
+                        SortOrder.ASC), },
+                    { PChar.INSTANCE.getKeyRange(Bytes.toBytes("B"), true, Bytes.toBytes("B"), true,
+                        SortOrder.ASC),
+                            PChar.INSTANCE.getKeyRange(Bytes.toBytes("C"), true, Bytes.toBytes("E"),
+                                true, SortOrder.ASC) } }));
         testCases.addAll(foreach(
-                new KeyRange[][] {{
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("b"), true, Bytes.toBytes("b"), true),
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("c"), true, Bytes.toBytes("e"), true),
-                    }, {
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("1"), true),
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("2"), true, Bytes.toBytes("4"), true),
-                    }, {
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("B"), true, Bytes.toBytes("B"), true),
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("C"), true, Bytes.toBytes("E"), true),
-                }},
-                new int[] {1,1,-1},
-                Bytes.toBytes("d0AA"),
-                Bytes.toBytes("d4FF"),
-                new KeyRange[][] {{
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("c"), true, Bytes.toBytes("e"), true),
-                    }, {
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("1"), true),
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("2"), true, Bytes.toBytes("4"), true),
-                    }, {
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("B"), true, Bytes.toBytes("B"), true),
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("C"), true, Bytes.toBytes("E"), true),
-                }}));
+            new KeyRange[][] {
+                    { PChar.INSTANCE.getKeyRange(Bytes.toBytes("b"), true, Bytes.toBytes("b"), true,
+                        SortOrder.ASC),
+                            PChar.INSTANCE.getKeyRange(Bytes.toBytes("c"), true, Bytes.toBytes("e"),
+                                true, SortOrder.ASC), },
+                    { PChar.INSTANCE.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("1"), true,
+                        SortOrder.ASC),
+                            PChar.INSTANCE.getKeyRange(Bytes.toBytes("2"), true, Bytes.toBytes("4"),
+                                true, SortOrder.ASC), },
+                    { PChar.INSTANCE.getKeyRange(Bytes.toBytes("B"), true, Bytes.toBytes("B"), true,
+                        SortOrder.ASC),
+                            PChar.INSTANCE.getKeyRange(Bytes.toBytes("C"), true, Bytes.toBytes("E"),
+                                true, SortOrder.ASC), } },
+            new int[] { 1, 1, -1 }, Bytes.toBytes("d0AA"), Bytes.toBytes("d4FF"),
+            new KeyRange[][] {
+                    { PChar.INSTANCE.getKeyRange(Bytes.toBytes("c"), true, Bytes.toBytes("e"), true,
+                        SortOrder.ASC), },
+                    { PChar.INSTANCE.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("1"), true,
+                        SortOrder.ASC),
+                            PChar.INSTANCE.getKeyRange(Bytes.toBytes("2"), true, Bytes.toBytes("4"),
+                                true, SortOrder.ASC), },
+                    { PChar.INSTANCE.getKeyRange(Bytes.toBytes("B"), true, Bytes.toBytes("B"), true,
+                        SortOrder.ASC),
+                            PChar.INSTANCE.getKeyRange(Bytes.toBytes("C"), true, Bytes.toBytes("E"),
+                                true, SortOrder.ASC), } }));
         testCases.addAll(foreach(
-                new KeyRange[][] {{
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("b"), true, Bytes.toBytes("b"), true),
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("c"), true, Bytes.toBytes("e"), true),
-                    }, {
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("1"), true),
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("2"), true, Bytes.toBytes("4"), true),
-                    }, {
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("B"), true, Bytes.toBytes("B"), true),
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("C"), true, Bytes.toBytes("E"), true),
-                }},
-                new int[] {1,1,-1},
-                Bytes.toBytes("a0AA"),
-                Bytes.toBytes("f4FF"),
-                new KeyRange[][] {{
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("b"), true, Bytes.toBytes("b"), true),
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("c"), true, Bytes.toBytes("e"), true),
-                    }, {
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("1"), true),
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("2"), true, Bytes.toBytes("4"), true),
-                    }, {
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("B"), true, Bytes.toBytes("B"), true),
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("C"), true, Bytes.toBytes("E"), true),
-                }}));
+            new KeyRange[][] {
+                    { PChar.INSTANCE.getKeyRange(Bytes.toBytes("b"), true, Bytes.toBytes("b"), true,
+                        SortOrder.ASC),
+                            PChar.INSTANCE.getKeyRange(Bytes.toBytes("c"), true, Bytes.toBytes("e"),
+                                true, SortOrder.ASC), },
+                    { PChar.INSTANCE.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("1"), true,
+                        SortOrder.ASC),
+                            PChar.INSTANCE.getKeyRange(Bytes.toBytes("2"), true, Bytes.toBytes("4"),
+                                true, SortOrder.ASC), },
+                    { PChar.INSTANCE.getKeyRange(Bytes.toBytes("B"), true, Bytes.toBytes("B"), true,
+                        SortOrder.ASC),
+                            PChar.INSTANCE.getKeyRange(Bytes.toBytes("C"), true, Bytes.toBytes("E"),
+                                true, SortOrder.ASC), } },
+            new int[] { 1, 1, -1 }, Bytes.toBytes("a0AA"), Bytes.toBytes("f4FF"),
+            new KeyRange[][] {
+                    { PChar.INSTANCE.getKeyRange(Bytes.toBytes("b"), true, Bytes.toBytes("b"), true,
+                        SortOrder.ASC),
+                            PChar.INSTANCE.getKeyRange(Bytes.toBytes("c"), true, Bytes.toBytes("e"),
+                                true, SortOrder.ASC), },
+                    { PChar.INSTANCE.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("1"), true,
+                        SortOrder.ASC),
+                            PChar.INSTANCE.getKeyRange(Bytes.toBytes("2"), true, Bytes.toBytes("4"),
+                                true, SortOrder.ASC), },
+                    { PChar.INSTANCE.getKeyRange(Bytes.toBytes("B"), true, Bytes.toBytes("B"), true,
+                        SortOrder.ASC),
+                            PChar.INSTANCE.getKeyRange(Bytes.toBytes("C"), true, Bytes.toBytes("E"),
+                                true, SortOrder.ASC), } }));
         return testCases;
     }
 
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/filter/SkipScanFilterTest.java b/phoenix-core/src/test/java/org/apache/phoenix/filter/SkipScanFilterTest.java
index 6cb80ef434..ceeaa6bcdd 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/filter/SkipScanFilterTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/filter/SkipScanFilterTest.java
@@ -114,18 +114,18 @@ public class SkipScanFilterTest extends TestCase {
         // Variable length tests
         testCases.addAll(
                 foreach(new KeyRange[][]{{
-                    PVarchar.INSTANCE.getKeyRange(Bytes.toBytes("a"), true, Bytes.toBytes("a"), true),
-                    PVarchar.INSTANCE.getKeyRange(Bytes.toBytes("e"), true, Bytes.toBytes("e"), true),
-                    PVarchar.INSTANCE.getKeyRange(Bytes.toBytes("f"), true, Bytes.toBytes("f"), true),
+                    PVarchar.INSTANCE.getKeyRange(Bytes.toBytes("a"), true, Bytes.toBytes("a"), true, SortOrder.ASC),
+                    PVarchar.INSTANCE.getKeyRange(Bytes.toBytes("e"), true, Bytes.toBytes("e"), true, SortOrder.ASC),
+                    PVarchar.INSTANCE.getKeyRange(Bytes.toBytes("f"), true, Bytes.toBytes("f"), true, SortOrder.ASC)
                 },
                 {
-                    PVarchar.INSTANCE.getKeyRange(Bytes.toBytes("b"), true, Bytes.toBytes("b"), true),
+                    PVarchar.INSTANCE.getKeyRange(Bytes.toBytes("b"), true, Bytes.toBytes("b"), true, SortOrder.ASC),
                 },
                 {
                     KeyRange.EVERYTHING_RANGE,
                 },
                 {
-                    PVarchar.INSTANCE.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("1"), true),
+                    PVarchar.INSTANCE.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("1"), true, SortOrder.ASC),
                 }},
                 new int[4],
                 null,
@@ -147,13 +147,13 @@ public class SkipScanFilterTest extends TestCase {
         );
         testCases.addAll(
                 foreach(new KeyRange[][]{{
-                                PVarchar.INSTANCE.getKeyRange(Bytes.toBytes("2018-02-10"), true, Bytes.toBytes("2019-02-19"), true),
+                                PVarchar.INSTANCE.getKeyRange(Bytes.toBytes("2018-02-10"), true, Bytes.toBytes("2019-02-19"), true, SortOrder.ASC),
                         },
                         {
-                                PVarchar.INSTANCE.getKeyRange(Bytes.toBytes("channel"), true, Bytes.toBytes("channel"), true),
+                                PVarchar.INSTANCE.getKeyRange(Bytes.toBytes("channel"), true, Bytes.toBytes("channel"), true, SortOrder.ASC),
                         },
                         {
-                                PChar.INSTANCE.getKeyRange(Bytes.toBytes("2"), true, Bytes.toBytes("2"), true),
+                                PChar.INSTANCE.getKeyRange(Bytes.toBytes("2"), true, Bytes.toBytes("2"), true, SortOrder.ASC),
                         },
                         {
                                 KeyRange.EVERYTHING_RANGE,
@@ -162,7 +162,7 @@ public class SkipScanFilterTest extends TestCase {
                                 KeyRange.EVERYTHING_RANGE,
                         },
                         {
-                                PVarchar.INSTANCE.getKeyRange(Bytes.toBytes("A004"), true, Bytes.toBytes("A004"), true),
+                                PVarchar.INSTANCE.getKeyRange(Bytes.toBytes("A004"), true, Bytes.toBytes("A004"), true, SortOrder.ASC),
                         },
                 },
                 new int[] {0, 0, 1, 0, 0, 0, 0, 0},
@@ -180,10 +180,10 @@ public class SkipScanFilterTest extends TestCase {
         );
         testCases.addAll(
                 foreach(new KeyRange[][]{{
-                    PVarchar.INSTANCE.getKeyRange(Bytes.toBytes("20160116121006"), true, Bytes.toBytes("20160116181006"), true),
+                    PVarchar.INSTANCE.getKeyRange(Bytes.toBytes("20160116121006"), true, Bytes.toBytes("20160116181006"), true, SortOrder.ASC),
                 },
                 {
-                    PVarchar.INSTANCE.getKeyRange(Bytes.toBytes("2404787"), true, Bytes.toBytes("2404787"), true),
+                    PVarchar.INSTANCE.getKeyRange(Bytes.toBytes("2404787"), true, Bytes.toBytes("2404787"), true, SortOrder.ASC),
                 }/*,
                 {
                     KeyRange.EVERYTHING_RANGE,
@@ -205,20 +205,20 @@ public class SkipScanFilterTest extends TestCase {
         // Fixed length tests
         testCases.addAll(
                 foreach(new KeyRange[][]{{
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("abc"), true, Bytes.toBytes("def"), true),
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("dzy"), false, Bytes.toBytes("xyz"), false),
+                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("abc"), true, Bytes.toBytes("def"), true, SortOrder.ASC),
+                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("dzy"), false, Bytes.toBytes("xyz"), false, SortOrder.ASC),
                 },
                 {
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("AA"), true, Bytes.toBytes("AB"), false),
+                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("AA"), true, Bytes.toBytes("AB"), false, SortOrder.ASC),
                 },
                 {
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("AA"), true, Bytes.toBytes("AB"), false),
+                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("AA"), true, Bytes.toBytes("AB"), false, SortOrder.ASC),
                 },
                 {
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("AA"), true, Bytes.toBytes("AB"), false),
+                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("AA"), true, Bytes.toBytes("AB"), false, SortOrder.ASC),
                 },
                 {
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("AA"), true, Bytes.toBytes("AB"), false),
+                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("AA"), true, Bytes.toBytes("AB"), false, SortOrder.ASC),
                 }},
                 new int[]{3,2,2,2,2},
                 null,
@@ -227,7 +227,7 @@ public class SkipScanFilterTest extends TestCase {
         );
         testCases.addAll(
                 foreach(new KeyRange[][]{{
-                        PVarchar.INSTANCE.getKeyRange(Bytes.toBytes("j"), false, Bytes.toBytes("k"), true),
+                        PVarchar.INSTANCE.getKeyRange(Bytes.toBytes("j"), false, Bytes.toBytes("k"), true, SortOrder.ASC),
                     }},
                     new int[]{0},
                     null,
@@ -238,9 +238,9 @@ public class SkipScanFilterTest extends TestCase {
                     new Finished("ka")));
         testCases.addAll(
             foreach(new KeyRange[][]{{
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("aaa"), true, Bytes.toBytes("aaa"), true),
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("aac"), true, Bytes.toBytes("aad"), true),
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("abc"), true, Bytes.toBytes("def"), true)
+                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("aaa"), true, Bytes.toBytes("aaa"), true, SortOrder.ASC),
+                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("aac"), true, Bytes.toBytes("aad"), true, SortOrder.ASC),
+                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("abc"), true, Bytes.toBytes("def"), true, SortOrder.ASC),
                 }},
                 new int[]{3},
                 null,
@@ -252,8 +252,8 @@ public class SkipScanFilterTest extends TestCase {
                 new Finished("deg")));
         testCases.addAll(
             foreach(new KeyRange[][]{{
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("aaa"), true, Bytes.toBytes("aaa"), true),
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("abc"), false, Bytes.toBytes("def"), true)
+                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("aaa"), true, Bytes.toBytes("aaa"), true, SortOrder.ASC),
+                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("abc"), false, Bytes.toBytes("def"), true, SortOrder.ASC)
                 }},
                 new int[]{3},
                 null,
@@ -263,8 +263,8 @@ public class SkipScanFilterTest extends TestCase {
                 new Finished("deg")));
         testCases.addAll(
             foreach(new KeyRange[][]{{
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("aaa"), true, Bytes.toBytes("aaa"), true),
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("abc"), false, Bytes.toBytes("def"), false)
+                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("aaa"), true, Bytes.toBytes("aaa"), true, SortOrder.ASC),
+                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("abc"), false, Bytes.toBytes("def"), false, SortOrder.ASC)
                 }},
                 new int[]{3},
                 null,
@@ -273,8 +273,8 @@ public class SkipScanFilterTest extends TestCase {
         );
         testCases.addAll(
             foreach(new KeyRange[][]{{
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("abc"), true, Bytes.toBytes("def"), true),
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("dzy"), false, Bytes.toBytes("xyz"), false),
+                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("abc"), true, Bytes.toBytes("def"), true, SortOrder.ASC),
+                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("dzy"), false, Bytes.toBytes("xyz"), false, SortOrder.ASC),
                 }},
                 new int[]{3},
                 null,
@@ -285,14 +285,14 @@ public class SkipScanFilterTest extends TestCase {
         );
         testCases.addAll(
             foreach(new KeyRange[][]{{
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("aaa"), true, Bytes.toBytes("aaa"), true),
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("abc"), true, Bytes.toBytes("abc"), true),
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("def"), true, Bytes.toBytes("def"), true),
+                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("aaa"), true, Bytes.toBytes("aaa"), true, SortOrder.ASC),
+                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("abc"), true, Bytes.toBytes("abc"), true, SortOrder.ASC),
+                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("def"), true, Bytes.toBytes("def"), true, SortOrder.ASC),
                 },
                 {
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("AB"), true, Bytes.toBytes("AX"), true),
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("EA"), false, Bytes.toBytes("EZ"), false),
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("PO"), true, Bytes.toBytes("PP"), false),
+                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("AB"), true, Bytes.toBytes("AX"), true, SortOrder.ASC),
+                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("EA"), false, Bytes.toBytes("EZ"), false, SortOrder.ASC),
+                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("PO"), true, Bytes.toBytes("PP"), false, SortOrder.ASC),
                 }},
                 new int[]{3,2},
                 null,
@@ -307,13 +307,13 @@ public class SkipScanFilterTest extends TestCase {
         );
         testCases.addAll(
             foreach(new KeyRange[][]{{
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("AB"), true, Bytes.toBytes("AX"), true),
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("EA"), false, Bytes.toBytes("EZ"), false),
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("PO"), true, Bytes.toBytes("PP"), false),
+                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("AB"), true, Bytes.toBytes("AX"), true, SortOrder.ASC),
+                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("EA"), false, Bytes.toBytes("EZ"), false, SortOrder.ASC),
+                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("PO"), true, Bytes.toBytes("PP"), false, SortOrder.ASC),
                 },
                 {
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("abc"), true, Bytes.toBytes("abc"), true),
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("def"), true, Bytes.toBytes("def"), true),
+                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("abc"), true, Bytes.toBytes("abc"), true, SortOrder.ASC),
+                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("def"), true, Bytes.toBytes("def"), true, SortOrder.ASC),
                 }},
                 new int[]{2,3},
                 null,
@@ -329,10 +329,10 @@ public class SkipScanFilterTest extends TestCase {
         );
         testCases.addAll(
             foreach(new KeyRange[][]{{
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("PO"), true, Bytes.toBytes("PP"), false),
+                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("PO"), true, Bytes.toBytes("PP"), false, SortOrder.ASC),
                 },
                 {
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("def"), true, Bytes.toBytes("def"), true),
+                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("def"), true, Bytes.toBytes("def"), true, SortOrder.ASC),
                 }},
                 new int[]{2,3},
                 null,
@@ -341,10 +341,10 @@ public class SkipScanFilterTest extends TestCase {
         );
         testCases.addAll(
             foreach(new KeyRange[][]{{
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("PO"), true, Bytes.toBytes("PO"), true),
+                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("PO"), true, Bytes.toBytes("PO"), true, SortOrder.ASC),
                 },
                 {
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("def"), true, Bytes.toBytes("def"), true),
+                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("def"), true, Bytes.toBytes("def"), true, SortOrder.ASC),
                 }},
                 new int[]{2,3},
                 null,
@@ -352,14 +352,14 @@ public class SkipScanFilterTest extends TestCase {
         );
         testCases.addAll(
                 foreach(new KeyRange[][]{{
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("AAA"), true, Bytes.toBytes("AAA"), true),
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("abc"), true, Bytes.toBytes("def"), true),
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("dzy"), false, Bytes.toBytes("xyz"), false),
+                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("AAA"), true, Bytes.toBytes("AAA"), true, SortOrder.ASC),
+                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("abc"), true, Bytes.toBytes("def"), true, SortOrder.ASC),
+                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("dzy"), false, Bytes.toBytes("xyz"), false, SortOrder.ASC),
                 },
                 {
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("AB"), true, Bytes.toBytes("AX"), true),
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("EA"), false, Bytes.toBytes("EZ"), false),
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("PO"), true, Bytes.toBytes("PP"), false),
+                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("AB"), true, Bytes.toBytes("AX"), true, SortOrder.ASC),
+                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("EA"), false, Bytes.toBytes("EZ"), false, SortOrder.ASC),
+                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("PO"), true, Bytes.toBytes("PP"), false, SortOrder.ASC),
                 }},
                 new int[]{3,2},
                 null,
@@ -386,9 +386,9 @@ public class SkipScanFilterTest extends TestCase {
         );
         testCases.addAll(
                 foreach(new KeyRange[][]{{
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("aaa"), true, Bytes.toBytes("aaa"), true),
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("abc"), true, Bytes.toBytes("def"), true),
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("dzz"), true, Bytes.toBytes("xyz"), false),
+                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("aaa"), true, Bytes.toBytes("aaa"), true, SortOrder.ASC),
+                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("abc"), true, Bytes.toBytes("def"), true, SortOrder.ASC),
+                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("dzz"), true, Bytes.toBytes("xyz"), false, SortOrder.ASC),
                 }},
                 new int[]{3},
                 null,
@@ -399,17 +399,17 @@ public class SkipScanFilterTest extends TestCase {
         );
         testCases.addAll(
                 foreach(new KeyRange[][]{{
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("abc"), true, Bytes.toBytes("def"), true),
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("dzy"), false, Bytes.toBytes("xyz"), false),
+                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("abc"), true, Bytes.toBytes("def"), true, SortOrder.ASC),
+                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("dzy"), false, Bytes.toBytes("xyz"), false, SortOrder.ASC),
                 },
                 {
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("AB"), true, Bytes.toBytes("AX"), true),
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("EA"), false, Bytes.toBytes("EZ"), false),
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("PO"), true, Bytes.toBytes("PP"), false),
+                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("AB"), true, Bytes.toBytes("AX"), true, SortOrder.ASC),
+                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("EA"), false, Bytes.toBytes("EZ"), false, SortOrder.ASC),
+                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("PO"), true, Bytes.toBytes("PP"), false, SortOrder.ASC),
                 },
                 {
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("100"), true, Bytes.toBytes("250"), false),
-                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("700"), false, Bytes.toBytes("901"), false),
+                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("100"), true, Bytes.toBytes("250"), false, SortOrder.ASC),
+                    PChar.INSTANCE.getKeyRange(Bytes.toBytes("700"), false, Bytes.toBytes("901"), false, SortOrder.ASC),
                 }},
                 new int[]{3,2,3},
                 null,
@@ -422,14 +422,14 @@ public class SkipScanFilterTest extends TestCase {
         testCases.addAll(
                 foreach(
                     new KeyRange[][]{{
-                        PInteger.INSTANCE.getKeyRange(PInteger.INSTANCE.toBytes(1), true, PInteger.INSTANCE.toBytes(4), true)
+                        PInteger.INSTANCE.getKeyRange(PInteger.INSTANCE.toBytes(1), true, PInteger.INSTANCE.toBytes(4), true, SortOrder.ASC)
                     },
                     {
                         KeyRange.getKeyRange(PInteger.INSTANCE.toBytes(5)),
                         KeyRange.getKeyRange(PInteger.INSTANCE.toBytes(7))
                     },
                     {
-                        PInteger.INSTANCE.getKeyRange(PInteger.INSTANCE.toBytes(9), true, PInteger.INSTANCE.toBytes(10), true)
+                        PInteger.INSTANCE.getKeyRange(PInteger.INSTANCE.toBytes(9), true, PInteger.INSTANCE.toBytes(10), true, SortOrder.ASC),
                     }},
                     new int[]{4,4,4},
                     null,
@@ -459,7 +459,7 @@ public class SkipScanFilterTest extends TestCase {
                     KeyRange.getKeyRange(PInteger.INSTANCE.toBytes(7))
                 },
                 {
-                    PInteger.INSTANCE.getKeyRange(PInteger.INSTANCE.toBytes(9), true, PInteger.INSTANCE.toBytes(10), true)
+                    PInteger.INSTANCE.getKeyRange(PInteger.INSTANCE.toBytes(9), true, PInteger.INSTANCE.toBytes(10), true, SortOrder.ASC),
                 }},
                 new int[]{4,4,4},
                 null,
@@ -536,7 +536,7 @@ public class SkipScanFilterTest extends TestCase {
                     KeyRange.getKeyRange(PInteger.INSTANCE.toBytes(7))
                 },
                 {
-                    PInteger.INSTANCE.getKeyRange(PInteger.INSTANCE.toBytes(9), true, PInteger.INSTANCE.toBytes(10), true)
+                    PInteger.INSTANCE.getKeyRange(PInteger.INSTANCE.toBytes(9), true, PInteger.INSTANCE.toBytes(10), true, SortOrder.ASC),
                 }},
                 new int[]{4,4,4,4},
                 new int[]{1,0,0},
@@ -579,7 +579,7 @@ public class SkipScanFilterTest extends TestCase {
                     KeyRange.getKeyRange(ByteUtil.concat(PInteger.INSTANCE.toBytes(7),PInteger.INSTANCE.toBytes(8)))
                 },
                 {
-                    PInteger.INSTANCE.getKeyRange(PInteger.INSTANCE.toBytes(9), true, PInteger.INSTANCE.toBytes(10), true)
+                    PInteger.INSTANCE.getKeyRange(PInteger.INSTANCE.toBytes(9), true, PInteger.INSTANCE.toBytes(10), true, SortOrder.ASC),
                 }},
                 new int[]{4,4,4,4,4},
                 new int[]{1,1,0},
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/query/KeyRangeCoalesceTest.java b/phoenix-core/src/test/java/org/apache/phoenix/query/KeyRangeCoalesceTest.java
index b5dffeecf3..f3a53b3103 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/query/KeyRangeCoalesceTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/query/KeyRangeCoalesceTest.java
@@ -27,6 +27,7 @@ import java.util.*;
 
 import junit.framework.TestCase;
 
+import org.apache.phoenix.schema.SortOrder;
 import org.apache.phoenix.schema.types.PChar;
 import org.junit.Test;
 import org.junit.runner.RunWith;
@@ -52,64 +53,64 @@ public class KeyRangeCoalesceTest extends TestCase {
                 input(
                 )},
                 {expect(
-                        PChar.INSTANCE.getKeyRange(toBytes("C"), true, toBytes("E"), true)
+                        PChar.INSTANCE.getKeyRange(toBytes("C"), true, toBytes("E"), true, SortOrder.ASC)
                 ),
                 input(
-                        PChar.INSTANCE.getKeyRange(toBytes("C"), true, toBytes("E"), true)
+                        PChar.INSTANCE.getKeyRange(toBytes("C"), true, toBytes("E"), true, SortOrder.ASC)
                 )},
                 {expect(
-                        PChar.INSTANCE.getKeyRange(toBytes("C"), true, toBytes("E"), true)
+                        PChar.INSTANCE.getKeyRange(toBytes("C"), true, toBytes("E"), true, SortOrder.ASC)
                 ),
                 input(
-                        PChar.INSTANCE.getKeyRange(toBytes("C"), true, toBytes("D"), true),
-                        PChar.INSTANCE.getKeyRange(toBytes("D"), true, toBytes("E"), true)
+                        PChar.INSTANCE.getKeyRange(toBytes("C"), true, toBytes("D"), true, SortOrder.ASC),
+                        PChar.INSTANCE.getKeyRange(toBytes("D"), true, toBytes("E"), true, SortOrder.ASC)
                 )},
                 {expect(
-                        PChar.INSTANCE.getKeyRange(toBytes("C"), true, toBytes("Z"), true)
+                        PChar.INSTANCE.getKeyRange(toBytes("C"), true, toBytes("Z"), true, SortOrder.ASC)
                 ),
                 input(
-                        PChar.INSTANCE.getKeyRange(toBytes("C"), true, toBytes("D"), true),
-                        PChar.INSTANCE.getKeyRange(toBytes("D"), true, toBytes("E"), true),
-                        PChar.INSTANCE.getKeyRange(toBytes("D"), true, toBytes("Z"), true)
+                        PChar.INSTANCE.getKeyRange(toBytes("C"), true, toBytes("D"), true, SortOrder.ASC),
+                        PChar.INSTANCE.getKeyRange(toBytes("D"), true, toBytes("E"), true, SortOrder.ASC),
+                        PChar.INSTANCE.getKeyRange(toBytes("D"), true, toBytes("Z"), true, SortOrder.ASC)
                 )},
                 {expect(
-                        PChar.INSTANCE.getKeyRange(toBytes("B"), true, toBytes("Z"), true)
+                        PChar.INSTANCE.getKeyRange(toBytes("B"), true, toBytes("Z"), true, SortOrder.ASC)
                 ),
                 input(
-                        PChar.INSTANCE.getKeyRange(toBytes("C"), true, toBytes("D"), true),
-                        PChar.INSTANCE.getKeyRange(toBytes("B"), true, toBytes("E"), true),
-                        PChar.INSTANCE.getKeyRange(toBytes("D"), true, toBytes("Z"), true)
+                        PChar.INSTANCE.getKeyRange(toBytes("C"), true, toBytes("D"), true, SortOrder.ASC),
+                        PChar.INSTANCE.getKeyRange(toBytes("B"), true, toBytes("E"), true, SortOrder.ASC),
+                        PChar.INSTANCE.getKeyRange(toBytes("D"), true, toBytes("Z"), true, SortOrder.ASC)
                 )},
                 {expect(
-                        PChar.INSTANCE.getKeyRange(toBytes("B"), true, toBytes("Z"), true)
+                        PChar.INSTANCE.getKeyRange(toBytes("B"), true, toBytes("Z"), true, SortOrder.ASC)
                 ),
                 input(
-                        PChar.INSTANCE.getKeyRange(toBytes("C"), true, toBytes("D"), true),
-                        PChar.INSTANCE.getKeyRange(toBytes("B"), true, toBytes("Z"), false),
-                        PChar.INSTANCE.getKeyRange(toBytes("D"), true, toBytes("Z"), true)
+                        PChar.INSTANCE.getKeyRange(toBytes("C"), true, toBytes("D"), true, SortOrder.ASC),
+                        PChar.INSTANCE.getKeyRange(toBytes("B"), true, toBytes("Z"), false, SortOrder.ASC),
+                        PChar.INSTANCE.getKeyRange(toBytes("D"), true, toBytes("Z"), true, SortOrder.ASC)
                 )},
                 {expect(
-                        PChar.INSTANCE.getKeyRange(toBytes("A"), true, toBytes("A"), true),
-                        PChar.INSTANCE.getKeyRange(toBytes("B"), true, toBytes("Z"), false)
+                        PChar.INSTANCE.getKeyRange(toBytes("A"), true, toBytes("A"), true, SortOrder.ASC),
+                        PChar.INSTANCE.getKeyRange(toBytes("B"), true, toBytes("Z"), false, SortOrder.ASC)
                 ),
                 input(
-                        PChar.INSTANCE.getKeyRange(toBytes("A"), true, toBytes("A"), true),
-                        PChar.INSTANCE.getKeyRange(toBytes("B"), true, toBytes("Z"), false)
+                        PChar.INSTANCE.getKeyRange(toBytes("A"), true, toBytes("A"), true, SortOrder.ASC),
+                        PChar.INSTANCE.getKeyRange(toBytes("B"), true, toBytes("Z"), false, SortOrder.ASC)
                 )},
                 {expect(
-                        PChar.INSTANCE.getKeyRange(toBytes("A"), true, toBytes("B"), false),
-                        PChar.INSTANCE.getKeyRange(toBytes("B"), false, toBytes("Z"), false)
+                        PChar.INSTANCE.getKeyRange(toBytes("A"), true, toBytes("B"), false, SortOrder.ASC),
+                        PChar.INSTANCE.getKeyRange(toBytes("B"), false, toBytes("Z"), false, SortOrder.ASC)
                 ),
                 input(
-                        PChar.INSTANCE.getKeyRange(toBytes("A"), true, toBytes("B"), false),
-                        PChar.INSTANCE.getKeyRange(toBytes("B"), false, toBytes("Z"), false)
+                        PChar.INSTANCE.getKeyRange(toBytes("A"), true, toBytes("B"), false, SortOrder.ASC),
+                        PChar.INSTANCE.getKeyRange(toBytes("B"), false, toBytes("Z"), false, SortOrder.ASC)
                 )},
                 {expect(
-                        PChar.INSTANCE.getKeyRange(toBytes("A"), true, toBytes("Z"), false)
+                        PChar.INSTANCE.getKeyRange(toBytes("A"), true, toBytes("Z"), false, SortOrder.ASC)
                 ),
                 input(
-                        PChar.INSTANCE.getKeyRange(toBytes("A"), true, toBytes("B"), false),
-                        PChar.INSTANCE.getKeyRange(toBytes("B"), true, toBytes("Z"), false)
+                        PChar.INSTANCE.getKeyRange(toBytes("A"), true, toBytes("B"), false, SortOrder.ASC),
+                        PChar.INSTANCE.getKeyRange(toBytes("B"), true, toBytes("Z"), false, SortOrder.ASC)
                 )},
                 {expect(
                     EVERYTHING_RANGE
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/query/KeyRangeIntersectTest.java b/phoenix-core/src/test/java/org/apache/phoenix/query/KeyRangeIntersectTest.java
index 8327415903..d8ae8a6e95 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/query/KeyRangeIntersectTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/query/KeyRangeIntersectTest.java
@@ -27,6 +27,7 @@ import java.util.Collection;
 
 import junit.framework.TestCase;
 
+import org.apache.phoenix.schema.SortOrder;
 import org.apache.phoenix.schema.types.PChar;
 import org.apache.phoenix.schema.types.PDate;
 import org.apache.phoenix.util.DateUtil;
@@ -49,29 +50,29 @@ public class KeyRangeIntersectTest extends TestCase {
     public static synchronized Collection<?> data() {
         return Arrays.asList(new Object[][] {
                 {
-                    PChar.INSTANCE.getKeyRange(toBytes("C"), true, toBytes("E"), true),
-                    PChar.INSTANCE.getKeyRange(toBytes("D"), true, toBytes("F"), true),
-                    PChar.INSTANCE.getKeyRange(toBytes("D"), true, toBytes("E"), true)
+                    PChar.INSTANCE.getKeyRange(toBytes("C"), true, toBytes("E"), true, SortOrder.ASC),
+                    PChar.INSTANCE.getKeyRange(toBytes("D"), true, toBytes("F"), true, SortOrder.ASC),
+                    PChar.INSTANCE.getKeyRange(toBytes("D"), true, toBytes("E"), true, SortOrder.ASC)
                 },
                 {
-                    PChar.INSTANCE.getKeyRange(toBytes("C"), true, toBytes("E"), true),
-                    PChar.INSTANCE.getKeyRange(toBytes("D"), false, toBytes("F"), true),
-                    PChar.INSTANCE.getKeyRange(toBytes("D"), false, toBytes("E"), true)
+                    PChar.INSTANCE.getKeyRange(toBytes("C"), true, toBytes("E"), true, SortOrder.ASC),
+                    PChar.INSTANCE.getKeyRange(toBytes("D"), false, toBytes("F"), true, SortOrder.ASC),
+                    PChar.INSTANCE.getKeyRange(toBytes("D"), false, toBytes("E"), true, SortOrder.ASC)
                 },
                 {
-                    PChar.INSTANCE.getKeyRange(toBytes("C"), true, toBytes("E"), false),
-                    PChar.INSTANCE.getKeyRange(toBytes("D"), false, toBytes("F"), true),
-                    PChar.INSTANCE.getKeyRange(toBytes("D"), false, toBytes("E"), false)
+                    PChar.INSTANCE.getKeyRange(toBytes("C"), true, toBytes("E"), false, SortOrder.ASC),
+                    PChar.INSTANCE.getKeyRange(toBytes("D"), false, toBytes("F"), true, SortOrder.ASC),
+                    PChar.INSTANCE.getKeyRange(toBytes("D"), false, toBytes("E"), false, SortOrder.ASC)
                 },
                 {
-                    PChar.INSTANCE.getKeyRange(toBytes("C"), true, toBytes("E"), false),
-                    PChar.INSTANCE.getKeyRange(toBytes("E"), false, toBytes("F"), true),
+                    PChar.INSTANCE.getKeyRange(toBytes("C"), true, toBytes("E"), false, SortOrder.ASC),
+                    PChar.INSTANCE.getKeyRange(toBytes("E"), false, toBytes("F"), true, SortOrder.ASC),
                     EMPTY_RANGE
                 },
                 {
                     EVERYTHING_RANGE,
-                    PChar.INSTANCE.getKeyRange(toBytes("E"), false, toBytes("F"), true),
-                    PChar.INSTANCE.getKeyRange(toBytes("E"), false, toBytes("F"), true),
+                    PChar.INSTANCE.getKeyRange(toBytes("E"), false, toBytes("F"), true, SortOrder.ASC),
+                    PChar.INSTANCE.getKeyRange(toBytes("E"), false, toBytes("F"), true, SortOrder.ASC),
                 },
                 {
                     EVERYTHING_RANGE,
@@ -85,13 +86,13 @@ public class KeyRangeIntersectTest extends TestCase {
                 },
                 {
                     EMPTY_RANGE,
-                    PChar.INSTANCE.getKeyRange(toBytes("E"), false, toBytes("F"), true),
+                    PChar.INSTANCE.getKeyRange(toBytes("E"), false, toBytes("F"), true, SortOrder.ASC),
                     EMPTY_RANGE
                 },
                 {
-                    PDate.INSTANCE.getKeyRange(PDate.INSTANCE.toBytes(DateUtil.parseDate("2011-01-01")), true, PDate.INSTANCE.toBytes(DateUtil.parseDate("2016-01-01")), true),
-                    PDate.INSTANCE.getKeyRange(PDate.INSTANCE.toBytes(DateUtil.parseDate("2012-10-21")), false, PDate.INSTANCE.toBytes(DateUtil.parseDate("2016-10-31")), false),
-                    PDate.INSTANCE.getKeyRange(PDate.INSTANCE.toBytes(DateUtil.parseDate("2012-10-21")), false, PDate.INSTANCE.toBytes(DateUtil.parseDate("2016-01-01")), true)
+                    PDate.INSTANCE.getKeyRange(PDate.INSTANCE.toBytes(DateUtil.parseDate("2011-01-01")), true, PDate.INSTANCE.toBytes(DateUtil.parseDate("2016-01-01")), true, SortOrder.ASC),
+                    PDate.INSTANCE.getKeyRange(PDate.INSTANCE.toBytes(DateUtil.parseDate("2012-10-21")), false, PDate.INSTANCE.toBytes(DateUtil.parseDate("2016-10-31")), false, SortOrder.ASC),
+                    PDate.INSTANCE.getKeyRange(PDate.INSTANCE.toBytes(DateUtil.parseDate("2012-10-21")), false, PDate.INSTANCE.toBytes(DateUtil.parseDate("2016-01-01")), true, SortOrder.ASC)
                 }
         });
     }
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/query/KeyRangeMoreTest.java b/phoenix-core/src/test/java/org/apache/phoenix/query/KeyRangeMoreTest.java
index b213c0782a..6ff133542a 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/query/KeyRangeMoreTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/query/KeyRangeMoreTest.java
@@ -25,6 +25,7 @@ import java.util.List;
 
 import org.apache.phoenix.thirdparty.com.google.common.collect.Lists;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.phoenix.schema.SortOrder;
 import org.apache.phoenix.schema.types.PInteger;
 import org.junit.Test;
 
@@ -54,7 +55,7 @@ public class KeyRangeMoreTest extends TestCase {
         List<KeyRange> rowKeyRanges2=new ArrayList<KeyRange>();
         for(int i=start1;i<=end1;i++) {
             rowKeyRanges1.add(
-                    PInteger.INSTANCE.getKeyRange(PInteger.INSTANCE.toBytes(i), true, PInteger.INSTANCE.toBytes(i+step1), true));
+                    PInteger.INSTANCE.getKeyRange(PInteger.INSTANCE.toBytes(i), true, PInteger.INSTANCE.toBytes(i+step1), true, SortOrder.ASC));
 
         }
         if(addEmptyRange) {
@@ -62,7 +63,7 @@ public class KeyRangeMoreTest extends TestCase {
         }
         for(int i=start2;i<=end2;i++) {
             rowKeyRanges2.add(
-                    PInteger.INSTANCE.getKeyRange(PInteger.INSTANCE.toBytes(i), true, PInteger.INSTANCE.toBytes(i+step2), true));
+                    PInteger.INSTANCE.getKeyRange(PInteger.INSTANCE.toBytes(i), true, PInteger.INSTANCE.toBytes(i+step2), true, SortOrder.ASC));
         }
         if(addEmptyRange) {
             rowKeyRanges2.add(KeyRange.EMPTY_RANGE);
@@ -100,7 +101,7 @@ public class KeyRangeMoreTest extends TestCase {
                             PInteger.INSTANCE.toBytes(start1+(i-1)*(step1+1)),
                             true,
                             PInteger.INSTANCE.toBytes(start1+i*(step1+1)-1),
-                            true));
+                            true, SortOrder.ASC));
 
         }
         if(addEmptyRange) {
@@ -112,7 +113,7 @@ public class KeyRangeMoreTest extends TestCase {
                             PInteger.INSTANCE.toBytes(start2+(i-1)*(step2+1)),
                             true,
                             PInteger.INSTANCE.toBytes(start2+i*(step2+1)-1),
-                            true));
+                            true, SortOrder.ASC));
         }
         if(addEmptyRange) {
             rowKeyRanges2.add(KeyRange.EMPTY_RANGE);
@@ -180,7 +181,7 @@ public class KeyRangeMoreTest extends TestCase {
                             PInteger.INSTANCE.toBytes(i),
                             true,
                             PInteger.INSTANCE.toBytes(i+2),
-                            true));
+                            true, SortOrder.ASC));
             i+=4;
         }
         List<KeyRange> expected=new ArrayList<KeyRange>(rowKeyRanges2);
@@ -258,7 +259,7 @@ public class KeyRangeMoreTest extends TestCase {
         for (int i = 0; i < size; i++) {
             byte[] startKey = keys.get(2*i).equals(Integer.MIN_VALUE) ? KeyRange.UNBOUND : PInteger.INSTANCE.toBytes(keys.get(2*i));
             byte[] endKey = keys.get(2*i + 1).equals(Integer.MAX_VALUE) ? KeyRange.UNBOUND : PInteger.INSTANCE.toBytes(keys.get(2*i + 1));
-            keyRangeList.add(PInteger.INSTANCE.getKeyRange(startKey, boundaryConditions.get(2*i), endKey, boundaryConditions.get(2*i+1)));
+            keyRangeList.add(PInteger.INSTANCE.getKeyRange(startKey, boundaryConditions.get(2*i), endKey, boundaryConditions.get(2*i+1), SortOrder.ASC));
         }
 
         return keyRangeList;
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/query/KeyRangeUnionTest.java b/phoenix-core/src/test/java/org/apache/phoenix/query/KeyRangeUnionTest.java
index 90b89bb303..174843fed0 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/query/KeyRangeUnionTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/query/KeyRangeUnionTest.java
@@ -27,6 +27,7 @@ import java.util.Collection;
 
 import junit.framework.TestCase;
 
+import org.apache.phoenix.schema.SortOrder;
 import org.apache.phoenix.schema.types.PChar;
 import org.junit.Test;
 import org.junit.runner.RunWith;
@@ -47,33 +48,33 @@ public class KeyRangeUnionTest extends TestCase {
     public static synchronized Collection<?> data() {
         return Arrays.asList(new Object[][] {
                 {
-                    PChar.INSTANCE.getKeyRange(toBytes("C"), true, toBytes("E"), true),
-                    PChar.INSTANCE.getKeyRange(toBytes("D"), true, toBytes("F"), true),
-                    PChar.INSTANCE.getKeyRange(toBytes("C"), true, toBytes("F"), true)
+                    PChar.INSTANCE.getKeyRange(toBytes("C"), true, toBytes("E"), true, SortOrder.ASC),
+                    PChar.INSTANCE.getKeyRange(toBytes("D"), true, toBytes("F"), true, SortOrder.ASC),
+                    PChar.INSTANCE.getKeyRange(toBytes("C"), true, toBytes("F"), true, SortOrder.ASC)
                 },
                 {
-                    PChar.INSTANCE.getKeyRange(toBytes("C"), false, toBytes("E"), false),
-                    PChar.INSTANCE.getKeyRange(toBytes("D"), true, toBytes("F"), true),
-                    PChar.INSTANCE.getKeyRange(toBytes("C"), false, toBytes("F"), true)
+                    PChar.INSTANCE.getKeyRange(toBytes("C"), false, toBytes("E"), false, SortOrder.ASC),
+                    PChar.INSTANCE.getKeyRange(toBytes("D"), true, toBytes("F"), true, SortOrder.ASC),
+                    PChar.INSTANCE.getKeyRange(toBytes("C"), false, toBytes("F"), true, SortOrder.ASC)
                 },
                 {
-                    PChar.INSTANCE.getKeyRange(toBytes("C"), false, toBytes("E"), false),
-                    PChar.INSTANCE.getKeyRange(toBytes("D"), true, toBytes("E"), true),
-                    PChar.INSTANCE.getKeyRange(toBytes("C"), false, toBytes("E"), true)
+                    PChar.INSTANCE.getKeyRange(toBytes("C"), false, toBytes("E"), false, SortOrder.ASC),
+                    PChar.INSTANCE.getKeyRange(toBytes("D"), true, toBytes("E"), true, SortOrder.ASC),
+                    PChar.INSTANCE.getKeyRange(toBytes("C"), false, toBytes("E"), true, SortOrder.ASC)
                 },
                 {
-                    PChar.INSTANCE.getKeyRange(toBytes("C"), false, toBytes("E"), false),
-                    PChar.INSTANCE.getKeyRange(toBytes("C"), true, toBytes("E"), true),
-                    PChar.INSTANCE.getKeyRange(toBytes("C"), true, toBytes("E"), true)
+                    PChar.INSTANCE.getKeyRange(toBytes("C"), false, toBytes("E"), false, SortOrder.ASC),
+                    PChar.INSTANCE.getKeyRange(toBytes("C"), true, toBytes("E"), true, SortOrder.ASC),
+                    PChar.INSTANCE.getKeyRange(toBytes("C"), true, toBytes("E"), true, SortOrder.ASC)
                 },
                 {
-                    PChar.INSTANCE.getKeyRange(toBytes("C"), true, toBytes("E"), false),
+                    PChar.INSTANCE.getKeyRange(toBytes("C"), true, toBytes("E"), false, SortOrder.ASC),
                     EMPTY_RANGE,
-                    PChar.INSTANCE.getKeyRange(toBytes("C"), true, toBytes("E"), false),
+                    PChar.INSTANCE.getKeyRange(toBytes("C"), true, toBytes("E"), false, SortOrder.ASC)
                 },
                 {
                     EVERYTHING_RANGE,
-                    PChar.INSTANCE.getKeyRange(toBytes("E"), false, toBytes("F"), true),
+                    PChar.INSTANCE.getKeyRange(toBytes("E"), false, toBytes("F"), true, SortOrder.ASC),
                     EVERYTHING_RANGE,
                 },
                 {
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/query/ParallelIteratorsSplitTest.java b/phoenix-core/src/test/java/org/apache/phoenix/query/ParallelIteratorsSplitTest.java
index 9fd9d8588a..4454ea5761 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/query/ParallelIteratorsSplitTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/query/ParallelIteratorsSplitTest.java
@@ -121,19 +121,19 @@ public class ParallelIteratorsSplitTest extends BaseConnectionlessQueryTest {
     }
 
     private static KeyRange getKeyRange(byte[] lowerRange, boolean lowerInclusive, byte[] upperRange, boolean upperInclusive) {
-        return PChar.INSTANCE.getKeyRange(lowerRange, lowerInclusive, upperRange, upperInclusive);
+        return PChar.INSTANCE.getKeyRange(lowerRange, lowerInclusive, upperRange, upperInclusive, SortOrder.ASC);
     }
 
     private static KeyRange getKeyRange(String lowerRange, boolean lowerInclusive, String upperRange, boolean upperInclusive) {
-        return PChar.INSTANCE.getKeyRange(Bytes.toBytes(lowerRange), lowerInclusive, Bytes.toBytes(upperRange), upperInclusive);
+        return PChar.INSTANCE.getKeyRange(Bytes.toBytes(lowerRange), lowerInclusive, Bytes.toBytes(upperRange), upperInclusive, SortOrder.ASC);
     }
     
     private static KeyRange getKeyRange(String lowerRange, boolean lowerInclusive, byte[] upperRange, boolean upperInclusive) {
-        return PChar.INSTANCE.getKeyRange(Bytes.toBytes(lowerRange), lowerInclusive, upperRange, upperInclusive);
+        return PChar.INSTANCE.getKeyRange(Bytes.toBytes(lowerRange), lowerInclusive, upperRange, upperInclusive, SortOrder.ASC);
     }
     
     private static KeyRange getKeyRange(byte[] lowerRange, boolean lowerInclusive, String upperRange, boolean upperInclusive) {
-        return PChar.INSTANCE.getKeyRange(lowerRange, lowerInclusive, Bytes.toBytes(upperRange), upperInclusive);
+        return PChar.INSTANCE.getKeyRange(lowerRange, lowerInclusive, Bytes.toBytes(upperRange), upperInclusive, SortOrder.ASC);
     }
     
     private static String nextKey(String s) {
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/schema/types/PDataTypeTest.java b/phoenix-core/src/test/java/org/apache/phoenix/schema/types/PDataTypeTest.java
index bb1dce7a34..02fd0c787f 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/schema/types/PDataTypeTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/schema/types/PDataTypeTest.java
@@ -1895,7 +1895,7 @@ public class PDataTypeTest {
         final byte[] lowerRange = PTimestamp.INSTANCE.toBytes(ts1);
         Timestamp ts2 = new Timestamp(now + MILLIS_IN_DAY);
         final byte[] upperRange = PTimestamp.INSTANCE.toBytes(ts2);
-        KeyRange range = PTimestamp.INSTANCE.getKeyRange(lowerRange, false, upperRange, false);
+        KeyRange range = PTimestamp.INSTANCE.getKeyRange(lowerRange, false, upperRange, false, SortOrder.ASC);
         Timestamp ts3 = new Timestamp(now + 1);
         // Rolled up to next millis
         final byte[] expectedLowerRange = PTimestamp.INSTANCE.toBytes(ts3);
@@ -1911,7 +1911,7 @@ public class PDataTypeTest {
         final byte[] lowerRange = PTimestamp.INSTANCE.toBytes(ts1, SortOrder.DESC);
         Timestamp ts2 = new Timestamp(now);
         final byte[] upperRange = PTimestamp.INSTANCE.toBytes(ts2, SortOrder.DESC);
-        KeyRange range = PTimestamp.INSTANCE.getKeyRange(lowerRange, false, upperRange, false);
+        KeyRange range = PTimestamp.INSTANCE.getKeyRange(lowerRange, false, upperRange, false, SortOrder.DESC);
         Timestamp ts3 = DateUtil.getTimestamp(now + MILLIS_IN_DAY - 1,  999999);
         // Rolled up to next millis
         final byte[] expectedLowerRange = PTimestamp.INSTANCE.toBytes(ts3, SortOrder.DESC);
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 e891ebce5b..06fa4eef93 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
@@ -149,127 +149,127 @@ public class ScanUtilTest {
             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, SortOrder.ASC), },
+                            { PChar.INSTANCE.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("1"), true, SortOrder.ASC), },
                             { PChar.INSTANCE.getKeyRange(Bytes.toBytes("A"), true, Bytes.toBytes("A"),
-                                    true), } }, new int[] { 1, 1, 1 }, PChar.INSTANCE.toBytes("a1A"),
+                                    true, SortOrder.ASC), } }, 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, SortOrder.ASC), },
+                            { PChar.INSTANCE.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("2"), true, SortOrder.ASC), },
                             { PChar.INSTANCE.getKeyRange(Bytes.toBytes("A"), true, Bytes.toBytes("B"),
-                                    true), } }, new int[] { 1, 1, 1 }, PChar.INSTANCE.toBytes("a1A"),
+                                    true, SortOrder.ASC), } }, 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, SortOrder.ASC), },
+                            { PChar.INSTANCE.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("2"), true, SortOrder.ASC), },
                             { PChar.INSTANCE.getKeyRange(Bytes.toBytes("A"), true, Bytes.toBytes("A"),
-                                    true), } }, new int[] { 1, 1, 1 }, PChar.INSTANCE.toBytes("a1A"),
+                                    true, SortOrder.ASC), } }, 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, SortOrder.ASC), },
+                            { PChar.INSTANCE.getKeyRange(Bytes.toBytes("1"), false, Bytes.toBytes("2"), true, SortOrder.ASC), },
                             { PChar.INSTANCE.getKeyRange(Bytes.toBytes("A"), false, Bytes.toBytes("B"),
-                                    true), } }, new int[] { 1, 1, 1 }, PChar.INSTANCE.toBytes("b2B"),
+                                    true, SortOrder.ASC), } }, 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, SortOrder.ASC), },
+                            { PChar.INSTANCE.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("2"), true, SortOrder.ASC), },
                             { PChar.INSTANCE.getKeyRange(Bytes.toBytes("A"), false, Bytes.toBytes("B"),
-                                    true), } }, new int[] { 1, 1, 1 }, PChar.INSTANCE.toBytes("b1B"),
+                                    true, SortOrder.ASC), } }, 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"), true, Bytes.toBytes("a"), true, SortOrder.ASC), },
+                            { PChar.INSTANCE.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("2"), true, SortOrder.ASC), },
                             { PChar.INSTANCE.getKeyRange(Bytes.toBytes("A"), false, Bytes.toBytes("B"),
-                                    true), } }, new int[] { 1, 1, 1 }, PChar.INSTANCE.toBytes("a1B"),
+                                    true, SortOrder.ASC), } }, 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), },
+                            { PChar.INSTANCE.getKeyRange(Bytes.toBytes("a"), true, Bytes.toBytes("a"), true, SortOrder.ASC), },
                             { KeyRange.EVERYTHING_RANGE, },
                             { PChar.INSTANCE.getKeyRange(Bytes.toBytes("A"), false, Bytes.toBytes("B"),
-                                    true), } }, new int[] { 1, 1, 1 }, PChar.INSTANCE.toBytes("a"),
+                                    true, SortOrder.ASC), } }, 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), },
+                            { PChar.INSTANCE.getKeyRange(Bytes.toBytes("a"), true, Bytes.toBytes("a"), true, SortOrder.ASC), },
                             { 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), },
+                            { PChar.INSTANCE.getKeyRange(Bytes.toBytes("a"), true, Bytes.toBytes("a"), true, SortOrder.ASC), },
                             { KeyRange.EVERYTHING_RANGE, },
                             { PChar.INSTANCE.getKeyRange(Bytes.toBytes("A"), true, Bytes.toBytes("B"),
-                                    true), } }, new int[] { 1, 1, 1 }, PChar.INSTANCE.toBytes("a"),
+                                    true, SortOrder.ASC), } }, 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, SortOrder.ASC), },
+                            { PChar.INSTANCE.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("1"), true, SortOrder.ASC), },
                             { PChar.INSTANCE.getKeyRange(Bytes.toBytes("A"), true, Bytes.toBytes("A"),
-                                    true), } }, new int[] { 1, 1, 1 }, PChar.INSTANCE.toBytes("a1B"),
+                                    true, SortOrder.ASC), } }, 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, SortOrder.ASC), },
+                            { PChar.INSTANCE.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("2"), true, SortOrder.ASC), },
                             { PChar.INSTANCE.getKeyRange(Bytes.toBytes("A"), true, Bytes.toBytes("B"),
-                                    true), } }, new int[] { 1, 1, 1 }, PChar.INSTANCE.toBytes("b2C"),
+                                    true, SortOrder.ASC), } }, 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, SortOrder.ASC), },
+                            { PChar.INSTANCE.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("2"), false, SortOrder.ASC), },
                             { PChar.INSTANCE.getKeyRange(Bytes.toBytes("A"), true, Bytes.toBytes("B"),
-                                    false), } }, new int[] { 1, 1, 1 }, PChar.INSTANCE.toBytes("b"),
+                                    false, SortOrder.ASC), } }, 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("a"), true, Bytes.toBytes("a"), true, SortOrder.ASC), },
                     { PChar.INSTANCE.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("2"),
-                            true), } }, new int[] { 1, 1 }, PChar.INSTANCE.toBytes("a3"), Bound.UPPER));
+                            true, SortOrder.ASC), } }, 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("a"), true, Bytes.toBytes("b"), false, SortOrder.ASC), },
                     { PChar.INSTANCE.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("1"),
-                            true), } }, new int[] { 1, 1 }, PChar.INSTANCE.toBytes("b"), Bound.UPPER));
+                            true, SortOrder.ASC), } }, 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("a"), true, Bytes.toBytes("b"), true, SortOrder.ASC), },
                     { PChar.INSTANCE.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("1"),
-                            true), } }, new int[] { 1, 1 }, PChar.INSTANCE.toBytes("b2"), Bound.UPPER));
+                            true, SortOrder.ASC), } }, 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("a"), true, Bytes.toBytes("a"), true, SortOrder.ASC), },
                             { PChar.INSTANCE.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("2"),
-                                    false), } }, new int[] { 1, 1 }, PChar.INSTANCE.toBytes("a2"),
+                                    false, SortOrder.ASC), } }, 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), },
+                            { PChar.INSTANCE.getKeyRange(Bytes.toBytes("a"), true, Bytes.toBytes("a"), true, SortOrder.ASC), },
                             { 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), },
+                            { PChar.INSTANCE.getKeyRange(Bytes.toBytes("a"), true, Bytes.toBytes("a"), true, SortOrder.ASC), },
                             { 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) } },
+                            { PChar.INSTANCE.getKeyRange(new byte[] { -1 }, true, new byte[] { -1 }, true, SortOrder.ASC) },
+                            { PChar.INSTANCE.getKeyRange(new byte[] { -1 }, true, new byte[] { -1 }, true, SortOrder.ASC) } },
                     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), },
+                            { PChar.INSTANCE.getKeyRange(Bytes.toBytes("a"), true, Bytes.toBytes("a"), true, SortOrder.ASC), },
                             { PVarchar.INSTANCE.getKeyRange(Bytes.toBytes("A"), true, Bytes.toBytes("B"),
-                                    true), } }, new int[] { 1, 0 },
+                                    true, SortOrder.ASC), } }, new int[] { 1, 0 },
                     ByteUtil.nextKey(ByteUtil.concat(PVarchar.INSTANCE.toBytes("aB"), QueryConstants.SEPARATOR_BYTE_ARRAY)),
                     Bound.UPPER));
             return testCases;