You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@phoenix.apache.org by ja...@apache.org on 2015/07/14 19:56:03 UTC

[7/7] phoenix git commit: PHOENIX-2067 Sort order incorrect for variable length DESC columns

PHOENIX-2067 Sort order incorrect for variable length DESC columns


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

Branch: refs/heads/master
Commit: 2620a80c1e35c0d214f06a1b16e99da5415a1a2c
Parents: 01b4f60
Author: James Taylor <jt...@salesforce.com>
Authored: Mon Jul 13 11:17:37 2015 -0700
Committer: James Taylor <jt...@salesforce.com>
Committed: Tue Jul 14 10:55:27 2015 -0700

----------------------------------------------------------------------
 dev/eclipse_prefs_phoenix.epf                   |    2 +-
 .../org/apache/phoenix/end2end/ArrayIT.java     |   59 +
 .../org/apache/phoenix/end2end/IsNullIT.java    |   52 +-
 .../apache/phoenix/end2end/LpadFunctionIT.java  |   24 +
 .../apache/phoenix/end2end/ReverseScanIT.java   |   30 +
 .../phoenix/end2end/RowValueConstructorIT.java  |    7 +-
 .../apache/phoenix/end2end/SortOrderFIT.java    |  563 -----
 .../org/apache/phoenix/end2end/SortOrderIT.java |  572 +++++
 .../apache/phoenix/compile/FromCompiler.java    |    3 +-
 .../apache/phoenix/compile/JoinCompiler.java    |    8 +-
 .../apache/phoenix/compile/OrderByCompiler.java |    4 +-
 .../phoenix/compile/OrderPreservingTracker.java |    7 +-
 .../org/apache/phoenix/compile/ScanRanges.java  |    5 +-
 .../compile/TupleProjectionCompiler.java        |    4 +-
 .../apache/phoenix/compile/UnionCompiler.java   |    5 +-
 .../apache/phoenix/compile/UpsertCompiler.java  |   16 +-
 .../apache/phoenix/compile/WhereOptimizer.java  |   53 +-
 .../coprocessor/BaseScannerRegionObserver.java  |    2 +
 .../coprocessor/MetaDataEndpointImpl.java       |   73 +-
 .../UngroupedAggregateRegionObserver.java       |  108 +-
 .../coprocessor/generated/PTableProtos.java     |  105 +-
 .../phoenix/exception/SQLExceptionCode.java     |    1 +
 .../apache/phoenix/execute/BaseQueryPlan.java   |   14 +-
 .../DescVarLengthFastByteComparisons.java       |  219 ++
 .../expression/ArrayConstructorExpression.java  |    2 +-
 .../phoenix/expression/OrderByExpression.java   |   13 +-
 .../RowValueConstructorExpression.java          |    8 +-
 .../function/ArrayConcatFunction.java           |   11 +-
 .../function/ArrayModifierFunction.java         |    3 +-
 .../expression/function/LpadFunction.java       |    8 +-
 .../expression/util/regex/JONIPattern.java      |    5 +-
 .../apache/phoenix/filter/SkipScanFilter.java   |    3 +-
 .../apache/phoenix/index/IndexMaintainer.java   |  127 +-
 .../phoenix/iterate/BaseResultIterators.java    |  109 +-
 .../phoenix/iterate/OrderedResultIterator.java  |   52 +-
 .../apache/phoenix/jdbc/PhoenixConnection.java  |   29 +-
 .../query/ConnectionQueryServicesImpl.java      |   17 +-
 .../java/org/apache/phoenix/query/KeyRange.java |   14 -
 .../apache/phoenix/query/QueryConstants.java    |    3 +
 .../apache/phoenix/schema/DelegateTable.java    |    5 +
 .../apache/phoenix/schema/MetaDataClient.java   |   31 +-
 .../java/org/apache/phoenix/schema/PTable.java  |    9 +
 .../org/apache/phoenix/schema/PTableImpl.java   |   78 +-
 .../org/apache/phoenix/schema/RowKeySchema.java |   44 +-
 .../phoenix/schema/RowKeyValueAccessor.java     |   12 +-
 .../org/apache/phoenix/schema/ValueSchema.java  |   30 +-
 .../phoenix/schema/stats/StatisticsUtil.java    |    4 +-
 .../phoenix/schema/types/PArrayDataType.java    |  682 +++---
 .../phoenix/schema/types/PBinaryArray.java      |  122 +-
 .../phoenix/schema/types/PBooleanArray.java     |  112 +-
 .../apache/phoenix/schema/types/PCharArray.java |  128 +-
 .../apache/phoenix/schema/types/PDataType.java  | 2037 +++++++++---------
 .../apache/phoenix/schema/types/PDateArray.java |  131 +-
 .../phoenix/schema/types/PDecimalArray.java     |  126 +-
 .../phoenix/schema/types/PDoubleArray.java      |  128 +-
 .../phoenix/schema/types/PFloatArray.java       |  130 +-
 .../phoenix/schema/types/PIntegerArray.java     |  130 +-
 .../apache/phoenix/schema/types/PLongArray.java |  130 +-
 .../phoenix/schema/types/PSmallintArray.java    |  130 +-
 .../apache/phoenix/schema/types/PTimeArray.java |  133 +-
 .../phoenix/schema/types/PTimestampArray.java   |  132 +-
 .../phoenix/schema/types/PTinyintArray.java     |  130 +-
 .../schema/types/PUnsignedDateArray.java        |  128 +-
 .../schema/types/PUnsignedDoubleArray.java      |  136 +-
 .../schema/types/PUnsignedFloatArray.java       |  130 +-
 .../phoenix/schema/types/PUnsignedIntArray.java |  130 +-
 .../schema/types/PUnsignedLongArray.java        |  130 +-
 .../schema/types/PUnsignedSmallintArray.java    |  132 +-
 .../schema/types/PUnsignedTimeArray.java        |  132 +-
 .../schema/types/PUnsignedTimestampArray.java   |  134 +-
 .../schema/types/PUnsignedTinyintArray.java     |  132 +-
 .../phoenix/schema/types/PVarbinaryArray.java   |  130 +-
 .../phoenix/schema/types/PVarcharArray.java     |  130 +-
 .../java/org/apache/phoenix/util/ByteUtil.java  |   12 +-
 .../org/apache/phoenix/util/MetaDataUtil.java   |    2 +-
 .../org/apache/phoenix/util/PhoenixRuntime.java |  154 +-
 .../java/org/apache/phoenix/util/ScanUtil.java  |   15 +-
 .../org/apache/phoenix/util/SchemaUtil.java     |   23 +
 .../java/org/apache/phoenix/util/TupleUtil.java |    7 +-
 .../org/apache/phoenix/util/UpgradeUtil.java    |  301 +++
 .../phoenix/compile/QueryCompilerTest.java      |   35 +-
 .../org/apache/phoenix/query/OrderByTest.java   |   84 +
 .../phoenix/schema/types/PDataTypeTest.java     |   12 +
 phoenix-protocol/src/main/PTable.proto          |    1 +
 84 files changed, 4997 insertions(+), 4027 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/2620a80c/dev/eclipse_prefs_phoenix.epf
----------------------------------------------------------------------
diff --git a/dev/eclipse_prefs_phoenix.epf b/dev/eclipse_prefs_phoenix.epf
index a218c97..65881ef 100644
--- a/dev/eclipse_prefs_phoenix.epf
+++ b/dev/eclipse_prefs_phoenix.epf
@@ -512,7 +512,7 @@ file_export_version=3.0
 /instance/org.eclipse.jdt.core/org.eclipse.jdt.core.compiler.problem.unusedParameter=ignore
 /instance/org.eclipse.jdt.core/org.eclipse.jdt.core.compiler.problem.unusedParameterWhenImplementingAbstract=disabled
 /instance/org.eclipse.jdt.core/org.eclipse.jdt.core.compiler.problem.unusedParameterWhenOverridingConcrete=disabled
-/instance/org.eclipse.jdt.core/org.eclipse.jdt.core.compiler.problem.unusedPrivateMember=error
+/instance/org.eclipse.jdt.core/org.eclipse.jdt.core.compiler.problem.unusedPrivateMember=warning
 /instance/org.eclipse.jdt.core/org.eclipse.jdt.core.compiler.problem.varargsArgumentNeedCast=error
 /instance/org.eclipse.jdt.core/org.eclipse.jdt.core.formatter.alignment_for_arguments_in_allocation_expression=16
 /instance/org.eclipse.jdt.core/org.eclipse.jdt.core.formatter.alignment_for_arguments_in_enum_constant=16

http://git-wip-us.apache.org/repos/asf/phoenix/blob/2620a80c/phoenix-core/src/it/java/org/apache/phoenix/end2end/ArrayIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ArrayIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ArrayIT.java
index 27887e4..89997f4 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ArrayIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ArrayIT.java
@@ -1980,4 +1980,63 @@ public class ArrayIT extends BaseClientManagedTimeIT {
         assertEquals(arr, rs.getArray(1));
         rs.next();
     }
+    
+    @Test
+    public void testPKWithDescArray() throws Exception {
+        Connection conn;
+        PreparedStatement stmt;
+        ResultSet rs;
+        long ts = nextTimestamp();
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 10));
+        conn = DriverManager.getConnection(getUrl(), props);
+        conn.createStatement()
+                .execute(
+                        "CREATE TABLE t ( a VARCHAR ARRAY PRIMARY KEY DESC)\n");
+        conn.close();
+        
+        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 30));
+        conn = DriverManager.getConnection(getUrl(), props);
+        stmt = conn.prepareStatement("UPSERT INTO t VALUES(?)");
+        Array a1 = conn.createArrayOf("VARCHAR", new String[] { "a", "ba" });
+        stmt.setArray(1, a1);
+        stmt.execute();
+        Array a2 = conn.createArrayOf("VARCHAR", new String[] { "a", "c" });
+        stmt.setArray(1, a2);
+        stmt.execute();
+        conn.commit();
+        conn.close();
+
+        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 40));
+        conn = DriverManager.getConnection(getUrl(), props);
+        rs = conn.createStatement().executeQuery("SELECT a FROM t ORDER BY a DESC");
+        assertTrue(rs.next());
+        assertEquals(a2, rs.getArray(1));
+        assertTrue(rs.next());
+        assertEquals(a1, rs.getArray(1));
+        assertFalse(rs.next());
+        conn.close();
+        
+        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 50));
+        conn = DriverManager.getConnection(getUrl(), props);
+        stmt = conn.prepareStatement("UPSERT INTO t VALUES(?)");
+        Array a3 = conn.createArrayOf("VARCHAR", new String[] { "a", "b" });
+        stmt.setArray(1, a3);
+        stmt.execute();
+        conn.commit();
+        conn.close();
+
+        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 60));
+        conn = DriverManager.getConnection(getUrl(), props);
+        rs = conn.createStatement().executeQuery("SELECT a FROM t ORDER BY a DESC");
+        assertTrue(rs.next());
+        assertEquals(a2, rs.getArray(1));
+        assertTrue(rs.next());
+        assertEquals(a1, rs.getArray(1));
+        assertTrue(rs.next());
+        assertEquals(a3, rs.getArray(1));
+        assertFalse(rs.next());
+        conn.close();
+    }
+    
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/2620a80c/phoenix-core/src/it/java/org/apache/phoenix/end2end/IsNullIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/IsNullIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/IsNullIT.java
index 9c60914..fb49378 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/IsNullIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/IsNullIT.java
@@ -19,26 +19,22 @@ package org.apache.phoenix.end2end;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
 
 import java.sql.Connection;
 import java.sql.DriverManager;
 import java.sql.PreparedStatement;
 import java.sql.ResultSet;
-import java.util.Properties;
 
-import org.apache.phoenix.util.PhoenixRuntime;
 import org.junit.Test;
 
 
-public class IsNullIT extends BaseClientManagedTimeIT {
+public class IsNullIT extends BaseHBaseManagedTimeIT {
     @Test
     public void testIsNullInPk() throws Exception {
-        long ts = nextTimestamp();
-        ensureTableCreated(getUrl(),"IntIntKeyTest",null, ts-2);
-        Properties props = new Properties();
-        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 1)); // Execute at timestamp 1
-        Connection conn = DriverManager.getConnection(getUrl(), props);
+        ensureTableCreated(getUrl(),"IntIntKeyTest");
+        Connection conn = DriverManager.getConnection(getUrl());
         String upsert = "UPSERT INTO IntIntKeyTest VALUES(4,2)";
         PreparedStatement upsertStmt = conn.prepareStatement(upsert);
         int rowsInserted = upsertStmt.executeUpdate();
@@ -48,10 +44,7 @@ public class IsNullIT extends BaseClientManagedTimeIT {
         rowsInserted = upsertStmt.executeUpdate();
         assertEquals(1, rowsInserted);
         conn.commit();
-        conn.close();
         
-        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 1
-        conn = DriverManager.getConnection(getUrl(), props);
         String select = "SELECT i/j FROM IntIntKeyTest WHERE j IS NULL";
         ResultSet rs;
         rs = conn.createStatement().executeQuery(select);
@@ -67,23 +60,38 @@ public class IsNullIT extends BaseClientManagedTimeIT {
     }
     
     @Test
+    public void testIsNullWithLastPKColDesc() throws Exception {
+        Connection conn = DriverManager.getConnection(getUrl());
+        conn.createStatement().execute("CREATE TABLE T(k1 VARCHAR NOT NULL, k2 VARCHAR, k3 VARCHAR, CONSTRAINT pk PRIMARY KEY (k1, k2, k3 DESC))");
+        conn.createStatement().execute("UPSERT INTO T VALUES ('a')");
+        conn.createStatement().execute("UPSERT INTO T VALUES ('b')");
+        conn.createStatement().execute("UPSERT INTO T VALUES ('b',null,'c')");
+        conn.createStatement().execute("UPSERT INTO T VALUES ('ba', null, 'd')");
+        conn.commit();
+        
+        ResultSet rs = conn.createStatement().executeQuery("SELECT k1,k2,k3 FROM T WHERE k1='b' AND k2 IS NULL");
+        assertTrue(rs.next());
+        assertEquals("b",rs.getString(1));
+        assertNull(rs.getString(2));
+        assertNull(rs.getString(3));
+
+        assertTrue(rs.next());
+        assertEquals("b",rs.getString(1));
+        assertNull(rs.getString(2));
+        assertEquals("c",rs.getString(3));
+        
+        assertFalse(rs.next());
+        conn.close();
+    }
+    
+    @Test
     public void testIsNullInCompositeKey() throws Exception {
-        long ts = nextTimestamp();
-        Properties props = new Properties();
-        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 10));
-        Connection conn = DriverManager.getConnection(getUrl(), props);
+        Connection conn = DriverManager.getConnection(getUrl());
         conn.createStatement().execute("CREATE TABLE T(k1 VARCHAR, k2 VARCHAR, CONSTRAINT pk PRIMARY KEY (k1,k2))");
-        conn.close();
-        
-        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 20));
-        conn = DriverManager.getConnection(getUrl(), props);
         conn.createStatement().execute("UPSERT INTO T VALUES (null,'a')");
         conn.createStatement().execute("UPSERT INTO T VALUES ('a','a')");
         conn.commit();
-        conn.close();
         
-        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 30));
-        conn = DriverManager.getConnection(getUrl(), props);
         ResultSet rs = conn.createStatement().executeQuery("SELECT count(*) FROM T");
         assertTrue(rs.next());
         assertEquals(2,rs.getInt(1));

http://git-wip-us.apache.org/repos/asf/phoenix/blob/2620a80c/phoenix-core/src/it/java/org/apache/phoenix/end2end/LpadFunctionIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/LpadFunctionIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/LpadFunctionIT.java
index 0a9bedf..4aa66c2 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/LpadFunctionIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/LpadFunctionIT.java
@@ -28,6 +28,7 @@ import java.util.ArrayList;
 import java.util.List;
 
 import org.apache.phoenix.util.TestUtil;
+import org.junit.Ignore;
 import org.junit.Test;
 
 import com.google.common.collect.Lists;
@@ -106,6 +107,29 @@ public class LpadFunctionIT extends BaseHBaseManagedTimeIT {
         testLpad(conn, inputList, length, fillStringList, "pk", expectedOutputList);
     }
 
+    @Ignore
+    @Test
+    public void testCharPadding() throws Exception {
+        ResultSet rs;
+        Connection conn = DriverManager.getConnection(getUrl());
+
+        conn.createStatement().execute("CREATE TABLE tdesc (k CHAR(3) PRIMARY KEY DESC)");
+        conn.createStatement().execute("UPSERT INTO tdesc VALUES('a')");
+        conn.commit();
+        rs = conn.createStatement().executeQuery("SELECT * FROM tdesc");
+        assertTrue(rs.next());
+        assertEquals("a", rs.getString(1));
+        assertFalse(rs.next());
+        
+        conn.createStatement().execute("CREATE TABLE t (k CHAR(3) PRIMARY KEY)");
+        conn.createStatement().execute("UPSERT INTO t VALUES('a')");
+        conn.commit();
+        rs = conn.createStatement().executeQuery("SELECT * FROM t");
+        assertTrue(rs.next());
+        assertEquals("a", rs.getString(1));
+        assertFalse(rs.next());
+    }
+
     @Test
     public void testNullInputStringSB() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());

http://git-wip-us.apache.org/repos/asf/phoenix/blob/2620a80c/phoenix-core/src/it/java/org/apache/phoenix/end2end/ReverseScanIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ReverseScanIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ReverseScanIT.java
index eca183b..35a8025 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ReverseScanIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ReverseScanIT.java
@@ -53,6 +53,8 @@ import com.google.common.collect.Maps;
 
 
 public class ReverseScanIT extends BaseHBaseManagedTimeIT {
+    protected static final String ATABLE_INDEX_NAME = "ATABLE_IDX";
+
     @BeforeClass
     @Shadower(classBeingShadowed = BaseHBaseManagedTimeIT.class)
     public static void doSetup() throws Exception {
@@ -165,4 +167,32 @@ public class ReverseScanIT extends BaseHBaseManagedTimeIT {
         }
     }
 
+    @Test
+    public void testReverseScanIndex() throws Exception {
+        String tenantId = getOrganizationId();
+        initATableValues(tenantId, getSplitsAtRowKeys(tenantId), getUrl());
+        
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        Connection conn = DriverManager.getConnection(getUrl(), props);
+        String ddl = "CREATE INDEX " + ATABLE_INDEX_NAME + " ON aTable (a_integer DESC) INCLUDE ("
+        + "    A_STRING, " + "    B_STRING, " + "    A_DATE)";
+        conn.createStatement().execute(ddl);
+        
+        String query = 
+                "SELECT a_integer FROM atable where a_integer is not null order by a_integer nulls last limit 1";
+
+        PreparedStatement statement = conn.prepareStatement(query);
+        ResultSet rs=statement.executeQuery();
+        assertTrue(rs.next());
+        assertEquals(1,rs.getInt(1));
+        assertFalse(rs.next());
+        
+        rs = conn.createStatement().executeQuery("EXPLAIN " + query);
+        assertEquals(
+                "CLIENT SERIAL 1-WAY REVERSE RANGE SCAN OVER ATABLE_IDX [not null]\n" + 
+                "    SERVER FILTER BY FIRST KEY ONLY\n" + 
+                "    SERVER 1 ROW LIMIT\n" + 
+                "CLIENT 1 ROW LIMIT",QueryUtil.getExplainPlan(rs));
+    }
+    
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/phoenix/blob/2620a80c/phoenix-core/src/it/java/org/apache/phoenix/end2end/RowValueConstructorIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/RowValueConstructorIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/RowValueConstructorIT.java
index b9e50a4..e5cfeb0 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/RowValueConstructorIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/RowValueConstructorIT.java
@@ -1232,8 +1232,7 @@ public class RowValueConstructorIT extends BaseClientManagedTimeIT {
     @Test
     public void testForceSkipScan() throws Exception {
         String tempTableWithCompositePK = "TEMP_TABLE_COMPOSITE_PK";
-        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
-        Connection conn = DriverManager.getConnection(getUrl(), props);
+        Connection conn = nextConnection(getUrl());
         try {
             conn.createStatement().execute("CREATE TABLE " + tempTableWithCompositePK
                     + "   (col0 INTEGER NOT NULL, "
@@ -1242,7 +1241,9 @@ public class RowValueConstructorIT extends BaseClientManagedTimeIT {
                     + "    col3 INTEGER "
                     + "   CONSTRAINT pk PRIMARY KEY (col0, col1, col2)) "
                     + "   SALT_BUCKETS=4");
+            conn.close();
 
+            conn = nextConnection(getUrl());
             PreparedStatement upsertStmt = conn.prepareStatement(
                     "upsert into " + tempTableWithCompositePK + "(col0, col1, col2, col3) " + "values (?, ?, ?, ?)");
             for (int i = 0; i < 3; i++) {
@@ -1253,7 +1254,9 @@ public class RowValueConstructorIT extends BaseClientManagedTimeIT {
                 upsertStmt.execute();
             }
             conn.commit();
+            conn.close();
 
+            conn = nextConnection(getUrl());
             String query = "SELECT * FROM " + tempTableWithCompositePK + " WHERE (col0, col1) in ((2, 3), (3, 4), (4, 5))";
             PreparedStatement statement = conn.prepareStatement(query);
             ResultSet rs = statement.executeQuery();

http://git-wip-us.apache.org/repos/asf/phoenix/blob/2620a80c/phoenix-core/src/it/java/org/apache/phoenix/end2end/SortOrderFIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SortOrderFIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SortOrderFIT.java
deleted file mode 100644
index 9fb7ea3..0000000
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SortOrderFIT.java
+++ /dev/null
@@ -1,563 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.phoenix.end2end;
-
-import static org.apache.phoenix.util.TestUtil.TEST_PROPERTIES;
-
-import java.math.BigDecimal;
-import java.math.BigInteger;
-import java.sql.Connection;
-import java.sql.Date;
-import java.sql.DriverManager;
-import java.sql.PreparedStatement;
-import java.sql.ResultSet;
-import java.util.Calendar;
-import java.util.GregorianCalendar;
-import java.util.List;
-import java.util.Properties;
-
-import org.apache.commons.lang.ArrayUtils;
-import org.apache.phoenix.util.PropertiesUtil;
-import org.junit.Assert;
-import org.junit.Test;
-
-import com.google.common.collect.Lists;
-
-/**
- * @since 1.2
- */
-
-
-public class SortOrderFIT extends BaseHBaseManagedTimeIT {
-    
-    private static final String TABLE = "DescColumnSortOrderTest";
-
-    @Test
-    public void noOrder() throws Exception {
-        String ddl = "CREATE TABLE " + TABLE + " (pk VARCHAR NOT NULL PRIMARY KEY)";
-        runQueryTest(ddl, "pk", new Object[][]{{"a"}, {"b"}, {"c"}}, new Object[][]{{"a"}, {"b"}, {"c"}});
-    }                                                           
-
-    @Test
-    public void noOrderCompositePK() throws Exception {
-        String ddl = "CREATE TABLE " + TABLE + " (oid CHAR(2) NOT NULL, code INTEGER NOT NULL constraint pk primary key (oid, code))";
-        Object[][] rows = new Object[][]{{"o1", 1}, {"o2", 2}, {"o3", 3}};
-        runQueryTest(ddl, upsert("oid", "code"), rows, rows);
-    }
-    
-    @Test
-    public void ascOrderInlinePK() throws Exception {
-        String ddl = "CREATE TABLE " + TABLE + " (pk VARCHAR NOT NULL PRIMARY KEY ASC)";
-        runQueryTest(ddl, "pk", new Object[][]{{"a"}, {"b"}, {"c"}}, new Object[][]{{"a"}, {"b"}, {"c"}});
-    }
-    
-    @Test
-    public void ascOrderCompositePK() throws Exception {
-        String ddl = "CREATE TABLE " + TABLE + " (oid CHAR(2) NOT NULL, code INTEGER NOT NULL constraint pk primary key (oid ASC, code DESC))";
-        Object[][] insertedRows = new Object[][]{{"o1", 1}, {"o1", 2}, {"o1", 3}};
-        Object[][] expectedRows = new Object[][]{{"o1", 3}, {"o1", 2}, {"o1", 1}};
-        runQueryTest(ddl, upsert("oid", "code"), insertedRows, expectedRows);        
-    }
-
-    @Test
-    public void descOrderInlinePK() throws Exception {
-        for (String type : new String[]{"CHAR(2)", "VARCHAR"}) {
-            String ddl = "CREATE TABLE " + TABLE + " (pk ${type} NOT NULL PRIMARY KEY DESC)".replace("${type}", type);
-            runQueryTest(ddl, "pk", new Object[][]{{"aa"}, {"bb"}, {"cc"}}, new Object[][]{{"cc"}, {"bb"}, {"aa"}});
-        }
-    }
-    
-    @Test
-    public void descOrderCompositePK1() throws Exception {
-        String ddl = "CREATE TABLE " + TABLE + " (oid CHAR(2) NOT NULL, code INTEGER NOT NULL constraint pk primary key (oid DESC, code))";
-        Object[][] insertedRows = new Object[][]{{"o1", 1}, {"o2", 2}, {"o3", 3}};
-        Object[][] expectedRows = new Object[][]{{"o3", 3}, {"o2", 2}, {"o1", 1}};
-        runQueryTest(ddl, upsert("oid", "code"), insertedRows, expectedRows);        
-    }
-    
-    @Test
-    public void descOrderCompositePK2() throws Exception {
-        String ddl = "CREATE TABLE " + TABLE + " (oid CHAR(2) NOT NULL, code INTEGER NOT NULL constraint pk primary key (oid DESC, code DESC))";
-        Object[][] insertedRows = new Object[][]{{"o1", 1}, {"o1", 2}, {"o1", 3}};
-        Object[][] expectedRows = new Object[][]{{"o1", 3}, {"o1", 2}, {"o1", 1}};
-        runQueryTest(ddl, upsert("oid", "code"), insertedRows, expectedRows);        
-    }    
-
-    @Test
-    public void equalityDescInlinePK() throws Exception {
-        String ddl = "CREATE TABLE " + TABLE + " (pk VARCHAR NOT NULL PRIMARY KEY DESC)";
-        runQueryTest(ddl, upsert("pk"), new Object[][]{{"a"}, {"b"}, {"c"}}, new Object[][]{{"b"}}, new WhereCondition("pk", "=", "'b'"));
-    }
-    
-    @Test
-    public void equalityDescCompositePK1() throws Exception {
-        String ddl = "CREATE TABLE " + TABLE + " (oid CHAR(2) NOT NULL, code INTEGER NOT NULL constraint pk primary key (oid DESC, code DESC))";
-        Object[][] insertedRows = new Object[][]{{"o1", 1}, {"o2", 2}, {"o3", 3}};
-        runQueryTest(ddl, upsert("oid", "code"), insertedRows, new Object[][]{{"o2", 2}}, new WhereCondition("oid", "=", "'o2'"));        
-    }
-    
-    @Test
-    public void equalityDescCompositePK2() throws Exception {
-        String ddl = "CREATE TABLE " + TABLE + " (oid CHAR(2) NOT NULL, code INTEGER NOT NULL constraint pk primary key (oid DESC, code DESC))";
-        Object[][] insertedRows = new Object[][]{{"o1", 1}, {"o1", 2}, {"o1", 3}};
-        runQueryTest(ddl, upsert("oid", "code"), insertedRows, new Object[][]{{"o1", 2}}, new WhereCondition("code", "=", "2"));        
-    }
-    
-    @Test
-    public void inDescCompositePK1() throws Exception {
-        String ddl = "CREATE TABLE " + TABLE + " (oid CHAR(2) NOT NULL, code INTEGER NOT NULL constraint pk primary key (oid DESC, code DESC))";
-        Object[][] insertedRows = new Object[][]{{"o1", 1}, {"o1", 2}, {"o1", 3}};
-        runQueryTest(ddl, upsert("oid", "code"), insertedRows, new Object[][]{{"o1", 2}}, new WhereCondition("code", "IN", "(2)"));        
-    }
-    
-    @Test
-    public void inDescCompositePK2() throws Exception {
-        String ddl = "CREATE TABLE " + TABLE + " (oid CHAR(2) NOT NULL, code INTEGER NOT NULL constraint pk primary key (oid DESC, code DESC))";
-        Object[][] insertedRows = new Object[][]{{"o1", 1}, {"o2", 2}, {"o3", 3}};
-        runQueryTest(ddl, upsert("oid", "code"), insertedRows, new Object[][]{{"o2", 2}}, new WhereCondition("oid", "IN", "('o2')"));        
-    }
-    
-    @Test
-    public void likeDescCompositePK1() throws Exception {
-        String ddl = "CREATE TABLE " + TABLE + " (oid CHAR(2) NOT NULL, code INTEGER NOT NULL constraint pk primary key (oid DESC, code DESC))";
-        Object[][] insertedRows = new Object[][]{{"a1", 1}, {"b2", 2}, {"c3", 3}};
-        runQueryTest(ddl, upsert("oid", "code"), insertedRows, new Object[][]{{"b2", 2}}, new WhereCondition("oid", "LIKE", "('b%')"));        
-    }
-    
-    @Test
-    public void likeDescCompositePK2() throws Exception {
-        String ddl = "CREATE TABLE " + TABLE + " (oid CHAR(2) NOT NULL, code CHAR(2) NOT NULL constraint pk primary key (oid DESC, code DESC))";
-        Object[][] insertedRows = new Object[][]{{"a1", "11"}, {"b2", "22"}, {"c3", "33"}};
-        runQueryTest(ddl, upsert("oid", "code"), insertedRows, new Object[][]{{"b2", "22"}}, new WhereCondition("code", "LIKE", "('2%')"));        
-    }
-    
-    @Test
-    public void greaterThanDescCompositePK3() throws Exception {
-        String ddl = "CREATE TABLE " + TABLE + " (oid CHAR(2) NOT NULL, code INTEGER NOT NULL constraint pk primary key (oid DESC, code DESC))";
-        Object[][] insertedRows = new Object[][]{{"o1", 1}, {"o1", 2}, {"o1", 3}};
-        Object[][] expectedRows = new Object[][]{{"o1", 2}, {"o1", 1}};
-        runQueryTest(ddl, upsert("oid", "code"), insertedRows, expectedRows, new WhereCondition("code", "<", "3"));        
-    }
-    
-    @Test
-    public void substrDescCompositePK1() throws Exception {
-        String ddl = "CREATE TABLE " + TABLE + " (oid CHAR(3) NOT NULL, code INTEGER NOT NULL constraint pk primary key (oid DESC, code ASC))";
-        Object[][] insertedRows = new Object[][]{{"ao1", 1}, {"bo2", 2}, {"co3", 3}};
-        Object[][] expectedRows = new Object[][]{{"co3", 3}, {"bo2", 2}};
-        runQueryTest(ddl, upsert("oid", "code"), insertedRows, expectedRows, new WhereCondition("SUBSTR(oid, 3, 1)", ">", "'1'"));
-    }
-        
-    @Test
-    public void substrDescCompositePK2() throws Exception {
-        String ddl = "CREATE TABLE " + TABLE + " (oid CHAR(4) NOT NULL, code INTEGER NOT NULL constraint pk primary key (oid DESC, code ASC))";
-        Object[][] insertedRows = new Object[][]{{"aaaa", 1}, {"bbbb", 2}, {"cccd", 3}};
-        Object[][] expectedRows = new Object[][]{{"cccd", 3}};
-        runQueryTest(ddl, upsert("oid", "code"), insertedRows, expectedRows, new WhereCondition("SUBSTR(oid, 4, 1)", "=", "'d'"));
-    }    
-    
-    @Test
-    public void lTrimDescCompositePK() throws Exception {
-        String ddl = "CREATE TABLE " + TABLE + " (oid VARCHAR NOT NULL, code INTEGER NOT NULL constraint pk primary key (oid DESC, code DESC))";
-        Object[][] insertedRows = new Object[][]{{" o1 ", 1}, {"  o2", 2}, {"  o3", 3}};
-        Object[][] expectedRows = new Object[][]{{"  o2", 2}};
-        runQueryTest(ddl, upsert("oid", "code"), insertedRows, expectedRows, new WhereCondition("LTRIM(oid)", "=", "'o2'"));
-    }
-    
-    @Test
-    public void lPadDescCompositePK() throws Exception {
-        String ddl = "CREATE TABLE " + TABLE + " (oid VARCHAR NOT NULL, code INTEGER NOT NULL constraint pk primary key (oid DESC, code DESC))";
-        Object[][] insertedRows = new Object[][]{{"aaaa", 1}, {"bbbb", 2}, {"cccc", 3}};
-        Object[][] expectedRows = new Object[][]{{"bbbb", 2}};
-        runQueryTest(ddl, upsert("oid", "code"), insertedRows, expectedRows, new WhereCondition("LPAD(oid, 8, '123')", "=", "'1231bbbb'"));
-    }
-
-    @Test
-    public void countDescCompositePK() throws Exception {
-        String ddl = "CREATE TABLE " + TABLE + " (oid CHAR(2) NOT NULL, code INTEGER NOT NULL constraint pk primary key (oid DESC, code ASC))";
-        Object[][] insertedRows = new Object[][]{{"o1", 1}, {"o2", 2}, {"o3", 3}};
-        Object[][] expectedRows = new Object[][]{{3l}};
-        runQueryTest(ddl, upsert("oid", "code"), select("COUNT(oid)"), insertedRows, expectedRows);
-    }
-    
-    @Test
-    public void sumDescCompositePK() throws Exception {
-        String ddl = "CREATE TABLE " + TABLE + " (n1 INTEGER NOT NULL, n2 DECIMAL(10, 2) NOT NULL, n3 BIGINT NOT NULL " + 
-            "constraint pk primary key (n1 DESC, n2 DESC, n3 DESC))";
-        Object[][] insertedRows = new Object[][]{{10, bdec(10.2), 21l}, {20, bdec(20.2), 32l}, {30, bdec(30.2), 43l}};
-        Object[][] expectedRows = new Object[][]{{60l, bdec(60.6), 96l}};
-        runQueryTest(ddl, upsert("n1", "n2", "n3"), select("SUM(n1), SUM(n2), SUM(n3)"), insertedRows, expectedRows);
-    }    
-    
-    @Test
-    public void avgDescCompositePK() throws Exception {
-        String ddl = "CREATE TABLE " + TABLE + " (n1 INTEGER NOT NULL, n2 DECIMAL(10, 2) NOT NULL, n3 BIGINT NOT NULL " + 
-            "constraint pk primary key (n1 DESC, n2 DESC, n3 DESC))";
-        Object[][] insertedRows = new Object[][]{{10, bdec(10.2), 21l}, {20, bdec(20.2), 32l}, {30, bdec(30.2), 43l}};
-        Object[][] expectedRows = new Object[][]{{new BigDecimal(bint(2), -1), bdec(20.2), BigDecimal.valueOf(32)}};
-        runQueryTest(ddl, upsert("n1", "n2", "n3"), select("AVG(n1), AVG(n2), AVG(n3)"), insertedRows, expectedRows);
-    }
-    
-    @Test
-    public void minDescCompositePK() throws Exception {
-        String ddl = "CREATE TABLE " + TABLE + " (n1 INTEGER NOT NULL, n2 DECIMAL(10, 2) NOT NULL, n3 BIGINT NOT NULL " + 
-            "constraint pk primary key (n1 DESC, n2 DESC, n3 DESC))";
-        Object[][] insertedRows = new Object[][]{{10, bdec(10.2), 21l}, {20, bdec(20.2), 32l}, {30, bdec(30.2), 43l}};
-        Object[][] expectedRows = new Object[][]{{10, bdec(10.2), 21l}};
-        runQueryTest(ddl, upsert("n1", "n2", "n3"), select("MIN(n1), MIN(n2), MIN(n3)"), insertedRows, expectedRows);
-    }
-    
-    @Test
-    public void maxDescCompositePK() throws Exception {
-        String ddl = "CREATE TABLE " + TABLE + " (n1 INTEGER NOT NULL, n2 DECIMAL(10, 2) NOT NULL, n3 BIGINT NOT NULL " + 
-            "constraint pk primary key (n1 DESC, n2 DESC, n3 DESC))";
-        Object[][] insertedRows = new Object[][]{{10, bdec(10.2), 21l}, {20, bdec(20.2), 32l}, {30, bdec(30.2), 43l}};
-        Object[][] expectedRows = new Object[][]{{30, bdec(30.2), 43l}};
-        runQueryTest(ddl, upsert("n1", "n2", "n3"), select("MAX(n1), MAX(n2), MAX(n3)"), insertedRows, expectedRows);
-    }
-    
-    @Test
-    public void havingSumDescCompositePK() throws Exception {
-        String ddl = "CREATE TABLE " + TABLE + " (name CHAR(1) NOT NULL, code INTEGER NOT NULL " + 
-            "constraint pk primary key (name DESC, code DESC))";
-        Object[][] insertedRows = new Object[][]{{"a", 10}, {"a", 20}, {"b", 100}}; 
-        Object[][] expectedRows = new Object[][]{{"a", 30l}};
-        runQueryTest(ddl, upsert("name", "code"), select("name", "SUM(code)"), insertedRows, expectedRows, 
-            new HavingCondition("name", "SUM(code) = 30"));
-    }
-    
-    @Test
-    public void queryDescDateWithExplicitOrderBy() throws Exception {
-        String ddl = "CREATE TABLE " + TABLE + " (c1 CHAR(1) NOT NULL, c2 CHAR(1) NOT NULL, d1 DATE NOT NULL, c3 CHAR(1) NOT NULL " + 
-            "constraint pk primary key (c1, c2, d1 DESC, c3))";
-        Object[] row1 = {"1", "2", date(10, 11, 2001), "3"};
-        Object[] row2 = {"1", "2", date(10, 11, 2003), "3"};
-        Object[][] insertedRows = new Object[][]{row1, row2};
-        runQueryTest(ddl, upsert("c1", "c2", "d1", "c3"), select("c1, c2, d1", "c3"), insertedRows, new Object[][]{row2, row1},
-            null, null, new OrderBy("d1", OrderBy.Direction.DESC));
-    }    
-    
-    @Test
-    public void additionOnDescCompositePK() throws Exception {
-        String ddl = "CREATE TABLE " + TABLE + " (n1 INTEGER NOT NULL, n2 DECIMAL(10, 2) NOT NULL, n3 BIGINT NOT NULL, d1 DATE NOT NULL " + 
-            "constraint pk primary key (n1 DESC, n2 DESC, n3 DESC, d1 DESC))";
-        Object[][] insertedRows = new Object[][]{
-            {10, bdec(10.2), 21l, date(1, 10, 2001)}, {20, bdec(20.2), 32l, date(2, 6, 2001)}, {30, bdec(30.2), 43l, date(3, 1, 2001)}};
-        Object[][] expectedRows = new Object[][]{
-            {31l, bdec(32.2), 46l, date(3, 5, 2001)}, {21l, bdec(22.2), 35l, date(2, 10, 2001)}, {11l, bdec(12.2), 24l, date(1, 14, 2001)}};
-        runQueryTest(ddl, upsert("n1", "n2", "n3", "d1"), select("n1+1, n2+2, n3+3", "d1+4"), insertedRows, expectedRows);
-    }
-    
-    @Test
-    public void subtractionOnDescCompositePK() throws Exception {
-        String ddl = "CREATE TABLE " + TABLE + " (n1 INTEGER NOT NULL, n2 DECIMAL(10, 2) NOT NULL, n3 BIGINT NOT NULL, d1 DATE NOT NULL " + 
-            "constraint pk primary key (n1 DESC, n2 DESC, n3 DESC, d1 DESC))";
-        Object[][] insertedRows = new Object[][]{
-            {10, bdec(10.2), 21l, date(1, 10, 2001)}, {20, bdec(20.2), 32l, date(2, 6, 2001)}, {30, bdec(30.2), 43l, date(3, 10, 2001)}};
-        Object[][] expectedRows = new Object[][]{
-            {29l, bdec(28.2), 40l, date(3, 6, 2001)}, {19l, bdec(18.2), 29l, date(2, 2, 2001)}, {9l, bdec(8.2), 18l, date(1, 6, 2001)}};
-        runQueryTest(ddl, upsert("n1", "n2", "n3", "d1"), select("n1-1, n2-2, n3-3", "d1-4"), insertedRows, expectedRows);
-    }
-    
-    @Test
-    public void lessThanLeadingDescCompositePK() throws Exception {
-        String ddl = "CREATE TABLE " + TABLE + " (id INTEGER NOT NULL, date DATE NOT NULL constraint pk primary key (id DESC, date))";
-        Object[][] insertedRows = new Object[][]{{1, date(1, 1, 2012)}, {3, date(1, 1, 2013)}, {2, date(1, 1, 2011)}};
-        Object[][] expectedRows = new Object[][]{{1, date(1, 1, 2012)}};
-        runQueryTest(ddl, upsert("id", "date"), insertedRows, expectedRows, new WhereCondition("id", "<", "2"));
-    }
-    
-    @Test
-    public void lessThanTrailingDescCompositePK() throws Exception {
-        String ddl = "CREATE TABLE " + TABLE + " (id INTEGER NOT NULL, date DATE NOT NULL constraint pk primary key (id DESC, date))";
-        Object[][] insertedRows = new Object[][]{{1, date(1, 1, 2002)}, {3, date(1, 1, 2003)}, {2, date(1, 1, 2001)}};
-        Object[][] expectedRows = new Object[][]{{2, date(1, 1, 2001)}};
-        runQueryTest(ddl, upsert("id", "date"), insertedRows, expectedRows, new WhereCondition("date", "<", "TO_DATE('02-02-2001','mm-dd-yyyy')"));
-    }
-    
-    private void runQueryTest(String ddl, String columnName, Object[][] rows, Object[][] expectedRows) throws Exception {
-        runQueryTest(ddl, new String[]{columnName}, rows, expectedRows, null);
-    }
-    
-    private void runQueryTest(String ddl, String[] columnNames, Object[][] rows, Object[][] expectedRows) throws Exception {
-        runQueryTest(ddl, columnNames, rows, expectedRows, null);
-    }
-    
-    private void runQueryTest(String ddl, String[] columnNames, Object[][] rows, Object[][] expectedRows, WhereCondition condition) throws Exception {
-        runQueryTest(ddl, columnNames, columnNames, rows, expectedRows, condition, null, null);
-    }
-    
-    private void runQueryTest(String ddl, String[] columnNames, String[] projections, Object[][] rows, Object[][] expectedRows) throws Exception {
-        runQueryTest(ddl, columnNames, projections, rows, expectedRows, null, null, null);
-    }
-    
-    private void runQueryTest(String ddl, String[] columnNames, String[] projections, Object[][] rows, Object[][] expectedRows, HavingCondition havingCondition) throws Exception {
-        runQueryTest(ddl, columnNames, projections, rows, expectedRows, null, havingCondition, null);
-    }
-    
-
-    private void runQueryTest(
-        String ddl, 
-        String[] columnNames, 
-        String[] projections, 
-        Object[][] rows, Object[][] expectedRows, 
-        WhereCondition whereCondition, 
-        HavingCondition havingCondition,
-        OrderBy orderBy) 
-        throws Exception 
-    {
-        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
-        Connection conn = DriverManager.getConnection(getUrl(), props);
-
-        try {
-
-            conn.setAutoCommit(false);
-
-            createTestTable(getUrl(), ddl);
-
-            String columns = appendColumns(columnNames);
-            String placeholders = appendPlaceholders(columnNames);
-            String dml = "UPSERT INTO " + TABLE + " (" + columns + ") VALUES(" + placeholders +")";
-            PreparedStatement stmt = conn.prepareStatement(dml);
-
-            for (int row = 0; row < rows.length; row++) {
-                for (int col = 0; col < rows[row].length; col++) {
-                    Object value = rows[row][col];
-                    stmt.setObject(col + 1, value);
-                }
-                stmt.execute();
-            }
-            conn.commit();
-            
-            String selectClause = "SELECT " + appendColumns(projections) + " FROM " + TABLE;
-            
-            for (WhereCondition whereConditionClause : new WhereCondition[]{whereCondition, WhereCondition.reverse(whereCondition)}) {
-                String query = WhereCondition.appendWhere(whereConditionClause, selectClause);
-                query = HavingCondition.appendHaving(havingCondition, query);
-                query = OrderBy.appendOrderBy(orderBy, query);
-                runQuery(conn, query, expectedRows);
-            }
-            
-            if (orderBy != null) {
-                orderBy = OrderBy.reverse(orderBy);
-                String query = WhereCondition.appendWhere(whereCondition, selectClause);
-                query = HavingCondition.appendHaving(havingCondition, query);
-                query = OrderBy.appendOrderBy(orderBy, query);
-                runQuery(conn, query, reverse(expectedRows));
-            }
-            
-        } finally {
-            conn.close();
-        }
-    }
-    
-    private String appendColumns(String[] columnNames) {
-        String appendedColumns = "";
-        for (int i = 0; i < columnNames.length; i++) {                
-            appendedColumns += columnNames[i];
-            if (i < columnNames.length - 1) {
-                appendedColumns += ",";
-            }
-        }
-        return appendedColumns;
-    }
-    
-    private String appendPlaceholders(String[] columnNames) {
-        String placeholderList = "";
-        for (int i = 0; i < columnNames.length; i++) {                
-            placeholderList += "?";
-            if (i < columnNames.length - 1) {
-                placeholderList += ",";
-            }
-        }
-        return placeholderList;        
-    }
-    
-    private static void runQuery(Connection connection, String query, Object[][] expectedValues) throws Exception {
-        PreparedStatement stmt = connection.prepareStatement(query);
-
-        ResultSet rs = stmt.executeQuery();
-        int rowCounter = 0;
-        while (rs.next()) {
-            if (rowCounter == expectedValues.length) {
-                Assert.assertEquals("Exceeded number of expected rows for query" + query, expectedValues.length, rowCounter+1);
-            }
-            Object[] cols = new Object[expectedValues[rowCounter].length];
-            for (int colCounter = 0; colCounter < expectedValues[rowCounter].length; colCounter++) {
-                cols[colCounter] = rs.getObject(colCounter+1);
-            }
-            Assert.assertArrayEquals("Unexpected result for query " + query, expectedValues[rowCounter], cols);
-            rowCounter++;
-        }
-        Assert.assertEquals("Unexpected number of rows for query " + query, expectedValues.length, rowCounter);
-    }
-    
-    private static Object[][] reverse(Object[][] rows) {
-        Object[][] reversedArray = new Object[rows.length][];
-        System.arraycopy(rows, 0, reversedArray, 0, rows.length);
-        ArrayUtils.reverse(reversedArray);
-        return reversedArray;
-    }
-    
-    private static Date date(int month, int day, int year) {
-        Calendar cal = new GregorianCalendar();
-        cal.set(Calendar.MONTH, month-1);
-        cal.set(Calendar.DAY_OF_MONTH, day);
-        cal.set(Calendar.YEAR, year);
-        cal.set(Calendar.HOUR_OF_DAY, 10);
-        cal.set(Calendar.MINUTE, 2);
-        cal.set(Calendar.SECOND, 5);
-        cal.set(Calendar.MILLISECOND, 101);
-        Date d = new Date(cal.getTimeInMillis()); 
-        return d;
-    }
-        
-    private static String[] upsert(String...args) {
-        return args;
-    }
-    
-    private static String[] select(String...args) {
-        return args;
-    }
-    
-    private static BigDecimal bdec(double d) {
-        return BigDecimal.valueOf(d);
-    }
-    
-    private static BigInteger bint(long l) {
-        return BigInteger.valueOf(l);
-    }    
-    
-    private static class WhereCondition {
-        final String lhs;
-        final String operator;
-        final String rhs;
-    
-        WhereCondition(String lhs, String operator, String rhs) {
-            this.lhs = lhs;
-            this.operator = operator;
-            this.rhs = rhs;
-        }
-        
-        static WhereCondition reverse(WhereCondition whereCondition) {
-            
-            if (whereCondition == null) {
-                return null; 
-            }
-            
-            if (whereCondition.operator.equalsIgnoreCase("IN") || whereCondition.operator.equalsIgnoreCase("LIKE")) {
-                return whereCondition;
-            } else {
-                return new WhereCondition(whereCondition.rhs, whereCondition.getReversedOperator(), whereCondition.lhs);
-            }
-        }
-        
-         static String appendWhere(WhereCondition whereCondition, String query) {
-             if (whereCondition == null) {
-                 return query;
-             }
-            return query + " WHERE " + whereCondition.lhs + " " + whereCondition.operator + " " + whereCondition.rhs;
-        }
-        
-        private String getReversedOperator() {
-            if (operator.equals("<")) {
-                return ">";
-            } else if (operator.equals(">")) {
-                return "<";
-            }
-            return operator;
-        }
-    }
-    
-    private static class HavingCondition {
-        
-        private String groupby;
-        private String having;
-        
-        HavingCondition(String groupby, String having) {
-            this.groupby = groupby;
-            this.having = having;
-        }
-        
-        static String appendHaving(HavingCondition havingCondition, String query) {
-            if (havingCondition == null) {
-                return query;
-            }
-            return query + " GROUP BY " + havingCondition.groupby + " HAVING " + havingCondition.having + " ";
-        }
-    }
-    
-    private static class OrderBy {
-        
-        enum Direction {
-            
-            ASC, DESC;
-            
-            Direction reverse() {
-                if (this == ASC) {
-                    return DESC;
-                }
-                return ASC;
-            }
-        }
-        
-        private List<String> columnNames = Lists.newArrayList();
-        private List<Direction> directions = Lists.newArrayList();
-        
-        OrderBy() {            
-        }
-        
-        OrderBy(String columnName, Direction orderBy) {
-            add(columnName, orderBy);
-        }
-        
-        void add(String columnName, Direction direction) {
-            columnNames.add(columnName);
-            directions.add(direction);
-        }
-        
-        static OrderBy reverse(OrderBy orderBy) {
-            
-            if (orderBy == null) {
-                return null;
-            }
-            
-            List<Direction> reversedDirections = Lists.newArrayList();
-            for (Direction dir : orderBy.directions) {
-                reversedDirections.add(dir.reverse());
-            }
-            OrderBy reversedOrderBy = new OrderBy();
-            reversedOrderBy.columnNames = orderBy.columnNames;
-            reversedOrderBy.directions = reversedDirections;
-            return reversedOrderBy;
-        }
-        
-        static String appendOrderBy(OrderBy orderBy, String query) {
-            if (orderBy == null || orderBy.columnNames.isEmpty()) {
-                return query;
-            }
-            query += " ORDER BY ";
-            for (int i = 0; i < orderBy.columnNames.size(); i++) {
-                query += orderBy.columnNames.get(i) + " " + orderBy.directions.get(i).toString() + " ";
-            }
-            
-            query += " LIMIT 1000 ";
-            
-            return query;
-        }        
-    }
-}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/2620a80c/phoenix-core/src/it/java/org/apache/phoenix/end2end/SortOrderIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SortOrderIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SortOrderIT.java
new file mode 100644
index 0000000..a03f71e
--- /dev/null
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SortOrderIT.java
@@ -0,0 +1,572 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.phoenix.end2end;
+
+import static org.apache.phoenix.util.TestUtil.TEST_PROPERTIES;
+
+import java.math.BigDecimal;
+import java.math.BigInteger;
+import java.sql.Connection;
+import java.sql.Date;
+import java.sql.DriverManager;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.util.Calendar;
+import java.util.GregorianCalendar;
+import java.util.List;
+import java.util.Properties;
+
+import org.apache.commons.lang.ArrayUtils;
+import org.apache.phoenix.util.PropertiesUtil;
+import org.junit.Assert;
+import org.junit.Test;
+
+import com.google.common.collect.Lists;
+
+/**
+ * @since 1.2
+ */
+
+
+public class SortOrderIT extends BaseHBaseManagedTimeIT {
+    
+    private static final String TABLE = "DescColumnSortOrderTest";
+
+    @Test
+    public void noOrder() throws Exception {
+        String ddl = "CREATE TABLE " + TABLE + " (pk VARCHAR NOT NULL PRIMARY KEY)";
+        runQueryTest(ddl, "pk", new Object[][]{{"a"}, {"b"}, {"c"}}, new Object[][]{{"a"}, {"b"}, {"c"}});
+    }                                                           
+
+    @Test
+    public void noOrderCompositePK() throws Exception {
+        String ddl = "CREATE TABLE " + TABLE + " (oid CHAR(2) NOT NULL, code INTEGER NOT NULL constraint pk primary key (oid, code))";
+        Object[][] rows = new Object[][]{{"o1", 1}, {"o2", 2}, {"o3", 3}};
+        runQueryTest(ddl, upsert("oid", "code"), rows, rows);
+    }
+    
+    @Test
+    public void ascOrderInlinePK() throws Exception {
+        String ddl = "CREATE TABLE " + TABLE + " (pk VARCHAR NOT NULL PRIMARY KEY ASC)";
+        runQueryTest(ddl, "pk", new Object[][]{{"a"}, {"b"}, {"c"}}, new Object[][]{{"a"}, {"b"}, {"c"}});
+    }
+    
+    @Test
+    public void ascOrderCompositePK() throws Exception {
+        String ddl = "CREATE TABLE " + TABLE + " (oid CHAR(2) NOT NULL, code INTEGER NOT NULL constraint pk primary key (oid ASC, code DESC))";
+        Object[][] insertedRows = new Object[][]{{"o1", 1}, {"o1", 2}, {"o1", 3}};
+        Object[][] expectedRows = new Object[][]{{"o1", 3}, {"o1", 2}, {"o1", 1}};
+        runQueryTest(ddl, upsert("oid", "code"), insertedRows, expectedRows);        
+    }
+
+    @Test
+    public void descOrderInlinePK() throws Exception {
+        for (String type : new String[]{"CHAR(2)", "VARCHAR"}) {
+            String ddl = "CREATE TABLE " + TABLE + " (pk ${type} NOT NULL PRIMARY KEY DESC)".replace("${type}", type);
+            runQueryTest(ddl, "pk", new Object[][]{{"aa"}, {"bb"}, {"cc"}}, new Object[][]{{"cc"}, {"bb"}, {"aa"}});
+        }
+    }
+    
+    @Test
+    public void descOrderCompositePK1() throws Exception {
+        String ddl = "CREATE TABLE " + TABLE + " (oid CHAR(2) NOT NULL, code INTEGER NOT NULL constraint pk primary key (oid DESC, code))";
+        Object[][] insertedRows = new Object[][]{{"o1", 1}, {"o2", 2}, {"o3", 3}};
+        Object[][] expectedRows = new Object[][]{{"o3", 3}, {"o2", 2}, {"o1", 1}};
+        runQueryTest(ddl, upsert("oid", "code"), insertedRows, expectedRows);        
+    }
+    
+    @Test
+    public void descOrderCompositePK2() throws Exception {
+        String ddl = "CREATE TABLE " + TABLE + " (oid CHAR(2) NOT NULL, code INTEGER NOT NULL constraint pk primary key (oid DESC, code DESC))";
+        Object[][] insertedRows = new Object[][]{{"o1", 1}, {"o1", 2}, {"o1", 3}};
+        Object[][] expectedRows = new Object[][]{{"o1", 3}, {"o1", 2}, {"o1", 1}};
+        runQueryTest(ddl, upsert("oid", "code"), insertedRows, expectedRows);        
+    }    
+
+    @Test
+    public void equalityDescInlinePK() throws Exception {
+        String ddl = "CREATE TABLE " + TABLE + " (pk VARCHAR NOT NULL PRIMARY KEY DESC)";
+        runQueryTest(ddl, upsert("pk"), new Object[][]{{"a"}, {"b"}, {"c"}}, new Object[][]{{"b"}}, new WhereCondition("pk", "=", "'b'"));
+    }
+    
+    @Test
+    public void equalityDescCompositePK1() throws Exception {
+        String ddl = "CREATE TABLE " + TABLE + " (oid CHAR(2) NOT NULL, code INTEGER NOT NULL constraint pk primary key (oid DESC, code DESC))";
+        Object[][] insertedRows = new Object[][]{{"o1", 1}, {"o2", 2}, {"o3", 3}};
+        runQueryTest(ddl, upsert("oid", "code"), insertedRows, new Object[][]{{"o2", 2}}, new WhereCondition("oid", "=", "'o2'"));        
+    }
+    
+    @Test
+    public void equalityDescCompositePK2() throws Exception {
+        String ddl = "CREATE TABLE " + TABLE + " (oid CHAR(2) NOT NULL, code INTEGER NOT NULL constraint pk primary key (oid DESC, code DESC))";
+        Object[][] insertedRows = new Object[][]{{"o1", 1}, {"o1", 2}, {"o1", 3}};
+        runQueryTest(ddl, upsert("oid", "code"), insertedRows, new Object[][]{{"o1", 2}}, new WhereCondition("code", "=", "2"));        
+    }
+    
+    @Test
+    public void inDescCompositePK1() throws Exception {
+        String ddl = "CREATE TABLE " + TABLE + " (oid CHAR(2) NOT NULL, code INTEGER NOT NULL constraint pk primary key (oid DESC, code DESC))";
+        Object[][] insertedRows = new Object[][]{{"o1", 1}, {"o1", 2}, {"o1", 3}};
+        runQueryTest(ddl, upsert("oid", "code"), insertedRows, new Object[][]{{"o1", 2}}, new WhereCondition("code", "IN", "(2)"));        
+    }
+    
+    @Test
+    public void inDescCompositePK2() throws Exception {
+        String ddl = "CREATE TABLE " + TABLE + " (oid CHAR(2) NOT NULL, code INTEGER NOT NULL constraint pk primary key (oid DESC, code DESC))";
+        Object[][] insertedRows = new Object[][]{{"o1", 1}, {"o2", 2}, {"o3", 3}};
+        runQueryTest(ddl, upsert("oid", "code"), insertedRows, new Object[][]{{"o2", 2}}, new WhereCondition("oid", "IN", "('o2')"));        
+    }
+    
+    @Test
+    public void likeDescCompositePK1() throws Exception {
+        String ddl = "CREATE TABLE " + TABLE + " (oid CHAR(2) NOT NULL, code INTEGER NOT NULL constraint pk primary key (oid DESC, code DESC))";
+        Object[][] insertedRows = new Object[][]{{"a1", 1}, {"b2", 2}, {"c3", 3}};
+        runQueryTest(ddl, upsert("oid", "code"), insertedRows, new Object[][]{{"b2", 2}}, new WhereCondition("oid", "LIKE", "('b%')"));        
+    }
+    
+    @Test
+    public void likeDescCompositePK2() throws Exception {
+        String ddl = "CREATE TABLE " + TABLE + " (oid CHAR(2) NOT NULL, code CHAR(2) NOT NULL constraint pk primary key (oid DESC, code DESC))";
+        Object[][] insertedRows = new Object[][]{{"a1", "11"}, {"b2", "22"}, {"c3", "33"}};
+        runQueryTest(ddl, upsert("oid", "code"), insertedRows, new Object[][]{{"b2", "22"}}, new WhereCondition("code", "LIKE", "('2%')"));        
+    }
+    
+    @Test
+    public void greaterThanDescCompositePK3() throws Exception {
+        String ddl = "CREATE TABLE " + TABLE + " (oid CHAR(2) NOT NULL, code INTEGER NOT NULL constraint pk primary key (oid DESC, code DESC))";
+        Object[][] insertedRows = new Object[][]{{"o1", 1}, {"o1", 2}, {"o1", 3}};
+        Object[][] expectedRows = new Object[][]{{"o1", 2}, {"o1", 1}};
+        runQueryTest(ddl, upsert("oid", "code"), insertedRows, expectedRows, new WhereCondition("code", "<", "3"));        
+    }
+    
+    @Test
+    public void substrDescCompositePK1() throws Exception {
+        String ddl = "CREATE TABLE " + TABLE + " (oid CHAR(3) NOT NULL, code INTEGER NOT NULL constraint pk primary key (oid DESC, code ASC))";
+        Object[][] insertedRows = new Object[][]{{"ao1", 1}, {"bo2", 2}, {"co3", 3}};
+        Object[][] expectedRows = new Object[][]{{"co3", 3}, {"bo2", 2}};
+        runQueryTest(ddl, upsert("oid", "code"), insertedRows, expectedRows, new WhereCondition("SUBSTR(oid, 3, 1)", ">", "'1'"));
+    }
+        
+    @Test
+    public void substrDescCompositePK2() throws Exception {
+        String ddl = "CREATE TABLE " + TABLE + " (oid CHAR(4) NOT NULL, code INTEGER NOT NULL constraint pk primary key (oid DESC, code ASC))";
+        Object[][] insertedRows = new Object[][]{{"aaaa", 1}, {"bbbb", 2}, {"cccd", 3}};
+        Object[][] expectedRows = new Object[][]{{"cccd", 3}};
+        runQueryTest(ddl, upsert("oid", "code"), insertedRows, expectedRows, new WhereCondition("SUBSTR(oid, 4, 1)", "=", "'d'"));
+    }    
+    
+    @Test
+    public void lTrimDescCompositePK() throws Exception {
+        String ddl = "CREATE TABLE " + TABLE + " (oid VARCHAR NOT NULL, code INTEGER NOT NULL constraint pk primary key (oid DESC, code DESC))";
+        Object[][] insertedRows = new Object[][]{{" o1 ", 1}, {"  o2", 2}, {"  o3", 3}};
+        Object[][] expectedRows = new Object[][]{{"  o2", 2}};
+        runQueryTest(ddl, upsert("oid", "code"), insertedRows, expectedRows, new WhereCondition("LTRIM(oid)", "=", "'o2'"));
+    }
+    
+    @Test
+    public void lPadDescCompositePK() throws Exception {
+        String ddl = "CREATE TABLE " + TABLE + " (oid VARCHAR NOT NULL, code INTEGER NOT NULL constraint pk primary key (oid DESC, code DESC))";
+        Object[][] insertedRows = new Object[][]{{"aaaa", 1}, {"bbbb", 2}, {"cccc", 3}};
+        Object[][] expectedRows = new Object[][]{{"bbbb", 2}};
+        runQueryTest(ddl, upsert("oid", "code"), insertedRows, expectedRows, new WhereCondition("LPAD(oid, 8, '123')", "=", "'1231bbbb'"));
+    }
+
+    @Test
+    public void countDescCompositePK() throws Exception {
+        String ddl = "CREATE TABLE " + TABLE + " (oid CHAR(2) NOT NULL, code INTEGER NOT NULL constraint pk primary key (oid DESC, code ASC))";
+        Object[][] insertedRows = new Object[][]{{"o1", 1}, {"o2", 2}, {"o3", 3}};
+        Object[][] expectedRows = new Object[][]{{3l}};
+        runQueryTest(ddl, upsert("oid", "code"), select("COUNT(oid)"), insertedRows, expectedRows);
+    }
+    
+    @Test
+    public void sumDescCompositePK() throws Exception {
+        String ddl = "CREATE TABLE " + TABLE + " (n1 INTEGER NOT NULL, n2 DECIMAL(10, 2) NOT NULL, n3 BIGINT NOT NULL " + 
+            "constraint pk primary key (n1 DESC, n2 DESC, n3 DESC))";
+        Object[][] insertedRows = new Object[][]{{10, bdec(10.2), 21l}, {20, bdec(20.2), 32l}, {30, bdec(30.2), 43l}};
+        Object[][] expectedRows = new Object[][]{{60l, bdec(60.6), 96l}};
+        runQueryTest(ddl, upsert("n1", "n2", "n3"), select("SUM(n1), SUM(n2), SUM(n3)"), insertedRows, expectedRows);
+    }    
+    
+    @Test
+    public void avgDescCompositePK() throws Exception {
+        String ddl = "CREATE TABLE " + TABLE + " (n1 INTEGER NOT NULL, n2 DECIMAL(10, 2) NOT NULL, n3 BIGINT NOT NULL " + 
+            "constraint pk primary key (n1 DESC, n2 DESC, n3 DESC))";
+        Object[][] insertedRows = new Object[][]{{10, bdec(10.2), 21l}, {20, bdec(20.2), 32l}, {30, bdec(30.2), 43l}};
+        Object[][] expectedRows = new Object[][]{{new BigDecimal(bint(2), -1), bdec(20.2), BigDecimal.valueOf(32)}};
+        runQueryTest(ddl, upsert("n1", "n2", "n3"), select("AVG(n1), AVG(n2), AVG(n3)"), insertedRows, expectedRows);
+    }
+    
+    @Test
+    public void minDescCompositePK() throws Exception {
+        String ddl = "CREATE TABLE " + TABLE + " (n1 INTEGER NOT NULL, n2 DECIMAL(10, 2) NOT NULL, n3 BIGINT NOT NULL " + 
+            "constraint pk primary key (n1 DESC, n2 DESC, n3 DESC))";
+        Object[][] insertedRows = new Object[][]{{10, bdec(10.2), 21l}, {20, bdec(20.2), 32l}, {30, bdec(30.2), 43l}};
+        Object[][] expectedRows = new Object[][]{{10, bdec(10.2), 21l}};
+        runQueryTest(ddl, upsert("n1", "n2", "n3"), select("MIN(n1), MIN(n2), MIN(n3)"), insertedRows, expectedRows);
+    }
+    
+    @Test
+    public void maxDescCompositePK() throws Exception {
+        String ddl = "CREATE TABLE " + TABLE + " (n1 INTEGER NOT NULL, n2 DECIMAL(10, 2) NOT NULL, n3 BIGINT NOT NULL " + 
+            "constraint pk primary key (n1 DESC, n2 DESC, n3 DESC))";
+        Object[][] insertedRows = new Object[][]{{10, bdec(10.2), 21l}, {20, bdec(20.2), 32l}, {30, bdec(30.2), 43l}};
+        Object[][] expectedRows = new Object[][]{{30, bdec(30.2), 43l}};
+        runQueryTest(ddl, upsert("n1", "n2", "n3"), select("MAX(n1), MAX(n2), MAX(n3)"), insertedRows, expectedRows);
+    }
+    
+    @Test
+    public void havingSumDescCompositePK() throws Exception {
+        String ddl = "CREATE TABLE " + TABLE + " (name CHAR(1) NOT NULL, code INTEGER NOT NULL " + 
+            "constraint pk primary key (name DESC, code DESC))";
+        Object[][] insertedRows = new Object[][]{{"a", 10}, {"a", 20}, {"b", 100}}; 
+        Object[][] expectedRows = new Object[][]{{"a", 30l}};
+        runQueryTest(ddl, upsert("name", "code"), select("name", "SUM(code)"), insertedRows, expectedRows, 
+            new HavingCondition("name", "SUM(code) = 30"));
+    }
+    
+    @Test
+    public void queryDescDateWithExplicitOrderBy() throws Exception {
+        String ddl = "CREATE TABLE " + TABLE + " (c1 CHAR(1) NOT NULL, c2 CHAR(1) NOT NULL, d1 DATE NOT NULL, c3 CHAR(1) NOT NULL " + 
+            "constraint pk primary key (c1, c2, d1 DESC, c3))";
+        Object[] row1 = {"1", "2", date(10, 11, 2001), "3"};
+        Object[] row2 = {"1", "2", date(10, 11, 2003), "3"};
+        Object[][] insertedRows = new Object[][]{row1, row2};
+        runQueryTest(ddl, upsert("c1", "c2", "d1", "c3"), select("c1, c2, d1", "c3"), insertedRows, new Object[][]{row2, row1},
+            null, null, new OrderBy("d1", OrderBy.Direction.DESC));
+    }    
+    
+    @Test
+    public void additionOnDescCompositePK() throws Exception {
+        String ddl = "CREATE TABLE " + TABLE + " (n1 INTEGER NOT NULL, n2 DECIMAL(10, 2) NOT NULL, n3 BIGINT NOT NULL, d1 DATE NOT NULL " + 
+            "constraint pk primary key (n1 DESC, n2 DESC, n3 DESC, d1 DESC))";
+        Object[][] insertedRows = new Object[][]{
+            {10, bdec(10.2), 21l, date(1, 10, 2001)}, {20, bdec(20.2), 32l, date(2, 6, 2001)}, {30, bdec(30.2), 43l, date(3, 1, 2001)}};
+        Object[][] expectedRows = new Object[][]{
+            {31l, bdec(32.2), 46l, date(3, 5, 2001)}, {21l, bdec(22.2), 35l, date(2, 10, 2001)}, {11l, bdec(12.2), 24l, date(1, 14, 2001)}};
+        runQueryTest(ddl, upsert("n1", "n2", "n3", "d1"), select("n1+1, n2+2, n3+3", "d1+4"), insertedRows, expectedRows);
+    }
+    
+    @Test
+    public void subtractionOnDescCompositePK() throws Exception {
+        String ddl = "CREATE TABLE " + TABLE + " (n1 INTEGER NOT NULL, n2 DECIMAL(10, 2) NOT NULL, n3 BIGINT NOT NULL, d1 DATE NOT NULL " + 
+            "constraint pk primary key (n1 DESC, n2 DESC, n3 DESC, d1 DESC))";
+        Object[][] insertedRows = new Object[][]{
+            {10, bdec(10.2), 21l, date(1, 10, 2001)}, {20, bdec(20.2), 32l, date(2, 6, 2001)}, {30, bdec(30.2), 43l, date(3, 10, 2001)}};
+        Object[][] expectedRows = new Object[][]{
+            {29l, bdec(28.2), 40l, date(3, 6, 2001)}, {19l, bdec(18.2), 29l, date(2, 2, 2001)}, {9l, bdec(8.2), 18l, date(1, 6, 2001)}};
+        runQueryTest(ddl, upsert("n1", "n2", "n3", "d1"), select("n1-1, n2-2, n3-3", "d1-4"), insertedRows, expectedRows);
+    }
+    
+    @Test
+    public void lessThanLeadingDescCompositePK() throws Exception {
+        String ddl = "CREATE TABLE " + TABLE + " (id INTEGER NOT NULL, date DATE NOT NULL constraint pk primary key (id DESC, date))";
+        Object[][] insertedRows = new Object[][]{{1, date(1, 1, 2012)}, {3, date(1, 1, 2013)}, {2, date(1, 1, 2011)}};
+        Object[][] expectedRows = new Object[][]{{1, date(1, 1, 2012)}};
+        runQueryTest(ddl, upsert("id", "date"), insertedRows, expectedRows, new WhereCondition("id", "<", "2"));
+    }
+    
+    @Test
+    public void lessThanTrailingDescCompositePK() throws Exception {
+        String ddl = "CREATE TABLE " + TABLE + " (id INTEGER NOT NULL, date DATE NOT NULL constraint pk primary key (id DESC, date))";
+        Object[][] insertedRows = new Object[][]{{1, date(1, 1, 2002)}, {3, date(1, 1, 2003)}, {2, date(1, 1, 2001)}};
+        Object[][] expectedRows = new Object[][]{{2, date(1, 1, 2001)}};
+        runQueryTest(ddl, upsert("id", "date"), insertedRows, expectedRows, new WhereCondition("date", "<", "TO_DATE('02-02-2001','mm-dd-yyyy')"));
+    }
+    
+    @Test
+    public void descVarLengthPK() throws Exception {
+        String ddl = "CREATE TABLE " + TABLE + " (id VARCHAR PRIMARY KEY DESC)";
+        Object[][] insertedRows = new Object[][]{{"a"}, {"ab"}, {"abc"}};
+        Object[][] expectedRows = new Object[][]{{"abc"}, {"ab"}, {"a"}};
+        runQueryTest(ddl, upsert("id"), select("id"), insertedRows, expectedRows,
+                null, null, new OrderBy("id", OrderBy.Direction.DESC));
+    }
+    
+    private void runQueryTest(String ddl, String columnName, Object[][] rows, Object[][] expectedRows) throws Exception {
+        runQueryTest(ddl, new String[]{columnName}, rows, expectedRows, null);
+    }
+    
+    private void runQueryTest(String ddl, String[] columnNames, Object[][] rows, Object[][] expectedRows) throws Exception {
+        runQueryTest(ddl, columnNames, rows, expectedRows, null);
+    }
+    
+    private void runQueryTest(String ddl, String[] columnNames, Object[][] rows, Object[][] expectedRows, WhereCondition condition) throws Exception {
+        runQueryTest(ddl, columnNames, columnNames, rows, expectedRows, condition, null, null);
+    }
+    
+    private void runQueryTest(String ddl, String[] columnNames, String[] projections, Object[][] rows, Object[][] expectedRows) throws Exception {
+        runQueryTest(ddl, columnNames, projections, rows, expectedRows, null, null, null);
+    }
+    
+    private void runQueryTest(String ddl, String[] columnNames, String[] projections, Object[][] rows, Object[][] expectedRows, HavingCondition havingCondition) throws Exception {
+        runQueryTest(ddl, columnNames, projections, rows, expectedRows, null, havingCondition, null);
+    }
+    
+
+    private void runQueryTest(
+        String ddl, 
+        String[] columnNames, 
+        String[] projections, 
+        Object[][] rows, Object[][] expectedRows, 
+        WhereCondition whereCondition, 
+        HavingCondition havingCondition,
+        OrderBy orderBy) 
+        throws Exception 
+    {
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        Connection conn = DriverManager.getConnection(getUrl(), props);
+
+        try {
+
+            conn.setAutoCommit(false);
+
+            createTestTable(getUrl(), ddl);
+
+            String columns = appendColumns(columnNames);
+            String placeholders = appendPlaceholders(columnNames);
+            String dml = "UPSERT INTO " + TABLE + " (" + columns + ") VALUES(" + placeholders +")";
+            PreparedStatement stmt = conn.prepareStatement(dml);
+
+            for (int row = 0; row < rows.length; row++) {
+                for (int col = 0; col < rows[row].length; col++) {
+                    Object value = rows[row][col];
+                    stmt.setObject(col + 1, value);
+                }
+                stmt.execute();
+            }
+            conn.commit();
+            
+            String selectClause = "SELECT " + appendColumns(projections) + " FROM " + TABLE;
+            
+            for (WhereCondition whereConditionClause : new WhereCondition[]{whereCondition, WhereCondition.reverse(whereCondition)}) {
+                String query = WhereCondition.appendWhere(whereConditionClause, selectClause);
+                query = HavingCondition.appendHaving(havingCondition, query);
+                query = OrderBy.appendOrderBy(orderBy, query);
+                runQuery(conn, query, expectedRows);
+            }
+            
+            if (orderBy != null) {
+                orderBy = OrderBy.reverse(orderBy);
+                String query = WhereCondition.appendWhere(whereCondition, selectClause);
+                query = HavingCondition.appendHaving(havingCondition, query);
+                query = OrderBy.appendOrderBy(orderBy, query);
+                runQuery(conn, query, reverse(expectedRows));
+            }
+            
+        } finally {
+            conn.close();
+        }
+    }
+    
+    private String appendColumns(String[] columnNames) {
+        String appendedColumns = "";
+        for (int i = 0; i < columnNames.length; i++) {                
+            appendedColumns += columnNames[i];
+            if (i < columnNames.length - 1) {
+                appendedColumns += ",";
+            }
+        }
+        return appendedColumns;
+    }
+    
+    private String appendPlaceholders(String[] columnNames) {
+        String placeholderList = "";
+        for (int i = 0; i < columnNames.length; i++) {                
+            placeholderList += "?";
+            if (i < columnNames.length - 1) {
+                placeholderList += ",";
+            }
+        }
+        return placeholderList;        
+    }
+    
+    private static void runQuery(Connection connection, String query, Object[][] expectedValues) throws Exception {
+        PreparedStatement stmt = connection.prepareStatement(query);
+
+        ResultSet rs = stmt.executeQuery();
+        int rowCounter = 0;
+        while (rs.next()) {
+            if (rowCounter == expectedValues.length) {
+                Assert.assertEquals("Exceeded number of expected rows for query" + query, expectedValues.length, rowCounter+1);
+            }
+            Object[] cols = new Object[expectedValues[rowCounter].length];
+            for (int colCounter = 0; colCounter < expectedValues[rowCounter].length; colCounter++) {
+                cols[colCounter] = rs.getObject(colCounter+1);
+            }
+            Assert.assertArrayEquals("Unexpected result for query " + query, expectedValues[rowCounter], cols);
+            rowCounter++;
+        }
+        Assert.assertEquals("Unexpected number of rows for query " + query, expectedValues.length, rowCounter);
+    }
+    
+    private static Object[][] reverse(Object[][] rows) {
+        Object[][] reversedArray = new Object[rows.length][];
+        System.arraycopy(rows, 0, reversedArray, 0, rows.length);
+        ArrayUtils.reverse(reversedArray);
+        return reversedArray;
+    }
+    
+    private static Date date(int month, int day, int year) {
+        Calendar cal = new GregorianCalendar();
+        cal.set(Calendar.MONTH, month-1);
+        cal.set(Calendar.DAY_OF_MONTH, day);
+        cal.set(Calendar.YEAR, year);
+        cal.set(Calendar.HOUR_OF_DAY, 10);
+        cal.set(Calendar.MINUTE, 2);
+        cal.set(Calendar.SECOND, 5);
+        cal.set(Calendar.MILLISECOND, 101);
+        Date d = new Date(cal.getTimeInMillis()); 
+        return d;
+    }
+        
+    private static String[] upsert(String...args) {
+        return args;
+    }
+    
+    private static String[] select(String...args) {
+        return args;
+    }
+    
+    private static BigDecimal bdec(double d) {
+        return BigDecimal.valueOf(d);
+    }
+    
+    private static BigInteger bint(long l) {
+        return BigInteger.valueOf(l);
+    }    
+    
+    private static class WhereCondition {
+        final String lhs;
+        final String operator;
+        final String rhs;
+    
+        WhereCondition(String lhs, String operator, String rhs) {
+            this.lhs = lhs;
+            this.operator = operator;
+            this.rhs = rhs;
+        }
+        
+        static WhereCondition reverse(WhereCondition whereCondition) {
+            
+            if (whereCondition == null) {
+                return null; 
+            }
+            
+            if (whereCondition.operator.equalsIgnoreCase("IN") || whereCondition.operator.equalsIgnoreCase("LIKE")) {
+                return whereCondition;
+            } else {
+                return new WhereCondition(whereCondition.rhs, whereCondition.getReversedOperator(), whereCondition.lhs);
+            }
+        }
+        
+         static String appendWhere(WhereCondition whereCondition, String query) {
+             if (whereCondition == null) {
+                 return query;
+             }
+            return query + " WHERE " + whereCondition.lhs + " " + whereCondition.operator + " " + whereCondition.rhs;
+        }
+        
+        private String getReversedOperator() {
+            if (operator.equals("<")) {
+                return ">";
+            } else if (operator.equals(">")) {
+                return "<";
+            }
+            return operator;
+        }
+    }
+    
+    private static class HavingCondition {
+        
+        private String groupby;
+        private String having;
+        
+        HavingCondition(String groupby, String having) {
+            this.groupby = groupby;
+            this.having = having;
+        }
+        
+        static String appendHaving(HavingCondition havingCondition, String query) {
+            if (havingCondition == null) {
+                return query;
+            }
+            return query + " GROUP BY " + havingCondition.groupby + " HAVING " + havingCondition.having + " ";
+        }
+    }
+    
+    private static class OrderBy {
+        
+        enum Direction {
+            
+            ASC, DESC;
+            
+            Direction reverse() {
+                if (this == ASC) {
+                    return DESC;
+                }
+                return ASC;
+            }
+        }
+        
+        private List<String> columnNames = Lists.newArrayList();
+        private List<Direction> directions = Lists.newArrayList();
+        
+        OrderBy() {            
+        }
+        
+        OrderBy(String columnName, Direction orderBy) {
+            add(columnName, orderBy);
+        }
+        
+        void add(String columnName, Direction direction) {
+            columnNames.add(columnName);
+            directions.add(direction);
+        }
+        
+        static OrderBy reverse(OrderBy orderBy) {
+            
+            if (orderBy == null) {
+                return null;
+            }
+            
+            List<Direction> reversedDirections = Lists.newArrayList();
+            for (Direction dir : orderBy.directions) {
+                reversedDirections.add(dir.reverse());
+            }
+            OrderBy reversedOrderBy = new OrderBy();
+            reversedOrderBy.columnNames = orderBy.columnNames;
+            reversedOrderBy.directions = reversedDirections;
+            return reversedOrderBy;
+        }
+        
+        static String appendOrderBy(OrderBy orderBy, String query) {
+            if (orderBy == null || orderBy.columnNames.isEmpty()) {
+                return query;
+            }
+            query += " ORDER BY ";
+            for (int i = 0; i < orderBy.columnNames.size(); i++) {
+                query += orderBy.columnNames.get(i) + " " + orderBy.directions.get(i).toString() + " ";
+            }
+            
+            query += " LIMIT 1000 ";
+            
+            return query;
+        }        
+    }
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/2620a80c/phoenix-core/src/main/java/org/apache/phoenix/compile/FromCompiler.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/FromCompiler.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/FromCompiler.java
index 30a2bb7..9845498 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/FromCompiler.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/FromCompiler.java
@@ -25,6 +25,7 @@ import java.util.HashMap;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
+
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.client.HTableInterface;
 import org.apache.phoenix.coprocessor.MetaDataProtocol;
@@ -650,7 +651,7 @@ public class FromCompiler {
                     PTableType.SUBQUERY, null, MetaDataProtocol.MIN_TABLE_TIMESTAMP, PTable.INITIAL_SEQ_NUM,
                     null, null, columns, null, null, Collections.<PTable>emptyList(),
                     false, Collections.<PName>emptyList(), null, null, false, false, false, null,
-                    null, null);
+                    null, null, false);
 
             String alias = subselectNode.getAlias();
             TableRef tableRef = new TableRef(alias, t, MetaDataProtocol.MIN_TABLE_TIMESTAMP, false);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/2620a80c/phoenix-core/src/main/java/org/apache/phoenix/compile/JoinCompiler.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/JoinCompiler.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/JoinCompiler.java
index 78f54e8..e4ca5d9 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/JoinCompiler.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/JoinCompiler.java
@@ -1297,8 +1297,12 @@ public class JoinCompiler {
         }
         
         return PTableImpl.makePTable(left.getTenantId(), left.getSchemaName(),
-                PNameFactory.newName(SchemaUtil.getTableName(left.getName().getString(), right.getName().getString())), left.getType(), left.getIndexState(), left.getTimeStamp(), left.getSequenceNumber(), left.getPKName(), left.getBucketNum(), merged,
-                left.getParentSchemaName(), left.getParentTableName(), left.getIndexes(), left.isImmutableRows(), Collections.<PName>emptyList(), null, null, PTable.DEFAULT_DISABLE_WAL, left.isMultiTenant(), left.getStoreNulls(), left.getViewType(), left.getViewIndexId(), left.getIndexType());
+                PNameFactory.newName(SchemaUtil.getTableName(left.getName().getString(), right.getName().getString())),
+                left.getType(), left.getIndexState(), left.getTimeStamp(), left.getSequenceNumber(), left.getPKName(), 
+                left.getBucketNum(), merged,left.getParentSchemaName(), left.getParentTableName(), left.getIndexes(),
+                left.isImmutableRows(), Collections.<PName>emptyList(), null, null, PTable.DEFAULT_DISABLE_WAL,
+                left.isMultiTenant(), left.getStoreNulls(), left.getViewType(), left.getViewIndexId(), left.getIndexType(),
+                left.rowKeyOrderOptimizable());
     }
 
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/2620a80c/phoenix-core/src/main/java/org/apache/phoenix/compile/OrderByCompiler.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/OrderByCompiler.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/OrderByCompiler.java
index f0406d4..7275b64 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/OrderByCompiler.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/OrderByCompiler.java
@@ -122,8 +122,8 @@ public class OrderByCompiler {
                 boolean isAscending = node.isAscending();
                 boolean isNullsLast = node.isNullsLast();
                 tracker.track(expression, isAscending ? SortOrder.ASC : SortOrder.DESC, isNullsLast);
-                // FIXME: this isn't correct. If we have a schema where column A is DESC,
-                // An ORDER BY A should still be ASC.
+                // If we have a schema where column A is DESC, reverse the sort order and nulls last
+                // since this is the order they actually are in.
                 if (expression.getSortOrder() == SortOrder.DESC) {
                     isAscending = !isAscending;
                     isNullsLast = !isNullsLast;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/2620a80c/phoenix-core/src/main/java/org/apache/phoenix/compile/OrderPreservingTracker.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/OrderPreservingTracker.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/OrderPreservingTracker.java
index 1c31606..70ae231 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/OrderPreservingTracker.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/OrderPreservingTracker.java
@@ -79,8 +79,9 @@ public class OrderPreservingTracker {
     
     public OrderPreservingTracker(StatementContext context, GroupBy groupBy, Ordering ordering, int nNodes, TupleProjector projector) {
         int pkPositionOffset = 0;
+        PTable table = context.getResolver().getTables().get(0).getTable();
+        isOrderPreserving = table.rowKeyOrderOptimizable();
         if (groupBy.isEmpty()) { // FIXME: would the below table have any of these set in the case of a GROUP BY?
-            PTable table = context.getResolver().getTables().get(0).getTable();
             boolean isSalted = table.getBucketNum() != null;
             boolean isMultiTenant = context.getConnection().getTenantId() != null && table.isMultiTenant();
             boolean isSharedViewIndex = table.getViewIndexId() != null;
@@ -117,10 +118,12 @@ public class OrderPreservingTracker {
                          */
                         if (!groupBy.isEmpty() && !groupBy.isOrderPreserving()) {
                             isOrderPreserving = false;
+                            isReverse = false;
                             return;
                         }
                     } else if (!isReverse){
                         isOrderPreserving = false;
+                        isReverse = false;
                         return;
                     }
                 } else {
@@ -128,12 +131,14 @@ public class OrderPreservingTracker {
                         isReverse = false;
                     } else if (isReverse){
                         isOrderPreserving = false;
+                        isReverse = false;
                         return;
                     }
                 }
                 if (node.isNullable()) {
                     if (!Boolean.valueOf(isNullsLast).equals(isReverse)) {
                         isOrderPreserving = false;
+                        isReverse = false;
                         return;
                     }
                 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/2620a80c/phoenix-core/src/main/java/org/apache/phoenix/compile/ScanRanges.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/ScanRanges.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/ScanRanges.java
index 473e579..80cfbfe 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/ScanRanges.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/ScanRanges.java
@@ -32,6 +32,7 @@ import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.phoenix.filter.SkipScanFilter;
 import org.apache.phoenix.query.KeyRange;
 import org.apache.phoenix.query.KeyRange.Bound;
+import org.apache.phoenix.query.QueryConstants;
 import org.apache.phoenix.schema.RowKeySchema;
 import org.apache.phoenix.schema.SaltingUtil;
 import org.apache.phoenix.util.ByteUtil;
@@ -82,7 +83,9 @@ public class ScanRanges {
                 }
             }
             ranges = Collections.singletonList(keyRanges);
-            if (keys.size() > 1) {
+            // Treat as binary if descending because we've got a separator byte at the end
+            // which is not part of the value.
+            if (keys.size() > 1 || SchemaUtil.getSeparatorByte(schema.rowKeyOrderOptimizable(), false, schema.getField(0)) == QueryConstants.DESC_SEPARATOR_BYTE) {
                 schema = SchemaUtil.VAR_BINARY_SCHEMA;
                 slotSpan = ScanUtil.SINGLE_COLUMN_SLOT_SPAN;
             } else {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/2620a80c/phoenix-core/src/main/java/org/apache/phoenix/compile/TupleProjectionCompiler.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/TupleProjectionCompiler.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/TupleProjectionCompiler.java
index ee65280..c6aa546 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/TupleProjectionCompiler.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/TupleProjectionCompiler.java
@@ -152,7 +152,7 @@ public class TupleProjectionCompiler {
                 table.getBucketNum(), projectedColumns, table.getParentSchemaName(),
                 table.getParentName(), table.getIndexes(), table.isImmutableRows(), Collections.<PName>emptyList(), null, null,
                 table.isWALDisabled(), table.isMultiTenant(), table.getStoreNulls(), table.getViewType(), table.getViewIndexId(),
-                table.getIndexType());
+                table.getIndexType(), table.rowKeyOrderOptimizable());
     }
 
     public static PTable createProjectedTable(TableRef tableRef, List<ColumnRef> sourceColumnRefs, boolean retainPKColumns) throws SQLException {
@@ -179,7 +179,7 @@ public class TupleProjectionCompiler {
                     retainPKColumns ? table.getBucketNum() : null, projectedColumns, null,
                     null, Collections.<PTable>emptyList(), table.isImmutableRows(), Collections.<PName>emptyList(), null, null,
                     table.isWALDisabled(), table.isMultiTenant(), table.getStoreNulls(), table.getViewType(), table.getViewIndexId(),
-                    null);
+                    null, table.rowKeyOrderOptimizable());
     }
 
     // For extracting column references from single select statement

http://git-wip-us.apache.org/repos/asf/phoenix/blob/2620a80c/phoenix-core/src/main/java/org/apache/phoenix/compile/UnionCompiler.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/UnionCompiler.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/UnionCompiler.java
index 269232e..942e244 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/UnionCompiler.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/UnionCompiler.java
@@ -80,8 +80,9 @@ public class UnionCompiler {
         }
         Long scn = statement.getConnection().getSCN();
         PTable tempTable = PTableImpl.makePTable(statement.getConnection().getTenantId(), UNION_SCHEMA_NAME, UNION_TABLE_NAME, 
-                PTableType.SUBQUERY, null, HConstants.LATEST_TIMESTAMP, scn == null ? HConstants.LATEST_TIMESTAMP : scn, null, null, projectedColumns, null, null, null,
-                        true, null, null, null, true, true, true, null, null, null);
+                PTableType.SUBQUERY, null, HConstants.LATEST_TIMESTAMP, scn == null ? HConstants.LATEST_TIMESTAMP : scn, null, null,
+                        projectedColumns, null, null, null,
+                        true, null, null, null, true, true, true, null, null, null, false);
         TableRef tableRef = new TableRef(null, tempTable, 0, false);
         return tableRef;
     }