You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@phoenix.apache.org by td...@apache.org on 2018/11/06 23:14:22 UTC

[1/6] phoenix git commit: PHOENIX-4981 Add tests for ORDER BY, GROUP BY and salted tables using phoenix-spark

Repository: phoenix
Updated Branches:
  refs/heads/4.x-HBase-1.2 0c3f43384 -> c509d58f1


http://git-wip-us.apache.org/repos/asf/phoenix/blob/c509d58f/phoenix-spark/src/main/java/org/apache/phoenix/spark/SparkResultSet.java
----------------------------------------------------------------------
diff --git a/phoenix-spark/src/main/java/org/apache/phoenix/spark/SparkResultSet.java b/phoenix-spark/src/main/java/org/apache/phoenix/spark/SparkResultSet.java
new file mode 100644
index 0000000..0cb8009
--- /dev/null
+++ b/phoenix-spark/src/main/java/org/apache/phoenix/spark/SparkResultSet.java
@@ -0,0 +1,1056 @@
+/*
+ * 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.spark;
+
+import org.apache.phoenix.exception.SQLExceptionCode;
+import org.apache.phoenix.exception.SQLExceptionInfo;
+import org.apache.phoenix.util.SQLCloseable;
+import org.apache.spark.sql.Row;
+
+import java.io.InputStream;
+import java.io.Reader;
+import java.math.BigDecimal;
+import java.net.MalformedURLException;
+import java.net.URL;
+import java.sql.Array;
+import java.sql.Blob;
+import java.sql.Clob;
+import java.sql.Date;
+import java.sql.NClob;
+import java.sql.Ref;
+import java.sql.ResultSet;
+import java.sql.ResultSetMetaData;
+import java.sql.RowId;
+import java.sql.SQLException;
+import java.sql.SQLFeatureNotSupportedException;
+import java.sql.SQLWarning;
+import java.sql.SQLXML;
+import java.sql.Statement;
+import java.sql.Time;
+import java.sql.Timestamp;
+import java.util.Arrays;
+import java.util.Calendar;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * Helper class to convert a List of Rows returned from a dataset to a sql ResultSet
+ */
+public class SparkResultSet implements ResultSet, SQLCloseable {
+
+    private int index = -1;
+    private List<Row> dataSetRows;
+    private List<String> columnNames;
+    private boolean wasNull = false;
+
+    public SparkResultSet(List<Row> rows, String[] columnNames) {
+        this.dataSetRows = rows;
+        this.columnNames = Arrays.asList(columnNames);
+    }
+
+    private Row getCurrentRow() {
+        return dataSetRows.get(index);
+    }
+
+    @Override
+    public boolean absolute(int row) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void afterLast() throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void beforeFirst() throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void cancelRowUpdates() throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void clearWarnings() throws SQLException {
+    }
+
+    @Override
+    public void close() throws SQLException {
+    }
+
+    @Override
+    public void deleteRow() throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public int findColumn(String columnLabel) throws SQLException {
+        return columnNames.indexOf(columnLabel.toUpperCase())+1;
+    }
+
+    @Override
+    public boolean first() throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public Array getArray(int columnIndex) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public Array getArray(String columnLabel) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public InputStream getAsciiStream(int columnIndex) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public InputStream getAsciiStream(String columnLabel) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    private void checkOpen() throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    private void checkCursorState() throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public BigDecimal getBigDecimal(int columnIndex) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public BigDecimal getBigDecimal(String columnLabel) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public BigDecimal getBigDecimal(int columnIndex, int scale) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public BigDecimal getBigDecimal(String columnLabel, int scale) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public InputStream getBinaryStream(int columnIndex) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public InputStream getBinaryStream(String columnLabel) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public Blob getBlob(int columnIndex) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public Blob getBlob(String columnLabel) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public boolean getBoolean(int columnIndex) throws SQLException {
+        wasNull = getCurrentRow().isNullAt(columnIndex-1);
+        return wasNull ? false : getCurrentRow().getBoolean(columnIndex-1);
+    }
+
+    @Override
+    public boolean getBoolean(String columnLabel) throws SQLException {
+        return getBoolean(findColumn(columnLabel));
+    }
+
+    @Override
+    public byte[] getBytes(int columnIndex) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public byte[] getBytes(String columnLabel) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public byte getByte(int columnIndex) throws SQLException {
+        wasNull = getCurrentRow().isNullAt(columnIndex-1);
+        return wasNull ? 0 : getCurrentRow().getByte(columnIndex-1);
+    }
+
+    @Override
+    public byte getByte(String columnLabel) throws SQLException {
+        return getByte(findColumn(columnLabel));
+    }
+
+    @Override
+    public Reader getCharacterStream(int columnIndex) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public Reader getCharacterStream(String columnLabel) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public Clob getClob(int columnIndex) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public Clob getClob(String columnLabel) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public int getConcurrency() throws SQLException {
+        return ResultSet.CONCUR_READ_ONLY;
+    }
+
+    @Override
+    public String getCursorName() throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public Date getDate(int columnIndex) throws SQLException {
+        wasNull = getCurrentRow().isNullAt(columnIndex-1);
+        return getCurrentRow().getDate(columnIndex-1);
+    }
+
+    @Override
+    public Date getDate(String columnLabel) throws SQLException {
+        return getDate(findColumn(columnLabel));
+    }
+
+    @Override
+    public Date getDate(int columnIndex, Calendar cal) throws SQLException {
+        cal.setTime(getCurrentRow().getDate(columnIndex-1));
+        return new Date(cal.getTimeInMillis());
+    }
+
+    @Override
+    public Date getDate(String columnLabel, Calendar cal) throws SQLException {
+        return getDate(findColumn(columnLabel), cal);
+    }
+
+    @Override
+    public double getDouble(int columnIndex) throws SQLException {
+        wasNull = getCurrentRow().isNullAt(columnIndex-1);
+        return wasNull ? 0 : getCurrentRow().getDouble(columnIndex-1);
+    }
+
+    @Override
+    public double getDouble(String columnLabel) throws SQLException {
+        return getDouble(findColumn(columnLabel));
+    }
+
+    @Override
+    public int getFetchDirection() throws SQLException {
+        return ResultSet.FETCH_FORWARD;
+    }
+
+    @Override
+    public int getFetchSize() throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public float getFloat(int columnIndex) throws SQLException {
+        wasNull = getCurrentRow().isNullAt(columnIndex-1);
+        return wasNull ? 0 : getCurrentRow().getFloat(columnIndex-1);
+    }
+
+    @Override
+    public float getFloat(String columnLabel) throws SQLException {
+        return getFloat(findColumn(columnLabel));
+    }
+
+    @Override
+    public int getHoldability() throws SQLException {
+        return ResultSet.CLOSE_CURSORS_AT_COMMIT;
+    }
+
+    @Override
+    public int getInt(int columnIndex) throws SQLException {
+        wasNull = getCurrentRow().isNullAt(columnIndex-1);
+        return wasNull ? 0 :  getCurrentRow().getInt(columnIndex-1);
+    }
+
+    @Override
+    public int getInt(String columnLabel) throws SQLException {
+        return getInt(findColumn(columnLabel));
+    }
+
+    @Override
+    public long getLong(int columnIndex) throws SQLException {
+        wasNull = getCurrentRow().isNullAt(columnIndex-1);
+        return wasNull ? 0 :  getCurrentRow().getLong(columnIndex-1);
+    }
+
+    @Override
+    public long getLong(String columnLabel) throws SQLException {
+        return getLong(findColumn(columnLabel));
+    }
+
+    @Override
+    public ResultSetMetaData getMetaData() throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public Reader getNCharacterStream(int columnIndex) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public Reader getNCharacterStream(String columnLabel) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public NClob getNClob(int columnIndex) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public NClob getNClob(String columnLabel) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public String getNString(int columnIndex) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public String getNString(String columnLabel) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public Object getObject(int columnIndex) throws SQLException {
+        return getCurrentRow().get(columnIndex-1);
+    }
+
+    @Override
+    public Object getObject(String columnLabel) throws SQLException {
+        return getObject(findColumn(columnLabel));
+    }
+
+    @Override
+    public Object getObject(int columnIndex, Map<String, Class<?>> map) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public Object getObject(String columnLabel, Map<String, Class<?>> map) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public Ref getRef(int columnIndex) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public Ref getRef(String columnLabel) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public int getRow() throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public RowId getRowId(int columnIndex) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public RowId getRowId(String columnLabel) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public SQLXML getSQLXML(int columnIndex) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public SQLXML getSQLXML(String columnLabel) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public short getShort(int columnIndex) throws SQLException {
+        return getCurrentRow().getShort(columnIndex-1);
+    }
+
+    @Override
+    public short getShort(String columnLabel) throws SQLException {
+        return getShort(findColumn(columnLabel));
+    }
+
+    @Override
+    public Statement getStatement() throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public String getString(int columnIndex) throws SQLException {
+        wasNull = getCurrentRow().isNullAt(columnIndex-1);
+        return wasNull ? null : getCurrentRow().getString(columnIndex-1);
+    }
+
+    @Override
+    public String getString(String columnLabel) throws SQLException {
+        return getString(findColumn(columnLabel));
+    }
+
+    @Override
+    public Time getTime(int columnIndex) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public Time getTime(String columnLabel) throws SQLException {
+        return getTime(findColumn(columnLabel));
+    }
+
+    @Override
+    public Time getTime(int columnIndex, Calendar cal) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public Time getTime(String columnLabel, Calendar cal) throws SQLException {
+        return getTime(findColumn(columnLabel),cal);
+    }
+
+    @Override
+    public Timestamp getTimestamp(int columnIndex) throws SQLException {
+        return getCurrentRow().getTimestamp(columnIndex-1);
+    }
+
+    @Override
+    public Timestamp getTimestamp(String columnLabel) throws SQLException {
+        return getTimestamp(findColumn(columnLabel));
+    }
+
+    @Override
+    public Timestamp getTimestamp(int columnIndex, Calendar cal) throws SQLException {
+        return getTimestamp(columnIndex-1);
+    }
+
+    @Override
+    public Timestamp getTimestamp(String columnLabel, Calendar cal) throws SQLException {
+        return getTimestamp(findColumn(columnLabel),cal);
+    }
+
+    @Override
+    public int getType() throws SQLException {
+        return ResultSet.TYPE_FORWARD_ONLY;
+    }
+
+    @Override
+    public URL getURL(int columnIndex) throws SQLException {
+        try {
+            return new URL(getCurrentRow().getString(columnIndex-1));
+        } catch (MalformedURLException e) {
+            throw new SQLExceptionInfo.Builder(SQLExceptionCode.MALFORMED_URL).setRootCause(e)
+                    .build().buildException();
+        }
+    }
+
+    @Override
+    public URL getURL(String columnLabel) throws SQLException {
+        return getURL(findColumn(columnLabel));
+    }
+
+    @Override
+    public InputStream getUnicodeStream(int columnIndex) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public InputStream getUnicodeStream(String columnLabel) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public SQLWarning getWarnings() throws SQLException {
+        return null;
+    }
+
+    @Override
+    public void insertRow() throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public boolean isAfterLast() throws SQLException {
+        return index >= dataSetRows.size();
+    }
+
+    @Override
+    public boolean isBeforeFirst() throws SQLException {
+        return index == -1;
+    }
+
+    @Override
+    public boolean isClosed() throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public boolean isFirst() throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public boolean isLast() throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public boolean last() throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void moveToCurrentRow() throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void moveToInsertRow() throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public boolean next() throws SQLException {
+        index++;
+        return index < dataSetRows.size();
+    }
+
+    @Override
+    public boolean previous() throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void refreshRow() throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public boolean relative(int rows) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public boolean rowDeleted() throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public boolean rowInserted() throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public boolean rowUpdated() throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void setFetchDirection(int direction) throws SQLException {
+        if (direction != ResultSet.FETCH_FORWARD) {
+            throw new SQLFeatureNotSupportedException();
+        }
+    }
+
+    @Override
+    public void setFetchSize(int rows) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void updateArray(int columnIndex, Array x) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void updateArray(String columnLabel, Array x) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void updateAsciiStream(int columnIndex, InputStream x) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void updateAsciiStream(String columnLabel, InputStream x) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void updateAsciiStream(int columnIndex, InputStream x, int length) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void updateAsciiStream(String columnLabel, InputStream x, int length) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void updateAsciiStream(int columnIndex, InputStream x, long length) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void updateAsciiStream(String columnLabel, InputStream x, long length) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void updateBigDecimal(int columnIndex, BigDecimal x) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void updateBigDecimal(String columnLabel, BigDecimal x) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void updateBinaryStream(int columnIndex, InputStream x) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void updateBinaryStream(String columnLabel, InputStream x) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void updateBinaryStream(int columnIndex, InputStream x, int length) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void updateBinaryStream(String columnLabel, InputStream x, int length) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void updateBinaryStream(int columnIndex, InputStream x, long length) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void updateBinaryStream(String columnLabel, InputStream x, long length) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void updateBlob(int columnIndex, Blob x) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void updateBlob(String columnLabel, Blob x) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void updateBlob(int columnIndex, InputStream inputStream) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void updateBlob(String columnLabel, InputStream inputStream) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void updateBlob(int columnIndex, InputStream inputStream, long length) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void updateBlob(String columnLabel, InputStream inputStream, long length) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void updateBoolean(int columnIndex, boolean x) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void updateBoolean(String columnLabel, boolean x) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void updateByte(int columnIndex, byte x) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void updateByte(String columnLabel, byte x) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void updateBytes(int columnIndex, byte[] x) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void updateBytes(String columnLabel, byte[] x) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void updateCharacterStream(int columnIndex, Reader x) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void updateCharacterStream(String columnLabel, Reader reader) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void updateCharacterStream(int columnIndex, Reader x, int length) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void updateCharacterStream(String columnLabel, Reader reader, int length) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void updateCharacterStream(int columnIndex, Reader x, long length) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void updateCharacterStream(String columnLabel, Reader reader, long length) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void updateClob(int columnIndex, Clob x) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void updateClob(String columnLabel, Clob x) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void updateClob(int columnIndex, Reader reader) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void updateClob(String columnLabel, Reader reader) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void updateClob(int columnIndex, Reader reader, long length) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void updateClob(String columnLabel, Reader reader, long length) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void updateDate(int columnIndex, Date x) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void updateDate(String columnLabel, Date x) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void updateDouble(int columnIndex, double x) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void updateDouble(String columnLabel, double x) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void updateFloat(int columnIndex, float x) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void updateFloat(String columnLabel, float x) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void updateInt(int columnIndex, int x) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void updateInt(String columnLabel, int x) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void updateLong(int columnIndex, long x) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void updateLong(String columnLabel, long x) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void updateNCharacterStream(int columnIndex, Reader x) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void updateNCharacterStream(String columnLabel, Reader reader) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void updateNCharacterStream(int columnIndex, Reader x, long length) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void updateNCharacterStream(String columnLabel, Reader reader, long length) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void updateNClob(int columnIndex, NClob nClob) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void updateNClob(String columnLabel, NClob nClob) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void updateNClob(int columnIndex, Reader reader) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void updateNClob(String columnLabel, Reader reader) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void updateNClob(int columnIndex, Reader reader, long length) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void updateNClob(String columnLabel, Reader reader, long length) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void updateNString(int columnIndex, String nString) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void updateNString(String columnLabel, String nString) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void updateNull(int columnIndex) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void updateNull(String columnLabel) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void updateObject(int columnIndex, Object x) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void updateObject(String columnLabel, Object x) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void updateObject(int columnIndex, Object x, int scaleOrLength) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void updateObject(String columnLabel, Object x, int scaleOrLength) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void updateRef(int columnIndex, Ref x) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void updateRef(String columnLabel, Ref x) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void updateRow() throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void updateRowId(int columnIndex, RowId x) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void updateRowId(String columnLabel, RowId x) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void updateSQLXML(int columnIndex, SQLXML xmlObject) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void updateSQLXML(String columnLabel, SQLXML xmlObject) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void updateShort(int columnIndex, short x) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void updateShort(String columnLabel, short x) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void updateString(int columnIndex, String x) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void updateString(String columnLabel, String x) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void updateTime(int columnIndex, Time x) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void updateTime(String columnLabel, Time x) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void updateTimestamp(int columnIndex, Timestamp x) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void updateTimestamp(String columnLabel, Timestamp x) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public boolean wasNull() throws SQLException {
+        return wasNull;
+    }
+
+    @Override
+    public boolean isWrapperFor(Class<?> iface) throws SQLException {
+        return iface.isInstance(this);
+    }
+
+    @SuppressWarnings("unchecked")
+    @Override
+    public <T> T unwrap(Class<T> iface) throws SQLException {
+        if (!iface.isInstance(this)) {
+            throw new SQLExceptionInfo.Builder(SQLExceptionCode.CLASS_NOT_UNWRAPPABLE)
+                    .setMessage(this.getClass().getName() + " not unwrappable from " + iface.getName())
+                    .build().buildException();
+        }
+        return (T)this;
+    }
+
+    @SuppressWarnings("unchecked")
+    @Override
+    public <T> T getObject(int columnIndex, Class<T> type) throws SQLException {
+        return (T) getObject(columnIndex-1); // Just ignore type since we only support built-in types
+    }
+
+    @SuppressWarnings("unchecked")
+    @Override
+    public <T> T getObject(String columnLabel, Class<T> type) throws SQLException {
+        return (T) getObject(columnLabel); // Just ignore type since we only support built-in types
+    }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/phoenix/blob/c509d58f/phoenix-spark/src/main/scala/org/apache/phoenix/spark/PhoenixRDD.scala
----------------------------------------------------------------------
diff --git a/phoenix-spark/src/main/scala/org/apache/phoenix/spark/PhoenixRDD.scala b/phoenix-spark/src/main/scala/org/apache/phoenix/spark/PhoenixRDD.scala
index 2c2c6e1..d604e0e 100644
--- a/phoenix-spark/src/main/scala/org/apache/phoenix/spark/PhoenixRDD.scala
+++ b/phoenix-spark/src/main/scala/org/apache/phoenix/spark/PhoenixRDD.scala
@@ -21,8 +21,9 @@ import org.apache.hadoop.io.NullWritable
 import org.apache.phoenix.jdbc.PhoenixDriver
 import org.apache.phoenix.mapreduce.PhoenixInputFormat
 import org.apache.phoenix.mapreduce.util.PhoenixConfigurationUtil
+import org.apache.phoenix.query.QueryConstants
 import org.apache.phoenix.schema.types._
-import org.apache.phoenix.util.ColumnInfo
+import org.apache.phoenix.util.{ColumnInfo, SchemaUtil}
 import org.apache.spark._
 import org.apache.spark.annotation.DeveloperApi
 import org.apache.spark.rdd.RDD
@@ -148,13 +149,25 @@ class PhoenixRDD(sc: SparkContext, table: String, columns: Seq[String],
     }), new StructType(structFields))
   }
 
-  def phoenixSchemaToCatalystSchema(columnList: Seq[ColumnInfo]) = {
-    columnList.map(ci => {
-      val structType = phoenixTypeToCatalystType(ci)
-      StructField(ci.getDisplayName, structType)
-    })
+  def normalizeColumnName(columnName: String) = {
+    val unescapedColumnName = SchemaUtil.getUnEscapedFullColumnName(columnName)
+    var normalizedColumnName = ""
+    if (unescapedColumnName.indexOf(QueryConstants.NAME_SEPARATOR) < 0) {
+      normalizedColumnName = unescapedColumnName
+    }
+    else {
+      // split by separator to get the column family and column name
+      val tokens = unescapedColumnName.split(QueryConstants.NAME_SEPARATOR_REGEX)
+      normalizedColumnName = if (tokens(0) == "0") tokens(1) else unescapedColumnName
+    }
+    normalizedColumnName
   }
 
+  def phoenixSchemaToCatalystSchema(columnList: Seq[ColumnInfo]) = columnList.map(ci => {
+    val structType = phoenixTypeToCatalystType(ci)
+    StructField(normalizeColumnName(ci.getColumnName), structType)
+  })
+
 
   // Lookup table for Phoenix types to Spark catalyst types
   def phoenixTypeToCatalystType(columnInfo: ColumnInfo): DataType = columnInfo.getPDataType match {
@@ -166,7 +179,7 @@ class PhoenixRDD(sc: SparkContext, table: String, columns: Seq[String],
     case t if t.isInstanceOf[PFloat] || t.isInstanceOf[PUnsignedFloat] => FloatType
     case t if t.isInstanceOf[PDouble] || t.isInstanceOf[PUnsignedDouble] => DoubleType
     // Use Spark system default precision for now (explicit to work with < 1.5)
-    case t if t.isInstanceOf[PDecimal] => 
+    case t if t.isInstanceOf[PDecimal] =>
       if (columnInfo.getPrecision == null || columnInfo.getPrecision < 0) DecimalType(38, 18) else DecimalType(columnInfo.getPrecision, columnInfo.getScale)
     case t if t.isInstanceOf[PTimestamp] || t.isInstanceOf[PUnsignedTimestamp] => TimestampType
     case t if t.isInstanceOf[PTime] || t.isInstanceOf[PUnsignedTime] => TimestampType

http://git-wip-us.apache.org/repos/asf/phoenix/blob/c509d58f/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 5594b88..0eaffc5 100644
--- a/pom.xml
+++ b/pom.xml
@@ -101,7 +101,7 @@
     <avatica.version>1.12.0</avatica.version>
     <jettyVersion>8.1.7.v20120910</jettyVersion>
     <tephra.version>0.15.0-incubating</tephra.version>
-    <spark.version>2.0.2</spark.version>
+    <spark.version>2.3.2</spark.version>
     <scala.version>2.11.8</scala.version>
     <scala.binary.version>2.11</scala.binary.version>
     <stream.version>2.9.5</stream.version>


[2/6] phoenix git commit: PHOENIX-4981 Add tests for ORDER BY, GROUP BY and salted tables using phoenix-spark

Posted by td...@apache.org.
http://git-wip-us.apache.org/repos/asf/phoenix/blob/c509d58f/phoenix-core/src/it/java/org/apache/phoenix/end2end/salted/SaltedTableIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/salted/SaltedTableIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/salted/SaltedTableIT.java
index c9168f1..69c9869 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/salted/SaltedTableIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/salted/SaltedTableIT.java
@@ -37,104 +37,18 @@ import org.apache.phoenix.util.QueryUtil;
 import org.apache.phoenix.util.SchemaUtil;
 import org.junit.Test;
 
-
 /**
  * Tests for table with transparent salting.
  */
 
-public class SaltedTableIT extends ParallelStatsDisabledIT {
-
-	private static String getUniqueTableName() {
-		return SchemaUtil.getTableName(generateUniqueName(), generateUniqueName());
-	}
-	
-    private static String initTableValues(byte[][] splits) throws Exception {
-    	String tableName = getUniqueTableName();    	
-        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
-        Connection conn = DriverManager.getConnection(getUrl(), props);
-        
-        // Rows we inserted:
-        // 1ab123abc111
-        // 1abc456abc111
-        // 1de123abc111
-        // 2abc123def222 
-        // 3abc123ghi333
-        // 4abc123jkl444
-        try {
-            // Upsert with no column specifies.
-            ensureTableCreated(getUrl(), tableName, TABLE_WITH_SALTING, splits, null, null);
-            String query = "UPSERT INTO " + tableName + " VALUES(?,?,?,?,?)";
-            PreparedStatement stmt = conn.prepareStatement(query);
-            stmt.setInt(1, 1);
-            stmt.setString(2, "ab");
-            stmt.setString(3, "123");
-            stmt.setString(4, "abc");
-            stmt.setInt(5, 111);
-            stmt.execute();
-            conn.commit();
-            
-            stmt.setInt(1, 1);
-            stmt.setString(2, "abc");
-            stmt.setString(3, "456");
-            stmt.setString(4, "abc");
-            stmt.setInt(5, 111);
-            stmt.execute();
-            conn.commit();
-            
-            // Test upsert when statement explicitly specifies the columns to upsert into.
-            query = "UPSERT INTO " + tableName +
-                    " (a_integer, a_string, a_id, b_string, b_integer) " + 
-                    " VALUES(?,?,?,?,?)";
-            stmt = conn.prepareStatement(query);
-            
-            stmt.setInt(1, 1);
-            stmt.setString(2, "de");
-            stmt.setString(3, "123");
-            stmt.setString(4, "abc");
-            stmt.setInt(5, 111);
-            stmt.execute();
-            conn.commit();
-            
-            stmt.setInt(1, 2);
-            stmt.setString(2, "abc");
-            stmt.setString(3, "123");
-            stmt.setString(4, "def");
-            stmt.setInt(5, 222);
-            stmt.execute();
-            conn.commit();
-            
-            // Test upsert when order of column is shuffled.
-            query = "UPSERT INTO " + tableName +
-                    " (a_string, a_integer, a_id, b_string, b_integer) " + 
-                    " VALUES(?,?,?,?,?)";
-            stmt = conn.prepareStatement(query);
-            stmt.setString(1, "abc");
-            stmt.setInt(2, 3);
-            stmt.setString(3, "123");
-            stmt.setString(4, "ghi");
-            stmt.setInt(5, 333);
-            stmt.execute();
-            conn.commit();
-            
-            stmt.setString(1, "abc");
-            stmt.setInt(2, 4);
-            stmt.setString(3, "123");
-            stmt.setString(4, "jkl");
-            stmt.setInt(5, 444);
-            stmt.execute();
-            conn.commit();
-        } finally {
-            conn.close();
-        }
-        return tableName;
-    }
+public class SaltedTableIT extends BaseSaltedTableIT {
 
     @Test
     public void testTableWithInvalidBucketNumber() throws Exception {
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         Connection conn = DriverManager.getConnection(getUrl(), props);
         try {
-            String query = "create table " + getUniqueTableName() + " (a_integer integer not null CONSTRAINT pk PRIMARY KEY (a_integer)) SALT_BUCKETS = 257";
+            String query = "create table " + generateUniqueName() + " (a_integer integer not null CONSTRAINT pk PRIMARY KEY (a_integer)) SALT_BUCKETS = 257";
             PreparedStatement stmt = conn.prepareStatement(query);
             stmt.execute();
             fail("Should have caught exception");
@@ -148,370 +62,12 @@ public class SaltedTableIT extends ParallelStatsDisabledIT {
     @Test
     public void testTableWithSplit() throws Exception {
         try {
-            createTestTable(getUrl(), "create table " + getUniqueTableName() + " (a_integer integer not null primary key) SALT_BUCKETS = 4",
+            createTestTable(getUrl(), "create table " + generateUniqueName() + " (a_integer integer not null primary key) SALT_BUCKETS = 4",
                     new byte[][] {{1}, {2,3}, {2,5}, {3}}, null);
             fail("Should have caught exception");
         } catch (SQLException e) {
             assertTrue(e.getMessage(), e.getMessage().contains("ERROR 1022 (42Y81): Should not specify split points on salted table with default row key order."));
         }
     }
-    
-    @Test
-    public void testSelectValueNoWhereClause() throws Exception {
-        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
-        Connection conn = DriverManager.getConnection(getUrl(), props);
-        try {
-            String tableName = initTableValues(null);
-            
-            String query = "SELECT * FROM " + tableName;
-            PreparedStatement statement = conn.prepareStatement(query);
-            ResultSet rs = statement.executeQuery();
-            
-            assertTrue(rs.next());
-            assertEquals(1, rs.getInt(1));
-            assertEquals("ab", rs.getString(2));
-            assertEquals("123", rs.getString(3));
-            assertEquals("abc", rs.getString(4));
-            assertEquals(111, rs.getInt(5));
-            
-            assertTrue(rs.next());
-            assertEquals(1, rs.getInt(1));
-            assertEquals("abc", rs.getString(2));
-            assertEquals("456", rs.getString(3));
-            assertEquals("abc", rs.getString(4));
-            assertEquals(111, rs.getInt(5));
-            
-            assertTrue(rs.next());
-            assertEquals(1, rs.getInt(1));
-            assertEquals("de", rs.getString(2));
-            assertEquals("123", rs.getString(3));
-            assertEquals("abc", rs.getString(4));
-            assertEquals(111, rs.getInt(5));
-            
-            assertTrue(rs.next());
-            assertEquals(2, rs.getInt(1));
-            assertEquals("abc", rs.getString(2));
-            assertEquals("123", rs.getString(3));
-            assertEquals("def", rs.getString(4));
-            assertEquals(222, rs.getInt(5));
-            
-            assertTrue(rs.next());
-            assertEquals(3, rs.getInt(1));
-            assertEquals("abc", rs.getString(2));
-            assertEquals("123", rs.getString(3));
-            assertEquals("ghi", rs.getString(4));
-            assertEquals(333, rs.getInt(5));
-            
-            assertTrue(rs.next());
-            assertEquals(4, rs.getInt(1));
-            assertEquals("abc", rs.getString(2));
-            assertEquals("123", rs.getString(3));
-            assertEquals("jkl", rs.getString(4));
-            assertEquals(444, rs.getInt(5));
-            
-            assertFalse(rs.next());
-        } finally {
-            conn.close();
-        }
-    }
-
-    @Test
-    public void testSelectValueWithFullyQualifiedWhereClause() throws Exception {
-        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
-        Connection conn = DriverManager.getConnection(getUrl(), props);
-        try {
-            String tableName = initTableValues(null);
-            String query;
-            PreparedStatement stmt;
-            ResultSet rs;
-            
-            // Variable length slot with bounded ranges.
-            query = "SELECT * FROM " + tableName + 
-                    " WHERE a_integer = 1 AND a_string >= 'ab' AND a_string < 'de' AND a_id = '123'";
-            stmt = conn.prepareStatement(query);
-            rs = stmt.executeQuery();
-            assertTrue(rs.next());
-            assertEquals(1, rs.getInt(1));
-            assertEquals("ab", rs.getString(2));
-            assertEquals("123", rs.getString(3));
-            assertEquals("abc", rs.getString(4));
-            assertEquals(111, rs.getInt(5));
-            assertFalse(rs.next());
-
-            // all single slots with one value.
-            query = "SELECT * FROM " + tableName + 
-                    " WHERE a_integer = 1 AND a_string = 'ab' AND a_id = '123'";
-            stmt = conn.prepareStatement(query);
-            
-            rs = stmt.executeQuery();
-            assertTrue(rs.next());
-            assertEquals(1, rs.getInt(1));
-            assertEquals("ab", rs.getString(2));
-            assertEquals("123", rs.getString(3));
-            assertEquals("abc", rs.getString(4));
-            assertEquals(111, rs.getInt(5));
-            assertFalse(rs.next());
-            
-            // all single slots with multiple values.
-            query = "SELECT * FROM " + tableName + 
-                    " WHERE a_integer in (2, 4) AND a_string = 'abc' AND a_id = '123'";
-            stmt = conn.prepareStatement(query);
-            rs = stmt.executeQuery();
-            
-            assertTrue(rs.next());
-            assertEquals(2, rs.getInt(1));
-            assertEquals("abc", rs.getString(2));
-            assertEquals("123", rs.getString(3));
-            assertEquals("def", rs.getString(4));
-            assertEquals(222, rs.getInt(5));
-            
-            assertTrue(rs.next());
-            assertEquals(4, rs.getInt(1));
-            assertEquals("abc", rs.getString(2));
-            assertEquals("123", rs.getString(3));
-            assertEquals("jkl", rs.getString(4));
-            assertEquals(444, rs.getInt(5));
-            assertFalse(rs.next());
-            
-            query = "SELECT a_integer, a_string FROM " + tableName +
-                    " WHERE a_integer in (1,2,3,4) AND a_string in ('a', 'abc', 'de') AND a_id = '123'";
-            stmt = conn.prepareStatement(query);
-            rs = stmt.executeQuery();
-            
-            assertTrue(rs.next());
-            assertEquals(1, rs.getInt(1));
-            assertEquals("de", rs.getString(2));
-            
-            assertTrue(rs.next());
-            assertEquals(2, rs.getInt(1));
-            assertEquals("abc", rs.getString(2));
-            
-            assertTrue(rs.next());
-            assertEquals(3, rs.getInt(1));
-            assertEquals("abc", rs.getString(2));
-            
-            assertTrue(rs.next());
-            assertEquals(4, rs.getInt(1));
-            assertEquals("abc", rs.getString(2));
-            assertFalse(rs.next());
-            
-            // fixed length slot with bounded ranges.
-            query = "SELECT a_string, a_id FROM " + tableName + 
-                    " WHERE a_integer > 1 AND a_integer < 4 AND a_string = 'abc' AND a_id = '123'";
-            stmt = conn.prepareStatement(query);
-            rs = stmt.executeQuery();
-            assertTrue(rs.next());
-            assertEquals("abc", rs.getString(1));
-            assertEquals("123", rs.getString(2));
-            
-            assertTrue(rs.next());
-            assertEquals("abc", rs.getString(1));
-            assertEquals("123", rs.getString(2));
-            assertFalse(rs.next());
-            
-            // fixed length slot with unbound ranges.
-            query = "SELECT b_string, b_integer FROM " + tableName + 
-                    " WHERE a_integer > 1 AND a_string = 'abc' AND a_id = '123'";
-            stmt = conn.prepareStatement(query);
-            rs = stmt.executeQuery();
-            assertTrue(rs.next());
-            assertEquals("def", rs.getString(1));
-            assertEquals(222, rs.getInt(2));
-            
-            assertTrue(rs.next());
-            assertEquals("ghi", rs.getString(1));
-            assertEquals(333, rs.getInt(2));
-            
-            assertTrue(rs.next());
-            assertEquals("jkl", rs.getString(1));
-            assertEquals(444, rs.getInt(2));
-            assertFalse(rs.next());
-            
-            // Variable length slot with unbounded ranges.
-            query = "SELECT * FROM " + tableName + 
-                    " WHERE a_integer = 1 AND a_string > 'ab' AND a_id = '123'";
-            stmt = conn.prepareStatement(query);
-            rs = stmt.executeQuery();
-            assertTrue(rs.next());
-            assertEquals(1, rs.getInt(1));
-            assertEquals("de", rs.getString(2));
-            assertEquals("123", rs.getString(3));
-            assertEquals("abc", rs.getString(4));
-            assertEquals(111, rs.getInt(5));
-            assertFalse(rs.next());
-
-        } finally {
-            conn.close();
-        }
-    }
-
-    @Test
-    public void testSelectValueWithNotFullyQualifiedWhereClause() throws Exception {
-        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
-        Connection conn = DriverManager.getConnection(getUrl(), props);
-        try {
-            String tableName = initTableValues(null);
-            
-            // Where without fully qualified key, point query.
-            String query = "SELECT * FROM " + tableName + " WHERE a_integer = ? AND a_string = ?";
-            PreparedStatement stmt = conn.prepareStatement(query);
-            
-            stmt.setInt(1, 1);
-            stmt.setString(2, "abc");
-            ResultSet rs = stmt.executeQuery();
-            assertTrue(rs.next());
-            assertEquals(1, rs.getInt(1));
-            assertEquals("abc", rs.getString(2));
-            assertEquals("456", rs.getString(3));
-            assertEquals("abc", rs.getString(4));
-            assertEquals(111, rs.getInt(5));
-            assertFalse(rs.next());
-            
-            // Where without fully qualified key, range query.
-            query = "SELECT * FROM " + tableName + " WHERE a_integer >= 2";
-            stmt = conn.prepareStatement(query);
-            rs = stmt.executeQuery();
-            assertTrue(rs.next());
-            assertEquals(2, rs.getInt(1));
-            assertEquals("abc", rs.getString(2));
-            assertEquals("123", rs.getString(3));
-            assertEquals("def", rs.getString(4));
-            assertEquals(222, rs.getInt(5));
-            
-            assertTrue(rs.next());
-            assertEquals(3, rs.getInt(1));
-            assertEquals("abc", rs.getString(2));
-            assertEquals("123", rs.getString(3));
-            assertEquals("ghi", rs.getString(4));
-            assertEquals(333, rs.getInt(5));
-            
-            assertTrue(rs.next());
-            assertEquals(4, rs.getInt(1));
-            assertEquals("abc", rs.getString(2));
-            assertEquals("123", rs.getString(3));
-            assertEquals("jkl", rs.getString(4));
-            assertEquals(444, rs.getInt(5));
-            assertFalse(rs.next());
-            
-            // With point query.
-            query = "SELECT a_string FROM " + tableName + " WHERE a_string = ?";
-            stmt = conn.prepareStatement(query);
-            stmt.setString(1, "de");
-            rs = stmt.executeQuery();
-            assertTrue(rs.next());
-            assertEquals("de", rs.getString(1));
-            assertFalse(rs.next());
-            
-            query = "SELECT a_id FROM " + tableName + " WHERE a_id = ?";
-            stmt = conn.prepareStatement(query);
-            stmt.setString(1, "456");
-            rs = stmt.executeQuery();
-            assertTrue(rs.next());
-            assertEquals("456", rs.getString(1));
-            assertFalse(rs.next());
-        } finally {
-            conn.close();
-        }
-    }
-
-    @Test
-    public void testSelectWithGroupBy() throws Exception {
-        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
-        Connection conn = DriverManager.getConnection(getUrl(), props);
-        try {
-            String tableName = initTableValues(null);
-            
-            String query = "SELECT a_integer FROM " + tableName + " GROUP BY a_integer";
-            PreparedStatement stmt = conn.prepareStatement(query);
-            ResultSet rs = stmt.executeQuery();
-            int count = 0;
-            while (rs.next()) {
-                count++;
-            }
-            assertEquals("Group by does not return the right count.", count, 4);
-        } finally {
-            conn.close();
-        }
-    }
 
-    @Test
-    public void testLimitScan() throws Exception {
-        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
-        Connection conn = DriverManager.getConnection(getUrl(), props);
-        try {
-            String tableName = initTableValues(null);
-            
-            String query = "SELECT a_integer FROM " + tableName + " WHERE a_string='abc' LIMIT 1";
-            PreparedStatement stmt = conn.prepareStatement(query);
-            ResultSet rs = stmt.executeQuery();
-            assertTrue(rs.next());
-            assertEquals(1, rs.getInt(1));
-            assertFalse(rs.next());
-        } finally {
-            conn.close();
-        }
-    }
-    
-    @Test
-    public void testSelectWithOrderByRowKey() throws Exception {
-        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
-        Connection conn = DriverManager.getConnection(getUrl(), props);
-        try {
-            String tableName = initTableValues(null);
-            
-            String query = "SELECT * FROM " + tableName + " ORDER  BY  a_integer, a_string, a_id";
-            PreparedStatement statement = conn.prepareStatement(query);
-            ResultSet explainPlan = statement.executeQuery("EXPLAIN " + query);
-            // Confirm that ORDER BY in row key order will be optimized out for salted table
-            assertEquals("CLIENT PARALLEL 4-WAY FULL SCAN OVER " + tableName + "\n" + 
-                    "CLIENT MERGE SORT", QueryUtil.getExplainPlan(explainPlan));
-            ResultSet rs = statement.executeQuery();
-            
-            assertTrue(rs.next());
-            assertEquals(1, rs.getInt(1));
-            assertEquals("ab", rs.getString(2));
-            assertEquals("123", rs.getString(3));
-            assertEquals("abc", rs.getString(4));
-            assertEquals(111, rs.getInt(5));
-            
-            assertTrue(rs.next());
-            assertEquals(1, rs.getInt(1));
-            assertEquals("abc", rs.getString(2));
-            assertEquals("456", rs.getString(3));
-            assertEquals("abc", rs.getString(4));
-            assertEquals(111, rs.getInt(5));
-            
-            assertTrue(rs.next());
-            assertEquals(1, rs.getInt(1));
-            assertEquals("de", rs.getString(2));
-            assertEquals("123", rs.getString(3));
-            assertEquals("abc", rs.getString(4));
-            assertEquals(111, rs.getInt(5));
-            
-            assertTrue(rs.next());
-            assertEquals(2, rs.getInt(1));
-            assertEquals("abc", rs.getString(2));
-            assertEquals("123", rs.getString(3));
-            assertEquals("def", rs.getString(4));
-            assertEquals(222, rs.getInt(5));
-            
-            assertTrue(rs.next());
-            assertEquals(3, rs.getInt(1));
-            assertEquals("abc", rs.getString(2));
-            assertEquals("123", rs.getString(3));
-            assertEquals("ghi", rs.getString(4));
-            assertEquals(333, rs.getInt(5));
-            
-            assertTrue(rs.next());
-            assertEquals(4, rs.getInt(1));
-            assertEquals("abc", rs.getString(2));
-            assertEquals("123", rs.getString(3));
-            assertEquals("jkl", rs.getString(4));
-            assertEquals(444, rs.getInt(5));
-            
-            assertFalse(rs.next());
-        } finally {
-            conn.close();
-        }
-    }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/c509d58f/phoenix-core/src/main/java/org/apache/phoenix/util/QueryBuilder.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/util/QueryBuilder.java b/phoenix-core/src/main/java/org/apache/phoenix/util/QueryBuilder.java
new file mode 100644
index 0000000..afce0dd
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/QueryBuilder.java
@@ -0,0 +1,211 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.phoenix.util;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.commons.lang.StringUtils;
+import org.apache.phoenix.parse.HintNode;
+
+import java.util.Collections;
+import java.util.List;
+
+import static org.apache.phoenix.util.SchemaUtil.getEscapedFullColumnName;
+
+public class QueryBuilder {
+
+    private String fullTableName;
+    // regular columns that are in the select clause
+    private List<String> selectColumns = Collections.emptyList();
+
+    // columns that are required for expressions in the select clause
+    private List<String> selectExpressionColumns  = Collections.emptyList();
+    // expression string in the select clause (for eg COL1 || COL2)
+    private String selectExpression;
+    private String whereClause;
+    private String orderByClause;
+    private String groupByClause;
+    private String havingClause;
+    private HintNode.Hint hint;
+    private boolean escapeCols;
+    private boolean distinct;
+    private int limit;
+
+    public String getFullTableName() {
+        return fullTableName;
+    }
+
+    /**
+     * @return column names required to evaluate this select statement
+     */
+    public List<String> getRequiredColumns() {
+        List<String> allColumns = Lists.newArrayList(selectColumns);
+        if (!CollectionUtils.isEmpty(selectExpressionColumns)) {
+            allColumns.addAll(selectExpressionColumns);
+        }
+        return allColumns;
+    }
+
+    public String getWhereClause() {
+        return whereClause;
+    }
+
+    public HintNode.Hint getHint() {
+        return hint;
+    }
+
+    public String getOrderByClause() {
+        return orderByClause;
+    }
+
+    public String getGroupByClause() {
+        return groupByClause;
+    }
+
+    public QueryBuilder setOrderByClause(String orderByClause) {
+        this.orderByClause = orderByClause;
+        return this;
+    }
+
+    public QueryBuilder setFullTableName(String fullTableName) {
+        this.fullTableName = fullTableName;
+        return this;
+    }
+
+    public QueryBuilder setSelectColumns(List<String> columns) {
+        this.selectColumns = columns;
+        return this;
+    }
+
+    public QueryBuilder setWhereClause(String whereClause) {
+        this.whereClause = whereClause;
+        return this;
+    }
+
+    public QueryBuilder setHint(HintNode.Hint hint) {
+        this.hint = hint;
+        return this;
+    }
+
+    public QueryBuilder setEscapeCols(boolean escapeCols) {
+        this.escapeCols = escapeCols;
+        return this;
+    }
+
+    public QueryBuilder setGroupByClause(String groupByClause) {
+        this.groupByClause = groupByClause;
+        return this;
+    }
+
+    public QueryBuilder setHavingClause(String havingClause) {
+        this.havingClause = havingClause;
+        return this;
+    }
+
+    public List<String> getSelectExpressionColumns() {
+        return selectExpressionColumns;
+    }
+
+    public QueryBuilder setSelectExpressionColumns(List<String> selectExpressionColumns) {
+        this.selectExpressionColumns = selectExpressionColumns;
+        return this;
+    }
+
+    public String getSelectExpression() {
+        return selectExpression;
+    }
+
+    public QueryBuilder setSelectExpression(String selectExpression) {
+        this.selectExpression = selectExpression;
+        return this;
+    }
+
+    public QueryBuilder setDistinct(boolean distinct) {
+        this.distinct = distinct;
+        return this;
+    }
+
+    public QueryBuilder setLimit(int limit) {
+        this.limit = limit;
+        return this;
+    }
+
+    public String build() {
+        Preconditions.checkNotNull(fullTableName, "Table name cannot be null");
+        if (CollectionUtils.isEmpty(selectColumns) && StringUtils.isBlank(selectExpression)) {
+            throw new IllegalArgumentException("At least one column or select expression must be provided");
+        }
+        StringBuilder query = new StringBuilder();
+        query.append("SELECT ");
+
+        if (distinct) {
+            query.append(" DISTINCT ");
+        }
+
+        if (hint != null) {
+            final HintNode node = new HintNode(hint.name());
+            String hintStr = node.toString();
+            query.append(hintStr);
+        }
+
+        StringBuilder selectClauseBuilder = new StringBuilder();
+        if (StringUtils.isNotBlank(selectExpression)) {
+            if (selectClauseBuilder.length()!=0) {
+                selectClauseBuilder.append(" , ");
+            }
+            selectClauseBuilder.append(selectExpression);
+        }
+
+        boolean first = true;
+        for (String col : selectColumns) {
+            if (StringUtils.isNotBlank(col)) {
+                if ((first && selectClauseBuilder.length()!=0) || !first) {
+                    selectClauseBuilder.append(" , ");
+                }
+                String fullColumnName = col;
+                if (escapeCols) {
+                    fullColumnName = getEscapedFullColumnName(col);
+                }
+                selectClauseBuilder.append(fullColumnName);
+                first = false;
+            }
+        }
+
+        query.append(selectClauseBuilder);
+        query.append(" FROM ");
+        query.append(fullTableName);
+        if (StringUtils.isNotBlank(whereClause)) {
+            query.append(" WHERE (").append(whereClause).append(")");
+        }
+        if (StringUtils.isNotBlank(groupByClause)) {
+            query.append(" GROUP BY ").append(groupByClause);
+        }
+        if (StringUtils.isNotBlank(havingClause)) {
+            query.append(" HAVING ").append(havingClause);
+        }
+        if (StringUtils.isNotBlank(orderByClause)) {
+            query.append(" ORDER BY ").append(orderByClause);
+        }
+        if (limit > 0) {
+            query.append(" LIMIT ").append(limit);
+        }
+        return query.toString();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/c509d58f/phoenix-core/src/main/java/org/apache/phoenix/util/QueryUtil.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/util/QueryUtil.java b/phoenix-core/src/main/java/org/apache/phoenix/util/QueryUtil.java
index 94cbfea..4501158 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/util/QueryUtil.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/QueryUtil.java
@@ -217,45 +217,15 @@ public final class QueryUtil {
      *
      * @param fullTableName name of the table for which the select statement needs to be created.
      * @param columns list of columns to be projected in the select statement.
-     * @param conditions The condition clause to be added to the WHERE condition
+     * @param whereClause The condition clause to be added to the WHERE condition
      * @param hint hint to use
      * @param escapeCols whether to escape the projected columns
      * @return Select Query
      */
     public static String constructSelectStatement(String fullTableName, List<String> columns,
-            final String conditions, Hint hint, boolean escapeCols) {
-        Preconditions.checkNotNull(fullTableName, "Table name cannot be null");
-        if (columns == null || columns.isEmpty()) {
-            throw new IllegalArgumentException("At least one column must be provided");
-        }
-        StringBuilder query = new StringBuilder();
-        query.append("SELECT ");
-
-        String hintStr = "";
-        if (hint != null) {
-            final HintNode node = new HintNode(hint.name());
-            hintStr = node.toString();
-        }
-        query.append(hintStr);
-
-        for (String col : columns) {
-            if (col != null) {
-                String fullColumnName = col;
-                if (escapeCols) {
-                    fullColumnName = getEscapedFullColumnName(col);
-                }
-                query.append(fullColumnName);
-                query.append(",");
-            }
-        }
-        // Remove the trailing comma
-        query.setLength(query.length() - 1);
-        query.append(" FROM ");
-        query.append(fullTableName);
-        if (conditions != null && conditions.length() > 0) {
-            query.append(" WHERE (").append(conditions).append(")");
-        }
-        return query.toString();
+            final String whereClause, Hint hint, boolean escapeCols) {
+        return new QueryBuilder().setFullTableName(fullTableName).setSelectColumns(columns)
+                .setWhereClause(whereClause).setHint(hint).setEscapeCols(escapeCols).build();
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/phoenix/blob/c509d58f/phoenix-core/src/test/java/org/apache/phoenix/mapreduce/index/IndexScrutinyTableOutputTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/mapreduce/index/IndexScrutinyTableOutputTest.java b/phoenix-core/src/test/java/org/apache/phoenix/mapreduce/index/IndexScrutinyTableOutputTest.java
index c6bb739..a904bca 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/mapreduce/index/IndexScrutinyTableOutputTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/mapreduce/index/IndexScrutinyTableOutputTest.java
@@ -47,7 +47,7 @@ public class IndexScrutinyTableOutputTest extends BaseIndexTest {
         String sqlStr =
                 IndexScrutinyTableOutput.getSqlQueryAllInvalidRows(conn, columnNames,
                     SCRUTINY_TIME_MILLIS);
-        assertEquals("SELECT \"SOURCE_TABLE\",\"TARGET_TABLE\",\"SCRUTINY_EXECUTE_TIME\",\"SOURCE_ROW_PK_HASH\",\"SOURCE_TS\",\"TARGET_TS\",\"HAS_TARGET_ROW\",\"ID\",\"PK_PART2\",\"NAME\",\"ZIP\",\":ID\",\":PK_PART2\",\"0:NAME\",\"0:ZIP\" FROM PHOENIX_INDEX_SCRUTINY(\"ID\" INTEGER,\"PK_PART2\" TINYINT,\"NAME\" VARCHAR,\"ZIP\" BIGINT,\":ID\" INTEGER,\":PK_PART2\" TINYINT,\"0:NAME\" VARCHAR,\"0:ZIP\" BIGINT) WHERE (\"SOURCE_TABLE\",\"TARGET_TABLE\",\"SCRUTINY_EXECUTE_TIME\") IN (('TEST_SCHEMA.TEST_INDEX_COLUMN_NAMES_UTIL','TEST_SCHEMA.TEST_ICN_INDEX',1502908914193))",
+        assertEquals("SELECT \"SOURCE_TABLE\" , \"TARGET_TABLE\" , \"SCRUTINY_EXECUTE_TIME\" , \"SOURCE_ROW_PK_HASH\" , \"SOURCE_TS\" , \"TARGET_TS\" , \"HAS_TARGET_ROW\" , \"ID\" , \"PK_PART2\" , \"NAME\" , \"ZIP\" , \":ID\" , \":PK_PART2\" , \"0:NAME\" , \"0:ZIP\" FROM PHOENIX_INDEX_SCRUTINY(\"ID\" INTEGER,\"PK_PART2\" TINYINT,\"NAME\" VARCHAR,\"ZIP\" BIGINT,\":ID\" INTEGER,\":PK_PART2\" TINYINT,\"0:NAME\" VARCHAR,\"0:ZIP\" BIGINT) WHERE (\"SOURCE_TABLE\",\"TARGET_TABLE\",\"SCRUTINY_EXECUTE_TIME\") IN (('TEST_SCHEMA.TEST_INDEX_COLUMN_NAMES_UTIL','TEST_SCHEMA.TEST_ICN_INDEX',1502908914193))",
             sqlStr);
     }
 
@@ -58,7 +58,7 @@ public class IndexScrutinyTableOutputTest extends BaseIndexTest {
         String query =
                 IndexScrutinyTableOutput.getSqlQueryMissingTargetRows(conn, columnNames,
                     SCRUTINY_TIME_MILLIS);
-        assertEquals("SELECT \"SOURCE_TABLE\",\"TARGET_TABLE\",\"SCRUTINY_EXECUTE_TIME\",\"SOURCE_ROW_PK_HASH\",\"SOURCE_TS\",\"TARGET_TS\",\"HAS_TARGET_ROW\",\"ID\",\"PK_PART2\",\"NAME\",\"ZIP\",\":ID\",\":PK_PART2\",\"0:NAME\",\"0:ZIP\" FROM PHOENIX_INDEX_SCRUTINY(\"ID\" INTEGER,\"PK_PART2\" TINYINT,\"NAME\" VARCHAR,\"ZIP\" BIGINT,\":ID\" INTEGER,\":PK_PART2\" TINYINT,\"0:NAME\" VARCHAR,\"0:ZIP\" BIGINT) WHERE (\"SOURCE_TABLE\",\"TARGET_TABLE\",\"SCRUTINY_EXECUTE_TIME\", \"HAS_TARGET_ROW\") IN (('TEST_SCHEMA.TEST_INDEX_COLUMN_NAMES_UTIL','TEST_SCHEMA.TEST_ICN_INDEX',1502908914193,false))",
+        assertEquals("SELECT \"SOURCE_TABLE\" , \"TARGET_TABLE\" , \"SCRUTINY_EXECUTE_TIME\" , \"SOURCE_ROW_PK_HASH\" , \"SOURCE_TS\" , \"TARGET_TS\" , \"HAS_TARGET_ROW\" , \"ID\" , \"PK_PART2\" , \"NAME\" , \"ZIP\" , \":ID\" , \":PK_PART2\" , \"0:NAME\" , \"0:ZIP\" FROM PHOENIX_INDEX_SCRUTINY(\"ID\" INTEGER,\"PK_PART2\" TINYINT,\"NAME\" VARCHAR,\"ZIP\" BIGINT,\":ID\" INTEGER,\":PK_PART2\" TINYINT,\"0:NAME\" VARCHAR,\"0:ZIP\" BIGINT) WHERE (\"SOURCE_TABLE\",\"TARGET_TABLE\",\"SCRUTINY_EXECUTE_TIME\", \"HAS_TARGET_ROW\") IN (('TEST_SCHEMA.TEST_INDEX_COLUMN_NAMES_UTIL','TEST_SCHEMA.TEST_ICN_INDEX',1502908914193,false))",
             query);
     }
 
@@ -69,7 +69,7 @@ public class IndexScrutinyTableOutputTest extends BaseIndexTest {
         String query =
                 IndexScrutinyTableOutput.getSqlQueryBadCoveredColVal(conn, columnNames,
                     SCRUTINY_TIME_MILLIS);
-        assertEquals("SELECT \"SOURCE_TABLE\",\"TARGET_TABLE\",\"SCRUTINY_EXECUTE_TIME\",\"SOURCE_ROW_PK_HASH\",\"SOURCE_TS\",\"TARGET_TS\",\"HAS_TARGET_ROW\",\"ID\",\"PK_PART2\",\"NAME\",\"ZIP\",\":ID\",\":PK_PART2\",\"0:NAME\",\"0:ZIP\" FROM PHOENIX_INDEX_SCRUTINY(\"ID\" INTEGER,\"PK_PART2\" TINYINT,\"NAME\" VARCHAR,\"ZIP\" BIGINT,\":ID\" INTEGER,\":PK_PART2\" TINYINT,\"0:NAME\" VARCHAR,\"0:ZIP\" BIGINT) WHERE (\"SOURCE_TABLE\",\"TARGET_TABLE\",\"SCRUTINY_EXECUTE_TIME\", \"HAS_TARGET_ROW\") IN (('TEST_SCHEMA.TEST_INDEX_COLUMN_NAMES_UTIL','TEST_SCHEMA.TEST_ICN_INDEX',1502908914193,true))",
+        assertEquals("SELECT \"SOURCE_TABLE\" , \"TARGET_TABLE\" , \"SCRUTINY_EXECUTE_TIME\" , \"SOURCE_ROW_PK_HASH\" , \"SOURCE_TS\" , \"TARGET_TS\" , \"HAS_TARGET_ROW\" , \"ID\" , \"PK_PART2\" , \"NAME\" , \"ZIP\" , \":ID\" , \":PK_PART2\" , \"0:NAME\" , \"0:ZIP\" FROM PHOENIX_INDEX_SCRUTINY(\"ID\" INTEGER,\"PK_PART2\" TINYINT,\"NAME\" VARCHAR,\"ZIP\" BIGINT,\":ID\" INTEGER,\":PK_PART2\" TINYINT,\"0:NAME\" VARCHAR,\"0:ZIP\" BIGINT) WHERE (\"SOURCE_TABLE\",\"TARGET_TABLE\",\"SCRUTINY_EXECUTE_TIME\", \"HAS_TARGET_ROW\") IN (('TEST_SCHEMA.TEST_INDEX_COLUMN_NAMES_UTIL','TEST_SCHEMA.TEST_ICN_INDEX',1502908914193,true))",
             query);
     }
 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/c509d58f/phoenix-core/src/test/java/org/apache/phoenix/mapreduce/util/PhoenixConfigurationUtilTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/mapreduce/util/PhoenixConfigurationUtilTest.java b/phoenix-core/src/test/java/org/apache/phoenix/mapreduce/util/PhoenixConfigurationUtilTest.java
index f864dd5..0c4c004 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/mapreduce/util/PhoenixConfigurationUtilTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/mapreduce/util/PhoenixConfigurationUtilTest.java
@@ -145,7 +145,7 @@ public class PhoenixConfigurationUtilTest extends BaseConnectionlessQueryTest {
             configuration.set(HConstants.ZOOKEEPER_QUORUM, getUrl());
             PhoenixConfigurationUtil.setInputTableName(configuration, tableName);
             final String selectStatement = PhoenixConfigurationUtil.getSelectStatement(configuration);
-            final String expectedSelectStatement = "SELECT \"A_STRING\",\"A_BINARY\",\"0\".\"COL1\" FROM " + tableName ; 
+            final String expectedSelectStatement = "SELECT \"A_STRING\" , \"A_BINARY\" , \"0\".\"COL1\" FROM " + tableName ;
             assertEquals(expectedSelectStatement, selectStatement);
         } finally {
             conn.close();
@@ -167,7 +167,7 @@ public class PhoenixConfigurationUtilTest extends BaseConnectionlessQueryTest {
             configuration.set(HConstants.ZOOKEEPER_QUORUM, getUrl());
             PhoenixConfigurationUtil.setInputTableName(configuration, fullTableName);
             final String selectStatement = PhoenixConfigurationUtil.getSelectStatement(configuration);
-            final String expectedSelectStatement = "SELECT \"A_STRING\",\"A_BINARY\",\"0\".\"COL1\" FROM " + fullTableName; 
+            final String expectedSelectStatement = "SELECT \"A_STRING\" , \"A_BINARY\" , \"0\".\"COL1\" FROM " + fullTableName;
             assertEquals(expectedSelectStatement, selectStatement);
         } finally {
             conn.close();
@@ -209,7 +209,7 @@ public class PhoenixConfigurationUtilTest extends BaseConnectionlessQueryTest {
             PhoenixConfigurationUtil.setSchemaType(configuration, SchemaType.QUERY);
             PhoenixConfigurationUtil.setInputTableName(configuration, tableName);
             final String selectStatement = PhoenixConfigurationUtil.getSelectStatement(configuration);
-            final String expectedSelectStatement = "SELECT \"ID\",\"0\".\"VCARRAY\" FROM " + tableName ; 
+            final String expectedSelectStatement = "SELECT \"ID\" , \"0\".\"VCARRAY\" FROM " + tableName ;
             assertEquals(expectedSelectStatement, selectStatement);
         } finally {
             conn.close();

http://git-wip-us.apache.org/repos/asf/phoenix/blob/c509d58f/phoenix-core/src/test/java/org/apache/phoenix/util/QueryUtilTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/util/QueryUtilTest.java b/phoenix-core/src/test/java/org/apache/phoenix/util/QueryUtilTest.java
index 2d094f6..8ee8f97 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/util/QueryUtilTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/util/QueryUtilTest.java
@@ -66,14 +66,14 @@ public class QueryUtilTest {
     @Test
     public void testConstructSelectStatement() {
         assertEquals(
-                "SELECT \"ID\",\"NAME\" FROM MYTAB",
+                "SELECT \"ID\" , \"NAME\" FROM MYTAB",
                 QueryUtil.constructSelectStatement("MYTAB", ImmutableList.of(ID_COLUMN,NAME_COLUMN),null));
     }
 
     @Test
     public void testConstructSelectStatementWithSchema() {
         assertEquals(
-                "SELECT \"ID\",\"NAME\" FROM A.MYTAB",
+                "SELECT \"ID\" , \"NAME\" FROM A.MYTAB",
                 QueryUtil.constructSelectStatement("A.MYTAB", ImmutableList.of(ID_COLUMN,NAME_COLUMN),null));
     }
     
@@ -83,7 +83,7 @@ public class QueryUtilTest {
         final String schemaName = SchemaUtil.getEscapedArgument("a");
         final String fullTableName = SchemaUtil.getTableName(schemaName, tableName);
         assertEquals(
-                "SELECT \"ID\",\"NAME\" FROM \"a\".MYTAB",
+                "SELECT \"ID\" , \"NAME\" FROM \"a\".MYTAB",
                 QueryUtil.constructSelectStatement(fullTableName, ImmutableList.of(ID_COLUMN,NAME_COLUMN),null));
     }
     
@@ -93,14 +93,14 @@ public class QueryUtilTest {
         final String schemaName = SchemaUtil.getEscapedArgument("a");
         final String fullTableName = SchemaUtil.getTableName(schemaName, tableName);
         assertEquals(
-                "SELECT \"ID\",\"NAME\" FROM \"a\".\"mytab\"",
+                "SELECT \"ID\" , \"NAME\" FROM \"a\".\"mytab\"",
                 QueryUtil.constructSelectStatement(fullTableName, ImmutableList.of(ID_COLUMN,NAME_COLUMN),null));
     }
 
     @Test
     public void testConstructSelectWithHint() {
         assertEquals(
-            "SELECT /*+ NO_INDEX */ \"col1\",\"col2\" FROM MYTAB WHERE (\"col2\"=? and \"col3\" is null)",
+            "SELECT /*+ NO_INDEX */ \"col1\" , \"col2\" FROM MYTAB WHERE (\"col2\"=? and \"col3\" is null)",
             QueryUtil.constructSelectStatement("MYTAB", Lists.newArrayList("col1", "col2"),
                 "\"col2\"=? and \"col3\" is null", Hint.NO_INDEX, true));
     }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/c509d58f/phoenix-spark/pom.xml
----------------------------------------------------------------------
diff --git a/phoenix-spark/pom.xml b/phoenix-spark/pom.xml
index 7e04bc1..cbf3808 100644
--- a/phoenix-spark/pom.xml
+++ b/phoenix-spark/pom.xml
@@ -488,6 +488,14 @@
     <testResources><testResource><directory>src/it/resources</directory></testResource></testResources>
     <plugins>
       <plugin>
+        <groupId>org.codehaus.mojo</groupId>
+        <artifactId>build-helper-maven-plugin</artifactId>
+      </plugin>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-failsafe-plugin</artifactId>
+      </plugin>
+      <plugin>
         <groupId>org.apache.maven.plugins</groupId>
         <artifactId>maven-compiler-plugin</artifactId>
       </plugin>

http://git-wip-us.apache.org/repos/asf/phoenix/blob/c509d58f/phoenix-spark/src/it/java/org/apache/phoenix/spark/AggregateIT.java
----------------------------------------------------------------------
diff --git a/phoenix-spark/src/it/java/org/apache/phoenix/spark/AggregateIT.java b/phoenix-spark/src/it/java/org/apache/phoenix/spark/AggregateIT.java
new file mode 100644
index 0000000..e4b96a3
--- /dev/null
+++ b/phoenix-spark/src/it/java/org/apache/phoenix/spark/AggregateIT.java
@@ -0,0 +1,91 @@
+/*
+ * 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.spark;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+
+import org.apache.phoenix.end2end.BaseAggregateIT;
+import org.apache.phoenix.util.QueryBuilder;
+
+public class AggregateIT extends BaseAggregateIT {
+
+    @Override
+    protected ResultSet executeQueryThrowsException(Connection conn, QueryBuilder queryBuilder,
+        String expectedPhoenixExceptionMsg, String expectedSparkExceptionMsg) {
+        ResultSet rs = null;
+        try {
+            rs = executeQuery(conn, queryBuilder);
+            fail();
+        }
+        catch(Exception e) {
+            assertTrue(e.getMessage().contains(expectedSparkExceptionMsg));
+        }
+        return rs;
+    }
+
+    @Override
+    protected ResultSet executeQuery(Connection conn, QueryBuilder queryBuilder) throws SQLException {
+        return SparkUtil.executeQuery(conn, queryBuilder, getUrl(), config);
+    }
+
+    @Override
+    protected void testCountNullInNonEmptyKeyValueCF(int columnEncodedBytes) throws Exception {
+        try (Connection conn = DriverManager.getConnection(getUrl())) {
+            //Type is INT
+            String intTableName=generateUniqueName();
+            String sql="create table " + intTableName + " (mykey integer not null primary key, A.COLA integer, B.COLB integer) "
+                    + "IMMUTABLE_ROWS=true, IMMUTABLE_STORAGE_SCHEME = ONE_CELL_PER_COLUMN, COLUMN_ENCODED_BYTES = " + columnEncodedBytes + ", DISABLE_WAL=true";
+
+            conn.createStatement().execute(sql);
+            conn.createStatement().execute("UPSERT INTO "+intTableName+" VALUES (1,1)");
+            conn.createStatement().execute("UPSERT INTO "+intTableName+" VALUES (2,1)");
+            conn.createStatement().execute("UPSERT INTO "+intTableName+" VALUES (3,1,2)");
+            conn.createStatement().execute("UPSERT INTO "+intTableName+" VALUES (4,1)");
+            conn.createStatement().execute("UPSERT INTO "+intTableName+" VALUES (5,1)");
+            conn.commit();
+
+            sql="select count(*) from "+intTableName;
+            QueryBuilder queryBuilder = new QueryBuilder()
+                .setSelectExpression("COUNT(*)")
+                .setFullTableName(intTableName);
+            ResultSet rs = executeQuery(conn, queryBuilder);
+            assertTrue(rs.next());
+            assertEquals(5, rs.getLong(1));
+
+            sql="select count(*) from "+intTableName + " where b.colb is not null";
+            queryBuilder.setWhereClause("`B.COLB` IS NOT NULL");
+            rs = executeQuery(conn, queryBuilder);
+            assertTrue(rs.next());
+            assertEquals(1, rs.getLong(1));
+
+            sql="select count(*) from "+intTableName + " where b.colb is null";
+            queryBuilder.setWhereClause("`B.COLB` IS NULL");
+            rs = executeQuery(conn, queryBuilder);
+            assertTrue(rs.next());
+            assertEquals(4, rs.getLong(1));
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/c509d58f/phoenix-spark/src/it/java/org/apache/phoenix/spark/OrderByIT.java
----------------------------------------------------------------------
diff --git a/phoenix-spark/src/it/java/org/apache/phoenix/spark/OrderByIT.java b/phoenix-spark/src/it/java/org/apache/phoenix/spark/OrderByIT.java
new file mode 100644
index 0000000..bdffaf5
--- /dev/null
+++ b/phoenix-spark/src/it/java/org/apache/phoenix/spark/OrderByIT.java
@@ -0,0 +1,460 @@
+package org.apache.phoenix.spark;
+
+import static org.apache.phoenix.util.TestUtil.TEST_PROPERTIES;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import java.sql.Connection;
+import java.sql.Date;
+import java.sql.DriverManager;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.List;
+import java.util.Properties;
+
+import org.apache.phoenix.end2end.BaseOrderByIT;
+import org.apache.phoenix.util.PropertiesUtil;
+import org.apache.phoenix.util.QueryBuilder;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SQLContext;
+import org.junit.Ignore;
+import org.junit.Test;
+
+import com.google.common.collect.Lists;
+
+import scala.Option;
+import scala.collection.JavaConverters;
+
+public class OrderByIT extends BaseOrderByIT {
+
+    @Override
+    protected ResultSet executeQueryThrowsException(Connection conn, QueryBuilder queryBuilder,
+                                                    String expectedPhoenixExceptionMsg, String expectedSparkExceptionMsg) {
+        ResultSet rs = null;
+        try {
+            rs = executeQuery(conn, queryBuilder);
+            fail();
+        }
+        catch(Exception e) {
+            assertTrue(e.getMessage().contains(expectedSparkExceptionMsg));
+        }
+        return rs;
+    }
+
+    @Override
+    protected ResultSet executeQuery(Connection conn, QueryBuilder queryBuilder) throws SQLException {
+        return SparkUtil.executeQuery(conn, queryBuilder, getUrl(), config);
+    }
+
+    @Test
+    public void testOrderByWithJoin() throws Exception {
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
+            conn.setAutoCommit(false);
+            String tableName1 = generateUniqueName();
+            String ddl = "CREATE TABLE " + tableName1 +
+                    "  (a_string varchar not null, cf1.a integer, cf1.b varchar, col1 integer, cf2.c varchar, cf2.d integer " +
+                    "  CONSTRAINT pk PRIMARY KEY (a_string))\n";
+            createTestTable(getUrl(), ddl);
+            String dml = "UPSERT INTO " + tableName1 + " VALUES(?,?,?,?,?,?)";
+            PreparedStatement stmt = conn.prepareStatement(dml);
+            stmt.setString(1, "a");
+            stmt.setInt(2, 40);
+            stmt.setString(3, "aa");
+            stmt.setInt(4, 10);
+            stmt.setString(5, "bb");
+            stmt.setInt(6, 20);
+            stmt.execute();
+            stmt.setString(1, "c");
+            stmt.setInt(2, 30);
+            stmt.setString(3, "cc");
+            stmt.setInt(4, 50);
+            stmt.setString(5, "dd");
+            stmt.setInt(6, 60);
+            stmt.execute();
+            stmt.setString(1, "b");
+            stmt.setInt(2, 40);
+            stmt.setString(3, "bb");
+            stmt.setInt(4, 5);
+            stmt.setString(5, "aa");
+            stmt.setInt(6, 80);
+            stmt.execute();
+            conn.commit();
+
+            String tableName2 = generateUniqueName();
+            ddl = "CREATE TABLE " + tableName2 +
+                    "  (a_string varchar not null, col1 integer" +
+                    "  CONSTRAINT pk PRIMARY KEY (a_string))\n";
+            createTestTable(getUrl(), ddl);
+
+            dml = "UPSERT INTO " + tableName2 + " VALUES(?, ?)";
+            stmt = conn.prepareStatement(dml);
+            stmt.setString(1, "a");
+            stmt.setInt(2, 40);
+            stmt.execute();
+            stmt.setString(1, "b");
+            stmt.setInt(2, 20);
+            stmt.execute();
+            stmt.setString(1, "c");
+            stmt.setInt(2, 30);
+            stmt.execute();
+            conn.commit();
+
+            // create two PhoenixRDDs  using the table names and columns that are required for the JOIN query
+            List<String> table1Columns = Lists.newArrayList("A_STRING", "CF1.A", "CF1.B", "COL1", "CF2.C", "CF2.D");
+            SQLContext sqlContext = SparkUtil.getSqlContext();
+            Dataset phoenixDataSet =
+                    new PhoenixRDD(SparkUtil.getSparkContext(), tableName1,
+                            JavaConverters.collectionAsScalaIterableConverter(table1Columns)
+                                    .asScala().toSeq(),
+                            Option.apply((String) null), Option.apply(getUrl()), config, false,
+                            null).toDataFrame(sqlContext);
+            phoenixDataSet.registerTempTable(tableName1);
+            List<String> table2Columns = Lists.newArrayList("A_STRING", "COL1");
+            phoenixDataSet =
+                    new PhoenixRDD(SparkUtil.getSparkContext(), tableName2,
+                            JavaConverters.collectionAsScalaIterableConverter(table2Columns)
+                                    .asScala().toSeq(),
+                            Option.apply((String) null), Option.apply(getUrl()), config, false,
+                            null).toDataFrame(sqlContext);
+            phoenixDataSet.registerTempTable(tableName2);
+
+            String query =
+                    "SELECT T1.* FROM " + tableName1 + " T1 JOIN " + tableName2
+                            + " T2 ON T1.A_STRING = T2.A_STRING ORDER BY T1.`CF1.B`";
+            Dataset<Row> dataset =
+                    sqlContext.sql(query);
+            List<Row> rows = dataset.collectAsList();
+            ResultSet rs = new SparkResultSet(rows, dataset.columns());
+
+            assertTrue(rs.next());
+            assertEquals("a",rs.getString(1));
+            assertEquals(40,rs.getInt(2));
+            assertEquals("aa",rs.getString(3));
+            assertEquals(10,rs.getInt(4));
+            assertEquals("bb",rs.getString(5));
+            assertEquals(20,rs.getInt(6));
+            assertTrue(rs.next());
+            assertEquals("b",rs.getString(1));
+            assertEquals(40,rs.getInt(2));
+            assertEquals("bb",rs.getString(3));
+            assertEquals(5,rs.getInt(4));
+            assertEquals("aa",rs.getString(5));
+            assertEquals(80,rs.getInt(6));
+            assertTrue(rs.next());
+            assertEquals("c",rs.getString(1));
+            assertEquals(30,rs.getInt(2));
+            assertEquals("cc",rs.getString(3));
+            assertEquals(50,rs.getInt(4));
+            assertEquals("dd",rs.getString(5));
+            assertEquals(60,rs.getInt(6));
+            assertFalse(rs.next());
+
+            query =
+                    "select t1.a_string, t2.col1 from " + tableName1 + " t1 join " + tableName2
+                            + " t2 on t1.a_string = t2.a_string order by t2.col1";
+            dataset =  sqlContext.sql(query);
+            rows = dataset.collectAsList();
+            rs = new SparkResultSet(rows, dataset.columns());
+            assertTrue(rs.next());
+            assertEquals("b",rs.getString(1));
+            assertEquals(20,rs.getInt(2));
+            assertTrue(rs.next());
+            assertEquals("c",rs.getString(1));
+            assertEquals(30,rs.getInt(2));
+            assertTrue(rs.next());
+            assertEquals("a",rs.getString(1));
+            assertEquals(40,rs.getInt(2));
+            assertFalse(rs.next());
+        }
+    }
+
+    @Test
+    public void testOrderByWithUnionAll() throws Exception {
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        try (Connection conn = DriverManager.getConnection(getUrl(), props)){
+            conn.setAutoCommit(false);
+            String tableName1 = generateUniqueName();
+            String ddl = "CREATE TABLE  " + tableName1 +
+                    "  (a_string varchar not null, cf1.a integer, cf1.b varchar, col1 integer, cf2.c varchar, cf2.d integer " +
+                    "  CONSTRAINT pk PRIMARY KEY (a_string))\n";
+            createTestTable(getUrl(), ddl);
+            String dml = "UPSERT INTO " + tableName1 + " VALUES(?,?,?,?,?,?)";
+            PreparedStatement stmt = conn.prepareStatement(dml);
+            stmt.setString(1, "a");
+            stmt.setInt(2, 40);
+            stmt.setString(3, "aa");
+            stmt.setInt(4, 10);
+            stmt.setString(5, "bb");
+            stmt.setInt(6, 20);
+            stmt.execute();
+            stmt.setString(1, "c");
+            stmt.setInt(2, 30);
+            stmt.setString(3, "cc");
+            stmt.setInt(4, 50);
+            stmt.setString(5, "dd");
+            stmt.setInt(6, 60);
+            stmt.execute();
+            stmt.setString(1, "b");
+            stmt.setInt(2, 40);
+            stmt.setString(3, "bb");
+            stmt.setInt(4, 5);
+            stmt.setString(5, "aa");
+            stmt.setInt(6, 80);
+            stmt.execute();
+            conn.commit();
+
+            String tableName2 = generateUniqueName();
+            ddl = "CREATE TABLE " + tableName2 +
+                    "  (a_string varchar not null, col1 integer" +
+                    "  CONSTRAINT pk PRIMARY KEY (a_string))\n";
+            createTestTable(getUrl(), ddl);
+
+            dml = "UPSERT INTO " + tableName2 + " VALUES(?, ?)";
+            stmt = conn.prepareStatement(dml);
+            stmt.setString(1, "aa");
+            stmt.setInt(2, 40);
+            stmt.execute();
+            stmt.setString(1, "bb");
+            stmt.setInt(2, 10);
+            stmt.execute();
+            stmt.setString(1, "cc");
+            stmt.setInt(2, 30);
+            stmt.execute();
+            conn.commit();
+
+
+            List<String> table1Columns = Lists.newArrayList("A_STRING", "CF1.A", "CF1.B", "COL1", "CF2.C", "CF2.D");
+            SQLContext sqlContext = SparkUtil.getSqlContext();
+            Dataset phoenixDataSet =
+                    new PhoenixRDD(SparkUtil.getSparkContext(), tableName1,
+                            JavaConverters.collectionAsScalaIterableConverter(table1Columns)
+                                    .asScala().toSeq(),
+                            Option.apply((String) null), Option.apply(getUrl()), config, false,
+                            null).toDataFrame(sqlContext);
+            phoenixDataSet.registerTempTable(tableName1);
+            List<String> table2Columns = Lists.newArrayList("A_STRING", "COL1");
+            phoenixDataSet =
+                    new PhoenixRDD(SparkUtil.getSparkContext(), tableName2,
+                            JavaConverters.collectionAsScalaIterableConverter(table2Columns)
+                                    .asScala().toSeq(),
+                            Option.apply((String) null), Option.apply(getUrl()), config, false,
+                            null).toDataFrame(sqlContext);
+            phoenixDataSet.registerTempTable(tableName2);
+
+            String query =
+                    "select a_string, `cf2.d` from " + tableName1 + " union all select * from "
+                            + tableName2 + " order by `cf2.d`";
+            Dataset<Row> dataset =
+                    sqlContext.sql(query);
+            List<Row> rows = dataset.collectAsList();
+            ResultSet rs = new SparkResultSet(rows, dataset.columns());
+            assertTrue(rs.next());
+            assertEquals("bb",rs.getString(1));
+            assertEquals(10,rs.getInt(2));
+            assertTrue(rs.next());
+            assertEquals("a",rs.getString(1));
+            assertEquals(20,rs.getInt(2));
+            assertTrue(rs.next());
+            assertEquals("cc",rs.getString(1));
+            assertEquals(30,rs.getInt(2));
+            assertTrue(rs.next());
+            assertEquals("aa",rs.getString(1));
+            assertEquals(40,rs.getInt(2));
+            assertTrue(rs.next());
+            assertEquals("c",rs.getString(1));
+            assertEquals(60,rs.getInt(2));
+            assertTrue(rs.next());
+            assertEquals("b",rs.getString(1));
+            assertEquals(80,rs.getInt(2));
+            assertFalse(rs.next());
+        }
+    }
+
+    @Test
+    public void testOrderByWithExpression() throws Exception {
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        Connection conn = DriverManager.getConnection(getUrl(), props);
+        conn.setAutoCommit(false);
+
+        try {
+            String tableName = generateUniqueName();
+            String ddl = "CREATE TABLE " + tableName +
+                    "  (a_string varchar not null, col1 integer, col2 integer, col3 timestamp, col4 varchar" +
+                    "  CONSTRAINT pk PRIMARY KEY (a_string))\n";
+            createTestTable(getUrl(), ddl);
+
+            Date date = new Date(System.currentTimeMillis());
+            String dml = "UPSERT INTO " + tableName + " VALUES(?, ?, ?, ?, ?)";
+            PreparedStatement stmt = conn.prepareStatement(dml);
+            stmt.setString(1, "a");
+            stmt.setInt(2, 40);
+            stmt.setInt(3, 20);
+            stmt.setDate(4, new Date(date.getTime()));
+            stmt.setString(5, "xxyy");
+            stmt.execute();
+            stmt.setString(1, "b");
+            stmt.setInt(2, 50);
+            stmt.setInt(3, 30);
+            stmt.setDate(4, new Date(date.getTime()-500));
+            stmt.setString(5, "yyzz");
+            stmt.execute();
+            stmt.setString(1, "c");
+            stmt.setInt(2, 60);
+            stmt.setInt(3, 20);
+            stmt.setDate(4, new Date(date.getTime()-300));
+            stmt.setString(5, "ddee");
+            stmt.execute();
+            conn.commit();
+
+            SQLContext sqlContext = SparkUtil.getSqlContext();
+            Dataset phoenixDataSet =
+                    new PhoenixRDD(SparkUtil.getSparkContext(), tableName,
+                            JavaConverters
+                                    .collectionAsScalaIterableConverter(
+                                        Lists.newArrayList("col1", "col2", "col4"))
+                                    .asScala().toSeq(),
+                            Option.apply((String) null), Option.apply(getUrl()), config, false,
+                            null).toDataFrame(sqlContext);
+
+            phoenixDataSet.registerTempTable(tableName);
+            Dataset<Row> dataset =
+                    sqlContext.sql("SELECT col1+col2, col4, a_string FROM " + tableName
+                            + " ORDER BY col1+col2, col4");
+            List<Row> rows = dataset.collectAsList();
+            ResultSet rs = new SparkResultSet(rows, dataset.columns());
+            assertTrue(rs.next());
+            assertEquals("a", rs.getString(3));
+            assertTrue(rs.next());
+            assertEquals("c", rs.getString(3));
+            assertTrue(rs.next());
+            assertEquals("b", rs.getString(3));
+            assertFalse(rs.next());
+        } catch (SQLException e) {
+        } finally {
+            conn.close();
+        }
+    }
+
+    @Test
+    public void testColumnFamily() throws Exception {
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
+            conn.setAutoCommit(false);
+            String tableName = generateUniqueName();
+            String ddl = "CREATE TABLE " + tableName +
+                    "  (a_string varchar not null, cf1.a integer, cf1.b varchar, col1 integer, cf2.c varchar, cf2.d integer, col2 integer" +
+                    "  CONSTRAINT pk PRIMARY KEY (a_string))\n";
+            createTestTable(getUrl(), ddl);
+            String dml = "UPSERT INTO " + tableName + " VALUES(?,?,?,?,?,?,?)";
+            PreparedStatement stmt = conn.prepareStatement(dml);
+            stmt.setString(1, "a");
+            stmt.setInt(2, 40);
+            stmt.setString(3, "aa");
+            stmt.setInt(4, 10);
+            stmt.setString(5, "bb");
+            stmt.setInt(6, 20);
+            stmt.setInt(7, 1);
+            stmt.execute();
+            stmt.setString(1, "c");
+            stmt.setInt(2, 30);
+            stmt.setString(3, "cc");
+            stmt.setInt(4, 50);
+            stmt.setString(5, "dd");
+            stmt.setInt(6, 60);
+            stmt.setInt(7, 3);
+            stmt.execute();
+            stmt.setString(1, "b");
+            stmt.setInt(2, 40);
+            stmt.setString(3, "bb");
+            stmt.setInt(4, 5);
+            stmt.setString(5, "aa");
+            stmt.setInt(6, 80);
+            stmt.setInt(7, 2);
+            stmt.execute();
+            conn.commit();
+
+
+            List<String> columns =
+                    Lists.newArrayList("A_STRING", "CF1.A", "CF1.B", "COL1", "CF2.C", "CF2.D",
+                        "COL2");
+
+            SQLContext sqlContext = SparkUtil.getSqlContext();
+            Dataset phoenixDataSet =
+                    new PhoenixRDD(SparkUtil.getSparkContext(), tableName,
+                            JavaConverters.collectionAsScalaIterableConverter(columns).asScala()
+                                    .toSeq(),
+                            Option.apply((String) null), Option.apply(url), config, false, null)
+                                    .toDataFrame(sqlContext);
+
+            phoenixDataSet.registerTempTable(tableName);
+            Dataset<Row> dataset =
+                    sqlContext.sql("SELECT A_STRING, `CF1.A`, `CF1.B`, COL1, `CF2.C`, `CF2.D`, COL2 from "
+                            + tableName + " ORDER BY `CF1.A`,`CF2.C`");
+            List<Row> rows = dataset.collectAsList();
+            ResultSet rs = new SparkResultSet(rows, dataset.columns());
+
+            assertTrue(rs.next());
+            assertEquals("c",rs.getString(1));
+            assertEquals(30,rs.getInt(2));
+            assertEquals("cc",rs.getString(3));
+            assertEquals(50,rs.getInt(4));
+            assertEquals("dd",rs.getString(5));
+            assertEquals(60,rs.getInt(6));
+            assertEquals(3,rs.getInt(7));
+            assertTrue(rs.next());
+            assertEquals("b",rs.getString(1));
+            assertEquals(40,rs.getInt(2));
+            assertEquals("bb",rs.getString(3));
+            assertEquals(5,rs.getInt(4));
+            assertEquals("aa",rs.getString(5));
+            assertEquals(80,rs.getInt(6));
+            assertEquals(2,rs.getInt(7));
+            assertTrue(rs.next());
+            assertEquals("a",rs.getString(1));
+            assertEquals(40,rs.getInt(2));
+            assertEquals("aa",rs.getString(3));
+            assertEquals(10,rs.getInt(4));
+            assertEquals("bb",rs.getString(5));
+            assertEquals(20,rs.getInt(6));
+            assertEquals(1,rs.getInt(7));
+            assertFalse(rs.next());
+
+            dataset =
+                    sqlContext.sql("SELECT A_STRING, `CF1.A`, `CF1.B`, COL1, `CF2.C`, `CF2.D`, COL2 from "
+                            + tableName + " ORDER BY COL2");
+            rows = dataset.collectAsList();
+            rs = new SparkResultSet(rows, dataset.columns());
+
+            assertTrue(rs.next());
+            assertEquals("a",rs.getString(1));
+            assertEquals(40,rs.getInt(2));
+            assertEquals("aa",rs.getString(3));
+            assertEquals(10,rs.getInt(4));
+            assertEquals("bb",rs.getString(5));
+            assertEquals(20,rs.getInt(6));
+            assertEquals(1,rs.getInt(7));
+            assertTrue(rs.next());
+            assertEquals("b",rs.getString(1));
+            assertEquals(40,rs.getInt(2));
+            assertEquals("bb",rs.getString(3));
+            assertEquals(5,rs.getInt(4));
+            assertEquals("aa",rs.getString(5));
+            assertEquals(80,rs.getInt(6));
+            assertEquals(2,rs.getInt(7));
+            assertTrue(rs.next());
+            assertEquals("c",rs.getString(1));
+            assertEquals(30,rs.getInt(2));
+            assertEquals("cc",rs.getString(3));
+            assertEquals(50,rs.getInt(4));
+            assertEquals("dd",rs.getString(5));
+            assertEquals(60,rs.getInt(6));
+            assertEquals(3,rs.getInt(7));
+            assertFalse(rs.next());
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/c509d58f/phoenix-spark/src/it/java/org/apache/phoenix/spark/SaltedTableIT.java
----------------------------------------------------------------------
diff --git a/phoenix-spark/src/it/java/org/apache/phoenix/spark/SaltedTableIT.java b/phoenix-spark/src/it/java/org/apache/phoenix/spark/SaltedTableIT.java
new file mode 100644
index 0000000..d72acbd
--- /dev/null
+++ b/phoenix-spark/src/it/java/org/apache/phoenix/spark/SaltedTableIT.java
@@ -0,0 +1,53 @@
+/*
+ * 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.spark;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import java.sql.Connection;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+
+import org.apache.phoenix.end2end.salted.BaseSaltedTableIT;
+import org.apache.phoenix.util.QueryBuilder;
+
+public class SaltedTableIT extends BaseSaltedTableIT {
+
+    @Override
+    protected ResultSet executeQueryThrowsException(Connection conn, QueryBuilder queryBuilder,
+                                                    String expectedPhoenixExceptionMsg, String expectedSparkExceptionMsg) {
+        ResultSet rs = null;
+        try {
+            rs = executeQuery(conn, queryBuilder);
+            fail();
+        }
+        catch(Exception e) {
+            assertTrue(e.getMessage().contains(expectedSparkExceptionMsg));
+        }
+        return rs;
+    }
+
+    @Override
+    protected ResultSet executeQuery(Connection conn, QueryBuilder queryBuilder) throws SQLException {
+        return SparkUtil.executeQuery(conn, queryBuilder, getUrl(), config);
+    }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/phoenix/blob/c509d58f/phoenix-spark/src/it/java/org/apache/phoenix/spark/SparkUtil.java
----------------------------------------------------------------------
diff --git a/phoenix-spark/src/it/java/org/apache/phoenix/spark/SparkUtil.java b/phoenix-spark/src/it/java/org/apache/phoenix/spark/SparkUtil.java
new file mode 100644
index 0000000..6285209
--- /dev/null
+++ b/phoenix-spark/src/it/java/org/apache/phoenix/spark/SparkUtil.java
@@ -0,0 +1,87 @@
+
+/*
+ * 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.spark;
+
+import com.google.common.base.Joiner;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.phoenix.jdbc.PhoenixConnection;
+import org.apache.phoenix.query.QueryServices;
+import org.apache.phoenix.util.QueryBuilder;
+import org.apache.spark.SparkContext;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SQLContext;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.execution.SparkPlan;
+import scala.Option;
+import scala.collection.JavaConverters;
+
+import java.sql.Connection;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.List;
+
+public class SparkUtil {
+
+    public static final String APP_NAME = "Java Spark Tests";
+    public static final String NUM_EXECUTORS = "local[2]";
+    public static final String UI_SHOW_CONSOLE_PROGRESS = "spark.ui.showConsoleProgress";
+
+    public static SparkContext getSparkContext() {
+        return SparkSession.builder().appName(APP_NAME).master(NUM_EXECUTORS)
+                .config(UI_SHOW_CONSOLE_PROGRESS, false).getOrCreate().sparkContext();
+    }
+
+    public static SQLContext getSqlContext() {
+        return SparkSession.builder().appName(APP_NAME).master(NUM_EXECUTORS)
+                .config(UI_SHOW_CONSOLE_PROGRESS, false).getOrCreate().sqlContext();
+    }
+
+    public static ResultSet executeQuery(Connection conn, QueryBuilder queryBuilder, String url, Configuration config)
+            throws SQLException {
+        SQLContext sqlContext = SparkUtil.getSqlContext();
+
+        boolean forceRowKeyOrder =
+                conn.unwrap(PhoenixConnection.class).getQueryServices().getProps()
+                        .getBoolean(QueryServices.FORCE_ROW_KEY_ORDER_ATTRIB, false);
+        // if we are forcing row key order we have to add an ORDER BY
+        // here we assume that the required columns are in the primary key column order
+        String prevOrderBy = queryBuilder.getOrderByClause();
+        if (forceRowKeyOrder &&  (queryBuilder.getOrderByClause()==null || queryBuilder.getOrderByClause().isEmpty())) {
+            queryBuilder.setOrderByClause(Joiner.on(", ").join(queryBuilder.getRequiredColumns()));
+        }
+
+        // create PhoenixRDD using the table name and columns that are required by the query
+        // since we don't set the predicate filtering is done after rows are returned from spark
+        Dataset phoenixDataSet =
+                new PhoenixRDD(SparkUtil.getSparkContext(), queryBuilder.getFullTableName(),
+                        JavaConverters.collectionAsScalaIterableConverter(queryBuilder.getRequiredColumns()).asScala()
+                                .toSeq(),
+                        Option.apply((String) null), Option.apply(url), config, false,
+                        null).toDataFrame(sqlContext);
+
+        phoenixDataSet.registerTempTable(queryBuilder.getFullTableName());
+        Dataset<Row> dataset = sqlContext.sql(queryBuilder.build());
+        SparkPlan plan = dataset.queryExecution().executedPlan();
+        List<Row> rows = dataset.collectAsList();
+        queryBuilder.setOrderByClause(prevOrderBy);
+        ResultSet rs = new SparkResultSet(rows, dataset.columns());
+        return rs;
+    }
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/c509d58f/phoenix-spark/src/it/scala/org/apache/phoenix/spark/PhoenixSparkIT.scala
----------------------------------------------------------------------
diff --git a/phoenix-spark/src/it/scala/org/apache/phoenix/spark/PhoenixSparkIT.scala b/phoenix-spark/src/it/scala/org/apache/phoenix/spark/PhoenixSparkIT.scala
index 4e11acc..d1e38fa 100644
--- a/phoenix-spark/src/it/scala/org/apache/phoenix/spark/PhoenixSparkIT.scala
+++ b/phoenix-spark/src/it/scala/org/apache/phoenix/spark/PhoenixSparkIT.scala
@@ -23,6 +23,7 @@ import org.joda.time.DateTime
 import org.apache.spark.{SparkConf, SparkContext}
 import scala.collection.mutable.ListBuffer
 import org.apache.hadoop.conf.Configuration
+
 /**
   * Note: If running directly from an IDE, these are the recommended VM parameters:
   * -Xmx1536m -XX:MaxPermSize=512m -XX:ReservedCodeCacheSize=512m
@@ -287,11 +288,11 @@ class PhoenixSparkIT extends AbstractPhoenixSparkIT {
 
     val plan = res.queryExecution.sparkPlan
     // filters should be pushed into phoenix relation
-    assert(plan.toString.contains("PushedFilters: [IsNotNull(COL1), IsNotNull(ID), " +
-      "EqualTo(COL1,test_row_1), EqualTo(ID,1)]"))
+    assert(plan.toString.contains("PushedFilters: [*IsNotNull(COL1), *IsNotNull(ID), " +
+      "*EqualTo(COL1,test_row_1), *EqualTo(ID,1)]"))
     // spark should run the filters on the rows returned by Phoenix
-    assert(!plan.toString.contains("Filter (((isnotnull(COL1#8) && isnotnull(ID#7L)) " +
-      "&& (COL1#8 = test_row_1)) && (ID#7L = 1))"))
+    assert(!plan.toString.matches(".*Filter (((isnotnull(COL1.*) && isnotnull(ID.*)) "
+      + " && (COL1.* = test_row_1)) && (ID.* = 1)).*"))
   }
 
   test("Can persist a dataframe using 'DataFrame.saveToPhoenix'") {


[6/6] phoenix git commit: PHOENIX-4981 Add tests for ORDER BY, GROUP BY and salted tables using phoenix-spark

Posted by td...@apache.org.
PHOENIX-4981 Add tests for ORDER BY, GROUP BY and salted tables using phoenix-spark


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

Branch: refs/heads/4.x-HBase-1.2
Commit: c509d58f12b73ba9e24b53d2e9ca0271666a400d
Parents: 0c3f433
Author: Thomas D'Silva <td...@apache.org>
Authored: Thu Oct 18 22:00:01 2018 -0700
Committer: Thomas D'Silva <td...@apache.org>
Committed: Tue Nov 6 14:52:08 2018 -0800

----------------------------------------------------------------------
 .../org/apache/phoenix/end2end/AggregateIT.java |  987 +---------------
 .../apache/phoenix/end2end/BaseAggregateIT.java | 1022 +++++++++++++++++
 .../apache/phoenix/end2end/BaseOrderByIT.java   |  940 ++++++++++++++++
 .../org/apache/phoenix/end2end/OrderByIT.java   |  943 ++--------------
 .../end2end/ParallelStatsDisabledIT.java        |   40 +
 .../end2end/salted/BaseSaltedTableIT.java       |  474 ++++++++
 .../phoenix/end2end/salted/SaltedTableIT.java   |  450 +-------
 .../org/apache/phoenix/util/QueryBuilder.java   |  211 ++++
 .../java/org/apache/phoenix/util/QueryUtil.java |   38 +-
 .../index/IndexScrutinyTableOutputTest.java     |    6 +-
 .../util/PhoenixConfigurationUtilTest.java      |    6 +-
 .../org/apache/phoenix/util/QueryUtilTest.java  |   10 +-
 phoenix-spark/pom.xml                           |    8 +
 .../org/apache/phoenix/spark/AggregateIT.java   |   91 ++
 .../org/apache/phoenix/spark/OrderByIT.java     |  460 ++++++++
 .../org/apache/phoenix/spark/SaltedTableIT.java |   53 +
 .../org/apache/phoenix/spark/SparkUtil.java     |   87 ++
 .../apache/phoenix/spark/PhoenixSparkIT.scala   |    9 +-
 .../apache/phoenix/spark/SparkResultSet.java    | 1056 ++++++++++++++++++
 .../org/apache/phoenix/spark/PhoenixRDD.scala   |   27 +-
 pom.xml                                         |    2 +-
 21 files changed, 4650 insertions(+), 2270 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/c509d58f/phoenix-core/src/it/java/org/apache/phoenix/end2end/AggregateIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/AggregateIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/AggregateIT.java
index 2059311..8916d4d 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/AggregateIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/AggregateIT.java
@@ -18,506 +18,28 @@
 package org.apache.phoenix.end2end;
 
 import static org.apache.phoenix.util.TestUtil.TEST_PROPERTIES;
+import static org.apache.phoenix.util.TestUtil.assertResultSet;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
-import static org.apache.phoenix.util.TestUtil.assertResultSet;
 
-import java.io.IOException;
 import java.sql.Connection;
 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.hadoop.hbase.util.Bytes;
-import org.apache.phoenix.compile.QueryPlan;
-import org.apache.phoenix.jdbc.PhoenixConnection;
 import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
-import org.apache.phoenix.jdbc.PhoenixStatement;
-import org.apache.phoenix.query.KeyRange;
 import org.apache.phoenix.schema.AmbiguousColumnException;
-import org.apache.phoenix.schema.types.PChar;
-import org.apache.phoenix.schema.types.PInteger;
-import org.apache.phoenix.util.ByteUtil;
 import org.apache.phoenix.util.PropertiesUtil;
-import org.apache.phoenix.util.QueryUtil;
+import org.apache.phoenix.util.QueryBuilder;
 import org.apache.phoenix.util.TestUtil;
 import org.junit.Test;
 
+public class AggregateIT extends BaseAggregateIT {
 
-public class AggregateIT extends ParallelStatsDisabledIT {
-    private static void initData(Connection conn, String tableName) throws SQLException {
-        conn.createStatement().execute("create table " + tableName +
-                "   (id varchar not null primary key,\n" +
-                "    uri varchar, appcpu integer)");
-        insertRow(conn, tableName, "Report1", 10, 1);
-        insertRow(conn, tableName, "Report2", 10, 2);
-        insertRow(conn, tableName, "Report3", 30, 3);
-        insertRow(conn, tableName, "Report4", 30, 4);
-        insertRow(conn, tableName, "SOQL1", 10, 5);
-        insertRow(conn, tableName, "SOQL2", 10, 6);
-        insertRow(conn, tableName, "SOQL3", 30, 7);
-        insertRow(conn, tableName, "SOQL4", 30, 8);
-        conn.commit();
-    }
-
-    private static void insertRow(Connection conn, String tableName, String uri, int appcpu, int id) throws SQLException {
-        PreparedStatement statement = conn.prepareStatement("UPSERT INTO " + tableName + "(id, uri, appcpu) values (?,?,?)");
-        statement.setString(1, "id" + id);
-        statement.setString(2, uri);
-        statement.setInt(3, appcpu);
-        statement.executeUpdate();
-    }
-
-    @Test
-    public void testDuplicateTrailingAggExpr() throws Exception {
-        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
-        Connection conn = DriverManager.getConnection(getUrl(), props);
-        String tableName = generateUniqueName();
-        conn.createStatement().execute("create table " + tableName +
-                "   (nam VARCHAR(20), address VARCHAR(20), id BIGINT "
-                + "constraint my_pk primary key (id))");
-        PreparedStatement statement = conn.prepareStatement("UPSERT INTO " + tableName + "(nam, address, id) values (?,?,?)");
-        statement.setString(1, "pulkit");
-        statement.setString(2, "badaun");
-        statement.setInt(3, 1);
-        statement.executeUpdate();
-        conn.commit();
-        Statement stmt = conn.createStatement();
-        ResultSet rs = stmt.executeQuery("select distinct 'harshit' as \"test_column\", trim(nam), trim(nam) from " + tableName);
-        assertTrue(rs.next());
-        assertEquals("harshit", rs.getString(1));
-        assertEquals("pulkit", rs.getString(2));
-        assertEquals("pulkit", rs.getString(3));
-        conn.close();
-    }
-
-    @Test
-    public void testExpressionInGroupBy() throws Exception {
-        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
-        Connection conn = DriverManager.getConnection(getUrl(), props);
-        String tableName = generateUniqueName();
-        String ddl = " create table " + tableName + "(tgb_id integer NOT NULL,utc_date_epoch integer NOT NULL,tgb_name varchar(40),ack_success_count integer" +
-                ",ack_success_one_ack_count integer, CONSTRAINT pk_tgb_counter PRIMARY KEY(tgb_id, utc_date_epoch))";
-        String query = "SELECT tgb_id, tgb_name, (utc_date_epoch/10)*10 AS utc_epoch_hour,SUM(ack_success_count + ack_success_one_ack_count) AS ack_tx_sum" +
-                " FROM " + tableName + " GROUP BY tgb_id, tgb_name, utc_epoch_hour";
-
-        createTestTable(getUrl(), ddl);
-        String dml = "UPSERT INTO " + tableName + " VALUES(?,?,?,?,?)";
-        PreparedStatement stmt = conn.prepareStatement(dml);
-        stmt.setInt(1, 1);
-        stmt.setInt(2, 1000);
-        stmt.setString(3, "aaa");
-        stmt.setInt(4, 1);
-        stmt.setInt(5, 1);
-        stmt.execute();
-        stmt.setInt(1, 2);
-        stmt.setInt(2, 2000);
-        stmt.setString(3, "bbb");
-        stmt.setInt(4, 2);
-        stmt.setInt(5, 2);
-        stmt.execute();
-        conn.commit();
-
-        ResultSet rs = conn.createStatement().executeQuery(query);
-        assertTrue(rs.next());
-        assertEquals(1,rs.getInt(1));
-        assertEquals("aaa",rs.getString(2));
-        assertEquals(1000,rs.getInt(3));
-        assertEquals(2,rs.getInt(4));
-        assertTrue(rs.next());
-        assertEquals(2,rs.getInt(1));
-        assertEquals("bbb",rs.getString(2));
-        assertEquals(2000,rs.getInt(3));
-        assertEquals(4,rs.getInt(4));
-        assertFalse(rs.next());
-        rs.close();
-        conn.close();
-    }
-    
-    @Test
-    public void testBooleanInGroupBy() throws Exception {
-        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
-        Connection conn = DriverManager.getConnection(getUrl(), props);
-        String tableName = generateUniqueName();
-        String ddl = " create table " + tableName + "(id varchar primary key,v1 boolean, v2 integer, v3 integer)";
-
-        createTestTable(getUrl(), ddl);
-        PreparedStatement stmt = conn.prepareStatement("UPSERT INTO " + tableName + "(id,v2,v3) VALUES(?,?,?)");
-        stmt.setString(1, "a");
-        stmt.setInt(2, 1);
-        stmt.setInt(3, 1);
-        stmt.execute();
-        stmt.close();
-        stmt = conn.prepareStatement("UPSERT INTO " + tableName + " VALUES(?,?,?,?)");
-        stmt.setString(1, "b");
-        stmt.setBoolean(2, false);
-        stmt.setInt(3, 2);
-        stmt.setInt(4, 2);
-        stmt.execute();
-        stmt.setString(1, "c");
-        stmt.setBoolean(2, true);
-        stmt.setInt(3, 3);
-        stmt.setInt(4, 3);
-        stmt.execute();
-        conn.commit();
-
-        String[] gbs = {"v1,v2,v3","v1,v3,v2","v2,v1,v3"};
-        for (String gb : gbs) {
-            ResultSet rs = conn.createStatement().executeQuery("SELECT v1, v2, v3 from " + tableName + " group by " + gb);
-            assertTrue(rs.next());
-            assertEquals(false,rs.getBoolean("v1"));
-            assertTrue(rs.wasNull());
-            assertEquals(1,rs.getInt("v2"));
-            assertEquals(1,rs.getInt("v3"));
-            assertTrue(rs.next());
-            assertEquals(false,rs.getBoolean("v1"));
-            assertFalse(rs.wasNull());
-            assertEquals(2,rs.getInt("v2"));
-            assertEquals(2,rs.getInt("v3"));
-            assertTrue(rs.next());
-            assertEquals(true,rs.getBoolean("v1"));
-            assertEquals(3,rs.getInt("v2"));
-            assertEquals(3,rs.getInt("v3"));
-            assertFalse(rs.next());
-            rs.close();
-        }
-        conn.close();
-    }
-    
-    @Test
-    public void testScanUri() throws Exception {
-        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
-        Connection conn = DriverManager.getConnection(getUrl(), props);
-        String tableName = generateUniqueName();
-        initData(conn, tableName);
-        Statement stmt = conn.createStatement();
-        ResultSet rs = stmt.executeQuery("select uri from " + tableName);
-        assertTrue(rs.next());
-        assertEquals("Report1", rs.getString(1));
-        assertTrue(rs.next());
-        assertEquals("Report2", rs.getString(1));
-        assertTrue(rs.next());
-        assertEquals("Report3", rs.getString(1));
-        assertTrue(rs.next());
-        assertEquals("Report4", rs.getString(1));
-        assertTrue(rs.next());
-        assertEquals("SOQL1", rs.getString(1));
-        assertTrue(rs.next());
-        assertEquals("SOQL2", rs.getString(1));
-        assertTrue(rs.next());
-        assertEquals("SOQL3", rs.getString(1));
-        assertTrue(rs.next());
-        assertEquals("SOQL4", rs.getString(1));
-        assertFalse(rs.next());
-        conn.close();
-    }
-
-    @Test
-    public void testCount() throws Exception {
-        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
-        Connection conn = DriverManager.getConnection(getUrl(), props);
-        String tableName = generateUniqueName();
-        initData(conn, tableName);
-        Statement stmt = conn.createStatement();
-        ResultSet rs = stmt.executeQuery("select count(1) from " + tableName);
-        assertTrue(rs.next());
-        assertEquals(8, rs.getInt(1));
-        assertFalse(rs.next());
-        conn.close();
-    }
-
-    @Test
-    public void testGroupByCase() throws Exception {
-        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
-        Connection conn = DriverManager.getConnection(getUrl(), props);
-        String tableName = generateUniqueName();
-        String groupBy1 = "select " +
-                "case when uri LIKE 'Report%' then 'Reports' else 'Other' END category" +
-                ", avg(appcpu) from " + tableName +
-                " group by category";
-
-        String groupBy2 = "select " +
-                "case uri when 'Report%' then 'Reports' else 'Other' END category" +
-                ", avg(appcpu) from " + tableName +
-                " group by appcpu, category";
-        
-        String groupBy3 = "select " +
-                "case uri when 'Report%' then 'Reports' else 'Other' END category" +
-                ", avg(appcpu) from " + tableName +
-                " group by avg(appcpu), category";
-        initData(conn, tableName);
-        conn.createStatement().executeQuery(groupBy1);
-        conn.createStatement().executeQuery(groupBy2);
-        // TODO: validate query results
-        try {
-            conn.createStatement().executeQuery(groupBy3);
-            fail();
-        } catch (SQLException e) {
-            assertTrue(e.getMessage().contains("Aggregate expressions may not be used in GROUP BY"));
-        }
-        conn.close();
-    }
-
-
-    @Test
-    public void testGroupByArray() throws Exception {
-        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
-        Connection conn = DriverManager.getConnection(getUrl(), props);
-
-        String tableName = generateUniqueName();
-        conn.createStatement().execute("CREATE TABLE " + tableName + "(\n" + 
-                "  a VARCHAR NOT NULL,\n" + 
-                "  b VARCHAR,\n" + 
-                "  c INTEGER,\n" + 
-                "  d VARCHAR,\n" + 
-                "  e VARCHAR ARRAY,\n" + 
-                "  f BIGINT,\n" + 
-                "  g BIGINT,\n" + 
-                "  CONSTRAINT pk PRIMARY KEY(a)\n" + 
-                ")");
-        conn.createStatement().execute("UPSERT INTO " + tableName + " VALUES('1', 'val', 100, 'a', ARRAY ['b'], 1, 2)");
-        conn.createStatement().execute("UPSERT INTO " + tableName + " VALUES('2', 'val', 100, 'a', ARRAY ['b'], 3, 4)");
-        conn.createStatement().execute("UPSERT INTO " + tableName + " VALUES('3', 'val', 100, 'a', ARRAY ['b','c'], 5, 6)");
-        conn.commit();
-        
-        ResultSet rs = conn.createStatement().executeQuery("SELECT c, SUM(f + g) AS sumone, d, e\n" + 
-                "FROM " + tableName + "\n" + 
-                "WHERE b = 'val'\n" + 
-                "  AND a IN ('1','2','3')\n" + 
-                "GROUP BY c, d, e\n" + 
-                "ORDER BY sumone DESC");
-        assertTrue(rs.next());
-        assertEquals(100, rs.getInt(1));
-        assertEquals(11, rs.getLong(2));
-        assertTrue(rs.next());
-        assertEquals(100, rs.getInt(1));
-        assertEquals(10, rs.getLong(2));
-        assertFalse(rs.next());
-        conn.close();
-    }
-    
-    @Test
-    public void testGroupByOrderPreserving() throws Exception {
-        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
-        Connection conn = DriverManager.getConnection(getUrl(), props);
-        String tableName = generateUniqueName();
-
-        conn.createStatement().execute("CREATE TABLE " + tableName + "(ORGANIZATION_ID char(15) not null, \n" + 
-                "JOURNEY_ID char(15) not null, \n" + 
-                "DATASOURCE SMALLINT not null, \n" + 
-                "MATCH_STATUS TINYINT not null, \n" + 
-                "EXTERNAL_DATASOURCE_KEY varchar(30), \n" + 
-                "ENTITY_ID char(15) not null, \n" + 
-                "CONSTRAINT PK PRIMARY KEY (\n" + 
-                "    ORGANIZATION_ID, \n" + 
-                "    JOURNEY_ID, \n" + 
-                "    DATASOURCE, \n" + 
-                "    MATCH_STATUS,\n" + 
-                "    EXTERNAL_DATASOURCE_KEY,\n" + 
-                "    ENTITY_ID))");
-        conn.createStatement().execute("UPSERT INTO " + tableName + " VALUES('000001111122222', '333334444455555', 0, 0, 'abc', '666667777788888')");
-        conn.createStatement().execute("UPSERT INTO " + tableName + " VALUES('000001111122222', '333334444455555', 0, 0, 'abcd', '666667777788889')");
-        conn.createStatement().execute("UPSERT INTO " + tableName + " VALUES('000001111122222', '333334444455555', 0, 0, 'abc', '666667777788899')");
-        conn.commit();
-        String query =
-                "SELECT COUNT(1), EXTERNAL_DATASOURCE_KEY As DUP_COUNT\n" + 
-                "    FROM " + tableName + " \n" + 
-                "   WHERE JOURNEY_ID='333334444455555' AND \n" + 
-                "                 DATASOURCE=0 AND MATCH_STATUS <= 1 and \n" + 
-                "                 ORGANIZATION_ID='000001111122222' \n" + 
-                "    GROUP BY MATCH_STATUS, EXTERNAL_DATASOURCE_KEY \n" + 
-                "    HAVING COUNT(1) > 1";
-        ResultSet rs = conn.createStatement().executeQuery(query);
-        assertTrue(rs.next());
-        assertEquals(2,rs.getInt(1));
-        assertEquals("abc", rs.getString(2));
-        assertFalse(rs.next());
-        
-        rs = conn.createStatement().executeQuery("EXPLAIN " + query);
-        assertEquals(
-                "CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + tableName + " ['000001111122222','333334444455555',0,*] - ['000001111122222','333334444455555',0,1]\n" + 
-                "    SERVER FILTER BY FIRST KEY ONLY\n" + 
-                "    SERVER AGGREGATE INTO ORDERED DISTINCT ROWS BY [MATCH_STATUS, EXTERNAL_DATASOURCE_KEY]\n" + 
-                "CLIENT FILTER BY COUNT(1) > 1",QueryUtil.getExplainPlan(rs));
-    }
-    
-    @Test
-    public void testGroupByOrderPreservingDescSort() throws Exception {
-        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
-        Connection conn = DriverManager.getConnection(getUrl(), props);
-        String tableName = generateUniqueName(); 
-        conn.createStatement().execute("CREATE TABLE " + tableName + " (k1 char(1) not null, k2 char(1) not null, constraint pk primary key (k1,k2)) split on ('ac','jc','nc')");
-        conn.createStatement().execute("UPSERT INTO " + tableName + " VALUES('a', 'a')");
-        conn.createStatement().execute("UPSERT INTO " + tableName + " VALUES('a', 'b')");
-        conn.createStatement().execute("UPSERT INTO " + tableName + " VALUES('a', 'c')");
-        conn.createStatement().execute("UPSERT INTO " + tableName + " VALUES('a', 'd')");
-        conn.createStatement().execute("UPSERT INTO " + tableName + " VALUES('j', 'a')");
-        conn.createStatement().execute("UPSERT INTO " + tableName + " VALUES('j', 'b')");
-        conn.createStatement().execute("UPSERT INTO " + tableName + " VALUES('j', 'c')");
-        conn.createStatement().execute("UPSERT INTO " + tableName + " VALUES('j', 'd')");
-        conn.createStatement().execute("UPSERT INTO " + tableName + " VALUES('n', 'a')");
-        conn.createStatement().execute("UPSERT INTO " + tableName + " VALUES('n', 'b')");
-        conn.createStatement().execute("UPSERT INTO " + tableName + " VALUES('n', 'c')");
-        conn.createStatement().execute("UPSERT INTO " + tableName + " VALUES('n', 'd')");
-        conn.commit();
-        String query = "SELECT k1,count(*) FROM " + tableName + " GROUP BY k1 ORDER BY k1 DESC";
-        ResultSet rs = conn.createStatement().executeQuery(query);
-        assertTrue(rs.next());
-        assertEquals("n", rs.getString(1));
-        assertEquals(4, rs.getInt(2));
-        assertTrue(rs.next());
-        assertEquals("j", rs.getString(1));
-        assertEquals(4, rs.getInt(2));
-        assertTrue(rs.next());
-        assertEquals("a", rs.getString(1));
-        assertEquals(4, rs.getInt(2));
-        assertFalse(rs.next());
-        rs = conn.createStatement().executeQuery("EXPLAIN " + query);
-        assertEquals(
-                "CLIENT PARALLEL 1-WAY REVERSE FULL SCAN OVER " + tableName + "\n" + 
-                "    SERVER FILTER BY FIRST KEY ONLY\n" + 
-                "    SERVER AGGREGATE INTO ORDERED DISTINCT ROWS BY [K1]", QueryUtil.getExplainPlan(rs));
-    }
-    
-    @Test
-    public void testSumGroupByOrderPreservingDesc() throws Exception {
-        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
-        Connection conn = DriverManager.getConnection(getUrl(), props);
-        String tableName = generateUniqueName();
-
-        PreparedStatement stmt = conn.prepareStatement("CREATE TABLE " + tableName + " (k1 char(1) not null, k2 integer not null, constraint pk primary key (k1,k2)) split on (?,?,?)");
-        stmt.setBytes(1, ByteUtil.concat(PChar.INSTANCE.toBytes("a"), PInteger.INSTANCE.toBytes(3)));
-        stmt.setBytes(2, ByteUtil.concat(PChar.INSTANCE.toBytes("j"), PInteger.INSTANCE.toBytes(3)));
-        stmt.setBytes(3, ByteUtil.concat(PChar.INSTANCE.toBytes("n"), PInteger.INSTANCE.toBytes(3)));
-        stmt.execute();
-        conn.createStatement().execute("UPSERT INTO " + tableName + " VALUES('a', 1)");
-        conn.createStatement().execute("UPSERT INTO " + tableName + " VALUES('a', 2)");
-        conn.createStatement().execute("UPSERT INTO " + tableName + " VALUES('a', 3)");
-        conn.createStatement().execute("UPSERT INTO " + tableName + " VALUES('a', 4)");
-        conn.createStatement().execute("UPSERT INTO " + tableName + " VALUES('b', 5)");
-        conn.createStatement().execute("UPSERT INTO " + tableName + " VALUES('j', 1)");
-        conn.createStatement().execute("UPSERT INTO " + tableName + " VALUES('j', 2)");
-        conn.createStatement().execute("UPSERT INTO " + tableName + " VALUES('j', 3)");
-        conn.createStatement().execute("UPSERT INTO " + tableName + " VALUES('j', 4)");
-        conn.createStatement().execute("UPSERT INTO " + tableName + " VALUES('n', 1)");
-        conn.createStatement().execute("UPSERT INTO " + tableName + " VALUES('n', 2)");
-        conn.createStatement().execute("UPSERT INTO " + tableName + " VALUES('n', 3)");
-        conn.createStatement().execute("UPSERT INTO " + tableName + " VALUES('n', 4)");
-        conn.commit();
-        String query = "SELECT k1,sum(k2) FROM " + tableName + " GROUP BY k1 ORDER BY k1 DESC";
-        ResultSet rs = conn.createStatement().executeQuery(query);
-        assertTrue(rs.next());
-        assertEquals("n", rs.getString(1));
-        assertEquals(10, rs.getInt(2));
-        assertTrue(rs.next());
-        assertEquals("j", rs.getString(1));
-        assertEquals(10, rs.getInt(2));
-        assertTrue(rs.next());
-        assertEquals("b", rs.getString(1));
-        assertEquals(5, rs.getInt(2));
-        assertTrue(rs.next());
-        assertEquals("a", rs.getString(1));
-        assertEquals(10, rs.getInt(2));
-        assertFalse(rs.next());
-        rs = conn.createStatement().executeQuery("EXPLAIN " + query);
-        assertEquals(
-                "CLIENT PARALLEL 1-WAY REVERSE FULL SCAN OVER " + tableName + "\n" + 
-                "    SERVER FILTER BY FIRST KEY ONLY\n" + 
-                "    SERVER AGGREGATE INTO ORDERED DISTINCT ROWS BY [K1]", QueryUtil.getExplainPlan(rs));
-    }
-
-    @Test
-    public void testAvgGroupByOrderPreservingWithStats() throws Exception {
-        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
-        Connection conn = DriverManager.getConnection(getUrl(), props);
-        String tableName = generateUniqueName();
-        ResultSet rs = conn.createStatement().executeQuery("SELECT COUNT(*) FROM " + "\"SYSTEM\".\"STATS\"" + " WHERE " + PhoenixDatabaseMetaData.PHYSICAL_NAME + " ='" + tableName + "'");
-        assertTrue(rs.next());
-        assertEquals(0,rs.getInt(1));
-        initAvgGroupTable(conn, tableName, PhoenixDatabaseMetaData.GUIDE_POSTS_WIDTH + "=20 ");
-        testAvgGroupByOrderPreserving(conn, tableName, 13);
-        rs = conn.createStatement().executeQuery("SELECT COUNT(*) FROM " + "\"SYSTEM\".\"STATS\"" + " WHERE " + PhoenixDatabaseMetaData.PHYSICAL_NAME + " ='" + tableName + "'");
-        assertTrue(rs.next());
-        assertEquals(13,rs.getInt(1));
-        conn.setAutoCommit(true);
-        conn.createStatement().execute("DELETE FROM " + "\"SYSTEM\".\"STATS\"");
-        rs = conn.createStatement().executeQuery("SELECT COUNT(*) FROM " + "\"SYSTEM\".\"STATS\"" + " WHERE " + PhoenixDatabaseMetaData.PHYSICAL_NAME + " ='" + tableName + "'");
-        assertTrue(rs.next());
-        assertEquals(0,rs.getInt(1));
-        TestUtil.doMajorCompaction(conn, tableName);
-        rs = conn.createStatement().executeQuery("SELECT COUNT(*) FROM " + "\"SYSTEM\".\"STATS\"" + " WHERE " + PhoenixDatabaseMetaData.PHYSICAL_NAME + " ='" + tableName + "'");
-        assertTrue(rs.next());
-        assertEquals(13,rs.getInt(1));
-        testAvgGroupByOrderPreserving(conn, tableName, 13);
-        conn.createStatement().execute("ALTER TABLE " + tableName + " SET " + PhoenixDatabaseMetaData.GUIDE_POSTS_WIDTH + "=100");
-        testAvgGroupByOrderPreserving(conn, tableName, 6);
-        conn.createStatement().execute("ALTER TABLE " + tableName + " SET " + PhoenixDatabaseMetaData.GUIDE_POSTS_WIDTH + "=null");
-        testAvgGroupByOrderPreserving(conn, tableName, 4);
-    }
-    
-    @Test
-    public void testAvgGroupByOrderPreservingWithNoStats() throws Exception {
-        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
-        Connection conn = DriverManager.getConnection(getUrl(), props);
-        String tableName = generateUniqueName();
-        initAvgGroupTable(conn, tableName, "");
-        testAvgGroupByOrderPreserving(conn, tableName, 4);
-    }
-    
-    private void initAvgGroupTable(Connection conn, String tableName, String tableProps) throws SQLException {
-        PreparedStatement stmt = conn.prepareStatement("CREATE TABLE " + tableName + " (k1 char(1) not null, k2 integer not null, constraint pk primary key (k1,k2)) " + tableProps + " split on (?,?,?)");
-        stmt.setBytes(1, ByteUtil.concat(PChar.INSTANCE.toBytes("a"), PInteger.INSTANCE.toBytes(3)));
-        stmt.setBytes(2, ByteUtil.concat(PChar.INSTANCE.toBytes("j"), PInteger.INSTANCE.toBytes(3)));
-        stmt.setBytes(3, ByteUtil.concat(PChar.INSTANCE.toBytes("n"), PInteger.INSTANCE.toBytes(3)));
-        stmt.execute();
-        conn.createStatement().execute("UPSERT INTO " + tableName + " VALUES('a', 1)");
-        conn.createStatement().execute("UPSERT INTO " + tableName + " VALUES('a', 2)");
-        conn.createStatement().execute("UPSERT INTO " + tableName + " VALUES('a', 3)");
-        conn.createStatement().execute("UPSERT INTO " + tableName + " VALUES('a', 6)");
-        conn.createStatement().execute("UPSERT INTO " + tableName + " VALUES('b', 5)");
-        conn.createStatement().execute("UPSERT INTO " + tableName + " VALUES('j', 1)");
-        conn.createStatement().execute("UPSERT INTO " + tableName + " VALUES('j', 2)");
-        conn.createStatement().execute("UPSERT INTO " + tableName + " VALUES('j', 3)");
-        conn.createStatement().execute("UPSERT INTO " + tableName + " VALUES('j', 10)");
-        conn.createStatement().execute("UPSERT INTO " + tableName + " VALUES('n', 1)");
-        conn.createStatement().execute("UPSERT INTO " + tableName + " VALUES('n', 2)");
-        conn.createStatement().execute("UPSERT INTO " + tableName + " VALUES('n', 3)");
-        conn.createStatement().execute("UPSERT INTO " + tableName + " VALUES('n', 2)");
-        conn.commit();
-    }
-    
-    private void testAvgGroupByOrderPreserving(Connection conn, String tableName, int nGuidePosts) throws SQLException, IOException {
-        String query = "SELECT k1,avg(k2) FROM " + tableName + " GROUP BY k1";
-        ResultSet rs = conn.createStatement().executeQuery(query);
-        assertTrue(rs.next());
-        assertEquals("a", rs.getString(1));
-        assertEquals(3, rs.getInt(2));
-        assertTrue(rs.next());
-        assertEquals("b", rs.getString(1));
-        assertEquals(5, rs.getInt(2));
-        assertTrue(rs.next());
-        assertEquals("j", rs.getString(1));
-        assertEquals(4, rs.getInt(2));
-        assertTrue(rs.next());
-        assertEquals("n", rs.getString(1));
-        assertEquals(2, rs.getInt(2));
-        assertFalse(rs.next());
-        rs = conn.createStatement().executeQuery("EXPLAIN " + query);
-        assertEquals(
-                "CLIENT PARALLEL 1-WAY FULL SCAN OVER " + tableName + "\n" + 
-                "    SERVER FILTER BY FIRST KEY ONLY\n" + 
-                "    SERVER AGGREGATE INTO ORDERED DISTINCT ROWS BY [K1]", QueryUtil.getExplainPlan(rs));
-        TestUtil.analyzeTable(conn, tableName);
-        List<KeyRange> splits = TestUtil.getAllSplits(conn, tableName);
-        assertEquals(nGuidePosts, splits.size());
-    }
-    
     @Test
     public void testGroupByWithAliasWithSameColumnName() throws SQLException {
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
@@ -557,344 +79,6 @@ public class AggregateIT extends ParallelStatsDisabledIT {
     }
 
     @Test
-    public void testDistinctGroupByBug3452WithoutMultiTenant() throws Exception {
-        doTestDistinctGroupByBug3452("");
-    }
-
-    @Test
-    public void testDistinctGroupByBug3452WithMultiTenant() throws Exception {
-        doTestDistinctGroupByBug3452("VERSIONS=1, MULTI_TENANT=TRUE, REPLICATION_SCOPE=1, TTL=31536000");
-    }
-
-    private void doTestDistinctGroupByBug3452(String options) throws Exception {
-        Connection conn=null;
-        try {
-            Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
-            conn = DriverManager.getConnection(getUrl(), props);
-
-            String tableName=generateUniqueName();
-            conn.createStatement().execute("DROP TABLE if exists "+tableName);
-            String sql="CREATE TABLE "+ tableName +" ( "+
-                    "ORGANIZATION_ID CHAR(15) NOT NULL,"+
-                    "CONTAINER_ID CHAR(15) NOT NULL,"+
-                    "ENTITY_ID CHAR(15) NOT NULL,"+
-                    "SCORE DOUBLE,"+
-                    "CONSTRAINT TEST_PK PRIMARY KEY ( "+
-                    "ORGANIZATION_ID,"+
-                    "CONTAINER_ID,"+
-                    "ENTITY_ID"+
-                    ")) "+options;
-            conn.createStatement().execute(sql);
-
-            String indexTableName=generateUniqueName();
-            conn.createStatement().execute("DROP INDEX IF EXISTS "+indexTableName+" ON "+tableName);
-            conn.createStatement().execute("CREATE INDEX "+indexTableName+" ON "+tableName+" (CONTAINER_ID, SCORE DESC, ENTITY_ID DESC)");
-
-            conn.createStatement().execute("UPSERT INTO "+tableName+" VALUES ('org1','container1','entityId6',1.1)");
-            conn.createStatement().execute("UPSERT INTO "+tableName+" VALUES ('org1','container1','entityId5',1.2)");
-            conn.createStatement().execute("UPSERT INTO "+tableName+" VALUES ('org1','container1','entityId4',1.3)");
-            conn.createStatement().execute("UPSERT INTO "+tableName+" VALUES ('org1','container1','entityId3',1.4)");
-            conn.createStatement().execute("UPSERT INTO "+tableName+" VALUES ('org1','container1','entityId2',1.5)");
-            conn.createStatement().execute("UPSERT INTO "+tableName+" VALUES ('org1','container1','entityId1',1.6)");
-            conn.commit();
-
-            sql="SELECT DISTINCT entity_id,score FROM "+tableName+" WHERE organization_id = 'org1' AND container_id = 'container1' ORDER BY score DESC";
-            ResultSet rs=conn.createStatement().executeQuery(sql);
-            assertTrue(rs.next());
-            assertTrue(rs.getString(1).equals("entityId1"));
-            assertEquals(rs.getDouble(2),1.6,0.0001);
-
-            assertTrue(rs.next());
-            assertTrue(rs.getString(1).equals("entityId2"));
-            assertEquals(rs.getDouble(2),1.5,0.0001);
-
-            assertTrue(rs.next());
-            assertTrue(rs.getString(1).equals("entityId3"));
-            assertEquals(rs.getDouble(2),1.4,0.0001);
-
-            assertTrue(rs.next());
-            assertTrue(rs.getString(1).equals("entityId4"));
-            assertEquals(rs.getDouble(2),1.3,0.0001);
-
-            assertTrue(rs.next());
-            assertTrue(rs.getString(1).equals("entityId5"));
-            assertEquals(rs.getDouble(2),1.2,0.0001);
-
-            assertTrue(rs.next());
-            assertTrue(rs.getString(1).equals("entityId6"));
-            assertEquals(rs.getDouble(2),1.1,0.0001);
-            assertTrue(!rs.next());
-        } finally {
-            if(conn!=null) {
-                conn.close();
-            }
-        }
-    }
-
-    @Test
-    public void testGroupByOrderByDescBug3451() throws Exception {
-        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
-        try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
-            String tableName=generateUniqueName();
-            String sql="CREATE TABLE " + tableName + " (\n" + 
-                    "            ORGANIZATION_ID CHAR(15) NOT NULL,\n" + 
-                    "            CONTAINER_ID CHAR(15) NOT NULL,\n" + 
-                    "            ENTITY_ID CHAR(15) NOT NULL,\n" + 
-                    "            SCORE DOUBLE,\n" + 
-                    "            CONSTRAINT TEST_PK PRIMARY KEY (\n" + 
-                    "               ORGANIZATION_ID,\n" + 
-                    "               CONTAINER_ID,\n" + 
-                    "               ENTITY_ID\n" + 
-                    "             )\n" + 
-                    "         )";
-            conn.createStatement().execute(sql);
-            String indexName=generateUniqueName();
-            conn.createStatement().execute("CREATE INDEX " + indexName + " ON " + tableName + "(ORGANIZATION_ID,CONTAINER_ID, SCORE DESC, ENTITY_ID DESC)");
-            conn.createStatement().execute("UPSERT INTO " + tableName + " VALUES ('org2','container2','entityId6',1.1)");
-            conn.createStatement().execute("UPSERT INTO " + tableName + " VALUES ('org2','container1','entityId5',1.2)");
-            conn.createStatement().execute("UPSERT INTO " + tableName + " VALUES ('org2','container2','entityId4',1.3)");
-            conn.createStatement().execute("UPSERT INTO " + tableName + " VALUES ('org2','container1','entityId5',1.2)");
-            conn.createStatement().execute("UPSERT INTO " + tableName + " VALUES ('org2','container1','entityId3',1.4)");
-            conn.createStatement().execute("UPSERT INTO " + tableName + " VALUES ('org2','container3','entityId7',1.35)");
-            conn.createStatement().execute("UPSERT INTO " + tableName + " VALUES ('org2','container3','entityId8',1.45)");
-            conn.commit();
-            String query = "SELECT DISTINCT entity_id, score\n" + 
-                    "    FROM " + tableName + "\n" +
-                    "    WHERE organization_id = 'org2'\n" + 
-                    "    AND container_id IN ( 'container1','container2','container3' )\n" + 
-                    "    ORDER BY score DESC\n" + 
-                    "    LIMIT 2";
-            Statement stmt = conn.createStatement();
-            ResultSet rs = stmt.executeQuery(query);
-            QueryPlan plan = stmt.unwrap(PhoenixStatement.class).getQueryPlan();
-            assertEquals(indexName, plan.getContext().getCurrentTable().getTable().getName().getString());
-            assertFalse(plan.getOrderBy().getOrderByExpressions().isEmpty());
-            assertTrue(rs.next());
-            assertEquals("entityId8", rs.getString(1));
-            assertEquals(1.45, rs.getDouble(2),0.001);
-            assertTrue(rs.next());
-            assertEquals("entityId3", rs.getString(1));
-            assertEquals(1.4, rs.getDouble(2),0.001);
-            assertFalse(rs.next());
-       }
-    }
-    
-    @Test
-    public void testGroupByDescColumnWithNullsLastBug3452() throws Exception {
-
-        Connection conn=null;
-        try
-        {
-            Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
-            conn = DriverManager.getConnection(getUrl(), props);
-
-            String tableName=generateUniqueName();
-            String sql="CREATE TABLE "+tableName+" ( "+
-                    "ORGANIZATION_ID VARCHAR,"+
-                    "CONTAINER_ID VARCHAR,"+
-                    "ENTITY_ID VARCHAR NOT NULL,"+
-                    "CONSTRAINT TEST_PK PRIMARY KEY ( "+
-                    "ORGANIZATION_ID DESC,"+
-                    "CONTAINER_ID DESC,"+
-                    "ENTITY_ID"+
-                    "))";
-            conn.createStatement().execute(sql);
-
-            conn.createStatement().execute("UPSERT INTO "+tableName+" VALUES ('a',null,'11')");
-            conn.createStatement().execute("UPSERT INTO "+tableName+" VALUES (null,'2','22')");
-            conn.createStatement().execute("UPSERT INTO "+tableName+" VALUES ('c','3','33')");
-            conn.commit();
-
-            //-----ORGANIZATION_ID
-
-            sql="SELECT CONTAINER_ID,ORGANIZATION_ID FROM "+tableName+" group by ORGANIZATION_ID, CONTAINER_ID order by ORGANIZATION_ID ASC NULLS FIRST";
-            ResultSet rs=conn.prepareStatement(sql).executeQuery();
-            assertResultSet(rs, new String[][]{{"2",null},{null,"a"},{"3","c"},});
-
-            sql="SELECT CONTAINER_ID,ORGANIZATION_ID FROM "+tableName+" group by ORGANIZATION_ID, CONTAINER_ID order by ORGANIZATION_ID ASC NULLS LAST";
-            rs=conn.prepareStatement(sql).executeQuery();
-            assertResultSet(rs, new String[][]{{null,"a"},{"3","c"},{"2",null}});
-
-            sql="SELECT CONTAINER_ID,ORGANIZATION_ID FROM "+tableName+" group by ORGANIZATION_ID, CONTAINER_ID order by ORGANIZATION_ID DESC NULLS FIRST";
-            rs=conn.prepareStatement(sql).executeQuery();
-            assertResultSet(rs, new String[][]{{"2",null},{"3","c"},{null,"a"}});
-
-            sql="SELECT CONTAINER_ID,ORGANIZATION_ID FROM "+tableName+" group by ORGANIZATION_ID, CONTAINER_ID order by ORGANIZATION_ID DESC NULLS LAST";
-            rs=conn.prepareStatement(sql).executeQuery();
-            assertResultSet(rs, new String[][]{{"3","c"},{null,"a"},{"2",null}});
-
-            //----CONTAINER_ID
-
-            sql="SELECT CONTAINER_ID,ORGANIZATION_ID FROM "+tableName+" group by ORGANIZATION_ID, CONTAINER_ID order by CONTAINER_ID ASC NULLS FIRST";
-            rs=conn.prepareStatement(sql).executeQuery();
-            assertResultSet(rs, new String[][]{{null,"a"},{"2",null},{"3","c"}});
-
-            sql="SELECT CONTAINER_ID,ORGANIZATION_ID FROM "+tableName+" group by ORGANIZATION_ID, CONTAINER_ID order by CONTAINER_ID ASC NULLS LAST";
-            rs=conn.prepareStatement(sql).executeQuery();
-            assertResultSet(rs, new String[][]{{"2",null},{"3","c"},{null,"a"}});
-
-            sql="SELECT CONTAINER_ID,ORGANIZATION_ID FROM "+tableName+" group by ORGANIZATION_ID, CONTAINER_ID order by CONTAINER_ID DESC NULLS FIRST";
-            rs=conn.prepareStatement(sql).executeQuery();
-            assertResultSet(rs, new String[][]{{null,"a"},{"3","c"},{"2",null}});
-
-            sql="SELECT CONTAINER_ID,ORGANIZATION_ID FROM "+tableName+" group by ORGANIZATION_ID, CONTAINER_ID order by CONTAINER_ID DESC NULLS LAST";
-            rs=conn.prepareStatement(sql).executeQuery();
-            assertResultSet(rs, new String[][]{{"3","c"},{"2",null},{null,"a"}});
-
-            conn.createStatement().execute("UPSERT INTO "+tableName+" VALUES (null,null,'44')");
-            conn.commit();
-
-            //-----ORGANIZATION_ID ASC  CONTAINER_ID ASC
-
-            sql="SELECT CONTAINER_ID,ORGANIZATION_ID FROM "+tableName+" group by ORGANIZATION_ID, CONTAINER_ID order by ORGANIZATION_ID NULLS FIRST,CONTAINER_ID NULLS FIRST";
-            rs=conn.prepareStatement(sql).executeQuery();
-            assertResultSet(rs, new String[][]{{null,null},{"2",null},{null,"a"},{"3","c"}});
-
-            sql="SELECT CONTAINER_ID,ORGANIZATION_ID FROM "+tableName+" group by ORGANIZATION_ID, CONTAINER_ID order by ORGANIZATION_ID NULLS FIRST,CONTAINER_ID NULLS LAST";
-            rs=conn.prepareStatement(sql).executeQuery();
-            assertResultSet(rs, new String[][]{{"2",null},{null,null},{null,"a"},{"3","c"}});
-
-            sql="SELECT CONTAINER_ID,ORGANIZATION_ID FROM "+tableName+" group by ORGANIZATION_ID, CONTAINER_ID order by ORGANIZATION_ID NULLS LAST,CONTAINER_ID NULLS FIRST";
-            rs=conn.prepareStatement(sql).executeQuery();
-            assertResultSet(rs, new String[][]{{null,"a"},{"3","c"},{null,null},{"2",null}});
-
-            sql="SELECT CONTAINER_ID,ORGANIZATION_ID FROM "+tableName+" group by ORGANIZATION_ID, CONTAINER_ID order by ORGANIZATION_ID NULLS LAST,CONTAINER_ID NULLS LAST";
-            rs=conn.prepareStatement(sql).executeQuery();
-            assertResultSet(rs, new String[][]{{null,"a"},{"3","c"},{"2",null},{null,null}});
-
-            //-----ORGANIZATION_ID ASC  CONTAINER_ID DESC
-
-            sql="SELECT CONTAINER_ID,ORGANIZATION_ID FROM "+tableName+" group by ORGANIZATION_ID, CONTAINER_ID order by ORGANIZATION_ID NULLS FIRST,CONTAINER_ID DESC NULLS FIRST";
-            rs=conn.prepareStatement(sql).executeQuery();
-            assertResultSet(rs, new String[][]{{null,null},{"2",null},{null,"a"},{"3","c"}});
-
-            sql="SELECT CONTAINER_ID,ORGANIZATION_ID FROM "+tableName+" group by ORGANIZATION_ID, CONTAINER_ID order by ORGANIZATION_ID NULLS FIRST,CONTAINER_ID DESC NULLS LAST";
-            rs=conn.prepareStatement(sql).executeQuery();
-            assertResultSet(rs, new String[][]{{"2",null},{null,null},{null,"a"},{"3","c"}});
-
-            sql="SELECT CONTAINER_ID,ORGANIZATION_ID FROM "+tableName+" group by ORGANIZATION_ID, CONTAINER_ID order by ORGANIZATION_ID NULLS LAST,CONTAINER_ID DESC NULLS FIRST";
-            rs=conn.prepareStatement(sql).executeQuery();
-            assertResultSet(rs, new String[][]{{null,"a"},{"3","c"},{null,null},{"2",null}});
-
-            sql="SELECT CONTAINER_ID,ORGANIZATION_ID FROM "+tableName+" group by ORGANIZATION_ID, CONTAINER_ID order by ORGANIZATION_ID NULLS LAST,CONTAINER_ID DESC NULLS LAST";
-            rs=conn.prepareStatement(sql).executeQuery();
-            assertResultSet(rs, new String[][]{{null,"a"},{"3","c"},{"2",null},{null,null}});
-
-            //-----ORGANIZATION_ID DESC  CONTAINER_ID ASC
-
-            sql="SELECT CONTAINER_ID,ORGANIZATION_ID FROM "+tableName+" group by ORGANIZATION_ID, CONTAINER_ID order by ORGANIZATION_ID DESC NULLS FIRST,CONTAINER_ID NULLS FIRST";
-            rs=conn.prepareStatement(sql).executeQuery();
-            assertResultSet(rs, new String[][]{{null,null},{"2",null},{"3","c"},{null,"a"}});
-
-            sql="SELECT CONTAINER_ID,ORGANIZATION_ID FROM "+tableName+" group by ORGANIZATION_ID, CONTAINER_ID order by ORGANIZATION_ID DESC NULLS FIRST,CONTAINER_ID NULLS LAST";
-            rs=conn.prepareStatement(sql).executeQuery();
-            assertResultSet(rs, new String[][]{{"2",null},{null,null},{"3","c"},{null,"a"}});
-
-            sql="SELECT CONTAINER_ID,ORGANIZATION_ID FROM "+tableName+" group by ORGANIZATION_ID, CONTAINER_ID order by ORGANIZATION_ID DESC NULLS LAST,CONTAINER_ID NULLS FIRST";
-            rs=conn.prepareStatement(sql).executeQuery();
-            assertResultSet(rs, new String[][]{{"3","c"},{null,"a"},{null,null},{"2",null}});
-
-            sql="SELECT CONTAINER_ID,ORGANIZATION_ID FROM "+tableName+" group by ORGANIZATION_ID, CONTAINER_ID order by ORGANIZATION_ID DESC NULLS LAST,CONTAINER_ID NULLS LAST";
-            rs=conn.prepareStatement(sql).executeQuery();
-            assertResultSet(rs, new String[][]{{"3","c"},{null,"a"},{"2",null},{null,null}});
-
-            //-----ORGANIZATION_ID DESC  CONTAINER_ID DESC
-
-            sql="SELECT CONTAINER_ID,ORGANIZATION_ID FROM "+tableName+" group by ORGANIZATION_ID, CONTAINER_ID order by ORGANIZATION_ID DESC NULLS FIRST,CONTAINER_ID DESC NULLS FIRST";
-            rs=conn.prepareStatement(sql).executeQuery();
-            assertResultSet(rs, new String[][]{{null,null},{"2",null},{"3","c"},{null,"a"}});
-
-            sql="SELECT CONTAINER_ID,ORGANIZATION_ID FROM "+tableName+" group by ORGANIZATION_ID, CONTAINER_ID order by ORGANIZATION_ID DESC NULLS FIRST,CONTAINER_ID DESC NULLS LAST";
-            rs=conn.prepareStatement(sql).executeQuery();
-            assertResultSet(rs, new String[][]{{"2",null},{null,null},{"3","c"},{null,"a"}});
-
-            sql="SELECT CONTAINER_ID,ORGANIZATION_ID FROM "+tableName+" group by ORGANIZATION_ID, CONTAINER_ID order by ORGANIZATION_ID DESC NULLS LAST,CONTAINER_ID DESC NULLS FIRST";
-            rs=conn.prepareStatement(sql).executeQuery();
-            assertResultSet(rs, new String[][]{{"3","c"},{null,"a"},{null,null},{"2",null}});
-
-            sql="SELECT CONTAINER_ID,ORGANIZATION_ID FROM "+tableName+" group by ORGANIZATION_ID, CONTAINER_ID order by ORGANIZATION_ID DESC NULLS LAST,CONTAINER_ID DESC NULLS LAST";
-            rs=conn.prepareStatement(sql).executeQuery();
-            assertResultSet(rs, new String[][]{{"3","c"},{null,"a"},{"2",null},{null,null}});
-
-            //-----CONTAINER_ID ASC  ORGANIZATION_ID ASC
-
-            sql="SELECT CONTAINER_ID,ORGANIZATION_ID FROM "+tableName+" group by ORGANIZATION_ID, CONTAINER_ID order by CONTAINER_ID NULLS FIRST,ORGANIZATION_ID NULLS FIRST";
-            rs=conn.prepareStatement(sql).executeQuery();
-            assertResultSet(rs, new String[][]{{null,null},{null,"a"},{"2",null},{"3","c"}});
-
-            sql="SELECT CONTAINER_ID,ORGANIZATION_ID FROM "+tableName+" group by ORGANIZATION_ID, CONTAINER_ID order by CONTAINER_ID NULLS FIRST,ORGANIZATION_ID NULLS LAST";
-            rs=conn.prepareStatement(sql).executeQuery();
-            assertResultSet(rs, new String[][]{{null,"a"},{null,null},{"2",null},{"3","c"}});
-
-            sql="SELECT CONTAINER_ID,ORGANIZATION_ID FROM "+tableName+" group by ORGANIZATION_ID, CONTAINER_ID order by CONTAINER_ID NULLS LAST,ORGANIZATION_ID NULLS FIRST";
-            rs=conn.prepareStatement(sql).executeQuery();
-            assertResultSet(rs, new String[][]{{"2",null},{"3","c"},{null,null},{null,"a"}});
-
-            sql="SELECT CONTAINER_ID,ORGANIZATION_ID FROM "+tableName+" group by ORGANIZATION_ID, CONTAINER_ID order by CONTAINER_ID NULLS LAST,ORGANIZATION_ID NULLS LAST";
-            rs=conn.prepareStatement(sql).executeQuery();
-            assertResultSet(rs, new String[][]{{"2",null},{"3","c"},{null,"a"},{null,null}});
-
-            //-----CONTAINER_ID ASC  ORGANIZATION_ID DESC
-
-            sql="SELECT CONTAINER_ID,ORGANIZATION_ID FROM "+tableName+" group by ORGANIZATION_ID, CONTAINER_ID order by CONTAINER_ID ASC NULLS FIRST,ORGANIZATION_ID DESC NULLS FIRST";
-            rs=conn.prepareStatement(sql).executeQuery();
-            assertResultSet(rs, new String[][]{{null,null},{null,"a"},{"2",null},{"3","c"}});
-
-            sql="SELECT CONTAINER_ID,ORGANIZATION_ID FROM "+tableName+" group by ORGANIZATION_ID, CONTAINER_ID order by CONTAINER_ID ASC NULLS FIRST,ORGANIZATION_ID DESC NULLS LAST";
-            rs=conn.prepareStatement(sql).executeQuery();
-            assertResultSet(rs, new String[][]{{null,"a"},{null,null},{"2",null},{"3","c"}});
-
-            sql="SELECT CONTAINER_ID,ORGANIZATION_ID FROM "+tableName+" group by ORGANIZATION_ID, CONTAINER_ID order by CONTAINER_ID ASC NULLS LAST,ORGANIZATION_ID DESC NULLS FIRST";
-            rs=conn.prepareStatement(sql).executeQuery();
-            assertResultSet(rs, new String[][]{{"2",null},{"3","c"},{null,null},{null,"a"}});
-
-            sql="SELECT CONTAINER_ID,ORGANIZATION_ID FROM "+tableName+" group by ORGANIZATION_ID, CONTAINER_ID  order by CONTAINER_ID ASC NULLS LAST,ORGANIZATION_ID DESC NULLS LAST";
-            rs=conn.prepareStatement(sql).executeQuery();
-            assertResultSet(rs, new String[][]{{"2",null},{"3","c"},{null,"a"},{null,null}});
-
-            //-----CONTAINER_ID DESC  ORGANIZATION_ID ASC
-
-            sql="SELECT CONTAINER_ID,ORGANIZATION_ID FROM "+tableName+" group by ORGANIZATION_ID, CONTAINER_ID order by CONTAINER_ID DESC NULLS FIRST,ORGANIZATION_ID ASC NULLS FIRST";
-            rs=conn.prepareStatement(sql).executeQuery();
-            assertResultSet(rs, new String[][]{{null,null},{null,"a"},{"3","c"},{"2",null}});
-
-            sql="SELECT CONTAINER_ID,ORGANIZATION_ID FROM "+tableName+" group by ORGANIZATION_ID, CONTAINER_ID order by CONTAINER_ID DESC NULLS FIRST,ORGANIZATION_ID ASC NULLS LAST";
-            rs=conn.prepareStatement(sql).executeQuery();
-            assertResultSet(rs, new String[][]{{null,"a"},{null,null},{"3","c"},{"2",null}});
-
-            sql="SELECT CONTAINER_ID,ORGANIZATION_ID FROM "+tableName+" group by ORGANIZATION_ID, CONTAINER_ID order by CONTAINER_ID DESC NULLS LAST,ORGANIZATION_ID ASC NULLS FIRST";
-            rs=conn.prepareStatement(sql).executeQuery();
-            assertResultSet(rs, new String[][]{{"3","c"},{"2",null},{null,null},{null,"a"}});
-
-            sql="SELECT CONTAINER_ID,ORGANIZATION_ID FROM "+tableName+" group by ORGANIZATION_ID, CONTAINER_ID order by CONTAINER_ID DESC NULLS LAST,ORGANIZATION_ID ASC NULLS LAST";
-            rs=conn.prepareStatement(sql).executeQuery();
-            assertResultSet(rs, new String[][]{{"3","c"},{"2",null},{null,"a"},{null,null}});
-
-            //-----CONTAINER_ID DESC  ORGANIZATION_ID DESC
-
-            sql="SELECT CONTAINER_ID,ORGANIZATION_ID FROM "+tableName+" group by ORGANIZATION_ID, CONTAINER_ID order by CONTAINER_ID DESC NULLS FIRST,ORGANIZATION_ID DESC NULLS FIRST";
-            rs=conn.prepareStatement(sql).executeQuery();
-            assertResultSet(rs, new String[][]{{null,null},{null,"a"},{"3","c"},{"2",null}});
-
-            sql="SELECT CONTAINER_ID,ORGANIZATION_ID FROM "+tableName+" group by ORGANIZATION_ID, CONTAINER_ID order by CONTAINER_ID DESC NULLS FIRST,ORGANIZATION_ID DESC NULLS LAST";
-            rs=conn.prepareStatement(sql).executeQuery();
-            assertResultSet(rs, new String[][]{{null,"a"},{null,null},{"3","c"},{"2",null}});
-
-            sql="SELECT CONTAINER_ID,ORGANIZATION_ID FROM "+tableName+" group by ORGANIZATION_ID, CONTAINER_ID order by CONTAINER_ID DESC NULLS LAST,ORGANIZATION_ID DESC NULLS FIRST";
-            rs=conn.prepareStatement(sql).executeQuery();
-            assertResultSet(rs, new String[][]{{"3","c"},{"2",null},{null,null},{null,"a"}});
-
-            sql="SELECT CONTAINER_ID,ORGANIZATION_ID FROM "+tableName+" group by ORGANIZATION_ID, CONTAINER_ID order by CONTAINER_ID DESC NULLS LAST,ORGANIZATION_ID DESC NULLS LAST";
-            rs=conn.prepareStatement(sql).executeQuery();
-            assertResultSet(rs, new String[][]{{"3","c"},{"2",null},{null,"a"},{null,null}});
-        } finally {
-            if(conn!=null) {
-                conn.close();
-            }
-        }
-    }
-
-    @Test
     public void testGroupByCoerceExpressionBug3453() throws Exception {
         final Connection conn = DriverManager.getConnection(getUrl());
         try {
@@ -955,16 +139,6 @@ public class AggregateIT extends ParallelStatsDisabledIT {
     }
 
     @Test
-    public void testCountNullInEncodedNonEmptyKeyValueCF() throws Exception {
-        testCountNullInNonEmptyKeyValueCF(1);
-    }
-    
-    @Test
-    public void testCountNullInNonEncodedNonEmptyKeyValueCF() throws Exception {
-        testCountNullInNonEmptyKeyValueCF(0);
-    }
-
-    @Test
     public void testNestedGroupedAggregationWithBigInt() throws Exception {
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         String tableName = generateUniqueName();
@@ -983,7 +157,41 @@ public class AggregateIT extends ParallelStatsDisabledIT {
         }
     }
 
-    private void testCountNullInNonEmptyKeyValueCF(int columnEncodedBytes) throws Exception {
+    @Test
+    public void testAvgGroupByOrderPreservingWithStats() throws Exception {
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        Connection conn = DriverManager.getConnection(getUrl(), props);
+        String tableName = generateUniqueName();
+        QueryBuilder queryBuilder = new QueryBuilder()
+            .setSelectExpression("COUNT(*)")
+            .setFullTableName(PhoenixDatabaseMetaData.SYSTEM_STATS_NAME)
+            .setWhereClause(PhoenixDatabaseMetaData.PHYSICAL_NAME + " ='" + tableName + "'");
+        ResultSet rs = executeQuery(conn, queryBuilder);
+        assertTrue(rs.next());
+        assertEquals(0,rs.getInt(1));
+        initAvgGroupTable(conn, tableName, PhoenixDatabaseMetaData.GUIDE_POSTS_WIDTH + "=20 ");
+        testAvgGroupByOrderPreserving(conn, tableName, 13);
+        rs = executeQuery(conn, queryBuilder);
+        assertTrue(rs.next());
+        assertEquals(13,rs.getInt(1));
+        conn.setAutoCommit(true);
+        conn.createStatement().execute("DELETE FROM " + "\"SYSTEM\".\"STATS\"");
+        rs = executeQuery(conn, queryBuilder);
+        assertTrue(rs.next());
+        assertEquals(0,rs.getInt(1));
+        TestUtil.doMajorCompaction(conn, tableName);
+        rs = executeQuery(conn, queryBuilder);
+        assertTrue(rs.next());
+        assertEquals(13,rs.getInt(1));
+        testAvgGroupByOrderPreserving(conn, tableName, 13);
+        conn.createStatement().execute("ALTER TABLE " + tableName + " SET " + PhoenixDatabaseMetaData.GUIDE_POSTS_WIDTH + "=100");
+        testAvgGroupByOrderPreserving(conn, tableName, 6);
+        conn.createStatement().execute("ALTER TABLE " + tableName + " SET " + PhoenixDatabaseMetaData.GUIDE_POSTS_WIDTH + "=null");
+        testAvgGroupByOrderPreserving(conn, tableName, 4);
+    }
+
+    @Override
+    protected void testCountNullInNonEmptyKeyValueCF(int columnEncodedBytes) throws Exception {
         try (Connection conn = DriverManager.getConnection(getUrl())) {
             //Type is INT
             String intTableName=generateUniqueName();
@@ -998,119 +206,26 @@ public class AggregateIT extends ParallelStatsDisabledIT {
             conn.createStatement().execute("UPSERT INTO "+intTableName+" VALUES (5,1)");
             conn.commit();
 
-            TestUtil.dumpTable(conn.unwrap(PhoenixConnection.class).getQueryServices().getTable(Bytes.toBytes(intTableName)));
-
             sql="select count(*) from "+intTableName;
-            ResultSet rs=conn.createStatement().executeQuery(sql);
+            QueryBuilder queryBuilder = new QueryBuilder()
+                .setSelectExpression("COUNT(*)")
+                .setFullTableName(intTableName);
+            ResultSet rs = executeQuery(conn, queryBuilder);
             assertTrue(rs.next());
-            assertEquals(5, rs.getInt(1));
-            
+            assertEquals(5, rs.getLong(1));
+
             sql="select count(*) from "+intTableName + " where b.colb is not null";
-            rs=conn.createStatement().executeQuery(sql);
+            queryBuilder.setWhereClause("B.COLB IS NOT NULL");
+            rs = executeQuery(conn, queryBuilder);
             assertTrue(rs.next());
-            assertEquals(1, rs.getInt(1));
+            assertEquals(1, rs.getLong(1));
 
             sql="select count(*) from "+intTableName + " where b.colb is null";
-            rs=conn.createStatement().executeQuery(sql);
+            queryBuilder.setWhereClause("B.COLB IS NULL");
+            rs = executeQuery(conn, queryBuilder);
             assertTrue(rs.next());
-            assertEquals(4, rs.getInt(1));
-        }
-    }
-
-    @Test
-    public void testGroupByOrderMatchPkColumnOrderBug4690() throws Exception {
-        this.doTestGroupByOrderMatchPkColumnOrderBug4690(false, false);
-        this.doTestGroupByOrderMatchPkColumnOrderBug4690(false, true);
-        this.doTestGroupByOrderMatchPkColumnOrderBug4690(true, false);
-        this.doTestGroupByOrderMatchPkColumnOrderBug4690(true, true);
-    }
-
-    private void doTestGroupByOrderMatchPkColumnOrderBug4690(boolean desc ,boolean salted) throws Exception {
-        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
-        Connection conn = null;
-        try {
-            conn = DriverManager.getConnection(getUrl(), props);
-            String tableName = generateUniqueName();
-            String sql = "create table " + tableName + "( "+
-                    " pk1 integer not null , " +
-                    " pk2 integer not null, " +
-                    " pk3 integer not null," +
-                    " pk4 integer not null,"+
-                    " v integer, " +
-                    " CONSTRAINT TEST_PK PRIMARY KEY ( "+
-                       "pk1 "+(desc ? "desc" : "")+", "+
-                       "pk2 "+(desc ? "desc" : "")+", "+
-                       "pk3 "+(desc ? "desc" : "")+", "+
-                       "pk4 "+(desc ? "desc" : "")+
-                    " )) "+(salted ? "SALT_BUCKETS =4" : "split on(2)");
-            conn.createStatement().execute(sql);
-
-            conn.createStatement().execute("UPSERT INTO "+tableName+" VALUES (1,8,10,20,30)");
-            conn.createStatement().execute("UPSERT INTO "+tableName+" VALUES (1,8,11,21,31)");
-            conn.createStatement().execute("UPSERT INTO "+tableName+" VALUES (1,9,5 ,22,32)");
-            conn.createStatement().execute("UPSERT INTO "+tableName+" VALUES (1,9,6 ,12,33)");
-            conn.createStatement().execute("UPSERT INTO "+tableName+" VALUES (1,9,6 ,13,34)");
-            conn.createStatement().execute("UPSERT INTO "+tableName+" VALUES (1,9,7 ,8,35)");
-            conn.createStatement().execute("UPSERT INTO "+tableName+" VALUES (2,3,15,25,35)");
-            conn.createStatement().execute("UPSERT INTO "+tableName+" VALUES (2,7,16,26,36)");
-            conn.createStatement().execute("UPSERT INTO "+tableName+" VALUES (2,7,17,27,37)");
-            conn.createStatement().execute("UPSERT INTO "+tableName+" VALUES (3,2,18,28,38)");
-            conn.createStatement().execute("UPSERT INTO "+tableName+" VALUES (3,2,19,29,39)");
-            conn.commit();
-
-            sql = "select pk2,pk1,count(v) from " + tableName + " group by pk2,pk1 order by pk2,pk1";
-            ResultSet rs = conn.prepareStatement(sql).executeQuery();
-            assertResultSet(rs, new Object[][]{{2,3,2L},{3,2,1L},{7,2,2L},{8,1,2L},{9,1,4L}});
-
-            sql = "select pk1,pk2,count(v) from " + tableName + " group by pk2,pk1 order by pk1,pk2";
-            rs = conn.prepareStatement(sql).executeQuery();
-            assertResultSet(rs, new Object[][]{{1,8,2L},{1,9,4L},{2,3,1L},{2,7,2L},{3,2,2L}});
-
-            sql = "select pk2,pk1,count(v) from " + tableName + " group by pk2,pk1 order by pk2 desc,pk1 desc";
-            rs = conn.prepareStatement(sql).executeQuery();
-            assertResultSet(rs, new Object[][]{{9,1,4L},{8,1,2L},{7,2,2L},{3,2,1L},{2,3,2L}});
-
-            sql = "select pk1,pk2,count(v) from " + tableName + " group by pk2,pk1 order by pk1 desc,pk2 desc";
-            rs = conn.prepareStatement(sql).executeQuery();
-            assertResultSet(rs, new Object[][]{{3,2,2L},{2,7,2L},{2,3,1L},{1,9,4L},{1,8,2L}});
-
-
-            sql = "select pk3,pk2,count(v) from " + tableName + " where pk1=1 group by pk3,pk2 order by pk3,pk2";
-            rs = conn.prepareStatement(sql).executeQuery();
-            assertResultSet(rs, new Object[][]{{5,9,1L},{6,9,2L},{7,9,1L},{10,8,1L},{11,8,1L}});
-
-            sql = "select pk2,pk3,count(v) from " + tableName + " where pk1=1 group by pk3,pk2 order by pk2,pk3";
-            rs = conn.prepareStatement(sql).executeQuery();
-            assertResultSet(rs, new Object[][]{{8,10,1L},{8,11,1L},{9,5,1L},{9,6,2L},{9,7,1L}});
-
-            sql = "select pk3,pk2,count(v) from " + tableName + " where pk1=1 group by pk3,pk2 order by pk3 desc,pk2 desc";
-            rs = conn.prepareStatement(sql).executeQuery();
-            assertResultSet(rs, new Object[][]{{11,8,1L},{10,8,1L},{7,9,1L},{6,9,2L},{5,9,1L}});
-
-            sql = "select pk2,pk3,count(v) from " + tableName + " where pk1=1 group by pk3,pk2 order by pk2 desc,pk3 desc";
-            rs = conn.prepareStatement(sql).executeQuery();
-            assertResultSet(rs, new Object[][]{{9,7,1L},{9,6,2L},{9,5,1L},{8,11,1L},{8,10,1L}});
-
-
-            sql = "select pk4,pk3,pk1,count(v) from " + tableName + " where pk2=9 group by pk4,pk3,pk1 order by pk4,pk3,pk1";
-            rs = conn.prepareStatement(sql).executeQuery();
-            assertResultSet(rs, new Object[][]{{8,7,1,1L},{12,6,1,1L},{13,6,1,1L},{22,5,1,1L}});
-
-            sql = "select pk1,pk3,pk4,count(v) from " + tableName + " where pk2=9 group by pk4,pk3,pk1 order by pk1,pk3,pk4";
-            rs = conn.prepareStatement(sql).executeQuery();
-            assertResultSet(rs, new Object[][]{{1,5,22,1L},{1,6,12,1L},{1,6,13,1L},{1,7,8,1L}});
-
-            sql = "select pk4,pk3,pk1,count(v) from " + tableName + " where pk2=9 group by pk4,pk3,pk1 order by pk4 desc,pk3 desc,pk1 desc";
-            rs = conn.prepareStatement(sql).executeQuery();
-            assertResultSet(rs, new Object[][]{{22,5,1,1L},{13,6,1,1L},{12,6,1,1L},{8,7,1,1L}});
-
-            sql = "select pk1,pk3,pk4,count(v) from " + tableName + " where pk2=9 group by pk4,pk3,pk1 order by pk1 desc,pk3 desc,pk4 desc";
-            rs = conn.prepareStatement(sql).executeQuery();
-            assertResultSet(rs, new Object[][]{{1,7,8,1L},{1,6,13,1L},{1,6,12,1L},{1,5,22,1L}});
-        } finally {
-            if(conn != null) {
-                conn.close();
-            }
+            assertEquals(4, rs.getLong(1));
         }
     }
 }
+


[5/6] phoenix git commit: PHOENIX-4981 Add tests for ORDER BY, GROUP BY and salted tables using phoenix-spark

Posted by td...@apache.org.
http://git-wip-us.apache.org/repos/asf/phoenix/blob/c509d58f/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseAggregateIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseAggregateIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseAggregateIT.java
new file mode 100644
index 0000000..5b466df
--- /dev/null
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseAggregateIT.java
@@ -0,0 +1,1022 @@
+/*
+ * 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 static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+import static org.apache.phoenix.util.TestUtil.assertResultSet;
+
+import java.io.IOException;
+import java.sql.Connection;
+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 com.google.common.collect.Lists;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.phoenix.compile.QueryPlan;
+import org.apache.phoenix.jdbc.PhoenixConnection;
+import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
+import org.apache.phoenix.jdbc.PhoenixStatement;
+import org.apache.phoenix.query.KeyRange;
+import org.apache.phoenix.query.QueryServices;
+import org.apache.phoenix.schema.AmbiguousColumnException;
+import org.apache.phoenix.schema.types.PChar;
+import org.apache.phoenix.schema.types.PInteger;
+import org.apache.phoenix.util.ByteUtil;
+import org.apache.phoenix.util.PropertiesUtil;
+import org.apache.phoenix.util.QueryBuilder;
+import org.apache.phoenix.util.QueryUtil;
+import org.apache.phoenix.util.TestUtil;
+import org.junit.Test;
+
+
+public abstract class BaseAggregateIT extends ParallelStatsDisabledIT {
+
+    private static void initData(Connection conn, String tableName) throws SQLException {
+        conn.createStatement().execute("create table " + tableName +
+                "   (id varchar not null primary key,\n" +
+                "    uri varchar, appcpu integer)");
+        insertRow(conn, tableName, "Report1", 10, 1);
+        insertRow(conn, tableName, "Report2", 10, 2);
+        insertRow(conn, tableName, "Report3", 30, 3);
+        insertRow(conn, tableName, "Report4", 30, 4);
+        insertRow(conn, tableName, "SOQL1", 10, 5);
+        insertRow(conn, tableName, "SOQL2", 10, 6);
+        insertRow(conn, tableName, "SOQL3", 30, 7);
+        insertRow(conn, tableName, "SOQL4", 30, 8);
+        conn.commit();
+    }
+
+    private static void insertRow(Connection conn, String tableName, String uri, int appcpu, int id) throws SQLException {
+        PreparedStatement statement = conn.prepareStatement("UPSERT INTO " + tableName + "(id, uri, appcpu) values (?,?,?)");
+        statement.setString(1, "id" + id);
+        statement.setString(2, uri);
+        statement.setInt(3, appcpu);
+        statement.executeUpdate();
+    }
+
+    @Test
+    public void testDuplicateTrailingAggExpr() throws Exception {
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        props.put(QueryServices.FORCE_ROW_KEY_ORDER_ATTRIB, Boolean.FALSE.toString());
+        Connection conn = DriverManager.getConnection(getUrl(), props);
+        String tableName = generateUniqueName();
+
+        conn.createStatement().execute("create table " + tableName +
+                "   (nam VARCHAR(20), address VARCHAR(20), id BIGINT "
+                + "constraint my_pk primary key (id))");
+        PreparedStatement statement = conn.prepareStatement("UPSERT INTO " + tableName + "(nam, address, id) values (?,?,?)");
+        statement.setString(1, "pulkit");
+        statement.setString(2, "badaun");
+        statement.setInt(3, 1);
+        statement.executeUpdate();
+        conn.commit();
+
+        QueryBuilder queryBuilder = new QueryBuilder()
+                .setDistinct(true)
+                .setSelectExpression("'harshit' as TEST_COLUMN, trim(NAM), trim(NAM)")
+                .setSelectExpressionColumns(Lists.newArrayList("NAM"))
+                .setFullTableName(tableName);
+
+        ResultSet rs = executeQuery(conn, queryBuilder);
+        assertTrue(rs.next());
+        assertEquals("harshit", rs.getString(1));
+        assertEquals("pulkit", rs.getString(2));
+        assertEquals("pulkit", rs.getString(3));
+        conn.close();
+    }
+
+    @Test
+    public void testExpressionInGroupBy() throws Exception {
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        Connection conn = DriverManager.getConnection(getUrl(), props);
+        String tableName = generateUniqueName();
+        String ddl = "create table " + tableName + "(tgb_id integer NOT NULL,utc_date_epoch integer NOT NULL,tgb_name varchar(40),ack_success_count integer" +
+                ",ack_success_one_ack_count integer, CONSTRAINT pk_tgb_counter PRIMARY KEY(tgb_id, utc_date_epoch))";
+
+        createTestTable(getUrl(), ddl);
+        String dml = "UPSERT INTO " + tableName + " VALUES(?,?,?,?,?)";
+        PreparedStatement stmt = conn.prepareStatement(dml);
+        stmt.setInt(1, 1);
+        stmt.setInt(2, 1000);
+        stmt.setString(3, "aaa");
+        stmt.setInt(4, 1);
+        stmt.setInt(5, 1);
+        stmt.execute();
+        stmt.setInt(1, 2);
+        stmt.setInt(2, 2000);
+        stmt.setString(3, "bbb");
+        stmt.setInt(4, 2);
+        stmt.setInt(5, 2);
+        stmt.execute();
+        conn.commit();
+
+        QueryBuilder queryBuilder = new QueryBuilder()
+            .setSelectExpression("TGB_ID, TGB_NAME, (UTC_DATE_EPOCH/10)*10 AS UTC_EPOCH_HOUR,SUM(ACK_SUCCESS_COUNT + " +
+                    "ACK_SUCCESS_ONE_ACK_COUNT) AS ACK_TX_SUM")
+            .setSelectExpressionColumns(Lists.newArrayList("TGB_ID", "TGB_NAME",
+                "UTC_DATE_EPOCH", "ACK_SUCCESS_COUNT", "ACK_SUCCESS_ONE_ACK_COUNT"))
+            .setGroupByClause("TGB_ID, TGB_NAME, UTC_EPOCH_HOUR")
+            .setFullTableName(tableName)
+            .setOrderByClause("TGB_ID, UTC_EPOCH_HOUR");
+
+        ResultSet rs = executeQuery(conn, queryBuilder);
+        assertTrue(rs.next());
+        assertEquals(1,rs.getInt(1));
+        assertEquals("aaa",rs.getString(2));
+        assertEquals(1000,rs.getDouble(3), 1e-6);
+        assertEquals(2,rs.getLong(4));
+        assertTrue(rs.next());
+        assertEquals(2,rs.getInt(1));
+        assertEquals("bbb",rs.getString(2));
+        assertEquals(2000,rs.getDouble(3), 1e-6);
+        assertEquals(4,rs.getLong(4));
+        assertFalse(rs.next());
+        rs.close();
+        conn.close();
+    }
+    
+    @Test
+    public void testBooleanInGroupBy() throws Exception {
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        Connection conn = DriverManager.getConnection(getUrl(), props);
+        String tableName = generateUniqueName();
+        String ddl = " create table " + tableName + "(id varchar primary key,v1 boolean, v2 integer, v3 integer)";
+
+        createTestTable(getUrl(), ddl);
+        PreparedStatement stmt = conn.prepareStatement("UPSERT INTO " + tableName + "(id,v2,v3) VALUES(?,?,?)");
+        stmt.setString(1, "a");
+        stmt.setInt(2, 1);
+        stmt.setInt(3, 1);
+        stmt.execute();
+        stmt.close();
+        stmt = conn.prepareStatement("UPSERT INTO " + tableName + " VALUES(?,?,?,?)");
+        stmt.setString(1, "b");
+        stmt.setBoolean(2, false);
+        stmt.setInt(3, 2);
+        stmt.setInt(4, 2);
+        stmt.execute();
+        stmt.setString(1, "c");
+        stmt.setBoolean(2, true);
+        stmt.setInt(3, 3);
+        stmt.setInt(4, 3);
+        stmt.execute();
+        conn.commit();
+
+        String[] gbs = {"V1,V2,V3","V1,V3,V2","V2,V1,V3"};
+        for (String gb : gbs) {
+            QueryBuilder queryBuilder = new QueryBuilder()
+                .setSelectColumns(Lists.newArrayList("V1", "V2", "V3"))
+                .setFullTableName(tableName)
+                .setGroupByClause(gb);
+
+            ResultSet rs = executeQuery(conn, queryBuilder);
+            assertTrue(rs.next());
+            assertEquals(false,rs.getBoolean("v1"));
+            assertTrue(rs.wasNull());
+            assertEquals(1,rs.getInt("v2"));
+            assertEquals(1,rs.getInt("v3"));
+            assertTrue(rs.next());
+            assertEquals(false,rs.getBoolean("v1"));
+            assertFalse(rs.wasNull());
+            assertEquals(2,rs.getInt("v2"));
+            assertEquals(2,rs.getInt("v3"));
+            assertTrue(rs.next());
+            assertEquals(true,rs.getBoolean("v1"));
+            assertEquals(3,rs.getInt("v2"));
+            assertEquals(3,rs.getInt("v3"));
+            assertFalse(rs.next());
+            rs.close();
+        }
+        conn.close();
+    }
+    
+    @Test
+    public void testScanUri() throws Exception {
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        Connection conn = DriverManager.getConnection(getUrl(), props);
+        String tableName = generateUniqueName();
+        initData(conn, tableName);
+        Statement stmt = conn.createStatement();
+        QueryBuilder queryBuilder = new QueryBuilder()
+            .setSelectColumns(Lists.newArrayList("URI"))
+            .setFullTableName(tableName);
+
+        ResultSet rs = executeQuery(conn, queryBuilder);
+        assertTrue(rs.next());
+        assertEquals("Report1", rs.getString(1));
+        assertTrue(rs.next());
+        assertEquals("Report2", rs.getString(1));
+        assertTrue(rs.next());
+        assertEquals("Report3", rs.getString(1));
+        assertTrue(rs.next());
+        assertEquals("Report4", rs.getString(1));
+        assertTrue(rs.next());
+        assertEquals("SOQL1", rs.getString(1));
+        assertTrue(rs.next());
+        assertEquals("SOQL2", rs.getString(1));
+        assertTrue(rs.next());
+        assertEquals("SOQL3", rs.getString(1));
+        assertTrue(rs.next());
+        assertEquals("SOQL4", rs.getString(1));
+        assertFalse(rs.next());
+        conn.close();
+    }
+
+    @Test
+    public void testCount() throws Exception {
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        Connection conn = DriverManager.getConnection(getUrl(), props);
+        String tableName = generateUniqueName();
+        initData(conn, tableName);
+        Statement stmt = conn.createStatement();
+        QueryBuilder queryBuilder = new QueryBuilder()
+            .setSelectExpression("count(1)")
+            .setFullTableName(tableName);
+        ResultSet rs = executeQuery(conn, queryBuilder);
+        assertTrue(rs.next());
+        assertEquals(8, rs.getLong(1));
+        assertFalse(rs.next());
+        conn.close();
+    }
+
+    @Test
+    public void testGroupByCase() throws Exception {
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        props.put(QueryServices.FORCE_ROW_KEY_ORDER_ATTRIB, Boolean.FALSE.toString());
+        Connection conn = DriverManager.getConnection(getUrl(), props);
+        String tableName = generateUniqueName();
+        initData(conn, tableName);
+        QueryBuilder queryBuilder = new QueryBuilder()
+            .setSelectExpression("CASE WHEN URI LIKE 'REPORT%' THEN 'REPORTS' ELSE 'OTHER' END CATEGORY, AVG(APPCPU)")
+            .setSelectExpressionColumns(Lists.newArrayList("URI", "APPCPU"))
+            .setFullTableName(tableName)
+            .setGroupByClause("CATEGORY");
+        executeQuery(conn, queryBuilder);
+
+        queryBuilder.setSelectExpression(
+                "CASE URI WHEN 'REPORT%' THEN 'REPORTS' ELSE 'OTHER' END CATEGORY, AVG(APPCPU)")
+            .setSelectExpressionColumns(Lists.newArrayList("URI", "APPCPU"))
+            .setFullTableName(tableName)
+            .setGroupByClause("APPCPU, CATEGORY");
+        executeQuery(conn, queryBuilder);
+
+        queryBuilder.setSelectExpression(
+                "CASE URI WHEN 'Report%' THEN 'Reports' ELSE 'Other' END CATEGORY, AVG(APPCPU)")
+            .setSelectColumns(Lists.newArrayList("URI", "APPCPU"))
+            .setFullTableName(tableName)
+            .setGroupByClause("AVG(APPCPU), CATEGORY");
+        executeQueryThrowsException(conn, queryBuilder, "Aggregate expressions may not be used in GROUP BY", "");
+        conn.close();
+    }
+
+
+    @Test
+    public void testGroupByArray() throws Exception {
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        Connection conn = DriverManager.getConnection(getUrl(), props);
+
+        String tableName = generateUniqueName();
+        conn.createStatement().execute("CREATE TABLE " + tableName + "(\n" + 
+                "  a VARCHAR NOT NULL,\n" + 
+                "  b VARCHAR,\n" + 
+                "  c INTEGER,\n" + 
+                "  d VARCHAR,\n" + 
+                "  e VARCHAR ARRAY,\n" + 
+                "  f BIGINT,\n" + 
+                "  g BIGINT,\n" + 
+                "  CONSTRAINT pk PRIMARY KEY(a)\n" + 
+                ")");
+        conn.createStatement().execute("UPSERT INTO " + tableName + " VALUES('1', 'val', 100, 'a', ARRAY ['b'], 1, 2)");
+        conn.createStatement().execute("UPSERT INTO " + tableName + " VALUES('2', 'val', 100, 'a', ARRAY ['b'], 3, 4)");
+        conn.createStatement().execute("UPSERT INTO " + tableName + " VALUES('3', 'val', 100, 'a', ARRAY ['b','c'], 5, 6)");
+        conn.commit();
+        
+        QueryBuilder queryBuilder = new QueryBuilder()
+            .setSelectExpression("C, SUM(F + G) AS SUMONE, D, E")
+            .setSelectExpressionColumns(Lists.newArrayList("A", "B", "C", "F", "G", "D", "E"))
+            .setWhereClause("B = 'val' AND A IN ('1','2','3')")
+            .setFullTableName(tableName)
+            .setGroupByClause("C, D, E")
+            .setOrderByClause("SUMONE desc");
+        ResultSet rs = executeQuery(conn, queryBuilder);
+        assertTrue(rs.next());
+        assertEquals(100, rs.getInt(1));
+        assertEquals(11, rs.getLong(2));
+        assertTrue(rs.next());
+        assertEquals(100, rs.getInt(1));
+        assertEquals(10, rs.getLong(2));
+        assertFalse(rs.next());
+        conn.close();
+    }
+    
+    @Test
+    public void testGroupByOrderPreserving() throws Exception {
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        props.put(QueryServices.FORCE_ROW_KEY_ORDER_ATTRIB, Boolean.FALSE.toString());
+        Connection conn = DriverManager.getConnection(getUrl(), props);
+        String tableName = generateUniqueName();
+
+        conn.createStatement().execute("CREATE TABLE " + tableName + "(ORGANIZATION_ID char(15) not null, \n" + 
+                "JOURNEY_ID char(15) not null, \n" + 
+                "DATASOURCE SMALLINT not null, \n" + 
+                "MATCH_STATUS TINYINT not null, \n" + 
+                "EXTERNAL_DATASOURCE_KEY varchar(30), \n" + 
+                "ENTITY_ID char(15) not null, \n" + 
+                "CONSTRAINT PK PRIMARY KEY (\n" + 
+                "    ORGANIZATION_ID, \n" + 
+                "    JOURNEY_ID, \n" + 
+                "    DATASOURCE, \n" + 
+                "    MATCH_STATUS,\n" + 
+                "    EXTERNAL_DATASOURCE_KEY,\n" + 
+                "    ENTITY_ID))");
+        conn.createStatement().execute("UPSERT INTO " + tableName
+                + " VALUES('000001111122222', '333334444455555', 0, 0, 'abc', '666667777788888')");
+        conn.createStatement().execute("UPSERT INTO " + tableName
+                + " VALUES('000001111122222', '333334444455555', 0, 0, 'abcd', '666667777788889')");
+        conn.createStatement().execute("UPSERT INTO " + tableName
+                + " VALUES('000001111122222', '333334444455555', 0, 0, 'abc', '666667777788899')");
+        conn.commit();
+        QueryBuilder queryBuilder = new QueryBuilder()
+            .setSelectExpression("COUNT(1), EXTERNAL_DATASOURCE_KEY As DUP_COUNT")
+            .setSelectExpressionColumns(Lists.newArrayList("EXTERNAL_DATASOURCE_KEY", "MATCH_STATUS",
+                "JOURNEY_ID", "DATASOURCE", "ORGANIZATION_ID"))
+            .setWhereClause(
+            "JOURNEY_ID='333334444455555' AND DATASOURCE=0 AND MATCH_STATUS <= 1 and ORGANIZATION_ID='000001111122222'")
+            .setFullTableName(tableName)
+            .setGroupByClause("MATCH_STATUS, EXTERNAL_DATASOURCE_KEY")
+            .setHavingClause("COUNT(1) > 1");
+        ResultSet rs = executeQuery(conn, queryBuilder);
+        assertTrue(rs.next());
+        assertEquals(2,rs.getLong(1));
+        assertEquals("abc", rs.getString(2));
+        assertFalse(rs.next());
+        
+        String expectedPhoenixPlan = "CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + tableName
+                + " ['000001111122222','333334444455555',0,*] - ['000001111122222','333334444455555',0,1]\n" +
+                "    SERVER FILTER BY FIRST KEY ONLY\n" +
+                "    SERVER AGGREGATE INTO ORDERED DISTINCT ROWS BY [MATCH_STATUS, EXTERNAL_DATASOURCE_KEY]\n" +
+                "CLIENT FILTER BY COUNT(1) > 1";
+        validateQueryPlan(conn, queryBuilder, expectedPhoenixPlan, null);
+    }
+    
+    @Test
+    public void testGroupByOrderPreservingDescSort() throws Exception {
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        Connection conn = DriverManager.getConnection(getUrl(), props);
+        String tableName = generateUniqueName();
+        conn.createStatement().execute("CREATE TABLE " + tableName + " (k1 char(1) not null, k2 char(1) not null," +
+                " constraint pk primary key (k1,k2)) split on ('ac','jc','nc')");
+        conn.createStatement().execute("UPSERT INTO " + tableName + " VALUES('a', 'a')");
+        conn.createStatement().execute("UPSERT INTO " + tableName + " VALUES('a', 'b')");
+        conn.createStatement().execute("UPSERT INTO " + tableName + " VALUES('a', 'c')");
+        conn.createStatement().execute("UPSERT INTO " + tableName + " VALUES('a', 'd')");
+        conn.createStatement().execute("UPSERT INTO " + tableName + " VALUES('j', 'a')");
+        conn.createStatement().execute("UPSERT INTO " + tableName + " VALUES('j', 'b')");
+        conn.createStatement().execute("UPSERT INTO " + tableName + " VALUES('j', 'c')");
+        conn.createStatement().execute("UPSERT INTO " + tableName + " VALUES('j', 'd')");
+        conn.createStatement().execute("UPSERT INTO " + tableName + " VALUES('n', 'a')");
+        conn.createStatement().execute("UPSERT INTO " + tableName + " VALUES('n', 'b')");
+        conn.createStatement().execute("UPSERT INTO " + tableName + " VALUES('n', 'c')");
+        conn.createStatement().execute("UPSERT INTO " + tableName + " VALUES('n', 'd')");
+        conn.commit();
+        QueryBuilder queryBuilder = new QueryBuilder()
+            .setSelectExpression("K1,COUNT(*)")
+            .setSelectColumns(Lists.newArrayList("K1"))
+            .setFullTableName(tableName)
+            .setGroupByClause("K1")
+            .setOrderByClause("K1 DESC");
+        ResultSet rs = executeQuery(conn, queryBuilder);
+        assertTrue(rs.next());
+        assertEquals("n", rs.getString(1));
+        assertEquals(4, rs.getLong(2));
+        assertTrue(rs.next());
+        assertEquals("j", rs.getString(1));
+        assertEquals(4, rs.getLong(2));
+        assertTrue(rs.next());
+        assertEquals("a", rs.getString(1));
+        assertEquals(4, rs.getLong(2));
+        assertFalse(rs.next());
+        String expectedPhoenixPlan = "CLIENT PARALLEL 1-WAY REVERSE FULL SCAN OVER " + tableName + "\n" +
+                "    SERVER FILTER BY FIRST KEY ONLY\n" +
+                "    SERVER AGGREGATE INTO ORDERED DISTINCT ROWS BY [K1]";
+        validateQueryPlan(conn, queryBuilder, expectedPhoenixPlan, null);
+    }
+    
+    @Test
+    public void testSumGroupByOrderPreservingDesc() throws Exception {
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        Connection conn = DriverManager.getConnection(getUrl(), props);
+        String tableName = generateUniqueName();
+
+        PreparedStatement stmt = conn.prepareStatement("CREATE TABLE " + tableName + " (k1 char(1) not null, k2 integer not null, constraint pk primary key (k1,k2)) split on (?,?,?)");
+        stmt.setBytes(1, ByteUtil.concat(PChar.INSTANCE.toBytes("a"), PInteger.INSTANCE.toBytes(3)));
+        stmt.setBytes(2, ByteUtil.concat(PChar.INSTANCE.toBytes("j"), PInteger.INSTANCE.toBytes(3)));
+        stmt.setBytes(3, ByteUtil.concat(PChar.INSTANCE.toBytes("n"), PInteger.INSTANCE.toBytes(3)));
+        stmt.execute();
+        conn.createStatement().execute("UPSERT INTO " + tableName + " VALUES('a', 1)");
+        conn.createStatement().execute("UPSERT INTO " + tableName + " VALUES('a', 2)");
+        conn.createStatement().execute("UPSERT INTO " + tableName + " VALUES('a', 3)");
+        conn.createStatement().execute("UPSERT INTO " + tableName + " VALUES('a', 4)");
+        conn.createStatement().execute("UPSERT INTO " + tableName + " VALUES('b', 5)");
+        conn.createStatement().execute("UPSERT INTO " + tableName + " VALUES('j', 1)");
+        conn.createStatement().execute("UPSERT INTO " + tableName + " VALUES('j', 2)");
+        conn.createStatement().execute("UPSERT INTO " + tableName + " VALUES('j', 3)");
+        conn.createStatement().execute("UPSERT INTO " + tableName + " VALUES('j', 4)");
+        conn.createStatement().execute("UPSERT INTO " + tableName + " VALUES('n', 1)");
+        conn.createStatement().execute("UPSERT INTO " + tableName + " VALUES('n', 2)");
+        conn.createStatement().execute("UPSERT INTO " + tableName + " VALUES('n', 3)");
+        conn.createStatement().execute("UPSERT INTO " + tableName + " VALUES('n', 4)");
+        conn.commit();
+        QueryBuilder queryBuilder = new QueryBuilder()
+            .setSelectExpression("K1,SUM(K2)")
+            .setSelectExpressionColumns(Lists.newArrayList("K1", "K2"))
+            .setFullTableName(tableName)
+            .setGroupByClause("K1")
+            .setOrderByClause("K1 DESC");
+        ResultSet rs = executeQuery(conn, queryBuilder);
+        assertTrue(rs.next());
+        assertEquals("n", rs.getString(1));
+        assertEquals(10, rs.getLong(2));
+        assertTrue(rs.next());
+        assertEquals("j", rs.getString(1));
+        assertEquals(10, rs.getLong(2));
+        assertTrue(rs.next());
+        assertEquals("b", rs.getString(1));
+        assertEquals(5, rs.getLong(2));
+        assertTrue(rs.next());
+        assertEquals("a", rs.getString(1));
+        assertEquals(10, rs.getLong(2));
+        assertFalse(rs.next());
+        String expectedPhoenixPlan = "CLIENT PARALLEL 1-WAY REVERSE FULL SCAN OVER " + tableName + "\n" +
+                "    SERVER FILTER BY FIRST KEY ONLY\n" +
+                "    SERVER AGGREGATE INTO ORDERED DISTINCT ROWS BY [K1]";
+        validateQueryPlan(conn, queryBuilder, expectedPhoenixPlan, null);
+    }
+
+    @Test
+    public void testAvgGroupByOrderPreservingWithNoStats() throws Exception {
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        Connection conn = DriverManager.getConnection(getUrl(), props);
+        String tableName = generateUniqueName();
+        initAvgGroupTable(conn, tableName, "");
+        testAvgGroupByOrderPreserving(conn, tableName, 4);
+    }
+    
+    protected void initAvgGroupTable(Connection conn, String tableName, String tableProps) throws SQLException {
+        PreparedStatement stmt = conn.prepareStatement("CREATE TABLE " + tableName + " (k1 char(1) not null, k2 integer not null, constraint pk primary key (k1,k2)) " + tableProps + " split on (?,?,?)");
+        stmt.setBytes(1, ByteUtil.concat(PChar.INSTANCE.toBytes("a"), PInteger.INSTANCE.toBytes(3)));
+        stmt.setBytes(2, ByteUtil.concat(PChar.INSTANCE.toBytes("j"), PInteger.INSTANCE.toBytes(3)));
+        stmt.setBytes(3, ByteUtil.concat(PChar.INSTANCE.toBytes("n"), PInteger.INSTANCE.toBytes(3)));
+        stmt.execute();
+        conn.createStatement().execute("UPSERT INTO " + tableName + " VALUES('a', 1)");
+        conn.createStatement().execute("UPSERT INTO " + tableName + " VALUES('a', 2)");
+        conn.createStatement().execute("UPSERT INTO " + tableName + " VALUES('a', 3)");
+        conn.createStatement().execute("UPSERT INTO " + tableName + " VALUES('a', 6)");
+        conn.createStatement().execute("UPSERT INTO " + tableName + " VALUES('b', 5)");
+        conn.createStatement().execute("UPSERT INTO " + tableName + " VALUES('j', 1)");
+        conn.createStatement().execute("UPSERT INTO " + tableName + " VALUES('j', 2)");
+        conn.createStatement().execute("UPSERT INTO " + tableName + " VALUES('j', 3)");
+        conn.createStatement().execute("UPSERT INTO " + tableName + " VALUES('j', 10)");
+        conn.createStatement().execute("UPSERT INTO " + tableName + " VALUES('n', 1)");
+        conn.createStatement().execute("UPSERT INTO " + tableName + " VALUES('n', 2)");
+        conn.createStatement().execute("UPSERT INTO " + tableName + " VALUES('n', 3)");
+        conn.createStatement().execute("UPSERT INTO " + tableName + " VALUES('n', 2)");
+        conn.commit();
+    }
+    
+    protected void testAvgGroupByOrderPreserving(Connection conn, String tableName, int nGuidePosts) throws SQLException, IOException {
+        QueryBuilder queryBuilder = new QueryBuilder()
+            .setSelectExpression("K1, AVG(K2)")
+            .setSelectExpressionColumns(Lists.newArrayList("K1", "K2"))
+            .setFullTableName(tableName)
+            .setGroupByClause("K1")
+            .setOrderByClause("K1");
+        ResultSet rs = executeQuery(conn, queryBuilder);
+        assertTrue(rs.next());
+        assertEquals("a", rs.getString(1));
+        assertEquals(3, rs.getDouble(2), 1e-6);
+        assertTrue(rs.next());
+        assertEquals("b", rs.getString(1));
+        assertEquals(5, rs.getDouble(2), 1e-6);
+        assertTrue(rs.next());
+        assertEquals("j", rs.getString(1));
+        assertEquals(4, rs.getDouble(2), 1e-6);
+        assertTrue(rs.next());
+        assertEquals("n", rs.getString(1));
+        assertEquals(2, rs.getDouble(2), 1e-6);
+        assertFalse(rs.next());
+        String expectedPhoenixPlan = "CLIENT PARALLEL 1-WAY FULL SCAN OVER " + tableName + "\n" +
+                "    SERVER FILTER BY FIRST KEY ONLY\n" +
+                "    SERVER AGGREGATE INTO ORDERED DISTINCT ROWS BY [K1]";
+        validateQueryPlan(conn, queryBuilder, expectedPhoenixPlan, null);
+        TestUtil.analyzeTable(conn, tableName);
+        List<KeyRange> splits = TestUtil.getAllSplits(conn, tableName);
+        assertEquals(nGuidePosts, splits.size());
+    }
+    
+    @Test
+    public void testDistinctGroupByBug3452WithoutMultiTenant() throws Exception {
+        doTestDistinctGroupByBug3452("");
+    }
+
+    @Test
+    public void testDistinctGroupByBug3452WithMultiTenant() throws Exception {
+        doTestDistinctGroupByBug3452("VERSIONS=1, MULTI_TENANT=TRUE, REPLICATION_SCOPE=1, TTL=31536000");
+    }
+
+    private void doTestDistinctGroupByBug3452(String options) throws Exception {
+        Connection conn=null;
+        try {
+            Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+            conn = DriverManager.getConnection(getUrl(), props);
+
+            String tableName=generateUniqueName();
+            conn.createStatement().execute("DROP TABLE if exists "+tableName);
+            String sql="CREATE TABLE "+ tableName +" ( "+
+                    "ORGANIZATION_ID CHAR(15) NOT NULL,"+
+                    "CONTAINER_ID CHAR(15) NOT NULL,"+
+                    "ENTITY_ID CHAR(15) NOT NULL,"+
+                    "SCORE DOUBLE,"+
+                    "CONSTRAINT TEST_PK PRIMARY KEY ( "+
+                    "ORGANIZATION_ID,"+
+                    "CONTAINER_ID,"+
+                    "ENTITY_ID"+
+                    ")) "+options;
+            conn.createStatement().execute(sql);
+
+            String indexTableName=generateUniqueName();
+            conn.createStatement().execute("DROP INDEX IF EXISTS "+indexTableName+" ON "+tableName);
+            conn.createStatement().execute("CREATE INDEX "+indexTableName+" ON "+tableName+" (CONTAINER_ID, SCORE DESC, ENTITY_ID DESC)");
+
+            conn.createStatement().execute("UPSERT INTO "+tableName+" VALUES ('org1','container1','entityId6',1.1)");
+            conn.createStatement().execute("UPSERT INTO "+tableName+" VALUES ('org1','container1','entityId5',1.2)");
+            conn.createStatement().execute("UPSERT INTO "+tableName+" VALUES ('org1','container1','entityId4',1.3)");
+            conn.createStatement().execute("UPSERT INTO "+tableName+" VALUES ('org1','container1','entityId3',1.4)");
+            conn.createStatement().execute("UPSERT INTO "+tableName+" VALUES ('org1','container1','entityId2',1.5)");
+            conn.createStatement().execute("UPSERT INTO "+tableName+" VALUES ('org1','container1','entityId1',1.6)");
+            conn.commit();
+
+            QueryBuilder queryBuilder = new QueryBuilder()
+                .setDistinct(true)
+                .setSelectColumns(Lists.newArrayList("ENTITY_ID", "SCORE", "ORGANIZATION_ID", "CONTAINER_ID"))
+                .setFullTableName(tableName)
+                .setWhereClause("ORGANIZATION_ID = 'org1' AND CONTAINER_ID = 'container1'")
+                .setOrderByClause("SCORE DESC");
+            ResultSet rs = executeQuery(conn, queryBuilder);
+            assertTrue(rs.next());
+            assertTrue(rs.getString(1).equals("entityId1"));
+            assertEquals(rs.getDouble(2),1.6,0.0001);
+
+            assertTrue(rs.next());
+            assertTrue(rs.getString(1).equals("entityId2"));
+            assertEquals(rs.getDouble(2),1.5,0.0001);
+
+            assertTrue(rs.next());
+            assertTrue(rs.getString(1).equals("entityId3"));
+            assertEquals(rs.getDouble(2),1.4,0.0001);
+
+            assertTrue(rs.next());
+            assertTrue(rs.getString(1).equals("entityId4"));
+            assertEquals(rs.getDouble(2),1.3,0.0001);
+
+            assertTrue(rs.next());
+            assertTrue(rs.getString(1).equals("entityId5"));
+            assertEquals(rs.getDouble(2),1.2,0.0001);
+
+            assertTrue(rs.next());
+            assertTrue(rs.getString(1).equals("entityId6"));
+            assertEquals(rs.getDouble(2),1.1,0.0001);
+            assertTrue(!rs.next());
+        } finally {
+            if(conn!=null) {
+                conn.close();
+            }
+        }
+    }
+
+    @Test
+    public void testGroupByOrderByDescBug3451() throws Exception {
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
+            String tableName=generateUniqueName();
+            String sql="CREATE TABLE " + tableName + " (\n" + 
+                    "            ORGANIZATION_ID CHAR(15) NOT NULL,\n" + 
+                    "            CONTAINER_ID CHAR(15) NOT NULL,\n" + 
+                    "            ENTITY_ID CHAR(15) NOT NULL,\n" + 
+                    "            SCORE DOUBLE,\n" + 
+                    "            CONSTRAINT TEST_PK PRIMARY KEY (\n" + 
+                    "               ORGANIZATION_ID,\n" + 
+                    "               CONTAINER_ID,\n" + 
+                    "               ENTITY_ID\n" + 
+                    "             )\n" + 
+                    "         )";
+            conn.createStatement().execute(sql);
+            String indexName=generateUniqueName();
+            conn.createStatement().execute("CREATE INDEX " + indexName + " ON " + tableName + "(ORGANIZATION_ID,CONTAINER_ID, SCORE DESC, ENTITY_ID DESC)");
+            conn.createStatement().execute("UPSERT INTO " + tableName + " VALUES ('org2','container2','entityId6',1.1)");
+            conn.createStatement().execute("UPSERT INTO " + tableName + " VALUES ('org2','container1','entityId5',1.2)");
+            conn.createStatement().execute("UPSERT INTO " + tableName + " VALUES ('org2','container2','entityId4',1.3)");
+            conn.createStatement().execute("UPSERT INTO " + tableName + " VALUES ('org2','container1','entityId5',1.2)");
+            conn.createStatement().execute("UPSERT INTO " + tableName + " VALUES ('org2','container1','entityId3',1.4)");
+            conn.createStatement().execute("UPSERT INTO " + tableName + " VALUES ('org2','container3','entityId7',1.35)");
+            conn.createStatement().execute("UPSERT INTO " + tableName + " VALUES ('org2','container3','entityId8',1.45)");
+            conn.commit();
+            QueryBuilder queryBuilder = new QueryBuilder()
+                .setDistinct(true)
+                .setSelectColumns(Lists.newArrayList("ENTITY_ID", "SCORE", "ORGANIZATION_ID", "CONTAINER_ID"))
+                .setFullTableName(tableName)
+                .setWhereClause(
+                    "ORGANIZATION_ID = 'org2' AND CONTAINER_ID IN ('container1','container2','container3')")
+                .setOrderByClause("SCORE DESC")
+                .setLimit(2);
+            ResultSet rs = executeQuery(conn, queryBuilder);
+            assertTrue(rs.next());
+            assertEquals("entityId8", rs.getString(1));
+            assertEquals(1.45, rs.getDouble(2),0.001);
+            assertTrue(rs.next());
+            assertEquals("entityId3", rs.getString(1));
+            assertEquals(1.4, rs.getDouble(2),0.001);
+            assertFalse(rs.next());
+
+            String expectedPhoenixPlan = "";
+            validateQueryPlan(conn, queryBuilder, expectedPhoenixPlan, null);
+       }
+    }
+    
+    @Test
+    public void testGroupByDescColumnWithNullsLastBug3452() throws Exception {
+
+        Connection conn=null;
+        try
+        {
+            Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+            conn = DriverManager.getConnection(getUrl(), props);
+
+            String tableName=generateUniqueName();
+            String sql="CREATE TABLE "+tableName+" ( "+
+                    "ORGANIZATION_ID VARCHAR,"+
+                    "CONTAINER_ID VARCHAR,"+
+                    "ENTITY_ID VARCHAR NOT NULL,"+
+                    "CONSTRAINT TEST_PK PRIMARY KEY ( "+
+                    "ORGANIZATION_ID DESC,"+
+                    "CONTAINER_ID DESC,"+
+                    "ENTITY_ID"+
+                    "))";
+            conn.createStatement().execute(sql);
+
+            conn.createStatement().execute("UPSERT INTO "+tableName+" VALUES ('a',null,'11')");
+            conn.createStatement().execute("UPSERT INTO "+tableName+" VALUES (null,'2','22')");
+            conn.createStatement().execute("UPSERT INTO "+tableName+" VALUES ('c','3','33')");
+            conn.commit();
+
+            //-----ORGANIZATION_ID
+
+            QueryBuilder queryBuilder = new QueryBuilder()
+                .setSelectColumns(
+                        Lists.newArrayList("CONTAINER_ID", "ORGANIZATION_ID"))
+                .setFullTableName(tableName)
+                .setGroupByClause("ORGANIZATION_ID, CONTAINER_ID")
+                .setOrderByClause("ORGANIZATION_ID ASC NULLS FIRST");
+            ResultSet rs = executeQuery(conn, queryBuilder);
+            assertResultSet(rs, new String[][]{{"2",null},{null,"a"},{"3","c"},});
+
+            queryBuilder.setOrderByClause("ORGANIZATION_ID ASC NULLS LAST");
+            rs = executeQuery(conn, queryBuilder);
+            assertResultSet(rs, new String[][]{{null,"a"},{"3","c"},{"2",null}});
+
+            queryBuilder.setOrderByClause("ORGANIZATION_ID DESC NULLS FIRST");
+            rs = executeQuery(conn, queryBuilder);
+            assertResultSet(rs, new String[][]{{"2",null},{"3","c"},{null,"a"}});
+
+            queryBuilder.setOrderByClause("ORGANIZATION_ID DESC NULLS LAST");
+            rs = executeQuery(conn, queryBuilder);
+            assertResultSet(rs, new String[][]{{"3","c"},{null,"a"},{"2",null}});
+
+            //----CONTAINER_ID
+
+            queryBuilder.setOrderByClause("CONTAINER_ID ASC NULLS FIRST");
+            rs = executeQuery(conn, queryBuilder);
+            assertResultSet(rs, new String[][]{{null,"a"},{"2",null},{"3","c"}});
+
+            queryBuilder.setOrderByClause("CONTAINER_ID ASC NULLS LAST");
+            rs = executeQuery(conn, queryBuilder);
+            assertResultSet(rs, new String[][]{{"2",null},{"3","c"},{null,"a"}});
+
+            queryBuilder.setOrderByClause("CONTAINER_ID DESC NULLS FIRST");
+            rs = executeQuery(conn, queryBuilder);
+            assertResultSet(rs, new String[][]{{null,"a"},{"3","c"},{"2",null}});
+
+            queryBuilder.setOrderByClause("CONTAINER_ID DESC NULLS LAST");
+            rs = executeQuery(conn, queryBuilder);
+            assertResultSet(rs, new String[][]{{"3","c"},{"2",null},{null,"a"}});
+
+            conn.createStatement().execute("UPSERT INTO "+tableName+" VALUES (null,null,'44')");
+            conn.commit();
+
+            //-----ORGANIZATION_ID ASC  CONTAINER_ID ASC
+
+            queryBuilder.setOrderByClause("ORGANIZATION_ID NULLS FIRST,CONTAINER_ID NULLS FIRST");
+            rs = executeQuery(conn, queryBuilder);
+            assertResultSet(rs, new String[][]{{null,null},{"2",null},{null,"a"},{"3","c"}});
+
+            queryBuilder.setOrderByClause("ORGANIZATION_ID NULLS FIRST,CONTAINER_ID NULLS LAST");
+            rs = executeQuery(conn, queryBuilder);
+            assertResultSet(rs, new String[][]{{"2",null},{null,null},{null,"a"},{"3","c"}});
+
+            queryBuilder.setOrderByClause("ORGANIZATION_ID NULLS LAST,CONTAINER_ID NULLS FIRST");
+            rs = executeQuery(conn, queryBuilder);
+            assertResultSet(rs, new String[][]{{null,"a"},{"3","c"},{null,null},{"2",null}});
+
+            queryBuilder.setOrderByClause("ORGANIZATION_ID NULLS LAST,CONTAINER_ID NULLS LAST");
+            rs = executeQuery(conn, queryBuilder);
+            assertResultSet(rs, new String[][]{{null,"a"},{"3","c"},{"2",null},{null,null}});
+
+            //-----ORGANIZATION_ID ASC  CONTAINER_ID DESC
+
+            queryBuilder.setOrderByClause("ORGANIZATION_ID NULLS FIRST,CONTAINER_ID DESC NULLS FIRST");
+            rs = executeQuery(conn, queryBuilder);
+            assertResultSet(rs, new String[][]{{null,null},{"2",null},{null,"a"},{"3","c"}});
+
+            queryBuilder.setOrderByClause("ORGANIZATION_ID NULLS FIRST,CONTAINER_ID DESC NULLS LAST");
+            rs = executeQuery(conn, queryBuilder);
+            assertResultSet(rs, new String[][]{{"2",null},{null,null},{null,"a"},{"3","c"}});
+
+            queryBuilder.setOrderByClause("ORGANIZATION_ID NULLS LAST,CONTAINER_ID DESC NULLS FIRST");
+            rs = executeQuery(conn, queryBuilder);
+            assertResultSet(rs, new String[][]{{null,"a"},{"3","c"},{null,null},{"2",null}});
+
+            queryBuilder.setOrderByClause("ORGANIZATION_ID NULLS LAST,CONTAINER_ID DESC NULLS LAST");
+            rs = executeQuery(conn, queryBuilder);
+            assertResultSet(rs, new String[][]{{null,"a"},{"3","c"},{"2",null},{null,null}});
+
+            //-----ORGANIZATION_ID DESC  CONTAINER_ID ASC
+
+            queryBuilder.setOrderByClause("ORGANIZATION_ID DESC NULLS FIRST,CONTAINER_ID NULLS FIRST");
+            rs = executeQuery(conn, queryBuilder);
+            assertResultSet(rs, new String[][]{{null,null},{"2",null},{"3","c"},{null,"a"}});
+
+            queryBuilder.setOrderByClause("ORGANIZATION_ID DESC NULLS FIRST,CONTAINER_ID NULLS LAST");
+            rs = executeQuery(conn, queryBuilder);
+            assertResultSet(rs, new String[][]{{"2",null},{null,null},{"3","c"},{null,"a"}});
+
+            queryBuilder.setOrderByClause("ORGANIZATION_ID DESC NULLS LAST,CONTAINER_ID NULLS FIRST");
+            rs = executeQuery(conn, queryBuilder);
+            assertResultSet(rs, new String[][]{{"3","c"},{null,"a"},{null,null},{"2",null}});
+
+            queryBuilder.setOrderByClause("ORGANIZATION_ID DESC NULLS LAST,CONTAINER_ID NULLS LAST");
+            rs = executeQuery(conn, queryBuilder);
+            assertResultSet(rs, new String[][]{{"3","c"},{null,"a"},{"2",null},{null,null}});
+
+            //-----ORGANIZATION_ID DESC  CONTAINER_ID DESC
+
+            queryBuilder.setOrderByClause("ORGANIZATION_ID DESC NULLS FIRST,CONTAINER_ID DESC NULLS FIRST");
+            rs = executeQuery(conn, queryBuilder);
+            assertResultSet(rs, new String[][]{{null,null},{"2",null},{"3","c"},{null,"a"}});
+
+            queryBuilder.setOrderByClause("ORGANIZATION_ID DESC NULLS FIRST,CONTAINER_ID DESC NULLS LAST");
+            rs = executeQuery(conn, queryBuilder);
+            assertResultSet(rs, new String[][]{{"2",null},{null,null},{"3","c"},{null,"a"}});
+
+            queryBuilder.setOrderByClause("ORGANIZATION_ID DESC NULLS LAST,CONTAINER_ID DESC NULLS FIRST");
+            rs = executeQuery(conn, queryBuilder);
+            assertResultSet(rs, new String[][]{{"3","c"},{null,"a"},{null,null},{"2",null}});
+
+            queryBuilder.setOrderByClause("ORGANIZATION_ID DESC NULLS LAST,CONTAINER_ID DESC NULLS LAST");
+            rs = executeQuery(conn, queryBuilder);
+            assertResultSet(rs, new String[][]{{"3","c"},{null,"a"},{"2",null},{null,null}});
+
+            //-----CONTAINER_ID ASC  ORGANIZATION_ID ASC
+
+            queryBuilder.setOrderByClause("CONTAINER_ID NULLS FIRST,ORGANIZATION_ID NULLS FIRST");
+            rs = executeQuery(conn, queryBuilder);
+            assertResultSet(rs, new String[][]{{null,null},{null,"a"},{"2",null},{"3","c"}});
+
+            queryBuilder.setOrderByClause("CONTAINER_ID NULLS FIRST,ORGANIZATION_ID NULLS LAST");
+            rs = executeQuery(conn, queryBuilder);
+            assertResultSet(rs, new String[][]{{null,"a"},{null,null},{"2",null},{"3","c"}});
+
+            queryBuilder.setOrderByClause("CONTAINER_ID NULLS LAST,ORGANIZATION_ID NULLS FIRST");
+            rs = executeQuery(conn, queryBuilder);
+            assertResultSet(rs, new String[][]{{"2",null},{"3","c"},{null,null},{null,"a"}});
+
+            queryBuilder.setOrderByClause("CONTAINER_ID NULLS LAST,ORGANIZATION_ID NULLS LAST");
+            rs = executeQuery(conn, queryBuilder);
+            assertResultSet(rs, new String[][]{{"2",null},{"3","c"},{null,"a"},{null,null}});
+
+            //-----CONTAINER_ID ASC  ORGANIZATION_ID DESC
+
+            queryBuilder.setOrderByClause("CONTAINER_ID ASC NULLS FIRST,ORGANIZATION_ID DESC NULLS FIRST");
+            rs = executeQuery(conn, queryBuilder);
+            assertResultSet(rs, new String[][]{{null,null},{null,"a"},{"2",null},{"3","c"}});
+
+            queryBuilder.setOrderByClause("CONTAINER_ID ASC NULLS FIRST,ORGANIZATION_ID DESC NULLS LAST");
+            rs = executeQuery(conn, queryBuilder);
+            assertResultSet(rs, new String[][]{{null,"a"},{null,null},{"2",null},{"3","c"}});
+
+            queryBuilder.setOrderByClause("CONTAINER_ID ASC NULLS LAST,ORGANIZATION_ID DESC NULLS FIRST");
+            rs = executeQuery(conn, queryBuilder);
+            assertResultSet(rs, new String[][]{{"2",null},{"3","c"},{null,null},{null,"a"}});
+
+            queryBuilder.setOrderByClause("CONTAINER_ID ASC NULLS LAST,ORGANIZATION_ID DESC NULLS LAST");
+            rs = executeQuery(conn, queryBuilder);
+            assertResultSet(rs, new String[][]{{"2",null},{"3","c"},{null,"a"},{null,null}});
+
+            //-----CONTAINER_ID DESC  ORGANIZATION_ID ASC
+
+            queryBuilder.setOrderByClause("CONTAINER_ID DESC NULLS FIRST,ORGANIZATION_ID ASC NULLS FIRST");
+            rs = executeQuery(conn, queryBuilder);
+            assertResultSet(rs, new String[][]{{null,null},{null,"a"},{"3","c"},{"2",null}});
+
+            queryBuilder.setOrderByClause("CONTAINER_ID DESC NULLS FIRST,ORGANIZATION_ID ASC NULLS LAST");
+            rs = executeQuery(conn, queryBuilder);
+            assertResultSet(rs, new String[][]{{null,"a"},{null,null},{"3","c"},{"2",null}});
+
+            queryBuilder.setOrderByClause("CONTAINER_ID DESC NULLS LAST,ORGANIZATION_ID ASC NULLS FIRST");
+            rs = executeQuery(conn, queryBuilder);
+            assertResultSet(rs, new String[][]{{"3","c"},{"2",null},{null,null},{null,"a"}});
+
+            queryBuilder.setOrderByClause("CONTAINER_ID DESC NULLS LAST,ORGANIZATION_ID ASC NULLS LAST");
+            rs = executeQuery(conn, queryBuilder);
+            assertResultSet(rs, new String[][]{{"3","c"},{"2",null},{null,"a"},{null,null}});
+
+            //-----CONTAINER_ID DESC  ORGANIZATION_ID DESC
+
+            queryBuilder.setOrderByClause("CONTAINER_ID DESC NULLS FIRST,ORGANIZATION_ID DESC NULLS FIRST");
+            rs = executeQuery(conn, queryBuilder);
+            assertResultSet(rs, new String[][]{{null,null},{null,"a"},{"3","c"},{"2",null}});
+
+            queryBuilder.setOrderByClause("CONTAINER_ID DESC NULLS FIRST,ORGANIZATION_ID DESC NULLS LAST");
+            rs = executeQuery(conn, queryBuilder);
+            assertResultSet(rs, new String[][]{{null,"a"},{null,null},{"3","c"},{"2",null}});
+
+            queryBuilder.setOrderByClause("CONTAINER_ID DESC NULLS LAST,ORGANIZATION_ID DESC NULLS FIRST");
+            rs = executeQuery(conn, queryBuilder);
+            assertResultSet(rs, new String[][]{{"3","c"},{"2",null},{null,null},{null,"a"}});
+
+            queryBuilder.setOrderByClause("CONTAINER_ID DESC NULLS LAST,ORGANIZATION_ID DESC NULLS LAST");
+            rs = executeQuery(conn, queryBuilder);
+            assertResultSet(rs, new String[][]{{"3","c"},{"2",null},{null,"a"},{null,null}});
+        } finally {
+            if(conn!=null) {
+                conn.close();
+            }
+        }
+    }
+
+    @Test
+    public void testCountNullInEncodedNonEmptyKeyValueCF() throws Exception {
+        testCountNullInNonEmptyKeyValueCF(1);
+    }
+    
+    @Test
+    public void testCountNullInNonEncodedNonEmptyKeyValueCF() throws Exception {
+        testCountNullInNonEmptyKeyValueCF(0);
+    }
+
+    protected abstract void testCountNullInNonEmptyKeyValueCF(int columnEncodedBytes) throws Exception;
+
+    @Test
+    public void testGroupByOrderMatchPkColumnOrderBug4690() throws Exception {
+        this.doTestGroupByOrderMatchPkColumnOrderBug4690(false, false);
+        this.doTestGroupByOrderMatchPkColumnOrderBug4690(false, true);
+        this.doTestGroupByOrderMatchPkColumnOrderBug4690(true, false);
+        this.doTestGroupByOrderMatchPkColumnOrderBug4690(true, true);
+    }
+
+    private void doTestGroupByOrderMatchPkColumnOrderBug4690(boolean desc ,boolean salted) throws Exception {
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        Connection conn = null;
+        try {
+            conn = DriverManager.getConnection(getUrl(), props);
+            String tableName = generateUniqueName();
+            String sql = "create table " + tableName + "( "+
+                    " pk1 integer not null , " +
+                    " pk2 integer not null, " +
+                    " pk3 integer not null," +
+                    " pk4 integer not null,"+
+                    " v integer, " +
+                    " CONSTRAINT TEST_PK PRIMARY KEY ( "+
+                       "pk1 "+(desc ? "desc" : "")+", "+
+                       "pk2 "+(desc ? "desc" : "")+", "+
+                       "pk3 "+(desc ? "desc" : "")+", "+
+                       "pk4 "+(desc ? "desc" : "")+
+                    " )) "+(salted ? "SALT_BUCKETS =4" : "split on(2)");
+            conn.createStatement().execute(sql);
+
+            conn.createStatement().execute("UPSERT INTO "+tableName+" VALUES (1,8,10,20,30)");
+            conn.createStatement().execute("UPSERT INTO "+tableName+" VALUES (1,8,11,21,31)");
+            conn.createStatement().execute("UPSERT INTO "+tableName+" VALUES (1,9,5 ,22,32)");
+            conn.createStatement().execute("UPSERT INTO "+tableName+" VALUES (1,9,6 ,12,33)");
+            conn.createStatement().execute("UPSERT INTO "+tableName+" VALUES (1,9,6 ,13,34)");
+            conn.createStatement().execute("UPSERT INTO "+tableName+" VALUES (1,9,7 ,8,35)");
+            conn.createStatement().execute("UPSERT INTO "+tableName+" VALUES (2,3,15,25,35)");
+            conn.createStatement().execute("UPSERT INTO "+tableName+" VALUES (2,7,16,26,36)");
+            conn.createStatement().execute("UPSERT INTO "+tableName+" VALUES (2,7,17,27,37)");
+            conn.createStatement().execute("UPSERT INTO "+tableName+" VALUES (3,2,18,28,38)");
+            conn.createStatement().execute("UPSERT INTO "+tableName+" VALUES (3,2,19,29,39)");
+            conn.commit();
+
+            QueryBuilder queryBuilder = new QueryBuilder()
+                .setSelectExpression("PK2,PK1,COUNT(V)")
+                .setSelectExpressionColumns(Lists.newArrayList("PK1", "PK2", "V"))
+                .setFullTableName(tableName)
+                .setGroupByClause("PK2, PK1")
+                .setOrderByClause("PK2, PK1");
+            ResultSet rs = executeQuery(conn, queryBuilder);
+            assertResultSet(rs, new Object[][]{{2,3,2L},{3,2,1L},{7,2,2L},{8,1,2L},{9,1,4L}});
+
+            queryBuilder.setSelectExpression("PK1, PK2, COUNT(V)");
+            queryBuilder.setOrderByClause("PK1, PK2");
+            rs = executeQuery(conn, queryBuilder);
+            assertResultSet(rs, new Object[][]{{1,8,2L},{1,9,4L},{2,3,1L},{2,7,2L},{3,2,2L}});
+
+            queryBuilder.setSelectExpression("PK2,PK1,COUNT(V)");
+            queryBuilder.setOrderByClause("PK2 DESC,PK1 DESC");
+            rs = executeQuery(conn, queryBuilder);
+            assertResultSet(rs, new Object[][]{{9,1,4L},{8,1,2L},{7,2,2L},{3,2,1L},{2,3,2L}});
+
+            queryBuilder.setSelectExpression("PK1,PK2,COUNT(V)");
+            queryBuilder.setOrderByClause("PK1 DESC,PK2 DESC");
+            rs = executeQuery(conn, queryBuilder);
+            assertResultSet(rs, new Object[][]{{3,2,2L},{2,7,2L},{2,3,1L},{1,9,4L},{1,8,2L}});
+
+
+            queryBuilder.setSelectExpression("PK3,PK2,COUNT(V)");
+            queryBuilder.setSelectExpressionColumns(Lists.newArrayList("PK1", "PK2", "PK3", "V"));
+            queryBuilder.setFullTableName(tableName);
+            queryBuilder.setGroupByClause("PK3,PK2");
+            queryBuilder.setOrderByClause("PK3,PK2");
+            queryBuilder.setWhereClause("PK1=1");
+            rs = executeQuery(conn, queryBuilder);
+            assertResultSet(rs, new Object[][]{{5,9,1L},{6,9,2L},{7,9,1L},{10,8,1L},{11,8,1L}});
+
+            queryBuilder.setSelectExpression("PK2,PK3,COUNT(V)");
+            queryBuilder.setOrderByClause("PK2,PK3");
+            rs = executeQuery(conn, queryBuilder);
+            assertResultSet(rs, new Object[][]{{8,10,1L},{8,11,1L},{9,5,1L},{9,6,2L},{9,7,1L}});
+
+            queryBuilder.setSelectExpression("PK3,PK2,COUNT(V)");
+            queryBuilder.setOrderByClause("PK3 DESC,PK2 DESC");
+            rs = executeQuery(conn, queryBuilder);
+            assertResultSet(rs, new Object[][]{{11,8,1L},{10,8,1L},{7,9,1L},{6,9,2L},{5,9,1L}});
+
+            queryBuilder.setSelectExpression("PK2,PK3,COUNT(V)");
+            queryBuilder.setOrderByClause("PK2 DESC,PK3 DESC");
+            rs = executeQuery(conn, queryBuilder);
+            assertResultSet(rs, new Object[][]{{9,7,1L},{9,6,2L},{9,5,1L},{8,11,1L},{8,10,1L}});
+
+
+            queryBuilder.setSelectExpression("PK4,PK3,PK1,COUNT(V)");
+            queryBuilder.setSelectExpressionColumns(Lists.newArrayList("PK1", "PK2", "PK3", "PK4", "V"));
+            queryBuilder.setFullTableName(tableName);
+            queryBuilder.setWhereClause("PK2=9 ");
+            queryBuilder.setGroupByClause("PK4,PK3,PK1");
+            queryBuilder.setOrderByClause("PK4,PK3,PK1");
+            rs = executeQuery(conn, queryBuilder);
+            assertResultSet(rs, new Object[][]{{8,7,1,1L},{12,6,1,1L},{13,6,1,1L},{22,5,1,1L}});
+
+            queryBuilder.setSelectExpression("PK1,PK3,PK4,COUNT(V)");
+            queryBuilder.setOrderByClause("PK1,PK3,PK4");
+            rs = executeQuery(conn, queryBuilder);
+            assertResultSet(rs, new Object[][]{{1,5,22,1L},{1,6,12,1L},{1,6,13,1L},{1,7,8,1L}});
+
+            queryBuilder.setSelectExpression("PK4,PK3,PK1,COUNT(V)");
+            queryBuilder.setOrderByClause("PK4 DESC,PK3 DESC,PK1 DESC");
+            rs = executeQuery(conn, queryBuilder);
+            assertResultSet(rs, new Object[][]{{22,5,1,1L},{13,6,1,1L},{12,6,1,1L},{8,7,1,1L}});
+
+            queryBuilder.setSelectExpression("PK1,PK3,PK4,COUNT(V)");
+            queryBuilder.setOrderByClause("PK1 DESC,PK3 DESC,PK4 DESC");
+            rs = executeQuery(conn, queryBuilder);
+            assertResultSet(rs, new Object[][]{{1,7,8,1L},{1,6,13,1L},{1,6,12,1L},{1,5,22,1L}});
+        } finally {
+            if(conn != null) {
+                conn.close();
+            }
+        }
+    }
+}


[4/6] phoenix git commit: PHOENIX-4981 Add tests for ORDER BY, GROUP BY and salted tables using phoenix-spark

Posted by td...@apache.org.
http://git-wip-us.apache.org/repos/asf/phoenix/blob/c509d58f/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseOrderByIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseOrderByIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseOrderByIT.java
new file mode 100644
index 0000000..31bf050
--- /dev/null
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseOrderByIT.java
@@ -0,0 +1,940 @@
+/*
+ * 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.ROW1;
+import static org.apache.phoenix.util.TestUtil.ROW2;
+import static org.apache.phoenix.util.TestUtil.ROW3;
+import static org.apache.phoenix.util.TestUtil.ROW4;
+import static org.apache.phoenix.util.TestUtil.ROW5;
+import static org.apache.phoenix.util.TestUtil.ROW6;
+import static org.apache.phoenix.util.TestUtil.ROW7;
+import static org.apache.phoenix.util.TestUtil.ROW8;
+import static org.apache.phoenix.util.TestUtil.ROW9;
+import static org.apache.phoenix.util.TestUtil.TEST_PROPERTIES;
+import static org.apache.phoenix.util.TestUtil.assertResultSet;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+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 com.google.common.collect.Lists;
+import org.apache.phoenix.util.PropertiesUtil;
+import org.apache.phoenix.util.QueryBuilder;
+import org.junit.Test;
+
+
+public abstract class BaseOrderByIT extends ParallelStatsDisabledIT {
+
+    @Test
+    public void testMultiOrderByExpr() throws Exception {
+        String tenantId = getOrganizationId();
+        String tableName = initATableValues(tenantId, getDefaultSplits(tenantId), getUrl());
+        QueryBuilder queryBuilder = new QueryBuilder()
+            .setSelectColumns(
+                    Lists.newArrayList("ENTITY_ID", "B_STRING"))
+            .setFullTableName(tableName)
+            .setOrderByClause("B_STRING, ENTITY_ID");
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
+            ResultSet rs = executeQuery(conn, queryBuilder);
+            assertTrue (rs.next());
+            assertEquals(ROW1,rs.getString(1));
+            assertTrue (rs.next());
+            assertEquals(ROW4,rs.getString(1));
+            assertTrue (rs.next());
+            assertEquals(ROW7,rs.getString(1));
+            assertTrue (rs.next());
+            assertEquals(ROW2,rs.getString(1));
+            assertTrue (rs.next());
+            assertEquals(ROW5,rs.getString(1));
+            assertTrue (rs.next());
+            assertEquals(ROW8,rs.getString(1));
+            assertTrue (rs.next());
+            assertEquals(ROW3,rs.getString(1));
+            assertTrue (rs.next());
+            assertEquals(ROW6,rs.getString(1));
+            assertTrue (rs.next());
+            assertEquals(ROW9,rs.getString(1));
+
+            assertFalse(rs.next());
+        }
+    }
+
+
+    @Test
+    public void testDescMultiOrderByExpr() throws Exception {
+        String tenantId = getOrganizationId();
+        String tableName = initATableValues(tenantId, getDefaultSplits(tenantId), getUrl());
+        QueryBuilder queryBuilder = new QueryBuilder()
+            .setSelectColumns(
+                    Lists.newArrayList("ENTITY_ID", "B_STRING"))
+            .setFullTableName(tableName)
+            .setOrderByClause("B_STRING || ENTITY_ID DESC");
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
+            ResultSet rs = executeQuery(conn, queryBuilder);
+            assertTrue (rs.next());
+            assertEquals(ROW9,rs.getString(1));
+            assertTrue (rs.next());
+            assertEquals(ROW6,rs.getString(1));
+            assertTrue (rs.next());
+            assertEquals(ROW3,rs.getString(1));
+            assertTrue (rs.next());
+            assertEquals(ROW8,rs.getString(1));
+            assertTrue (rs.next());
+            assertEquals(ROW5,rs.getString(1));
+            assertTrue (rs.next());
+            assertEquals(ROW2,rs.getString(1));
+            assertTrue (rs.next());
+            assertEquals(ROW7,rs.getString(1));
+            assertTrue (rs.next());
+            assertEquals(ROW4,rs.getString(1));
+            assertTrue (rs.next());
+            assertEquals(ROW1,rs.getString(1));
+
+            assertFalse(rs.next());
+        }
+    }
+
+    @Test
+    public void testOrderByDifferentColumns() throws Exception {
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
+            conn.setAutoCommit(false);
+            String tableName = generateUniqueName();
+            String ddl = "CREATE TABLE " + tableName +
+                    "  (a_string varchar not null, col1 integer" +
+                    "  CONSTRAINT pk PRIMARY KEY (a_string))\n";
+            createTestTable(getUrl(), ddl);
+
+            String dml = "UPSERT INTO " + tableName + " VALUES(?, ?)";
+            PreparedStatement stmt = conn.prepareStatement(dml);
+            stmt.setString(1, "a");
+            stmt.setInt(2, 40);
+            stmt.execute();
+            stmt.setString(1, "b");
+            stmt.setInt(2, 20);
+            stmt.execute();
+            stmt.setString(1, "c");
+            stmt.setInt(2, 30);
+            stmt.execute();
+            conn.commit();
+
+            QueryBuilder queryBuilder = new QueryBuilder()
+                .setSelectColumns(
+                        Lists.newArrayList("COL1"))
+                .setFullTableName(tableName)
+                .setSelectExpression("count(*)")
+                .setGroupByClause("COL1")
+                .setOrderByClause("COL1");
+            ResultSet rs = executeQuery(conn, queryBuilder);
+            assertTrue(rs.next());
+            assertEquals(1,rs.getLong(1));
+            assertTrue(rs.next());
+            assertEquals(1,rs.getLong(1));
+            assertTrue(rs.next());
+            assertEquals(1,rs.getLong(1));
+            assertFalse(rs.next());
+
+            queryBuilder = new QueryBuilder();
+            queryBuilder.setSelectColumns(
+                    Lists.newArrayList("A_STRING",  "COL1"));
+            queryBuilder.setFullTableName(tableName);
+            queryBuilder.setOrderByClause("A_STRING");
+            rs = executeQuery(conn, queryBuilder);
+            assertTrue(rs.next());
+            assertEquals("a",rs.getString(1));
+            assertEquals(40,rs.getInt(2));
+            assertTrue(rs.next());
+            assertEquals("b",rs.getString(1));
+            assertEquals(20,rs.getInt(2));
+            assertTrue(rs.next());
+            assertEquals("c",rs.getString(1));  
+            assertEquals(30,rs.getInt(2));
+            assertFalse(rs.next());  
+
+            queryBuilder.setSelectColumns(
+                    Lists.newArrayList("A_STRING", "COL1"));
+            queryBuilder.setFullTableName(tableName);
+            queryBuilder.setOrderByClause("COL1");
+            rs = executeQuery(conn, queryBuilder);
+            assertTrue(rs.next());
+            assertEquals("b",rs.getString(1));
+            assertEquals(20,rs.getInt(2));
+            assertTrue(rs.next());
+            assertEquals("c",rs.getString(1));  
+            assertEquals(30,rs.getInt(2));
+            assertTrue(rs.next());
+            assertEquals("a",rs.getString(1));
+            assertEquals(40,rs.getInt(2));
+            assertFalse(rs.next());  
+        }
+    }
+
+    @Test
+    public void testAggregateOrderBy() throws Exception {
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        Connection conn = DriverManager.getConnection(getUrl(), props);
+        String tableName = generateUniqueName();
+        String ddl = "create table " + tableName + " (ID VARCHAR NOT NULL PRIMARY KEY, VAL1 VARCHAR, VAL2 INTEGER)";
+        conn.createStatement().execute(ddl);
+
+        conn.createStatement().execute("upsert into " + tableName + " values ('ABC','aa123', 11)");
+        conn.createStatement().execute("upsert into " + tableName + " values ('ABD','ba124', 1)");
+        conn.createStatement().execute("upsert into " + tableName + " values ('ABE','cf125', 13)");
+        conn.createStatement().execute("upsert into " + tableName + " values ('ABF','dan126', 4)");
+        conn.createStatement().execute("upsert into " + tableName + " values ('ABG','elf127', 15)");
+        conn.createStatement().execute("upsert into " + tableName + " values ('ABH','fan128', 6)");
+        conn.createStatement().execute("upsert into " + tableName + " values ('AAA','get211', 100)");
+        conn.createStatement().execute("upsert into " + tableName + " values ('AAB','hat212', 7)");
+        conn.createStatement().execute("upsert into " + tableName + " values ('AAC','aap12', 2)");
+        conn.createStatement().execute("upsert into " + tableName + " values ('AAD','ball12', 3)");
+        conn.createStatement().execute("upsert into " + tableName + " values ('AAE','inn2110', 13)");
+        conn.createStatement().execute("upsert into " + tableName + " values ('AAF','key2112', 40)");
+        conn.commit();
+
+        QueryBuilder queryBuilder = new QueryBuilder()
+            .setSelectColumns(
+                    Lists.newArrayList("VAL1", "VAL2"))
+            .setFullTableName(tableName)
+            .setOrderByClause("VAL1")
+            .setSelectExpression("DISTINCT(ID)")
+            .setSelectExpressionColumns(Lists.newArrayList("ID"))
+            .setWhereClause(
+                "ID in ('ABC','ABD','ABE','ABF','ABG','ABH','AAA', 'AAB', 'AAC','AAD','AAE','AAF')");
+        ResultSet rs = executeQuery(conn, queryBuilder);
+        assertTrue(rs.next());
+        assertEquals("ABC", rs.getString(1));
+        assertEquals("aa123", rs.getString(2));
+        assertTrue(rs.next());
+        assertEquals("aap12", rs.getString(2));
+        assertTrue(rs.next());
+        assertEquals("ba124", rs.getString(2));
+        assertTrue(rs.next());
+        assertEquals("ball12", rs.getString(2));
+        assertTrue(rs.next());
+        assertEquals("cf125", rs.getString(2));
+        assertTrue(rs.next());
+        assertEquals("dan126", rs.getString(2));
+        assertTrue(rs.next());
+        assertEquals("elf127", rs.getString(2));
+        assertTrue(rs.next());
+        assertEquals("fan128", rs.getString(2));
+        assertTrue(rs.next());
+        assertEquals("get211", rs.getString(2));
+        assertTrue(rs.next());
+        assertEquals("hat212", rs.getString(2));
+        assertTrue(rs.next());
+        assertEquals("inn2110", rs.getString(2));
+        assertTrue(rs.next());
+        assertEquals("AAF", rs.getString(1));
+        assertEquals("key2112", rs.getString(2));
+        assertFalse(rs.next());
+    }
+
+    @Test
+    public void testAggregateOptimizedOutOrderBy() throws Exception {
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        Connection conn = DriverManager.getConnection(getUrl(), props);
+        String tableName = generateUniqueName();
+        String ddl = "create table " + tableName + " (K1 VARCHAR NOT NULL, K2 VARCHAR NOT NULL, VAL1 VARCHAR, VAL2 INTEGER, CONSTRAINT pk PRIMARY KEY(K1,K2))";
+        conn.createStatement().execute(ddl);
+
+        conn.createStatement().execute("upsert into " + tableName + " values ('ABC','ABC','aa123', 11)");
+        conn.createStatement().execute("upsert into " + tableName + " values ('ABD','ABC','ba124', 1)");
+        conn.createStatement().execute("upsert into " + tableName + " values ('ABE','ABC','cf125', 13)");
+        conn.createStatement().execute("upsert into " + tableName + " values ('ABF','ABC','dan126', 4)");
+        conn.createStatement().execute("upsert into " + tableName + " values ('ABG','ABC','elf127', 15)");
+        conn.createStatement().execute("upsert into " + tableName + " values ('ABH','ABC','fan128', 6)");
+        conn.createStatement().execute("upsert into " + tableName + " values ('AAA','ABC','get211', 100)");
+        conn.createStatement().execute("upsert into " + tableName + " values ('AAB','ABC','hat212', 7)");
+        conn.createStatement().execute("upsert into " + tableName + " values ('AAC','ABC','aap12', 2)");
+        conn.createStatement().execute("upsert into " + tableName + " values ('AAD','ABC','ball12', 3)");
+        conn.createStatement().execute("upsert into " + tableName + " values ('AAE','ABC','inn2110', 13)");
+        conn.createStatement().execute("upsert into " + tableName + " values ('AAF','ABC','key2112', 40)");
+        conn.commit();
+
+        QueryBuilder queryBuilder = new QueryBuilder()
+            .setSelectColumns(
+                    Lists.newArrayList("VAL1", "VAL2"))
+            .setFullTableName(tableName)
+            .setOrderByClause("VAL1")
+            .setSelectExpressionColumns(Lists.newArrayList("K2"))
+            .setSelectExpression("DISTINCT(K2)")
+        .setWhereClause("K2 = 'ABC'");
+
+        // verify that the phoenix query plan doesn't contain an order by
+        String expectedPhoenixPlan = "CLIENT PARALLEL 1-WAY FULL SCAN OVER " + tableName +"\n" +
+                "    SERVER FILTER BY K2 = 'ABC'\n" +
+                "    SERVER AGGREGATE INTO DISTINCT ROWS BY [K2, VAL1, VAL2]\n" +
+                "CLIENT MERGE SORT";
+        validateQueryPlan(conn, queryBuilder, expectedPhoenixPlan, null);
+
+        ResultSet rs = executeQuery(conn, queryBuilder);
+        assertTrue(rs.next());
+        assertEquals("ABC", rs.getString(1));
+        assertEquals("aa123", rs.getString(2));
+        assertTrue(rs.next());
+        assertEquals("aap12", rs.getString(2));
+        assertTrue(rs.next());
+        assertEquals("ba124", rs.getString(2));
+        assertTrue(rs.next());
+        assertEquals("ball12", rs.getString(2));
+        assertTrue(rs.next());
+        assertEquals("cf125", rs.getString(2));
+        assertTrue(rs.next());
+        assertEquals("dan126", rs.getString(2));
+        assertTrue(rs.next());
+        assertEquals("elf127", rs.getString(2));
+        assertTrue(rs.next());
+        assertEquals("fan128", rs.getString(2));
+        assertTrue(rs.next());
+        assertEquals("get211", rs.getString(2));
+        assertTrue(rs.next());
+        assertEquals("hat212", rs.getString(2));
+        assertTrue(rs.next());
+        assertEquals("inn2110", rs.getString(2));
+        assertTrue(rs.next());
+        assertEquals("ABC", rs.getString(1));
+        assertEquals("key2112", rs.getString(2));
+        assertFalse(rs.next());
+    }
+
+    @Test
+    public void testNullsLastWithDesc() throws Exception {
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
+            String tableName=generateUniqueName();
+            String sql="CREATE TABLE "+tableName+" ( "+
+                "ORGANIZATION_ID VARCHAR,"+
+                "CONTAINER_ID VARCHAR,"+
+                "ENTITY_ID VARCHAR NOT NULL,"+
+                "CONSTRAINT TEST_PK PRIMARY KEY ( "+
+                "ORGANIZATION_ID DESC,"+
+                "CONTAINER_ID DESC,"+
+                "ENTITY_ID"+
+                "))";
+            conn.createStatement().execute(sql);
+
+            conn.createStatement().execute("UPSERT INTO "+tableName+" VALUES ('a',null,'11')");
+            conn.createStatement().execute("UPSERT INTO "+tableName+" VALUES (null,'2','22')");
+            conn.createStatement().execute("UPSERT INTO "+tableName+" VALUES ('c','3','33')");
+            conn.commit();
+
+            //-----ORGANIZATION_ID
+
+//            sql="SELECT CONTAINER_ID,ORGANIZATION_ID FROM "+tableName+" order by ORGANIZATION_ID ASC NULLS FIRST";
+            QueryBuilder queryBuilder = new QueryBuilder()
+                .setSelectColumns(
+                        Lists.newArrayList("CONTAINER_ID", "ORGANIZATION_ID"))
+                .setFullTableName(tableName)
+                .setOrderByClause("ORGANIZATION_ID ASC NULLS FIRST");
+            ResultSet rs = executeQuery(conn, queryBuilder);
+            assertResultSet(rs, new String[][]{{"2",null},{null,"a"},{"3","c"},});
+
+//            sql="SELECT CONTAINER_ID,ORGANIZATION_ID FROM "+tableName+" order by ORGANIZATION_ID ASC NULLS LAST";
+            queryBuilder.setOrderByClause("ORGANIZATION_ID ASC NULLS LAST");
+            rs = executeQuery(conn, queryBuilder);
+            assertResultSet(rs, new String[][]{{null,"a"},{"3","c"},{"2",null}});
+
+//            sql="SELECT CONTAINER_ID,ORGANIZATION_ID FROM "+tableName+" order by ORGANIZATION_ID DESC NULLS FIRST";
+            queryBuilder.setOrderByClause("ORGANIZATION_ID DESC NULLS FIRST");
+            rs = executeQuery(conn, queryBuilder);
+            assertResultSet(rs, new String[][]{{"2",null},{"3","c"},{null,"a"}});
+
+//            sql="SELECT CONTAINER_ID,ORGANIZATION_ID FROM "+tableName+" order by ORGANIZATION_ID DESC NULLS LAST";
+            queryBuilder.setOrderByClause("ORGANIZATION_ID DESC NULLS LAST");
+            rs = executeQuery(conn, queryBuilder);
+            assertResultSet(rs, new String[][]{{"3","c"},{null,"a"},{"2",null}});
+
+            //----CONTAINER_ID
+
+//            sql="SELECT CONTAINER_ID,ORGANIZATION_ID FROM "+tableName+" order by CONTAINER_ID ASC NULLS FIRST";
+            queryBuilder.setOrderByClause("CONTAINER_ID ASC NULLS FIRST");
+            rs = executeQuery(conn, queryBuilder);
+            assertResultSet(rs, new String[][]{{null,"a"},{"2",null},{"3","c"}});
+
+//            sql="SELECT CONTAINER_ID,ORGANIZATION_ID FROM "+tableName+" order by CONTAINER_ID ASC NULLS LAST";
+            queryBuilder.setOrderByClause("CONTAINER_ID ASC NULLS LAST");
+            rs = executeQuery(conn, queryBuilder);
+            assertResultSet(rs, new String[][]{{"2",null},{"3","c"},{null,"a"}});
+
+//            sql="SELECT CONTAINER_ID,ORGANIZATION_ID FROM "+tableName+" order by CONTAINER_ID DESC NULLS FIRST";
+            queryBuilder.setOrderByClause("CONTAINER_ID DESC NULLS FIRST");
+            rs = executeQuery(conn, queryBuilder);
+            assertResultSet(rs, new String[][]{{null,"a"},{"3","c"},{"2",null}});
+
+//            sql="SELECT CONTAINER_ID,ORGANIZATION_ID FROM "+tableName+" order by CONTAINER_ID DESC NULLS LAST";
+            queryBuilder.setOrderByClause("CONTAINER_ID DESC NULLS LAST");
+            rs = executeQuery(conn, queryBuilder);
+            assertResultSet(rs, new String[][]{{"3","c"},{"2",null},{null,"a"}});
+
+            conn.createStatement().execute("UPSERT INTO "+tableName+" VALUES (null,null,'44')");
+            conn.commit();
+
+            //-----ORGANIZATION_ID ASC  CONTAINER_ID ASC
+
+//            sql="SELECT CONTAINER_ID,ORGANIZATION_ID FROM "+tableName+" order by ORGANIZATION_ID NULLS FIRST,CONTAINER_ID NULLS FIRST";
+            queryBuilder.setOrderByClause("ORGANIZATION_ID NULLS FIRST,CONTAINER_ID NULLS FIRST");
+            rs = executeQuery(conn, queryBuilder);
+            assertResultSet(rs, new String[][]{{null,null},{"2",null},{null,"a"},{"3","c"}});
+
+//            sql="SELECT CONTAINER_ID,ORGANIZATION_ID FROM "+tableName+" order by ORGANIZATION_ID NULLS FIRST,CONTAINER_ID NULLS LAST";
+            queryBuilder.setOrderByClause("ORGANIZATION_ID NULLS FIRST,CONTAINER_ID NULLS LAST");
+            rs = executeQuery(conn, queryBuilder);
+            assertResultSet(rs, new String[][]{{"2",null},{null,null},{null,"a"},{"3","c"}});
+
+//            sql="SELECT CONTAINER_ID,ORGANIZATION_ID FROM "+tableName+" order by ORGANIZATION_ID NULLS LAST,CONTAINER_ID NULLS FIRST";
+            queryBuilder.setOrderByClause("ORGANIZATION_ID NULLS LAST,CONTAINER_ID NULLS FIRST");
+            rs = executeQuery(conn, queryBuilder);
+            assertResultSet(rs, new String[][]{{null,"a"},{"3","c"},{null,null},{"2",null}});
+
+//            sql="SELECT CONTAINER_ID,ORGANIZATION_ID FROM "+tableName+" order by ORGANIZATION_ID NULLS LAST,CONTAINER_ID NULLS LAST";
+            queryBuilder.setOrderByClause("ORGANIZATION_ID NULLS LAST,CONTAINER_ID NULLS LAST");
+            rs = executeQuery(conn, queryBuilder);
+            assertResultSet(rs, new String[][]{{null,"a"},{"3","c"},{"2",null},{null,null}});
+
+
+            //-----ORGANIZATION_ID ASC  CONTAINER_ID DESC
+
+//            sql="SELECT CONTAINER_ID,ORGANIZATION_ID FROM "+tableName+" order by ORGANIZATION_ID NULLS FIRST,CONTAINER_ID DESC NULLS FIRST";
+            queryBuilder.setOrderByClause("ORGANIZATION_ID NULLS FIRST,CONTAINER_ID DESC NULLS FIRST");
+            rs = executeQuery(conn, queryBuilder);
+            assertResultSet(rs, new String[][]{{null,null},{"2",null},{null,"a"},{"3","c"}});
+
+//            sql="SELECT CONTAINER_ID,ORGANIZATION_ID FROM "+tableName+" order by ORGANIZATION_ID NULLS FIRST,CONTAINER_ID DESC NULLS LAST";
+            queryBuilder.setOrderByClause("ORGANIZATION_ID NULLS FIRST,CONTAINER_ID DESC NULLS LAST");
+            rs = executeQuery(conn, queryBuilder);
+            assertResultSet(rs, new String[][]{{"2",null},{null,null},{null,"a"},{"3","c"}});
+
+//            sql="SELECT CONTAINER_ID,ORGANIZATION_ID FROM "+tableName+" order by ORGANIZATION_ID NULLS LAST,CONTAINER_ID DESC NULLS FIRST";
+            queryBuilder.setOrderByClause("ORGANIZATION_ID NULLS LAST,CONTAINER_ID DESC NULLS FIRST");
+            rs = executeQuery(conn, queryBuilder);
+            assertResultSet(rs, new String[][]{{null,"a"},{"3","c"},{null,null},{"2",null}});
+
+//            sql="SELECT CONTAINER_ID,ORGANIZATION_ID FROM "+tableName+" order by ORGANIZATION_ID NULLS LAST,CONTAINER_ID DESC NULLS LAST";
+            queryBuilder.setOrderByClause("ORGANIZATION_ID NULLS LAST,CONTAINER_ID DESC NULLS LAST");
+            rs = executeQuery(conn, queryBuilder);
+            assertResultSet(rs, new String[][]{{null,"a"},{"3","c"},{"2",null},{null,null}});
+
+            //-----ORGANIZATION_ID DESC  CONTAINER_ID ASC
+
+//            sql="SELECT CONTAINER_ID,ORGANIZATION_ID FROM "+tableName+" order by ORGANIZATION_ID DESC NULLS FIRST,CONTAINER_ID NULLS FIRST";
+            queryBuilder.setOrderByClause("ORGANIZATION_ID DESC NULLS FIRST,CONTAINER_ID NULLS FIRST");
+            rs = executeQuery(conn, queryBuilder);
+            assertResultSet(rs, new String[][]{{null,null},{"2",null},{"3","c"},{null,"a"}});
+
+//            sql="SELECT CONTAINER_ID,ORGANIZATION_ID FROM "+tableName+" order by ORGANIZATION_ID DESC NULLS FIRST,CONTAINER_ID NULLS LAST";
+            queryBuilder.setOrderByClause("ORGANIZATION_ID DESC NULLS FIRST,CONTAINER_ID NULLS LAST");
+            rs = executeQuery(conn, queryBuilder);
+            assertResultSet(rs, new String[][]{{"2",null},{null,null},{"3","c"},{null,"a"}});
+
+//            sql="SELECT CONTAINER_ID,ORGANIZATION_ID FROM "+tableName+" order by ORGANIZATION_ID DESC NULLS LAST,CONTAINER_ID NULLS FIRST";
+            queryBuilder.setOrderByClause("ORGANIZATION_ID DESC NULLS LAST,CONTAINER_ID NULLS FIRST");
+            rs = executeQuery(conn, queryBuilder);
+            assertResultSet(rs, new String[][]{{"3","c"},{null,"a"},{null,null},{"2",null}});
+
+//            sql="SELECT CONTAINER_ID,ORGANIZATION_ID FROM "+tableName+" order by ORGANIZATION_ID DESC NULLS LAST,CONTAINER_ID NULLS LAST";
+            queryBuilder.setOrderByClause("ORGANIZATION_ID DESC NULLS LAST,CONTAINER_ID NULLS LAST");
+            rs = executeQuery(conn, queryBuilder);
+            assertResultSet(rs, new String[][]{{"3","c"},{null,"a"},{"2",null},{null,null}});
+
+            //-----ORGANIZATION_ID DESC  CONTAINER_ID DESC
+
+//            sql="SELECT CONTAINER_ID,ORGANIZATION_ID FROM "+tableName+" order by ORGANIZATION_ID DESC NULLS FIRST,CONTAINER_ID DESC NULLS FIRST";
+            queryBuilder.setOrderByClause("ORGANIZATION_ID DESC NULLS FIRST,CONTAINER_ID DESC NULLS FIRST");
+            rs = executeQuery(conn, queryBuilder);
+            assertResultSet(rs, new String[][]{{null,null},{"2",null},{"3","c"},{null,"a"}});
+
+//            sql="SELECT CONTAINER_ID,ORGANIZATION_ID FROM "+tableName+" order by ORGANIZATION_ID DESC NULLS FIRST,CONTAINER_ID DESC NULLS LAST";
+            queryBuilder.setOrderByClause("ORGANIZATION_ID DESC NULLS FIRST,CONTAINER_ID DESC NULLS LAST");
+            rs = executeQuery(conn, queryBuilder);
+            assertResultSet(rs, new String[][]{{"2",null},{null,null},{"3","c"},{null,"a"}});
+
+//            sql="SELECT CONTAINER_ID,ORGANIZATION_ID FROM "+tableName+" order by ORGANIZATION_ID DESC NULLS LAST,CONTAINER_ID DESC NULLS FIRST";
+            queryBuilder.setOrderByClause("ORGANIZATION_ID DESC NULLS LAST,CONTAINER_ID DESC NULLS FIRST");
+            rs = executeQuery(conn, queryBuilder);
+            assertResultSet(rs, new String[][]{{"3","c"},{null,"a"},{null,null},{"2",null}});
+
+//            sql="SELECT CONTAINER_ID,ORGANIZATION_ID FROM "+tableName+" order by ORGANIZATION_ID DESC NULLS LAST,CONTAINER_ID DESC NULLS LAST";
+            queryBuilder.setOrderByClause("ORGANIZATION_ID DESC NULLS LAST,CONTAINER_ID DESC NULLS LAST");
+            rs = executeQuery(conn, queryBuilder);
+            assertResultSet(rs, new String[][]{{"3","c"},{null,"a"},{"2",null},{null,null}});
+
+            //-----CONTAINER_ID ASC  ORGANIZATION_ID ASC
+
+//            sql="SELECT CONTAINER_ID,ORGANIZATION_ID FROM "+tableName+" order by CONTAINER_ID NULLS FIRST,ORGANIZATION_ID NULLS FIRST";
+            queryBuilder.setOrderByClause("CONTAINER_ID NULLS FIRST,ORGANIZATION_ID NULLS FIRST");
+            rs = executeQuery(conn, queryBuilder);
+            assertResultSet(rs, new String[][]{{null,null},{null,"a"},{"2",null},{"3","c"}});
+
+//            sql="SELECT CONTAINER_ID,ORGANIZATION_ID FROM "+tableName+" order by CONTAINER_ID NULLS FIRST,ORGANIZATION_ID NULLS LAST";
+            queryBuilder.setOrderByClause("CONTAINER_ID NULLS FIRST,ORGANIZATION_ID NULLS LAST");
+            rs = executeQuery(conn, queryBuilder);
+            assertResultSet(rs, new String[][]{{null,"a"},{null,null},{"2",null},{"3","c"}});
+
+//            sql="SELECT CONTAINER_ID,ORGANIZATION_ID FROM "+tableName+" order by CONTAINER_ID NULLS LAST,ORGANIZATION_ID NULLS FIRST";
+            queryBuilder.setOrderByClause("CONTAINER_ID NULLS LAST,ORGANIZATION_ID NULLS FIRST");
+            rs = executeQuery(conn, queryBuilder);
+            assertResultSet(rs, new String[][]{{"2",null},{"3","c"},{null,null},{null,"a"}});
+
+//            sql="SELECT CONTAINER_ID,ORGANIZATION_ID FROM "+tableName+" order by CONTAINER_ID NULLS LAST,ORGANIZATION_ID NULLS LAST";
+            queryBuilder.setOrderByClause("CONTAINER_ID NULLS LAST,ORGANIZATION_ID NULLS LAST");
+            rs = executeQuery(conn, queryBuilder);
+            assertResultSet(rs, new String[][]{{"2",null},{"3","c"},{null,"a"},{null,null}});
+
+            //-----CONTAINER_ID ASC  ORGANIZATION_ID DESC
+
+//            sql="SELECT CONTAINER_ID,ORGANIZATION_ID FROM "+tableName+" order by CONTAINER_ID ASC NULLS FIRST,ORGANIZATION_ID DESC NULLS FIRST";
+            queryBuilder.setOrderByClause("CONTAINER_ID ASC NULLS FIRST,ORGANIZATION_ID DESC NULLS FIRST");
+            rs = executeQuery(conn, queryBuilder);
+            assertResultSet(rs, new String[][]{{null,null},{null,"a"},{"2",null},{"3","c"}});
+
+//            sql="SELECT CONTAINER_ID,ORGANIZATION_ID FROM "+tableName+" order by CONTAINER_ID ASC NULLS FIRST,ORGANIZATION_ID DESC NULLS LAST";
+            queryBuilder.setOrderByClause("CONTAINER_ID ASC NULLS FIRST,ORGANIZATION_ID DESC NULLS LAST");
+            rs = executeQuery(conn, queryBuilder);
+            assertResultSet(rs, new String[][]{{null,"a"},{null,null},{"2",null},{"3","c"}});
+
+//            sql="SELECT CONTAINER_ID,ORGANIZATION_ID FROM "+tableName+" order by CONTAINER_ID ASC NULLS LAST,ORGANIZATION_ID DESC NULLS FIRST";
+            queryBuilder.setOrderByClause("CONTAINER_ID ASC NULLS LAST,ORGANIZATION_ID DESC NULLS FIRST");
+            rs = executeQuery(conn, queryBuilder);
+            assertResultSet(rs, new String[][]{{"2",null},{"3","c"},{null,null},{null,"a"}});
+
+//            sql="SELECT CONTAINER_ID,ORGANIZATION_ID FROM "+tableName+" order by CONTAINER_ID ASC NULLS LAST,ORGANIZATION_ID DESC NULLS LAST";
+            queryBuilder.setOrderByClause("CONTAINER_ID ASC NULLS LAST,ORGANIZATION_ID DESC NULLS LAST");
+            rs = executeQuery(conn, queryBuilder);
+            assertResultSet(rs, new String[][]{{"2",null},{"3","c"},{null,"a"},{null,null}});
+
+            //-----CONTAINER_ID DESC  ORGANIZATION_ID ASC
+
+//            sql="SELECT CONTAINER_ID,ORGANIZATION_ID FROM "+tableName+" order by CONTAINER_ID DESC NULLS FIRST,ORGANIZATION_ID ASC NULLS FIRST";
+            queryBuilder.setOrderByClause("CONTAINER_ID DESC NULLS FIRST,ORGANIZATION_ID ASC NULLS FIRST");
+            rs = executeQuery(conn, queryBuilder);
+            assertResultSet(rs, new String[][]{{null,null},{null,"a"},{"3","c"},{"2",null}});
+
+//            sql="SELECT CONTAINER_ID,ORGANIZATION_ID FROM "+tableName+" order by CONTAINER_ID DESC NULLS FIRST,ORGANIZATION_ID ASC NULLS LAST";
+            queryBuilder.setOrderByClause("CONTAINER_ID DESC NULLS FIRST,ORGANIZATION_ID ASC NULLS LAST");
+            rs = executeQuery(conn, queryBuilder);
+            assertResultSet(rs, new String[][]{{null,"a"},{null,null},{"3","c"},{"2",null}});
+
+//            sql="SELECT CONTAINER_ID,ORGANIZATION_ID FROM "+tableName+" order by CONTAINER_ID DESC NULLS LAST,ORGANIZATION_ID ASC NULLS FIRST";
+            queryBuilder.setOrderByClause("CONTAINER_ID DESC NULLS LAST,ORGANIZATION_ID ASC NULLS FIRST");
+            rs = executeQuery(conn, queryBuilder);
+            assertResultSet(rs, new String[][]{{"3","c"},{"2",null},{null,null},{null,"a"}});
+
+//            sql="SELECT CONTAINER_ID,ORGANIZATION_ID FROM "+tableName+" order by CONTAINER_ID DESC NULLS LAST,ORGANIZATION_ID ASC NULLS LAST";
+            queryBuilder.setOrderByClause("CONTAINER_ID DESC NULLS LAST,ORGANIZATION_ID ASC NULLS LAST");
+            rs = executeQuery(conn, queryBuilder);
+            assertResultSet(rs, new String[][]{{"3","c"},{"2",null},{null,"a"},{null,null}});
+
+            //-----CONTAINER_ID DESC  ORGANIZATION_ID DESC
+
+//            sql="SELECT CONTAINER_ID,ORGANIZATION_ID FROM "+tableName+" order by CONTAINER_ID DESC NULLS FIRST,ORGANIZATION_ID DESC NULLS FIRST";
+            queryBuilder.setOrderByClause("CONTAINER_ID DESC NULLS FIRST,ORGANIZATION_ID DESC NULLS FIRST");
+            rs = executeQuery(conn, queryBuilder);
+            assertResultSet(rs, new String[][]{{null,null},{null,"a"},{"3","c"},{"2",null}});
+
+//            sql="SELECT CONTAINER_ID,ORGANIZATION_ID FROM "+tableName+" order by CONTAINER_ID DESC NULLS FIRST,ORGANIZATION_ID DESC NULLS LAST";
+            queryBuilder.setOrderByClause("CONTAINER_ID DESC NULLS FIRST,ORGANIZATION_ID DESC NULLS LAST");
+            rs = executeQuery(conn, queryBuilder);
+            assertResultSet(rs, new String[][]{{null,"a"},{null,null},{"3","c"},{"2",null}});
+
+//            sql="SELECT CONTAINER_ID,ORGANIZATION_ID FROM "+tableName+" order by CONTAINER_ID DESC NULLS LAST,ORGANIZATION_ID DESC NULLS FIRST";
+            queryBuilder.setOrderByClause("CONTAINER_ID DESC NULLS LAST,ORGANIZATION_ID DESC NULLS FIRST");
+            rs = executeQuery(conn, queryBuilder);
+            assertResultSet(rs, new String[][]{{"3","c"},{"2",null},{null,null},{null,"a"}});
+
+//            sql="SELECT CONTAINER_ID,ORGANIZATION_ID FROM "+tableName+" order by CONTAINER_ID DESC NULLS LAST,ORGANIZATION_ID DESC NULLS LAST";
+            queryBuilder.setOrderByClause("CONTAINER_ID DESC NULLS LAST,ORGANIZATION_ID DESC NULLS LAST");
+            rs = executeQuery(conn, queryBuilder);
+            assertResultSet(rs, new String[][]{{"3","c"},{"2",null},{null,"a"},{null,null}});
+        }
+    }
+
+    @Test
+    public void testOrderByReverseOptimization() throws Exception {
+        for(boolean salted: new boolean[]{true,false}) {
+            doTestOrderByReverseOptimization(salted,true,true,true);
+            doTestOrderByReverseOptimization(salted,true,true,false);
+            doTestOrderByReverseOptimization(salted,true,false,true);
+            doTestOrderByReverseOptimization(salted,true,false,false);
+            doTestOrderByReverseOptimization(salted,false,true,true);
+            doTestOrderByReverseOptimization(salted,false,true,false);
+            doTestOrderByReverseOptimization(salted,false,false,true);
+            doTestOrderByReverseOptimization(salted,false,false,false);
+        }
+    }
+
+    private void doTestOrderByReverseOptimization(boolean salted,boolean desc1,boolean desc2,boolean desc3) throws Exception {
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
+            String tableName=generateUniqueName();
+            String sql="CREATE TABLE "+tableName+" ( "+
+                    "ORGANIZATION_ID INTEGER NOT NULL,"+
+                    "CONTAINER_ID INTEGER NOT NULL,"+
+                    "SCORE INTEGER NOT NULL,"+
+                    "ENTITY_ID INTEGER NOT NULL,"+
+                    "CONSTRAINT TEST_PK PRIMARY KEY ( "+
+                    "ORGANIZATION_ID" +(desc1 ? " DESC" : "" )+","+
+                    "CONTAINER_ID"+(desc2 ? " DESC" : "" )+","+
+                    "SCORE"+(desc3 ? " DESC" : "" )+","+
+                    "ENTITY_ID"+
+                    ")) "+(salted ? "SALT_BUCKETS =4" : "split on(4)");
+            conn.createStatement().execute(sql);
+
+            conn.createStatement().execute("UPSERT INTO "+tableName+" VALUES (1,1,1,1)");
+            conn.createStatement().execute("UPSERT INTO "+tableName+" VALUES (2,2,2,2)");
+            conn.createStatement().execute("UPSERT INTO "+tableName+" VALUES (3,3,3,3)");
+            conn.createStatement().execute("UPSERT INTO "+tableName+" VALUES (4,4,4,4)");
+            conn.createStatement().execute("UPSERT INTO "+tableName+" VALUES (5,5,5,5)");
+            conn.createStatement().execute("UPSERT INTO "+tableName+" VALUES (6,6,6,6)");
+            conn.createStatement().execute("UPSERT INTO "+tableName+" VALUES (1,1,1,11)");
+            conn.createStatement().execute("UPSERT INTO "+tableName+" VALUES (2,2,2,22)");
+            conn.createStatement().execute("UPSERT INTO "+tableName+" VALUES (3,3,3,33)");
+            conn.createStatement().execute("UPSERT INTO "+tableName+" VALUES (4,4,4,44)");
+            conn.createStatement().execute("UPSERT INTO "+tableName+" VALUES (5,5,5,55)");
+            conn.createStatement().execute("UPSERT INTO "+tableName+" VALUES (6,6,6,66)");
+            conn.commit();
+
+            QueryBuilder queryBuilder = new QueryBuilder()
+                .setSelectColumns(
+                        Lists.newArrayList("CONTAINER_ID", "ORGANIZATION_ID"))
+                .setFullTableName(tableName)
+                .setGroupByClause("ORGANIZATION_ID, CONTAINER_ID")
+            .setOrderByClause(  "ORGANIZATION_ID ASC, CONTAINER_ID ASC");
+            //groupBy orderPreserving orderBy asc asc
+//            sql="SELECT ORGANIZATION_ID,CONTAINER_ID FROM "+tableName+" group by ORGANIZATION_ID, CONTAINER_ID ORDER BY ORGANIZATION_ID ASC, CONTAINER_ID ASC";
+            ResultSet rs = executeQuery(conn, queryBuilder);
+            assertResultSet(rs, new Object[][]{{1,1},{2,2},{3,3},{4,4},{5,5},{6,6}});
+
+            //groupBy orderPreserving orderBy asc desc
+//            sql="SELECT ORGANIZATION_ID,CONTAINER_ID FROM "+tableName+" group by ORGANIZATION_ID, CONTAINER_ID ORDER BY ORGANIZATION_ID ASC, CONTAINER_ID desc";
+            queryBuilder.setOrderByClause("ORGANIZATION_ID ASC, CONTAINER_ID DESC");
+            rs = executeQuery(conn, queryBuilder);
+            assertResultSet(rs, new Object[][]{{1,1},{2,2},{3,3},{4,4},{5,5},{6,6}});
+
+            //groupBy orderPreserving orderBy desc asc
+//            sql="SELECT ORGANIZATION_ID,CONTAINER_ID FROM "+tableName+" group by ORGANIZATION_ID, CONTAINER_ID ORDER BY ORGANIZATION_ID DESC, CONTAINER_ID ASC";
+            queryBuilder.setOrderByClause("ORGANIZATION_ID DESC, CONTAINER_ID ASC");
+            rs = executeQuery(conn, queryBuilder);
+            assertResultSet(rs, new Object[][]{{6,6},{5,5},{4,4},{3,3},{2,2},{1,1}});
+
+            //groupBy orderPreserving orderBy desc desc
+//            sql="SELECT ORGANIZATION_ID,CONTAINER_ID FROM "+tableName+" group by ORGANIZATION_ID, CONTAINER_ID ORDER BY ORGANIZATION_ID DESC, CONTAINER_ID DESC";
+            queryBuilder.setOrderByClause("ORGANIZATION_ID DESC, CONTAINER_ID DESC");
+            rs = executeQuery(conn, queryBuilder);
+            assertResultSet(rs, new Object[][]{{6,6},{5,5},{4,4},{3,3},{2,2},{1,1}});
+
+            //groupBy not orderPreserving orderBy asc asc
+//            sql="SELECT ORGANIZATION_ID,SCORE FROM "+tableName+" group by ORGANIZATION_ID, SCORE ORDER BY ORGANIZATION_ID ASC, SCORE ASC";
+            queryBuilder.setSelectColumns(
+                    Lists.newArrayList("ORGANIZATION_ID", "SCORE"));
+            queryBuilder.setGroupByClause("ORGANIZATION_ID, SCORE");
+            queryBuilder.setOrderByClause("ORGANIZATION_ID ASC, SCORE ASC");
+            rs = executeQuery(conn, queryBuilder);
+            assertResultSet(rs, new Object[][]{{1,1},{2,2},{3,3},{4,4},{5,5},{6,6}});
+
+            //groupBy not orderPreserving orderBy asc desc
+//            sql="SELECT ORGANIZATION_ID,SCORE FROM "+tableName+" group by ORGANIZATION_ID, SCORE ORDER BY ORGANIZATION_ID ASC, SCORE DESC";
+            queryBuilder.setGroupByClause("ORGANIZATION_ID, SCORE");
+            queryBuilder.setOrderByClause("ORGANIZATION_ID ASC, SCORE DESC");
+            rs = executeQuery(conn, queryBuilder);
+            assertResultSet(rs, new Object[][]{{1,1},{2,2},{3,3},{4,4},{5,5},{6,6}});
+
+            //groupBy not orderPreserving orderBy desc asc
+//            sql="SELECT ORGANIZATION_ID,SCORE FROM "+tableName+" group by ORGANIZATION_ID, SCORE ORDER BY ORGANIZATION_ID DESC, SCORE ASC";
+            queryBuilder.setGroupByClause("ORGANIZATION_ID, SCORE");
+            queryBuilder.setOrderByClause("ORGANIZATION_ID DESC, SCORE ASC");
+            rs = executeQuery(conn, queryBuilder);
+            assertResultSet(rs, new Object[][]{{6,6},{5,5},{4,4},{3,3},{2,2},{1,1}});
+
+            //groupBy not orderPreserving orderBy desc desc
+//            sql="SELECT ORGANIZATION_ID,SCORE FROM "+tableName+" group by ORGANIZATION_ID, SCORE ORDER BY ORGANIZATION_ID DESC, SCORE DESC";
+            queryBuilder.setGroupByClause("ORGANIZATION_ID, SCORE");
+            queryBuilder.setOrderByClause("ORGANIZATION_ID DESC, SCORE DESC");
+            rs = executeQuery(conn, queryBuilder);
+            assertResultSet(rs, new Object[][]{{6,6},{5,5},{4,4},{3,3},{2,2},{1,1}});
+        }
+    }
+
+    @Test
+    public void testOrderByReverseOptimizationWithNullsLast() throws Exception{
+        for(boolean salted: new boolean[]{true,false}) {
+            doTestOrderByReverseOptimizationWithNullsLast(salted,true,true,true);
+            doTestOrderByReverseOptimizationWithNullsLast(salted,true,true,false);
+            doTestOrderByReverseOptimizationWithNullsLast(salted,true,false,true);
+            doTestOrderByReverseOptimizationWithNullsLast(salted,true,false,false);
+            doTestOrderByReverseOptimizationWithNullsLast(salted,false,true,true);
+            doTestOrderByReverseOptimizationWithNullsLast(salted,false,true,false);
+            doTestOrderByReverseOptimizationWithNullsLast(salted,false,false,true);
+            doTestOrderByReverseOptimizationWithNullsLast(salted,false,false,false);
+        }
+    }
+
+    private void doTestOrderByReverseOptimizationWithNullsLast(boolean salted,boolean desc1,boolean desc2,boolean desc3) throws Exception {
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
+            String tableName=generateUniqueName();
+            String sql="CREATE TABLE "+tableName+" ( "+
+                    "ORGANIZATION_ID VARCHAR,"+
+                    "CONTAINER_ID VARCHAR,"+
+                    "SCORE VARCHAR,"+
+                    "ENTITY_ID VARCHAR NOT NULL,"+
+                    "CONSTRAINT TEST_PK PRIMARY KEY ( "+
+                    "ORGANIZATION_ID" +(desc1 ? " DESC" : "" )+","+
+                    "CONTAINER_ID"+(desc2 ? " DESC" : "" )+","+
+                    "SCORE"+(desc3 ? " DESC" : "" )+","+
+                    "ENTITY_ID"+
+                    ")) "+(salted ? "SALT_BUCKETS =4" : "split on('4')");
+            conn.createStatement().execute(sql);
+
+            for(int i=1;i<=6;i++) {
+                conn.createStatement().execute("UPSERT INTO "+tableName+" VALUES (null,'"+i+"','"+i+"','"+i+"')");
+                conn.createStatement().execute("UPSERT INTO "+tableName+" VALUES (null,'"+i+"',null,'"+i+"')");
+                conn.createStatement().execute("UPSERT INTO "+tableName+" VALUES (null,null,'"+i+"','"+i+"')");
+                conn.createStatement().execute("UPSERT INTO "+tableName+" VALUES (null,null,null,'"+i+"')");
+                conn.createStatement().execute("UPSERT INTO "+tableName+" VALUES ('"+i+"','"+i+"','"+i+"','"+i+"')");
+                conn.createStatement().execute("UPSERT INTO "+tableName+" VALUES ('"+i+"','"+i+"',null,'"+i+"')");
+                conn.createStatement().execute("UPSERT INTO "+tableName+" VALUES ('"+i+"',null,'"+i+"','"+i+"')");
+                conn.createStatement().execute("UPSERT INTO "+tableName+" VALUES ('"+i+"',null,null,'"+i+"')");
+            }
+            conn.createStatement().execute("UPSERT INTO "+tableName+" VALUES (null,null,null,'66')");
+            conn.commit();
+
+            //groupBy orderPreserving orderBy asc asc
+
+//            sql="SELECT ORGANIZATION_ID,CONTAINER_ID FROM "+tableName+" group by ORGANIZATION_ID, CONTAINER_ID ORDER BY ORGANIZATION_ID ASC NULLS FIRST, CONTAINER_ID ASC NULLS FIRST";
+            QueryBuilder queryBuilder = new QueryBuilder()
+            .setSelectColumns(
+                    Lists.newArrayList("ORGANIZATION_ID", "CONTAINER_ID"))
+            .setFullTableName(tableName)
+            .setGroupByClause("ORGANIZATION_ID, CONTAINER_ID")
+            .setOrderByClause("ORGANIZATION_ID ASC NULLS FIRST, CONTAINER_ID ASC NULLS FIRST");
+            ResultSet rs = executeQuery(conn, queryBuilder);
+            assertResultSet(rs, new Object[][]{{null,null},{null,"1"},{null,"2"},{null,"3"},{null,"4"},{null,"5"},{null,"6"},{"1",null},{"1","1"},{"2",null},{"2","2"},{"3",null},{"3","3"},{"4",null},{"4","4"},{"5",null},{"5","5"},{"6",null},{"6","6"}});
+
+//            sql="SELECT ORGANIZATION_ID,CONTAINER_ID FROM "+tableName+" group by ORGANIZATION_ID, CONTAINER_ID ORDER BY ORGANIZATION_ID ASC NULLS FIRST, CONTAINER_ID ASC NULLS LAST";
+            queryBuilder.setOrderByClause("ORGANIZATION_ID ASC NULLS FIRST, CONTAINER_ID ASC NULLS LAST");
+            rs = executeQuery(conn, queryBuilder);
+            assertResultSet(rs, new Object[][]{{null,"1"},{null,"2"},{null,"3"},{null,"4"},{null,"5"},{null,"6"},{null,null},{"1","1"},{"1",null},{"2","2"},{"2",null},{"3","3"},{"3",null},{"4","4"},{"4",null},{"5","5"},{"5",null},{"6","6"},{"6",null}});
+
+//            sql="SELECT ORGANIZATION_ID,CONTAINER_ID FROM "+tableName+" group by ORGANIZATION_ID, CONTAINER_ID ORDER BY ORDER BY ORGANIZATION_ID ASC NULLS FIRST, CONTAINER_ID ASC NULLS LAST";
+            queryBuilder.setOrderByClause("ORGANIZATION_ID ASC NULLS FIRST, CONTAINER_ID ASC NULLS LAST");
+            rs = executeQuery(conn, queryBuilder);
+            assertResultSet(rs, new Object[][]{{null,"1"},{null,"2"},{null,"3"},{null,"4"},{null,"5"},{null,"6"},{null,null},{"1","1"},{"1",null},{"2","2"},{"2",null},{"3","3"},{"3",null},{"4","4"},{"4",null},{"5","5"},{"5",null},{"6","6"},{"6",null}});
+
+//            sql="SELECT ORGANIZATION_ID,CONTAINER_ID FROM "+tableName+" group by ORGANIZATION_ID, CONTAINER_ID ORDER BY ORGANIZATION_ID ASC NULLS LAST, CONTAINER_ID ASC NULLS LAST";
+            queryBuilder.setOrderByClause("ORGANIZATION_ID ASC NULLS LAST, CONTAINER_ID ASC NULLS LAST");
+            rs = executeQuery(conn, queryBuilder);
+            assertResultSet(rs, new Object[][]{{"1","1"},{"1",null},{"2","2"},{"2",null},{"3","3"},{"3",null},{"4","4"},{"4",null},{"5","5"},{"5",null},{"6","6"},{"6",null},{null,"1"},{null,"2"},{null,"3"},{null,"4"},{null,"5"},{null,"6"},{null,null}});
+
+            //groupBy orderPreserving orderBy asc desc
+
+//            sql="SELECT ORGANIZATION_ID,CONTAINER_ID FROM "+tableName+" group by ORGANIZATION_ID, CONTAINER_ID ORDER BY ORGANIZATION_ID ASC NULLS FIRST, CONTAINER_ID DESC NULLS FIRST";
+            queryBuilder.setOrderByClause("ORGANIZATION_ID ASC NULLS FIRST, CONTAINER_ID DESC NULLS FIRST");
+            rs = executeQuery(conn, queryBuilder);
+            assertResultSet(rs, new Object[][]{{null,null},{null,"6"},{null,"5"},{null,"4"},{null,"3"},{null,"2"},{null,"1"},{"1",null},{"1","1"},{"2",null},{"2","2"},{"3",null},{"3","3"},{"4",null},{"4","4"},{"5",null},{"5","5"},{"6",null},{"6","6"}});
+
+//            sql="SELECT ORGANIZATION_ID,CONTAINER_ID FROM "+tableName+" group by ORGANIZATION_ID, CONTAINER_ID ORDER BY ORGANIZATION_ID ASC NULLS FIRST, CONTAINER_ID DESC NULLS LAST";
+            queryBuilder.setOrderByClause("ORGANIZATION_ID ASC NULLS FIRST, CONTAINER_ID DESC NULLS LAST");
+            rs = executeQuery(conn, queryBuilder);
+            assertResultSet(rs, new Object[][]{{null,"6"},{null,"5"},{null,"4"},{null,"3"},{null,"2"},{null,"1"},{null,null},{"1","1"},{"1",null},{"2","2"},{"2",null},{"3","3"},{"3",null},{"4","4"},{"4",null},{"5","5"},{"5",null},{"6","6"},{"6",null}});
+
+//            sql="SELECT ORGANIZATION_ID,CONTAINER_ID FROM "+tableName+" group by ORGANIZATION_ID, CONTAINER_ID ORDER BY ORGANIZATION_ID ASC NULLS LAST, CONTAINER_ID DESC NULLS FIRST";
+            queryBuilder.setOrderByClause("ORGANIZATION_ID ASC NULLS LAST, CONTAINER_ID DESC NULLS FIRST");
+            rs = executeQuery(conn, queryBuilder);
+            assertResultSet(rs, new Object[][]{{"1",null},{"1","1"},{"2",null},{"2","2"},{"3",null},{"3","3"},{"4",null},{"4","4"},{"5",null},{"5","5"},{"6",null},{"6","6"},{null,null},{null,"6"},{null,"5"},{null,"4"},{null,"3"},{null,"2"},{null,"1"}});
+
+//            sql="SELECT ORGANIZATION_ID,CONTAINER_ID FROM "+tableName+" group by ORGANIZATION_ID, CONTAINER_ID ORDER BY ORGANIZATION_ID ASC NULLS LAST, CONTAINER_ID DESC NULLS LAST";
+            queryBuilder.setOrderByClause("ORGANIZATION_ID ASC NULLS LAST, CONTAINER_ID DESC NULLS LAST");
+            rs = executeQuery(conn, queryBuilder);
+            assertResultSet(rs, new Object[][]{{"1","1"},{"1",null},{"2","2"},{"2",null},{"3","3"},{"3",null},{"4","4"},{"4",null},{"5","5"},{"5",null},{"6","6"},{"6",null},{null,"6"},{null,"5"},{null,"4"},{null,"3"},{null,"2"},{null,"1"},{null,null}});
+
+            //groupBy orderPreserving orderBy desc asc
+
+//            sql="SELECT ORGANIZATION_ID,CONTAINER_ID FROM "+tableName+" group by ORGANIZATION_ID, CONTAINER_ID ORDER BY ORGANIZATION_ID DESC NULLS FIRST, CONTAINER_ID ASC NULLS FIRST";
+            queryBuilder.setOrderByClause("ORGANIZATION_ID DESC NULLS FIRST, CONTAINER_ID ASC NULLS FIRST");
+            rs = executeQuery(conn, queryBuilder);
+            assertResultSet(rs, new Object[][]{{null,null},{null,"1"},{null,"2"},{null,"3"},{null,"4"},{null,"5"},{null,"6"},{"6",null},{"6","6"},{"5",null},{"5","5"},{"4",null},{"4","4"},{"3",null},{"3","3"},{"2",null},{"2","2"},{"1",null},{"1","1"}});
+
+//            sql="SELECT ORGANIZATION_ID,CONTAINER_ID FROM "+tableName+" group by ORGANIZATION_ID, CONTAINER_ID ORDER BY ORGANIZATION_ID DESC NULLS FIRST, CONTAINER_ID ASC NULLS LAST";
+            queryBuilder.setOrderByClause("ORGANIZATION_ID DESC NULLS FIRST, CONTAINER_ID ASC NULLS LAST");
+            rs = executeQuery(conn, queryBuilder);
+            assertResultSet(rs, new Object[][]{{null,"1"},{null,"2"},{null,"3"},{null,"4"},{null,"5"},{null,"6"},{null,null},{"6","6"},{"6",null},{"5","5"},{"5",null},{"4","4"},{"4",null},{"3","3"},{"3",null},{"2","2"},{"2",null},{"1","1"},{"1",null}});
+
+//            sql="SELECT ORGANIZATION_ID,CONTAINER_ID FROM "+tableName+" group by ORGANIZATION_ID, CONTAINER_ID ORDER BY ORGANIZATION_ID DESC NULLS LAST, CONTAINER_ID ASC NULLS FIRST";
+            queryBuilder.setOrderByClause("ORGANIZATION_ID DESC NULLS LAST, CONTAINER_ID ASC NULLS FIRST");
+            rs = executeQuery(conn, queryBuilder);
+            assertResultSet(rs, new Object[][]{{"6",null},{"6","6"},{"5",null},{"5","5"},{"4",null},{"4","4"},{"3",null},{"3","3"},{"2",null},{"2","2"},{"1",null},{"1","1"},{null,null},{null,"1"},{null,"2"},{null,"3"},{null,"4"},{null,"5"},{null,"6"}});
+
+//            sql="SELECT ORGANIZATION_ID,CONTAINER_ID FROM "+tableName+" group by ORGANIZATION_ID, CONTAINER_ID ORDER BY ORGANIZATION_ID DESC NULLS LAST, CONTAINER_ID ASC NULLS LAST";
+            queryBuilder.setOrderByClause("ORGANIZATION_ID DESC NULLS LAST, CONTAINER_ID ASC NULLS LAST");
+            rs = executeQuery(conn, queryBuilder);
+            assertResultSet(rs, new Object[][]{{"6","6"},{"6",null},{"5","5"},{"5",null},{"4","4"},{"4",null},{"3","3"},{"3",null},{"2","2"},{"2",null},{"1","1"},{"1",null},{null,"1"},{null,"2"},{null,"3"},{null,"4"},{null,"5"},{null,"6"},{null,null}});
+
+            //groupBy orderPreserving orderBy desc desc
+
+//            sql="SELECT ORGANIZATION_ID,CONTAINER_ID FROM "+tableName+" group by ORGANIZATION_ID, CONTAINER_ID ORDER BY ORGANIZATION_ID DESC NULLS FIRST, CONTAINER_ID DESC NULLS FIRST";
+            queryBuilder.setOrderByClause("ORGANIZATION_ID DESC NULLS FIRST, CONTAINER_ID DESC NULLS FIRST");
+            rs = executeQuery(conn, queryBuilder);
+            assertResultSet(rs, new Object[][]{{null,null},{null,"6"},{null,"5"},{null,"4"},{null,"3"},{null,"2"},{null,"1"},{"6",null},{"6","6"},{"5",null},{"5","5"},{"4",null},{"4","4"},{"3",null},{"3","3"},{"2",null},{"2","2"},{"1",null},{"1","1"}});
+
+//            sql="SELECT ORGANIZATION_ID,CONTAINER_ID FROM "+tableName+" group by ORGANIZATION_ID, CONTAINER_ID ORDER BY ORGANIZATION_ID DESC NULLS FIRST, CONTAINER_ID DESC NULLS LAST";
+            queryBuilder.setOrderByClause("ORGANIZATION_ID DESC NULLS FIRST, CONTAINER_ID DESC NULLS LAST");
+            rs = executeQuery(conn, queryBuilder);
+            assertResultSet(rs, new Object[][]{{null,"6"},{null,"5"},{null,"4"},{null,"3"},{null,"2"},{null,"1"},{null,null},{"6","6"},{"6",null},{"5","5"},{"5",null},{"4","4"},{"4",null},{"3","3"},{"3",null},{"2","2"},{"2",null},{"1","1"},{"1",null}});
+
+//            sql="SELECT ORGANIZATION_ID,CONTAINER_ID FROM "+tableName+" group by ORGANIZATION_ID, CONTAINER_ID ORDER BY ORGANIZATION_ID DESC NULLS LAST, CONTAINER_ID DESC NULLS FIRST";
+            queryBuilder.setOrderByClause("ORGANIZATION_ID DESC NULLS LAST, CONTAINER_ID DESC NULLS FIRST");
+            rs = executeQuery(conn, queryBuilder);
+            assertResultSet(rs, new Object[][]{{"6",null},{"6","6"},{"5",null},{"5","5"},{"4",null},{"4","4"},{"3",null},{"3","3"},{"2",null},{"2","2"},{"1",null},{"1","1"},{null,null},{null,"6"},{null,"5"},{null,"4"},{null,"3"},{null,"2"},{null,"1"}});
+
+//            sql="SELECT ORGANIZATION_ID,CONTAINER_ID FROM "+tableName+" group by ORGANIZATION_ID, CONTAINER_ID ORDER BY ORGANIZATION_ID DESC NULLS LAST, CONTAINER_ID DESC NULLS LAST";
+            queryBuilder.setOrderByClause("ORGANIZATION_ID DESC NULLS LAST, CONTAINER_ID DESC NULLS LAST");
+            rs = executeQuery(conn, queryBuilder);
+            assertResultSet(rs, new Object[][]{{"6","6"},{"6",null},{"5","5"},{"5",null},{"4","4"},{"4",null},{"3","3"},{"3",null},{"2","2"},{"2",null},{"1","1"},{"1",null},{null,"6"},{null,"5"},{null,"4"},{null,"3"},{null,"2"},{null,"1"},{null,null}});
+
+            //-----groupBy not orderPreserving--
+
+            //groupBy not orderPreserving orderBy asc asc
+
+//            sql="SELECT ORGANIZATION_ID,SCORE FROM "+tableName+" group by ORGANIZATION_ID, SCORE ORDER BY ORGANIZATION_ID ASC NULLS FIRST, SCORE ASC NULLS FIRST";
+            queryBuilder.setSelectColumns(
+                    Lists.newArrayList("ORGANIZATION_ID", "SCORE"))
+                .setFullTableName(tableName)
+                .setGroupByClause("ORGANIZATION_ID, SCORE")
+                .setOrderByClause("ORGANIZATION_ID ASC NULLS FIRST, SCORE ASC NULLS FIRST");
+            rs = executeQuery(conn, queryBuilder);
+            assertResultSet(rs, new Object[][]{{null,null},{null,"1"},{null,"2"},{null,"3"},{null,"4"},{null,"5"},{null,"6"},{"1",null},{"1","1"},{"2",null},{"2","2"},{"3",null},{"3","3"},{"4",null},{"4","4"},{"5",null},{"5","5"},{"6",null},{"6","6"}});
+
+//            sql="SELECT ORGANIZATION_ID,SCORE FROM "+tableName+" group by ORGANIZATION_ID, SCORE ORDER BY ORGANIZATION_ID ASC NULLS FIRST, SCORE ASC NULLS LAST";
+            queryBuilder.setOrderByClause("ORGANIZATION_ID ASC NULLS FIRST, SCORE ASC NULLS LAST");
+            rs = executeQuery(conn, queryBuilder);
+            assertResultSet(rs, new Object[][]{{null,"1"},{null,"2"},{null,"3"},{null,"4"},{null,"5"},{null,"6"},{null,null},{"1","1"},{"1",null},{"2","2"},{"2",null},{"3","3"},{"3",null},{"4","4"},{"4",null},{"5","5"},{"5",null},{"6","6"},{"6",null}});
+
+//            sql="SELECT ORGANIZATION_ID,SCORE FROM "+tableName+" group by ORGANIZATION_ID, SCORE ORDER BY ORGANIZATION_ID ASC NULLS LAST, SCORE ASC NULLS FIRST";
+            queryBuilder.setOrderByClause("ORGANIZATION_ID ASC NULLS LAST, SCORE ASC NULLS FIRST");
+            rs = executeQuery(conn, queryBuilder);
+            assertResultSet(rs, new Object[][]{{"1",null},{"1","1"},{"2",null},{"2","2"},{"3",null},{"3","3"},{"4",null},{"4","4"},{"5",null},{"5","5"},{"6",null},{"6","6"},{null,null},{null,"1"},{null,"2"},{null,"3"},{null,"4"},{null,"5"},{null,"6"}});
+
+//            sql="SELECT ORGANIZATION_ID,SCORE FROM "+tableName+" group by ORGANIZATION_ID, SCORE ORDER BY ORGANIZATION_ID ASC NULLS LAST, SCORE ASC NULLS LAST";
+            queryBuilder.setOrderByClause("ORGANIZATION_ID ASC NULLS LAST, SCORE ASC NULLS LAST");
+            rs = executeQuery(conn, queryBuilder);
+            assertResultSet(rs, new Object[][]{{"1","1"},{"1",null},{"2","2"},{"2",null},{"3","3"},{"3",null},{"4","4"},{"4",null},{"5","5"},{"5",null},{"6","6"},{"6",null},{null,"1"},{null,"2"},{null,"3"},{null,"4"},{null,"5"},{null,"6"},{null,null}});
+
+            //groupBy not orderPreserving orderBy asc desc
+
+//            sql="SELECT ORGANIZATION_ID,SCORE FROM "+tableName+" group by ORGANIZATION_ID, SCORE ORDER BY ORGANIZATION_ID ASC NULLS FIRST, SCORE DESC NULLS FIRST";
+            queryBuilder.setOrderByClause("ORGANIZATION_ID ASC NULLS FIRST, SCORE DESC NULLS FIRST");
+            rs = executeQuery(conn, queryBuilder);
+            assertResultSet(rs, new Object[][]{{null,null},{null,"6"},{null,"5"},{null,"4"},{null,"3"},{null,"2"},{null,"1"},{"1",null},{"1","1"},{"2",null},{"2","2"},{"3",null},{"3","3"},{"4",null},{"4","4"},{"5",null},{"5","5"},{"6",null},{"6","6"}});
+
+//            sql="SELECT ORGANIZATION_ID,SCORE FROM "+tableName+" group by ORGANIZATION_ID, SCORE ORDER BY ORGANIZATION_ID ASC NULLS FIRST, SCORE DESC NULLS LAST";
+            queryBuilder.setOrderByClause("ORGANIZATION_ID ASC NULLS FIRST, SCORE DESC NULLS LAST");
+            rs = executeQuery(conn, queryBuilder);
+            assertResultSet(rs, new Object[][]{{null,"6"},{null,"5"},{null,"4"},{null,"3"},{null,"2"},{null,"1"},{null,null},{"1","1"},{"1",null},{"2","2"},{"2",null},{"3","3"},{"3",null},{"4","4"},{"4",null},{"5","5"},{"5",null},{"6","6"},{"6",null}});
+
+//            sql="SELECT ORGANIZATION_ID,SCORE FROM "+tableName+" group by ORGANIZATION_ID, SCORE ORDER BY ORGANIZATION_ID ASC NULLS LAST, SCORE DESC NULLS FIRST";
+            queryBuilder.setOrderByClause("ORGANIZATION_ID ASC NULLS LAST, SCORE DESC NULLS FIRST");
+            rs = executeQuery(conn, queryBuilder);
+            assertResultSet(rs, new Object[][]{{"1",null},{"1","1"},{"2",null},{"2","2"},{"3",null},{"3","3"},{"4",null},{"4","4"},{"5",null},{"5","5"},{"6",null},{"6","6"},{null,null},{null,"6"},{null,"5"},{null,"4"},{null,"3"},{null,"2"},{null,"1"}});
+
+//            sql="SELECT ORGANIZATION_ID,SCORE FROM "+tableName+" group by ORGANIZATION_ID, SCORE ORDER BY ORGANIZATION_ID ASC NULLS LAST, SCORE DESC NULLS LAST";
+            queryBuilder.setOrderByClause("ORGANIZATION_ID ASC NULLS LAST, SCORE DESC NULLS LAST");
+            rs = executeQuery(conn, queryBuilder);
+            assertResultSet(rs, new Object[][]{{"1","1"},{"1",null},{"2","2"},{"2",null},{"3","3"},{"3",null},{"4","4"},{"4",null},{"5","5"},{"5",null},{"6","6"},{"6",null},{null,"6"},{null,"5"},{null,"4"},{null,"3"},{null,"2"},{null,"1"},{null,null}});
+
+            //groupBy not orderPreserving orderBy desc asc
+
+//            sql="SELECT ORGANIZATION_ID,SCORE FROM "+tableName+" group by ORGANIZATION_ID, SCORE ORDER BY ORGANIZATION_ID DESC NULLS FIRST, SCORE ASC NULLS FIRST";
+            queryBuilder.setOrderByClause("ORGANIZATION_ID DESC NULLS FIRST, SCORE ASC NULLS FIRST");
+            rs = executeQuery(conn, queryBuilder);
+            assertResultSet(rs, new Object[][]{{null,null},{null,"1"},{null,"2"},{null,"3"},{null,"4"},{null,"5"},{null,"6"},{"6",null},{"6","6"},{"5",null},{"5","5"},{"4",null},{"4","4"},{"3",null},{"3","3"},{"2",null},{"2","2"},{"1",null},{"1","1"}});
+
+//            sql="SELECT ORGANIZATION_ID,SCORE FROM "+tableName+" group by ORGANIZATION_ID, SCORE ORDER BY ORGANIZATION_ID DESC NULLS FIRST, SCORE ASC NULLS LAST";
+            queryBuilder.setOrderByClause("ORGANIZATION_ID DESC NULLS FIRST, SCORE ASC NULLS LAST");
+            rs = executeQuery(conn, queryBuilder);
+            assertResultSet(rs, new Object[][]{{null,"1"},{null,"2"},{null,"3"},{null,"4"},{null,"5"},{null,"6"},{null,null},{"6","6"},{"6",null},{"5","5"},{"5",null},{"4","4"},{"4",null},{"3","3"},{"3",null},{"2","2"},{"2",null},{"1","1"},{"1",null}});
+
+//            sql="SELECT ORGANIZATION_ID,SCORE FROM "+tableName+" group by ORGANIZATION_ID, SCORE ORDER BY ORGANIZATION_ID DESC NULLS LAST, SCORE ASC NULLS FIRST";
+            queryBuilder.setOrderByClause("ORGANIZATION_ID DESC NULLS LAST, SCORE ASC NULLS FIRST");
+            rs = executeQuery(conn, queryBuilder);
+            assertResultSet(rs, new Object[][]{{"6",null},{"6","6"},{"5",null},{"5","5"},{"4",null},{"4","4"},{"3",null},{"3","3"},{"2",null},{"2","2"},{"1",null},{"1","1"},{null,null},{null,"1"},{null,"2"},{null,"3"},{null,"4"},{null,"5"},{null,"6"}});
+
+//            sql="SELECT ORGANIZATION_ID,SCORE FROM "+tableName+" group by ORGANIZATION_ID, SCORE ORDER BY ORGANIZATION_ID DESC NULLS LAST, SCORE ASC NULLS LAST";
+            queryBuilder.setOrderByClause("ORGANIZATION_ID DESC NULLS LAST, SCORE ASC NULLS LAST");
+            rs = executeQuery(conn, queryBuilder);
+            assertResultSet(rs, new Object[][]{{"6","6"},{"6",null},{"5","5"},{"5",null},{"4","4"},{"4",null},{"3","3"},{"3",null},{"2","2"},{"2",null},{"1","1"},{"1",null},{null,"1"},{null,"2"},{null,"3"},{null,"4"},{null,"5"},{null,"6"},{null,null}});
+
+            //groupBy not orderPreserving orderBy desc desc
+
+//            sql="SELECT ORGANIZATION_ID,SCORE FROM "+tableName+" group by ORGANIZATION_ID, SCORE ORDER BY ORGANIZATION_ID DESC NULLS FIRST, SCORE DESC NULLS FIRST";
+            queryBuilder.setOrderByClause("ORGANIZATION_ID DESC NULLS FIRST, SCORE DESC NULLS FIRST");
+            rs = executeQuery(conn, queryBuilder);
+            assertResultSet(rs, new Object[][]{{null,null},{null,"6"},{null,"5"},{null,"4"},{null,"3"},{null,"2"},{null,"1"},{"6",null},{"6","6"},{"5",null},{"5","5"},{"4",null},{"4","4"},{"3",null},{"3","3"},{"2",null},{"2","2"},{"1",null},{"1","1"}});
+
+//            sql="SELECT ORGANIZATION_ID,SCORE FROM "+tableName+" group by ORGANIZATION_ID, SCORE ORDER BY ORGANIZATION_ID DESC NULLS FIRST, SCORE DESC NULLS LAST";
+            queryBuilder.setOrderByClause("ORGANIZATION_ID DESC NULLS FIRST, SCORE DESC NULLS LAST");
+            rs = executeQuery(conn, queryBuilder);
+            assertResultSet(rs, new Object[][]{{null,"6"},{null,"5"},{null,"4"},{null,"3"},{null,"2"},{null,"1"},{null,null},{"6","6"},{"6",null},{"5","5"},{"5",null},{"4","4"},{"4",null},{"3","3"},{"3",null},{"2","2"},{"2",null},{"1","1"},{"1",null}});
+
+//            sql="SELECT ORGANIZATION_ID,SCORE FROM "+tableName+" group by ORGANIZATION_ID, SCORE ORDER BY ORGANIZATION_ID DESC NULLS LAST, SCORE DESC NULLS FIRST";
+            queryBuilder.setOrderByClause("ORGANIZATION_ID DESC NULLS LAST, SCORE DESC NULLS FIRST");
+            rs = executeQuery(conn, queryBuilder);
+            assertResultSet(rs, new Object[][]{{"6",null},{"6","6"},{"5",null},{"5","5"},{"4",null},{"4","4"},{"3",null},{"3","3"},{"2",null},{"2","2"},{"1",null},{"1","1"},{null,null},{null,"6"},{null,"5"},{null,"4"},{null,"3"},{null,"2"},{null,"1"}});
+
+//            sql="SELECT ORGANIZATION_ID,SCORE FROM "+tableName+" group by ORGANIZATION_ID, SCORE ORDER BY ORGANIZATION_ID DESC NULLS LAST, SCORE DESC NULLS LAST";
+            queryBuilder.setOrderByClause("ORGANIZATION_ID DESC NULLS LAST, SCORE DESC NULLS LAST");
+            rs = executeQuery(conn, queryBuilder);
+            assertResultSet(rs, new Object[][]{{"6","6"},{"6",null},{"5","5"},{"5",null},{"4","4"},{"4",null},{"3","3"},{"3",null},{"2","2"},{"2",null},{"1","1"},{"1",null},{null,"6"},{null,"5"},{null,"4"},{null,"3"},{null,"2"},{null,"1"},{null,null}});
+
+            //-------test only one return column----------------------------------
+
+//            sql="SELECT SCORE FROM "+tableName+" group by SCORE ORDER BY SCORE ASC NULLS FIRST";
+            queryBuilder.setSelectColumns(
+                    Lists.newArrayList("SCORE"))
+                .setFullTableName(tableName)
+                .setGroupByClause("SCORE")
+                .setOrderByClause("SCORE ASC NULLS FIRST");
+            rs = executeQuery(conn, queryBuilder);
+            assertResultSet(rs, new Object[][]{{null},{"1"},{"2"},{"3"},{"4"},{"5"},{"6"}});
+
+//            sql="SELECT SCORE FROM "+tableName+" group by SCORE ORDER BY SCORE ASC NULLS LAST";
+            queryBuilder.setOrderByClause("SCORE ASC NULLS LAST");
+            rs = executeQuery(conn, queryBuilder);
+            assertResultSet(rs, new Object[][]{{"1"},{"2"},{"3"},{"4"},{"5"},{"6"},{null}});
+
+//            sql="SELECT SCORE FROM "+tableName+" group by SCORE ORDER BY SCORE DESC NULLS FIRST";
+            queryBuilder.setOrderByClause("SCORE DESC NULLS FIRST");
+            rs = executeQuery(conn, queryBuilder);
+            assertResultSet(rs, new Object[][]{{null},{"6"},{"5"},{"4"},{"3"},{"2"},{"1"}});
+
+//            sql="SELECT SCORE FROM "+tableName+" group by SCORE ORDER BY SCORE DESC NULLS LAST";
+            queryBuilder.setOrderByClause("SCORE DESC NULLS LAST");
+            rs = executeQuery(conn, queryBuilder);
+            assertResultSet(rs, new Object[][]{{"6"},{"5"},{"4"},{"3"},{"2"},{"1"},{null}});
+        }
+    }
+
+}
\ No newline at end of file


[3/6] phoenix git commit: PHOENIX-4981 Add tests for ORDER BY, GROUP BY and salted tables using phoenix-spark

Posted by td...@apache.org.
http://git-wip-us.apache.org/repos/asf/phoenix/blob/c509d58f/phoenix-core/src/it/java/org/apache/phoenix/end2end/OrderByIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/OrderByIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/OrderByIT.java
index 578a3af..792d08f 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/OrderByIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/OrderByIT.java
@@ -17,17 +17,7 @@
  */
 package org.apache.phoenix.end2end;
 
-import static org.apache.phoenix.util.TestUtil.ROW1;
-import static org.apache.phoenix.util.TestUtil.ROW2;
-import static org.apache.phoenix.util.TestUtil.ROW3;
-import static org.apache.phoenix.util.TestUtil.ROW4;
-import static org.apache.phoenix.util.TestUtil.ROW5;
-import static org.apache.phoenix.util.TestUtil.ROW6;
-import static org.apache.phoenix.util.TestUtil.ROW7;
-import static org.apache.phoenix.util.TestUtil.ROW8;
-import static org.apache.phoenix.util.TestUtil.ROW9;
 import static org.apache.phoenix.util.TestUtil.TEST_PROPERTIES;
-import static org.apache.phoenix.util.TestUtil.assertResultSet;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
@@ -40,83 +30,10 @@ import java.sql.ResultSet;
 import java.sql.SQLException;
 import java.util.Properties;
 
-import org.apache.phoenix.jdbc.PhoenixStatement;
 import org.apache.phoenix.util.PropertiesUtil;
 import org.junit.Test;
 
-
-public class OrderByIT extends ParallelStatsDisabledIT {
-
-    @Test
-    public void testMultiOrderByExpr() throws Exception {
-        String tenantId = getOrganizationId();
-        String tableName = initATableValues(tenantId, getDefaultSplits(tenantId), getUrl());
-        String query = "SELECT entity_id FROM " + tableName + " ORDER BY b_string, entity_id";
-        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
-        Connection conn = DriverManager.getConnection(getUrl(), props);
-        try {
-            PreparedStatement statement = conn.prepareStatement(query);
-            ResultSet rs = statement.executeQuery();
-            assertTrue (rs.next());
-            assertEquals(ROW1,rs.getString(1));
-            assertTrue (rs.next());
-            assertEquals(ROW4,rs.getString(1));
-            assertTrue (rs.next());
-            assertEquals(ROW7,rs.getString(1));
-            assertTrue (rs.next());
-            assertEquals(ROW2,rs.getString(1));
-            assertTrue (rs.next());
-            assertEquals(ROW5,rs.getString(1));
-            assertTrue (rs.next());
-            assertEquals(ROW8,rs.getString(1));
-            assertTrue (rs.next());
-            assertEquals(ROW3,rs.getString(1));
-            assertTrue (rs.next());
-            assertEquals(ROW6,rs.getString(1));
-            assertTrue (rs.next());
-            assertEquals(ROW9,rs.getString(1));
-
-            assertFalse(rs.next());
-        } finally {
-            conn.close();
-        }
-    }
-
-
-    @Test
-    public void testDescMultiOrderByExpr() throws Exception {
-        String tenantId = getOrganizationId();
-        String tableName = initATableValues(tenantId, getDefaultSplits(tenantId), getUrl());
-        String query = "SELECT entity_id FROM " + tableName + " ORDER BY b_string || entity_id desc";
-        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
-        Connection conn = DriverManager.getConnection(getUrl(), props);
-        try {
-            PreparedStatement statement = conn.prepareStatement(query);
-            ResultSet rs = statement.executeQuery();
-            assertTrue (rs.next());
-            assertEquals(ROW9,rs.getString(1));
-            assertTrue (rs.next());
-            assertEquals(ROW6,rs.getString(1));
-            assertTrue (rs.next());
-            assertEquals(ROW3,rs.getString(1));
-            assertTrue (rs.next());
-            assertEquals(ROW8,rs.getString(1));
-            assertTrue (rs.next());
-            assertEquals(ROW5,rs.getString(1));
-            assertTrue (rs.next());
-            assertEquals(ROW2,rs.getString(1));
-            assertTrue (rs.next());
-            assertEquals(ROW7,rs.getString(1));
-            assertTrue (rs.next());
-            assertEquals(ROW4,rs.getString(1));
-            assertTrue (rs.next());
-            assertEquals(ROW1,rs.getString(1));
-
-            assertFalse(rs.next());
-        } finally {
-            conn.close();
-        }
-    }
+public class OrderByIT extends BaseOrderByIT {
 
     @Test
     public void testOrderByWithPosition() throws Exception {
@@ -151,8 +68,8 @@ public class OrderByIT extends ParallelStatsDisabledIT {
             assertTrue(rs.next());
             assertEquals(1,rs.getInt(1));
             assertTrue(rs.next());
-            assertEquals(1,rs.getInt(1));  
-            assertFalse(rs.next());  
+            assertEquals(1,rs.getInt(1));
+            assertFalse(rs.next());
 
             query = "select a_string x, col1 y from " + tableName + " order by x";
             rs = conn.createStatement().executeQuery(query);
@@ -163,9 +80,9 @@ public class OrderByIT extends ParallelStatsDisabledIT {
             assertEquals("b",rs.getString(1));
             assertEquals(20,rs.getInt(2));
             assertTrue(rs.next());
-            assertEquals("c",rs.getString(1));  
+            assertEquals("c",rs.getString(1));
             assertEquals(30,rs.getInt(2));
-            assertFalse(rs.next());  
+            assertFalse(rs.next());
 
             query = "select * from " + tableName + " order by 2";
             rs = conn.createStatement().executeQuery(query);
@@ -173,113 +90,12 @@ public class OrderByIT extends ParallelStatsDisabledIT {
             assertEquals("b",rs.getString(1));
             assertEquals(20,rs.getInt(2));
             assertTrue(rs.next());
-            assertEquals("c",rs.getString(1));  
-            assertEquals(30,rs.getInt(2));
-            assertTrue(rs.next());
-            assertEquals("a",rs.getString(1));
-            assertEquals(40,rs.getInt(2));
-            assertFalse(rs.next());  
-        } finally {
-            conn.close();
-        }
-    }
-
-
-    @Test
-    public void testColumnFamily() throws Exception {
-        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
-        Connection conn = DriverManager.getConnection(getUrl(), props);
-        conn.setAutoCommit(false);
-
-        try {
-            String tableName = generateUniqueName();
-            String ddl = "CREATE TABLE " + tableName +
-                    "  (a_string varchar not null, cf1.a integer, cf1.b varchar, col1 integer, cf2.c varchar, cf2.d integer, col2 integer" +
-                    "  CONSTRAINT pk PRIMARY KEY (a_string))\n";
-            createTestTable(getUrl(), ddl);
-            String dml = "UPSERT INTO " + tableName + " VALUES(?,?,?,?,?,?,?)";
-            PreparedStatement stmt = conn.prepareStatement(dml);
-            stmt.setString(1, "a");
-            stmt.setInt(2, 40);
-            stmt.setString(3, "aa");
-            stmt.setInt(4, 10);
-            stmt.setString(5, "bb");
-            stmt.setInt(6, 20);
-            stmt.setInt(7, 1);
-            stmt.execute();
-            stmt.setString(1, "c");
-            stmt.setInt(2, 30);
-            stmt.setString(3, "cc");
-            stmt.setInt(4, 50);
-            stmt.setString(5, "dd");
-            stmt.setInt(6, 60);
-            stmt.setInt(7, 3);
-            stmt.execute();
-            stmt.setString(1, "b");
-            stmt.setInt(2, 40);
-            stmt.setString(3, "bb");
-            stmt.setInt(4, 5);
-            stmt.setString(5, "aa");
-            stmt.setInt(6, 80);
-            stmt.setInt(7, 2);
-            stmt.execute();
-            conn.commit();
-
-            String query = "select * from " + tableName + " order by 2, 5";
-            ResultSet rs = conn.createStatement().executeQuery(query);
-            assertTrue(rs.next());
             assertEquals("c",rs.getString(1));
             assertEquals(30,rs.getInt(2));
-            assertEquals("cc",rs.getString(3));
-            assertEquals(50,rs.getInt(4));
-            assertEquals("dd",rs.getString(5));
-            assertEquals(60,rs.getInt(6));
-            assertEquals(3,rs.getInt(7));
-            assertTrue(rs.next());
-            assertEquals("b",rs.getString(1));  
-            assertEquals(40,rs.getInt(2));
-            assertEquals("bb",rs.getString(3));
-            assertEquals(5,rs.getInt(4));
-            assertEquals("aa",rs.getString(5));
-            assertEquals(80,rs.getInt(6));
-            assertEquals(2,rs.getInt(7));   
-            assertTrue(rs.next());
-            assertEquals("a",rs.getString(1));  
-            assertEquals(40,rs.getInt(2));
-            assertEquals("aa",rs.getString(3));
-            assertEquals(10,rs.getInt(4));
-            assertEquals("bb",rs.getString(5));
-            assertEquals(20,rs.getInt(6));
-            assertEquals(1,rs.getInt(7));         
-            assertFalse(rs.next());  
-
-            query = "select * from " + tableName + " order by 7";
-            rs = conn.createStatement().executeQuery(query);
             assertTrue(rs.next());
-            assertEquals("a",rs.getString(1));  
-            assertEquals(40,rs.getInt(2));
-            assertEquals("aa",rs.getString(3));
-            assertEquals(10,rs.getInt(4));
-            assertEquals("bb",rs.getString(5));
-            assertEquals(20,rs.getInt(6));
-            assertEquals(1,rs.getInt(7));  
-            assertTrue(rs.next());
-            assertEquals("b",rs.getString(1));  
+            assertEquals("a",rs.getString(1));
             assertEquals(40,rs.getInt(2));
-            assertEquals("bb",rs.getString(3));
-            assertEquals(5,rs.getInt(4));
-            assertEquals("aa",rs.getString(5));
-            assertEquals(80,rs.getInt(6));
-            assertEquals(2,rs.getInt(7));  
-            assertTrue(rs.next());
-            assertEquals("c",rs.getString(1));
-            assertEquals(30,rs.getInt(2));
-            assertEquals("cc",rs.getString(3));
-            assertEquals(50,rs.getInt(4));
-            assertEquals("dd",rs.getString(5));
-            assertEquals(60,rs.getInt(6));
-            assertEquals(3,rs.getInt(7));
-            assertFalse(rs.next());  
+            assertFalse(rs.next());
         } finally {
             conn.close();
         }
@@ -343,19 +159,19 @@ public class OrderByIT extends ParallelStatsDisabledIT {
             String query = "select t1.* from " + tableName1 + " t1 join " + tableName2 + " t2 on t1.a_string = t2.a_string order by 3";
             ResultSet rs = conn.createStatement().executeQuery(query);
             assertTrue(rs.next());
-            assertEquals("a",rs.getString(1));  
+            assertEquals("a",rs.getString(1));
             assertEquals(40,rs.getInt(2));
             assertEquals("aa",rs.getString(3));
             assertEquals(10,rs.getInt(4));
             assertEquals("bb",rs.getString(5));
             assertEquals(20,rs.getInt(6));
             assertTrue(rs.next());
-            assertEquals("b",rs.getString(1));  
+            assertEquals("b",rs.getString(1));
             assertEquals(40,rs.getInt(2));
             assertEquals("bb",rs.getString(3));
             assertEquals(5,rs.getInt(4));
             assertEquals("aa",rs.getString(5));
-            assertEquals(80,rs.getInt(6));         
+            assertEquals(80,rs.getInt(6));
             assertTrue(rs.next());
             assertEquals("c",rs.getString(1));
             assertEquals(30,rs.getInt(2));
@@ -363,20 +179,20 @@ public class OrderByIT extends ParallelStatsDisabledIT {
             assertEquals(50,rs.getInt(4));
             assertEquals("dd",rs.getString(5));
             assertEquals(60,rs.getInt(6));
-            assertFalse(rs.next());  
+            assertFalse(rs.next());
 
             query = "select t1.a_string, t2.col1 from " + tableName1 + " t1 join " + tableName2 + " t2 on t1.a_string = t2.a_string order by 2";
             rs = conn.createStatement().executeQuery(query);
             assertTrue(rs.next());
-            assertEquals("b",rs.getString(1));  
+            assertEquals("b",rs.getString(1));
             assertEquals(20,rs.getInt(2));
             assertTrue(rs.next());
-            assertEquals("c",rs.getString(1));  
+            assertEquals("c",rs.getString(1));
             assertEquals(30,rs.getInt(2));
             assertTrue(rs.next());
-            assertEquals("a",rs.getString(1));  
+            assertEquals("a",rs.getString(1));
             assertEquals(40,rs.getInt(2));
-            assertFalse(rs.next()); 
+            assertFalse(rs.next());
         } finally {
             conn.close();
         }
@@ -441,24 +257,24 @@ public class OrderByIT extends ParallelStatsDisabledIT {
             String query = "select a_string, cf2.d from " + tableName1 + " union all select * from " + tableName2 + " order by 2";
             ResultSet rs = conn.createStatement().executeQuery(query);
             assertTrue(rs.next());
-            assertEquals("bb",rs.getString(1));  
+            assertEquals("bb",rs.getString(1));
             assertEquals(10,rs.getInt(2));
             assertTrue(rs.next());
-            assertEquals("a",rs.getString(1));  
-            assertEquals(20,rs.getInt(2));      
+            assertEquals("a",rs.getString(1));
+            assertEquals(20,rs.getInt(2));
             assertTrue(rs.next());
             assertEquals("cc",rs.getString(1));
             assertEquals(30,rs.getInt(2));
             assertTrue(rs.next());
-            assertEquals("aa",rs.getString(1));  
+            assertEquals("aa",rs.getString(1));
             assertEquals(40,rs.getInt(2));
             assertTrue(rs.next());
-            assertEquals("c",rs.getString(1));  
-            assertEquals(60,rs.getInt(2));      
+            assertEquals("c",rs.getString(1));
+            assertEquals(60,rs.getInt(2));
             assertTrue(rs.next());
             assertEquals("b",rs.getString(1));
             assertEquals(80,rs.getInt(2));
-            assertFalse(rs.next());  
+            assertFalse(rs.next());
         } finally {
             conn.close();
         }
@@ -514,7 +330,8 @@ public class OrderByIT extends ParallelStatsDisabledIT {
             conn.close();
         }
     }
-    
+
+
     @Test
     public void testOrderByRVC() throws Exception {
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
@@ -546,636 +363,102 @@ public class OrderByIT extends ParallelStatsDisabledIT {
     }
 
     @Test
-    public void testAggregateOrderBy() throws Exception {
-        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
-        Connection conn = DriverManager.getConnection(getUrl(), props);
-        String tableName = generateUniqueName();
-        String ddl = "create table " + tableName + " (ID VARCHAR NOT NULL PRIMARY KEY, VAL1 VARCHAR, VAL2 INTEGER)";
-        conn.createStatement().execute(ddl);
-
-        conn.createStatement().execute("upsert into " + tableName + " values ('ABC','aa123', 11)");
-        conn.createStatement().execute("upsert into " + tableName + " values ('ABD','ba124', 1)");
-        conn.createStatement().execute("upsert into " + tableName + " values ('ABE','cf125', 13)");
-        conn.createStatement().execute("upsert into " + tableName + " values ('ABF','dan126', 4)");
-        conn.createStatement().execute("upsert into " + tableName + " values ('ABG','elf127', 15)");
-        conn.createStatement().execute("upsert into " + tableName + " values ('ABH','fan128', 6)");
-        conn.createStatement().execute("upsert into " + tableName + " values ('AAA','get211', 100)");
-        conn.createStatement().execute("upsert into " + tableName + " values ('AAB','hat212', 7)");
-        conn.createStatement().execute("upsert into " + tableName + " values ('AAC','aap12', 2)");
-        conn.createStatement().execute("upsert into " + tableName + " values ('AAD','ball12', 3)");
-        conn.createStatement().execute("upsert into " + tableName + " values ('AAE','inn2110', 13)");
-        conn.createStatement().execute("upsert into " + tableName + " values ('AAF','key2112', 40)");
-        conn.commit();
-
-        ResultSet rs;
-        PhoenixStatement stmt = conn.createStatement().unwrap(PhoenixStatement.class);
-        rs = stmt.executeQuery("select distinct ID, VAL1, VAL2 from " + tableName + " where ID in ('ABC','ABD','ABE','ABF','ABG','ABH','AAA', 'AAB', 'AAC','AAD','AAE','AAF') order by VAL1");
-        assertFalse(stmt.getQueryPlan().getOrderBy().getOrderByExpressions().isEmpty());
-        assertTrue(rs.next());
-        assertEquals("ABC", rs.getString(1));
-        assertEquals("aa123", rs.getString(2));
-        assertTrue(rs.next());
-        assertEquals("aap12", rs.getString(2));
-        assertTrue(rs.next());
-        assertEquals("ba124", rs.getString(2));
-        assertTrue(rs.next());
-        assertEquals("ball12", rs.getString(2));
-        assertTrue(rs.next());
-        assertEquals("cf125", rs.getString(2));
-        assertTrue(rs.next());
-        assertEquals("dan126", rs.getString(2));
-        assertTrue(rs.next());
-        assertEquals("elf127", rs.getString(2));
-        assertTrue(rs.next());
-        assertEquals("fan128", rs.getString(2));
-        assertTrue(rs.next());
-        assertEquals("get211", rs.getString(2));
-        assertTrue(rs.next());
-        assertEquals("hat212", rs.getString(2));
-        assertTrue(rs.next());
-        assertEquals("inn2110", rs.getString(2));
-        assertTrue(rs.next());
-        assertEquals("AAF", rs.getString(1));
-        assertEquals("key2112", rs.getString(2));
-        assertFalse(rs.next());
-    }
-
-    @Test
-    public void testAggregateOptimizedOutOrderBy() throws Exception {
+    public void testColumnFamily() throws Exception {
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         Connection conn = DriverManager.getConnection(getUrl(), props);
-        String tableName = generateUniqueName();
-        String ddl = "create table " + tableName + " (K1 VARCHAR NOT NULL, K2 VARCHAR NOT NULL, VAL1 VARCHAR, VAL2 INTEGER, CONSTRAINT pk PRIMARY KEY(K1,K2))";
-        conn.createStatement().execute(ddl);
-
-        conn.createStatement().execute("upsert into " + tableName + " values ('ABC','ABC','aa123', 11)");
-        conn.createStatement().execute("upsert into " + tableName + " values ('ABD','ABC','ba124', 1)");
-        conn.createStatement().execute("upsert into " + tableName + " values ('ABE','ABC','cf125', 13)");
-        conn.createStatement().execute("upsert into " + tableName + " values ('ABF','ABC','dan126', 4)");
-        conn.createStatement().execute("upsert into " + tableName + " values ('ABG','ABC','elf127', 15)");
-        conn.createStatement().execute("upsert into " + tableName + " values ('ABH','ABC','fan128', 6)");
-        conn.createStatement().execute("upsert into " + tableName + " values ('AAA','ABC','get211', 100)");
-        conn.createStatement().execute("upsert into " + tableName + " values ('AAB','ABC','hat212', 7)");
-        conn.createStatement().execute("upsert into " + tableName + " values ('AAC','ABC','aap12', 2)");
-        conn.createStatement().execute("upsert into " + tableName + " values ('AAD','ABC','ball12', 3)");
-        conn.createStatement().execute("upsert into " + tableName + " values ('AAE','ABC','inn2110', 13)");
-        conn.createStatement().execute("upsert into " + tableName + " values ('AAF','ABC','key2112', 40)");
-        conn.commit();
-
-        ResultSet rs;
-        PhoenixStatement stmt = conn.createStatement().unwrap(PhoenixStatement.class);
-        rs = stmt.executeQuery("select distinct K2, VAL1, VAL2 from " + tableName + " where K2 = 'ABC' order by VAL1");
-        assertTrue(stmt.getQueryPlan().getOrderBy().getOrderByExpressions().isEmpty());
-        assertTrue(rs.next());
-        assertEquals("ABC", rs.getString(1));
-        assertEquals("aa123", rs.getString(2));
-        assertTrue(rs.next());
-        assertEquals("aap12", rs.getString(2));
-        assertTrue(rs.next());
-        assertEquals("ba124", rs.getString(2));
-        assertTrue(rs.next());
-        assertEquals("ball12", rs.getString(2));
-        assertTrue(rs.next());
-        assertEquals("cf125", rs.getString(2));
-        assertTrue(rs.next());
-        assertEquals("dan126", rs.getString(2));
-        assertTrue(rs.next());
-        assertEquals("elf127", rs.getString(2));
-        assertTrue(rs.next());
-        assertEquals("fan128", rs.getString(2));
-        assertTrue(rs.next());
-        assertEquals("get211", rs.getString(2));
-        assertTrue(rs.next());
-        assertEquals("hat212", rs.getString(2));
-        assertTrue(rs.next());
-        assertEquals("inn2110", rs.getString(2));
-        assertTrue(rs.next());
-        assertEquals("ABC", rs.getString(1));
-        assertEquals("key2112", rs.getString(2));
-        assertFalse(rs.next());
-    }
-
-    @Test
-    public void testNullsLastWithDesc() throws Exception {
-        Connection conn=null;
-        try {
-            Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
-            conn = DriverManager.getConnection(getUrl(), props);
-
-            String tableName=generateUniqueName();
-            String sql="CREATE TABLE "+tableName+" ( "+
-                "ORGANIZATION_ID VARCHAR,"+
-                "CONTAINER_ID VARCHAR,"+
-                "ENTITY_ID VARCHAR NOT NULL,"+
-                "CONSTRAINT TEST_PK PRIMARY KEY ( "+
-                "ORGANIZATION_ID DESC,"+
-                "CONTAINER_ID DESC,"+
-                "ENTITY_ID"+
-                "))";
-            conn.createStatement().execute(sql);
-
-            conn.createStatement().execute("UPSERT INTO "+tableName+" VALUES ('a',null,'11')");
-            conn.createStatement().execute("UPSERT INTO "+tableName+" VALUES (null,'2','22')");
-            conn.createStatement().execute("UPSERT INTO "+tableName+" VALUES ('c','3','33')");
-            conn.commit();
-
-            //-----ORGANIZATION_ID
-
-            sql="SELECT CONTAINER_ID,ORGANIZATION_ID FROM "+tableName+" order by ORGANIZATION_ID ASC NULLS FIRST";
-            ResultSet rs=conn.prepareStatement(sql).executeQuery();
-            assertResultSet(rs, new String[][]{{"2",null},{null,"a"},{"3","c"},});
-
-            sql="SELECT CONTAINER_ID,ORGANIZATION_ID FROM "+tableName+" order by ORGANIZATION_ID ASC NULLS LAST";
-            rs=conn.prepareStatement(sql).executeQuery();
-            assertResultSet(rs, new String[][]{{null,"a"},{"3","c"},{"2",null}});
-
-            sql="SELECT CONTAINER_ID,ORGANIZATION_ID FROM "+tableName+" order by ORGANIZATION_ID DESC NULLS FIRST";
-            rs=conn.prepareStatement(sql).executeQuery();
-            assertResultSet(rs, new String[][]{{"2",null},{"3","c"},{null,"a"}});
-
-            sql="SELECT CONTAINER_ID,ORGANIZATION_ID FROM "+tableName+" order by ORGANIZATION_ID DESC NULLS LAST";
-            rs=conn.prepareStatement(sql).executeQuery();
-            assertResultSet(rs, new String[][]{{"3","c"},{null,"a"},{"2",null}});
-
-            //----CONTAINER_ID
-
-            sql="SELECT CONTAINER_ID,ORGANIZATION_ID FROM "+tableName+" order by CONTAINER_ID ASC NULLS FIRST";
-            rs=conn.prepareStatement(sql).executeQuery();
-            assertResultSet(rs, new String[][]{{null,"a"},{"2",null},{"3","c"}});
-
-            sql="SELECT CONTAINER_ID,ORGANIZATION_ID FROM "+tableName+" order by CONTAINER_ID ASC NULLS LAST";
-            rs=conn.prepareStatement(sql).executeQuery();
-            assertResultSet(rs, new String[][]{{"2",null},{"3","c"},{null,"a"}});
-
-            sql="SELECT CONTAINER_ID,ORGANIZATION_ID FROM "+tableName+" order by CONTAINER_ID DESC NULLS FIRST";
-            rs=conn.prepareStatement(sql).executeQuery();
-            assertResultSet(rs, new String[][]{{null,"a"},{"3","c"},{"2",null}});
-
-            sql="SELECT CONTAINER_ID,ORGANIZATION_ID FROM "+tableName+" order by CONTAINER_ID DESC NULLS LAST";
-            rs=conn.prepareStatement(sql).executeQuery();
-            assertResultSet(rs, new String[][]{{"3","c"},{"2",null},{null,"a"}});
-
-            conn.createStatement().execute("UPSERT INTO "+tableName+" VALUES (null,null,'44')");
-            conn.commit();
-
-            //-----ORGANIZATION_ID ASC  CONTAINER_ID ASC
-
-            sql="SELECT CONTAINER_ID,ORGANIZATION_ID FROM "+tableName+" order by ORGANIZATION_ID NULLS FIRST,CONTAINER_ID NULLS FIRST";
-            rs=conn.prepareStatement(sql).executeQuery();
-            assertResultSet(rs, new String[][]{{null,null},{"2",null},{null,"a"},{"3","c"}});
-
-            sql="SELECT CONTAINER_ID,ORGANIZATION_ID FROM "+tableName+" order by ORGANIZATION_ID NULLS FIRST,CONTAINER_ID NULLS LAST";
-            rs=conn.prepareStatement(sql).executeQuery();
-            assertResultSet(rs, new String[][]{{"2",null},{null,null},{null,"a"},{"3","c"}});
-
-            sql="SELECT CONTAINER_ID,ORGANIZATION_ID FROM "+tableName+" order by ORGANIZATION_ID NULLS LAST,CONTAINER_ID NULLS FIRST";
-            rs=conn.prepareStatement(sql).executeQuery();
-            assertResultSet(rs, new String[][]{{null,"a"},{"3","c"},{null,null},{"2",null}});
-
-            sql="SELECT CONTAINER_ID,ORGANIZATION_ID FROM "+tableName+" order by ORGANIZATION_ID NULLS LAST,CONTAINER_ID NULLS LAST";
-            rs=conn.prepareStatement(sql).executeQuery();
-            assertResultSet(rs, new String[][]{{null,"a"},{"3","c"},{"2",null},{null,null}});
-
-
-            //-----ORGANIZATION_ID ASC  CONTAINER_ID DESC
-
-            sql="SELECT CONTAINER_ID,ORGANIZATION_ID FROM "+tableName+" order by ORGANIZATION_ID NULLS FIRST,CONTAINER_ID DESC NULLS FIRST";
-            rs=conn.prepareStatement(sql).executeQuery();
-            assertResultSet(rs, new String[][]{{null,null},{"2",null},{null,"a"},{"3","c"}});
-
-            sql="SELECT CONTAINER_ID,ORGANIZATION_ID FROM "+tableName+" order by ORGANIZATION_ID NULLS FIRST,CONTAINER_ID DESC NULLS LAST";
-            rs=conn.prepareStatement(sql).executeQuery();
-            assertResultSet(rs, new String[][]{{"2",null},{null,null},{null,"a"},{"3","c"}});
-
-            sql="SELECT CONTAINER_ID,ORGANIZATION_ID FROM "+tableName+" order by ORGANIZATION_ID NULLS LAST,CONTAINER_ID DESC NULLS FIRST";
-            rs=conn.prepareStatement(sql).executeQuery();
-            assertResultSet(rs, new String[][]{{null,"a"},{"3","c"},{null,null},{"2",null}});
-
-            sql="SELECT CONTAINER_ID,ORGANIZATION_ID FROM "+tableName+" order by ORGANIZATION_ID NULLS LAST,CONTAINER_ID DESC NULLS LAST";
-            rs=conn.prepareStatement(sql).executeQuery();
-            assertResultSet(rs, new String[][]{{null,"a"},{"3","c"},{"2",null},{null,null}});
-
-            //-----ORGANIZATION_ID DESC  CONTAINER_ID ASC
-
-            sql="SELECT CONTAINER_ID,ORGANIZATION_ID FROM "+tableName+" order by ORGANIZATION_ID DESC NULLS FIRST,CONTAINER_ID NULLS FIRST";
-            rs=conn.prepareStatement(sql).executeQuery();
-            assertResultSet(rs, new String[][]{{null,null},{"2",null},{"3","c"},{null,"a"}});
-
-            sql="SELECT CONTAINER_ID,ORGANIZATION_ID FROM "+tableName+" order by ORGANIZATION_ID DESC NULLS FIRST,CONTAINER_ID NULLS LAST";
-            rs=conn.prepareStatement(sql).executeQuery();
-            assertResultSet(rs, new String[][]{{"2",null},{null,null},{"3","c"},{null,"a"}});
-
-            sql="SELECT CONTAINER_ID,ORGANIZATION_ID FROM "+tableName+" order by ORGANIZATION_ID DESC NULLS LAST,CONTAINER_ID NULLS FIRST";
-            rs=conn.prepareStatement(sql).executeQuery();
-            assertResultSet(rs, new String[][]{{"3","c"},{null,"a"},{null,null},{"2",null}});
-
-            sql="SELECT CONTAINER_ID,ORGANIZATION_ID FROM "+tableName+" order by ORGANIZATION_ID DESC NULLS LAST,CONTAINER_ID NULLS LAST";
-            rs=conn.prepareStatement(sql).executeQuery();
-            assertResultSet(rs, new String[][]{{"3","c"},{null,"a"},{"2",null},{null,null}});
-
-            //-----ORGANIZATION_ID DESC  CONTAINER_ID DESC
-
-            sql="SELECT CONTAINER_ID,ORGANIZATION_ID FROM "+tableName+" order by ORGANIZATION_ID DESC NULLS FIRST,CONTAINER_ID DESC NULLS FIRST";
-            rs=conn.prepareStatement(sql).executeQuery();
-            assertResultSet(rs, new String[][]{{null,null},{"2",null},{"3","c"},{null,"a"}});
-
-            sql="SELECT CONTAINER_ID,ORGANIZATION_ID FROM "+tableName+" order by ORGANIZATION_ID DESC NULLS FIRST,CONTAINER_ID DESC NULLS LAST";
-            rs=conn.prepareStatement(sql).executeQuery();
-            assertResultSet(rs, new String[][]{{"2",null},{null,null},{"3","c"},{null,"a"}});
-
-            sql="SELECT CONTAINER_ID,ORGANIZATION_ID FROM "+tableName+" order by ORGANIZATION_ID DESC NULLS LAST,CONTAINER_ID DESC NULLS FIRST";
-            rs=conn.prepareStatement(sql).executeQuery();
-            assertResultSet(rs, new String[][]{{"3","c"},{null,"a"},{null,null},{"2",null}});
-
-            sql="SELECT CONTAINER_ID,ORGANIZATION_ID FROM "+tableName+" order by ORGANIZATION_ID DESC NULLS LAST,CONTAINER_ID DESC NULLS LAST";
-            rs=conn.prepareStatement(sql).executeQuery();
-            assertResultSet(rs, new String[][]{{"3","c"},{null,"a"},{"2",null},{null,null}});
-
-            //-----CONTAINER_ID ASC  ORGANIZATION_ID ASC
-
-            sql="SELECT CONTAINER_ID,ORGANIZATION_ID FROM "+tableName+" order by CONTAINER_ID NULLS FIRST,ORGANIZATION_ID NULLS FIRST";
-            rs=conn.prepareStatement(sql).executeQuery();
-            assertResultSet(rs, new String[][]{{null,null},{null,"a"},{"2",null},{"3","c"}});
-
-            sql="SELECT CONTAINER_ID,ORGANIZATION_ID FROM "+tableName+" order by CONTAINER_ID NULLS FIRST,ORGANIZATION_ID NULLS LAST";
-            rs=conn.prepareStatement(sql).executeQuery();
-            assertResultSet(rs, new String[][]{{null,"a"},{null,null},{"2",null},{"3","c"}});
-
-            sql="SELECT CONTAINER_ID,ORGANIZATION_ID FROM "+tableName+" order by CONTAINER_ID NULLS LAST,ORGANIZATION_ID NULLS FIRST";
-            rs=conn.prepareStatement(sql).executeQuery();
-            assertResultSet(rs, new String[][]{{"2",null},{"3","c"},{null,null},{null,"a"}});
-
-            sql="SELECT CONTAINER_ID,ORGANIZATION_ID FROM "+tableName+" order by CONTAINER_ID NULLS LAST,ORGANIZATION_ID NULLS LAST";
-            rs=conn.prepareStatement(sql).executeQuery();
-            assertResultSet(rs, new String[][]{{"2",null},{"3","c"},{null,"a"},{null,null}});
-
-            //-----CONTAINER_ID ASC  ORGANIZATION_ID DESC
-
-            sql="SELECT CONTAINER_ID,ORGANIZATION_ID FROM "+tableName+" order by CONTAINER_ID ASC NULLS FIRST,ORGANIZATION_ID DESC NULLS FIRST";
-            rs=conn.prepareStatement(sql).executeQuery();
-            assertResultSet(rs, new String[][]{{null,null},{null,"a"},{"2",null},{"3","c"}});
-
-            sql="SELECT CONTAINER_ID,ORGANIZATION_ID FROM "+tableName+" order by CONTAINER_ID ASC NULLS FIRST,ORGANIZATION_ID DESC NULLS LAST";
-            rs=conn.prepareStatement(sql).executeQuery();
-            assertResultSet(rs, new String[][]{{null,"a"},{null,null},{"2",null},{"3","c"}});
-
-            sql="SELECT CONTAINER_ID,ORGANIZATION_ID FROM "+tableName+" order by CONTAINER_ID ASC NULLS LAST,ORGANIZATION_ID DESC NULLS FIRST";
-            rs=conn.prepareStatement(sql).executeQuery();
-            assertResultSet(rs, new String[][]{{"2",null},{"3","c"},{null,null},{null,"a"}});
-
-            sql="SELECT CONTAINER_ID,ORGANIZATION_ID FROM "+tableName+" order by CONTAINER_ID ASC NULLS LAST,ORGANIZATION_ID DESC NULLS LAST";
-            rs=conn.prepareStatement(sql).executeQuery();
-            assertResultSet(rs, new String[][]{{"2",null},{"3","c"},{null,"a"},{null,null}});
-
-            //-----CONTAINER_ID DESC  ORGANIZATION_ID ASC
-
-            sql="SELECT CONTAINER_ID,ORGANIZATION_ID FROM "+tableName+" order by CONTAINER_ID DESC NULLS FIRST,ORGANIZATION_ID ASC NULLS FIRST";
-            rs=conn.prepareStatement(sql).executeQuery();
-            assertResultSet(rs, new String[][]{{null,null},{null,"a"},{"3","c"},{"2",null}});
-
-            sql="SELECT CONTAINER_ID,ORGANIZATION_ID FROM "+tableName+" order by CONTAINER_ID DESC NULLS FIRST,ORGANIZATION_ID ASC NULLS LAST";
-            rs=conn.prepareStatement(sql).executeQuery();
-            assertResultSet(rs, new String[][]{{null,"a"},{null,null},{"3","c"},{"2",null}});
-
-            sql="SELECT CONTAINER_ID,ORGANIZATION_ID FROM "+tableName+" order by CONTAINER_ID DESC NULLS LAST,ORGANIZATION_ID ASC NULLS FIRST";
-            rs=conn.prepareStatement(sql).executeQuery();
-            assertResultSet(rs, new String[][]{{"3","c"},{"2",null},{null,null},{null,"a"}});
-
-            sql="SELECT CONTAINER_ID,ORGANIZATION_ID FROM "+tableName+" order by CONTAINER_ID DESC NULLS LAST,ORGANIZATION_ID ASC NULLS LAST";
-            rs=conn.prepareStatement(sql).executeQuery();
-            assertResultSet(rs, new String[][]{{"3","c"},{"2",null},{null,"a"},{null,null}});
-
-            //-----CONTAINER_ID DESC  ORGANIZATION_ID DESC
-
-            sql="SELECT CONTAINER_ID,ORGANIZATION_ID FROM "+tableName+" order by CONTAINER_ID DESC NULLS FIRST,ORGANIZATION_ID DESC NULLS FIRST";
-            rs=conn.prepareStatement(sql).executeQuery();
-            assertResultSet(rs, new String[][]{{null,null},{null,"a"},{"3","c"},{"2",null}});
-
-            sql="SELECT CONTAINER_ID,ORGANIZATION_ID FROM "+tableName+" order by CONTAINER_ID DESC NULLS FIRST,ORGANIZATION_ID DESC NULLS LAST";
-            rs=conn.prepareStatement(sql).executeQuery();
-            assertResultSet(rs, new String[][]{{null,"a"},{null,null},{"3","c"},{"2",null}});
-
-            sql="SELECT CONTAINER_ID,ORGANIZATION_ID FROM "+tableName+" order by CONTAINER_ID DESC NULLS LAST,ORGANIZATION_ID DESC NULLS FIRST";
-            rs=conn.prepareStatement(sql).executeQuery();
-            assertResultSet(rs, new String[][]{{"3","c"},{"2",null},{null,null},{null,"a"}});
-
-            sql="SELECT CONTAINER_ID,ORGANIZATION_ID FROM "+tableName+" order by CONTAINER_ID DESC NULLS LAST,ORGANIZATION_ID DESC NULLS LAST";
-            rs=conn.prepareStatement(sql).executeQuery();
-            assertResultSet(rs, new String[][]{{"3","c"},{"2",null},{null,"a"},{null,null}});
-        } finally {
-            if(conn!=null) {
-                conn.close();
-            }
-        }
-    }
-
-    @Test
-    public void testOrderByReverseOptimization() throws Exception {
-        for(boolean salted: new boolean[]{true,false}) {
-            doTestOrderByReverseOptimization(salted,true,true,true);
-            doTestOrderByReverseOptimization(salted,true,true,false);
-            doTestOrderByReverseOptimization(salted,true,false,true);
-            doTestOrderByReverseOptimization(salted,true,false,false);
-            doTestOrderByReverseOptimization(salted,false,true,true);
-            doTestOrderByReverseOptimization(salted,false,true,false);
-            doTestOrderByReverseOptimization(salted,false,false,true);
-            doTestOrderByReverseOptimization(salted,false,false,false);
-        }
-    }
-
-    private void doTestOrderByReverseOptimization(boolean salted,boolean desc1,boolean desc2,boolean desc3) throws Exception {
-        Connection conn = null;
-        try {
-            Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
-            conn = DriverManager.getConnection(getUrl(), props);
-            String tableName=generateUniqueName();
-            String sql="CREATE TABLE "+tableName+" ( "+
-                    "ORGANIZATION_ID INTEGER NOT NULL,"+
-                    "CONTAINER_ID INTEGER NOT NULL,"+
-                    "SCORE INTEGER NOT NULL,"+
-                    "ENTITY_ID INTEGER NOT NULL,"+
-                    "CONSTRAINT TEST_PK PRIMARY KEY ( "+
-                    "ORGANIZATION_ID" +(desc1 ? " DESC" : "" )+","+
-                    "CONTAINER_ID"+(desc2 ? " DESC" : "" )+","+
-                    "SCORE"+(desc3 ? " DESC" : "" )+","+
-                    "ENTITY_ID"+
-                    ")) "+(salted ? "SALT_BUCKETS =4" : "split on(4)");
-            conn.createStatement().execute(sql);
-
-            conn.createStatement().execute("UPSERT INTO "+tableName+" VALUES (1,1,1,1)");
-            conn.createStatement().execute("UPSERT INTO "+tableName+" VALUES (2,2,2,2)");
-            conn.createStatement().execute("UPSERT INTO "+tableName+" VALUES (3,3,3,3)");
-            conn.createStatement().execute("UPSERT INTO "+tableName+" VALUES (4,4,4,4)");
-            conn.createStatement().execute("UPSERT INTO "+tableName+" VALUES (5,5,5,5)");
-            conn.createStatement().execute("UPSERT INTO "+tableName+" VALUES (6,6,6,6)");
-            conn.createStatement().execute("UPSERT INTO "+tableName+" VALUES (1,1,1,11)");
-            conn.createStatement().execute("UPSERT INTO "+tableName+" VALUES (2,2,2,22)");
-            conn.createStatement().execute("UPSERT INTO "+tableName+" VALUES (3,3,3,33)");
-            conn.createStatement().execute("UPSERT INTO "+tableName+" VALUES (4,4,4,44)");
-            conn.createStatement().execute("UPSERT INTO "+tableName+" VALUES (5,5,5,55)");
-            conn.createStatement().execute("UPSERT INTO "+tableName+" VALUES (6,6,6,66)");
-            conn.commit();
-
-            //groupBy orderPreserving orderBy asc asc
-            sql="SELECT ORGANIZATION_ID,CONTAINER_ID FROM "+tableName+" group by ORGANIZATION_ID, CONTAINER_ID ORDER BY ORGANIZATION_ID ASC, CONTAINER_ID ASC";
-            ResultSet rs=conn.prepareStatement(sql).executeQuery();
-            assertResultSet(rs, new Object[][]{{1,1},{2,2},{3,3},{4,4},{5,5},{6,6}});
-
-            //groupBy orderPreserving orderBy asc desc
-            sql="SELECT ORGANIZATION_ID,CONTAINER_ID FROM "+tableName+" group by ORGANIZATION_ID, CONTAINER_ID ORDER BY ORGANIZATION_ID ASC, CONTAINER_ID desc";
-            rs=conn.prepareStatement(sql).executeQuery();
-            assertResultSet(rs, new Object[][]{{1,1},{2,2},{3,3},{4,4},{5,5},{6,6}});
-
-            //groupBy orderPreserving orderBy desc asc
-            sql="SELECT ORGANIZATION_ID,CONTAINER_ID FROM "+tableName+" group by ORGANIZATION_ID, CONTAINER_ID ORDER BY ORGANIZATION_ID DESC, CONTAINER_ID DESC";
-            rs=conn.prepareStatement(sql).executeQuery();
-            assertResultSet(rs, new Object[][]{{6,6},{5,5},{4,4},{3,3},{2,2},{1,1}});
-
-            //groupBy orderPreserving orderBy desc desc
-            sql="SELECT ORGANIZATION_ID,CONTAINER_ID FROM "+tableName+" group by ORGANIZATION_ID, CONTAINER_ID ORDER BY ORGANIZATION_ID DESC, CONTAINER_ID DESC";
-            rs=conn.prepareStatement(sql).executeQuery();
-            assertResultSet(rs, new Object[][]{{6,6},{5,5},{4,4},{3,3},{2,2},{1,1}});
-
-            //groupBy not orderPreserving orderBy asc asc
-            sql="SELECT ORGANIZATION_ID,SCORE FROM "+tableName+" group by ORGANIZATION_ID, SCORE ORDER BY ORGANIZATION_ID ASC, SCORE ASC";
-            rs=conn.prepareStatement(sql).executeQuery();
-            assertResultSet(rs, new Object[][]{{1,1},{2,2},{3,3},{4,4},{5,5},{6,6}});
-
-            //groupBy not orderPreserving orderBy asc desc
-            sql="SELECT ORGANIZATION_ID,SCORE FROM "+tableName+" group by ORGANIZATION_ID, SCORE ORDER BY ORGANIZATION_ID ASC, SCORE ASC";
-            rs=conn.prepareStatement(sql).executeQuery();
-            assertResultSet(rs, new Object[][]{{1,1},{2,2},{3,3},{4,4},{5,5},{6,6}});
-
-            //groupBy not orderPreserving orderBy desc asc
-            sql="SELECT ORGANIZATION_ID,SCORE FROM "+tableName+" group by ORGANIZATION_ID, SCORE ORDER BY ORGANIZATION_ID DESC, SCORE DESC";
-            rs=conn.prepareStatement(sql).executeQuery();
-            assertResultSet(rs, new Object[][]{{6,6},{5,5},{4,4},{3,3},{2,2},{1,1}});
-
-            //groupBy not orderPreserving orderBy desc desc
-            sql="SELECT ORGANIZATION_ID,SCORE FROM "+tableName+" group by ORGANIZATION_ID, SCORE ORDER BY ORGANIZATION_ID DESC, SCORE DESC";
-            rs=conn.prepareStatement(sql).executeQuery();
-            assertResultSet(rs, new Object[][]{{6,6},{5,5},{4,4},{3,3},{2,2},{1,1}});
-        } finally {
-            if(conn!=null) {
-                conn.close();
-            }
-        }
-    }
-
-    @Test
-    public void testOrderByReverseOptimizationWithNullsLast() throws Exception{
-        for(boolean salted: new boolean[]{true,false}) {
-            doTestOrderByReverseOptimizationWithNullsLast(salted,true,true,true);
-            doTestOrderByReverseOptimizationWithNullsLast(salted,true,true,false);
-            doTestOrderByReverseOptimizationWithNullsLast(salted,true,false,true);
-            doTestOrderByReverseOptimizationWithNullsLast(salted,true,false,false);
-            doTestOrderByReverseOptimizationWithNullsLast(salted,false,true,true);
-            doTestOrderByReverseOptimizationWithNullsLast(salted,false,true,false);
-            doTestOrderByReverseOptimizationWithNullsLast(salted,false,false,true);
-            doTestOrderByReverseOptimizationWithNullsLast(salted,false,false,false);
-        }
-    }
+        conn.setAutoCommit(false);
 
-    private void doTestOrderByReverseOptimizationWithNullsLast(boolean salted,boolean desc1,boolean desc2,boolean desc3) throws Exception {
-        Connection conn = null;
         try {
-            Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
-            conn = DriverManager.getConnection(getUrl(), props);
-            String tableName=generateUniqueName();
-            String sql="CREATE TABLE "+tableName+" ( "+
-                    "ORGANIZATION_ID VARCHAR,"+
-                    "CONTAINER_ID VARCHAR,"+
-                    "SCORE VARCHAR,"+
-                    "ENTITY_ID VARCHAR NOT NULL,"+
-                    "CONSTRAINT TEST_PK PRIMARY KEY ( "+
-                    "ORGANIZATION_ID" +(desc1 ? " DESC" : "" )+","+
-                    "CONTAINER_ID"+(desc2 ? " DESC" : "" )+","+
-                    "SCORE"+(desc3 ? " DESC" : "" )+","+
-                    "ENTITY_ID"+
-                    ")) "+(salted ? "SALT_BUCKETS =4" : "split on('4')");
-            conn.createStatement().execute(sql);
-
-            for(int i=1;i<=6;i++) {
-                conn.createStatement().execute("UPSERT INTO "+tableName+" VALUES (null,'"+i+"','"+i+"','"+i+"')");
-                conn.createStatement().execute("UPSERT INTO "+tableName+" VALUES (null,'"+i+"',null,'"+i+"')");
-                conn.createStatement().execute("UPSERT INTO "+tableName+" VALUES (null,null,'"+i+"','"+i+"')");
-                conn.createStatement().execute("UPSERT INTO "+tableName+" VALUES (null,null,null,'"+i+"')");
-                conn.createStatement().execute("UPSERT INTO "+tableName+" VALUES ('"+i+"','"+i+"','"+i+"','"+i+"')");
-                conn.createStatement().execute("UPSERT INTO "+tableName+" VALUES ('"+i+"','"+i+"',null,'"+i+"')");
-                conn.createStatement().execute("UPSERT INTO "+tableName+" VALUES ('"+i+"',null,'"+i+"','"+i+"')");
-                conn.createStatement().execute("UPSERT INTO "+tableName+" VALUES ('"+i+"',null,null,'"+i+"')");
-            }
-            conn.createStatement().execute("UPSERT INTO "+tableName+" VALUES (null,null,null,'66')");
+            String tableName = generateUniqueName();
+            String ddl = "CREATE TABLE " + tableName +
+                    "  (a_string varchar not null, cf1.a integer, cf1.b varchar, col1 integer, cf2.c varchar, cf2.d integer, col2 integer" +
+                    "  CONSTRAINT pk PRIMARY KEY (a_string))\n";
+            createTestTable(getUrl(), ddl);
+            String dml = "UPSERT INTO " + tableName + " VALUES(?,?,?,?,?,?,?)";
+            PreparedStatement stmt = conn.prepareStatement(dml);
+            stmt.setString(1, "a");
+            stmt.setInt(2, 40);
+            stmt.setString(3, "aa");
+            stmt.setInt(4, 10);
+            stmt.setString(5, "bb");
+            stmt.setInt(6, 20);
+            stmt.setInt(7, 1);
+            stmt.execute();
+            stmt.setString(1, "c");
+            stmt.setInt(2, 30);
+            stmt.setString(3, "cc");
+            stmt.setInt(4, 50);
+            stmt.setString(5, "dd");
+            stmt.setInt(6, 60);
+            stmt.setInt(7, 3);
+            stmt.execute();
+            stmt.setString(1, "b");
+            stmt.setInt(2, 40);
+            stmt.setString(3, "bb");
+            stmt.setInt(4, 5);
+            stmt.setString(5, "aa");
+            stmt.setInt(6, 80);
+            stmt.setInt(7, 2);
+            stmt.execute();
             conn.commit();
 
-            //groupBy orderPreserving orderBy asc asc
-
-            sql="SELECT ORGANIZATION_ID,CONTAINER_ID FROM "+tableName+" group by ORGANIZATION_ID, CONTAINER_ID ORDER BY ORGANIZATION_ID ASC NULLS FIRST, CONTAINER_ID ASC NULLS FIRST";
-            ResultSet rs=conn.prepareStatement(sql).executeQuery();
-            assertResultSet(rs, new Object[][]{{null,null},{null,"1"},{null,"2"},{null,"3"},{null,"4"},{null,"5"},{null,"6"},{"1",null},{"1","1"},{"2",null},{"2","2"},{"3",null},{"3","3"},{"4",null},{"4","4"},{"5",null},{"5","5"},{"6",null},{"6","6"}});
-
-            sql="SELECT ORGANIZATION_ID,CONTAINER_ID FROM "+tableName+" group by ORGANIZATION_ID, CONTAINER_ID ORDER BY ORGANIZATION_ID ASC NULLS FIRST, CONTAINER_ID ASC NULLS LAST";
-            rs=conn.prepareStatement(sql).executeQuery();
-            assertResultSet(rs, new Object[][]{{null,"1"},{null,"2"},{null,"3"},{null,"4"},{null,"5"},{null,"6"},{null,null},{"1","1"},{"1",null},{"2","2"},{"2",null},{"3","3"},{"3",null},{"4","4"},{"4",null},{"5","5"},{"5",null},{"6","6"},{"6",null}});
-
-            sql="SELECT ORGANIZATION_ID,CONTAINER_ID FROM "+tableName+" group by ORGANIZATION_ID, CONTAINER_ID ORDER BY ORGANIZATION_ID ASC NULLS LAST, CONTAINER_ID ASC NULLS FIRST";
-            rs=conn.prepareStatement(sql).executeQuery();
-            assertResultSet(rs, new Object[][]{{"1",null},{"1","1"},{"2",null},{"2","2"},{"3",null},{"3","3"},{"4",null},{"4","4"},{"5",null},{"5","5"},{"6",null},{"6","6"},{null,null},{null,"1"},{null,"2"},{null,"3"},{null,"4"},{null,"5"},{null,"6"}});
-
-            sql="SELECT ORGANIZATION_ID,CONTAINER_ID FROM "+tableName+" group by ORGANIZATION_ID, CONTAINER_ID ORDER BY ORGANIZATION_ID ASC NULLS LAST, CONTAINER_ID ASC NULLS LAST";
-            rs=conn.prepareStatement(sql).executeQuery();
-            assertResultSet(rs, new Object[][]{{"1","1"},{"1",null},{"2","2"},{"2",null},{"3","3"},{"3",null},{"4","4"},{"4",null},{"5","5"},{"5",null},{"6","6"},{"6",null},{null,"1"},{null,"2"},{null,"3"},{null,"4"},{null,"5"},{null,"6"},{null,null}});
-
-            //groupBy orderPreserving orderBy asc desc
-
-            sql="SELECT ORGANIZATION_ID,CONTAINER_ID FROM "+tableName+" group by ORGANIZATION_ID, CONTAINER_ID ORDER BY ORGANIZATION_ID ASC NULLS FIRST, CONTAINER_ID DESC NULLS FIRST";
-            rs=conn.prepareStatement(sql).executeQuery();
-            assertResultSet(rs, new Object[][]{{null,null},{null,"6"},{null,"5"},{null,"4"},{null,"3"},{null,"2"},{null,"1"},{"1",null},{"1","1"},{"2",null},{"2","2"},{"3",null},{"3","3"},{"4",null},{"4","4"},{"5",null},{"5","5"},{"6",null},{"6","6"}});
-
-            sql="SELECT ORGANIZATION_ID,CONTAINER_ID FROM "+tableName+" group by ORGANIZATION_ID, CONTAINER_ID ORDER BY ORGANIZATION_ID ASC NULLS FIRST, CONTAINER_ID DESC NULLS LAST";
-            rs=conn.prepareStatement(sql).executeQuery();
-            assertResultSet(rs, new Object[][]{{null,"6"},{null,"5"},{null,"4"},{null,"3"},{null,"2"},{null,"1"},{null,null},{"1","1"},{"1",null},{"2","2"},{"2",null},{"3","3"},{"3",null},{"4","4"},{"4",null},{"5","5"},{"5",null},{"6","6"},{"6",null}});
-
-            sql="SELECT ORGANIZATION_ID,CONTAINER_ID FROM "+tableName+" group by ORGANIZATION_ID, CONTAINER_ID ORDER BY ORGANIZATION_ID ASC NULLS LAST, CONTAINER_ID DESC NULLS FIRST";
-            rs=conn.prepareStatement(sql).executeQuery();
-            assertResultSet(rs, new Object[][]{{"1",null},{"1","1"},{"2",null},{"2","2"},{"3",null},{"3","3"},{"4",null},{"4","4"},{"5",null},{"5","5"},{"6",null},{"6","6"},{null,null},{null,"6"},{null,"5"},{null,"4"},{null,"3"},{null,"2"},{null,"1"}});
-
-            sql="SELECT ORGANIZATION_ID,CONTAINER_ID FROM "+tableName+" group by ORGANIZATION_ID, CONTAINER_ID ORDER BY ORGANIZATION_ID ASC NULLS LAST, CONTAINER_ID DESC NULLS LAST";
-            rs=conn.prepareStatement(sql).executeQuery();
-            assertResultSet(rs, new Object[][]{{"1","1"},{"1",null},{"2","2"},{"2",null},{"3","3"},{"3",null},{"4","4"},{"4",null},{"5","5"},{"5",null},{"6","6"},{"6",null},{null,"6"},{null,"5"},{null,"4"},{null,"3"},{null,"2"},{null,"1"},{null,null}});
-
-            //groupBy orderPreserving orderBy desc asc
-
-            sql="SELECT ORGANIZATION_ID,CONTAINER_ID FROM "+tableName+" group by ORGANIZATION_ID, CONTAINER_ID ORDER BY ORGANIZATION_ID DESC NULLS FIRST, CONTAINER_ID ASC NULLS FIRST";
-            rs=conn.prepareStatement(sql).executeQuery();
-            assertResultSet(rs, new Object[][]{{null,null},{null,"1"},{null,"2"},{null,"3"},{null,"4"},{null,"5"},{null,"6"},{"6",null},{"6","6"},{"5",null},{"5","5"},{"4",null},{"4","4"},{"3",null},{"3","3"},{"2",null},{"2","2"},{"1",null},{"1","1"}});
-
-            sql="SELECT ORGANIZATION_ID,CONTAINER_ID FROM "+tableName+" group by ORGANIZATION_ID, CONTAINER_ID ORDER BY ORGANIZATION_ID DESC NULLS FIRST, CONTAINER_ID ASC NULLS LAST";
-            rs=conn.prepareStatement(sql).executeQuery();
-            assertResultSet(rs, new Object[][]{{null,"1"},{null,"2"},{null,"3"},{null,"4"},{null,"5"},{null,"6"},{null,null},{"6","6"},{"6",null},{"5","5"},{"5",null},{"4","4"},{"4",null},{"3","3"},{"3",null},{"2","2"},{"2",null},{"1","1"},{"1",null}});
-
-            sql="SELECT ORGANIZATION_ID,CONTAINER_ID FROM "+tableName+" group by ORGANIZATION_ID, CONTAINER_ID ORDER BY ORGANIZATION_ID DESC NULLS LAST, CONTAINER_ID ASC NULLS FIRST";
-            rs=conn.prepareStatement(sql).executeQuery();
-            assertResultSet(rs, new Object[][]{{"6",null},{"6","6"},{"5",null},{"5","5"},{"4",null},{"4","4"},{"3",null},{"3","3"},{"2",null},{"2","2"},{"1",null},{"1","1"},{null,null},{null,"1"},{null,"2"},{null,"3"},{null,"4"},{null,"5"},{null,"6"}});
-
-            sql="SELECT ORGANIZATION_ID,CONTAINER_ID FROM "+tableName+" group by ORGANIZATION_ID, CONTAINER_ID ORDER BY ORGANIZATION_ID DESC NULLS LAST, CONTAINER_ID ASC NULLS LAST";
-            rs=conn.prepareStatement(sql).executeQuery();
-            assertResultSet(rs, new Object[][]{{"6","6"},{"6",null},{"5","5"},{"5",null},{"4","4"},{"4",null},{"3","3"},{"3",null},{"2","2"},{"2",null},{"1","1"},{"1",null},{null,"1"},{null,"2"},{null,"3"},{null,"4"},{null,"5"},{null,"6"},{null,null}});
-
-            //groupBy orderPreserving orderBy desc desc
-
-            sql="SELECT ORGANIZATION_ID,CONTAINER_ID FROM "+tableName+" group by ORGANIZATION_ID, CONTAINER_ID ORDER BY ORGANIZATION_ID DESC NULLS FIRST, CONTAINER_ID DESC NULLS FIRST";
-            rs=conn.prepareStatement(sql).executeQuery();
-            assertResultSet(rs, new Object[][]{{null,null},{null,"6"},{null,"5"},{null,"4"},{null,"3"},{null,"2"},{null,"1"},{"6",null},{"6","6"},{"5",null},{"5","5"},{"4",null},{"4","4"},{"3",null},{"3","3"},{"2",null},{"2","2"},{"1",null},{"1","1"}});
-
-            sql="SELECT ORGANIZATION_ID,CONTAINER_ID FROM "+tableName+" group by ORGANIZATION_ID, CONTAINER_ID ORDER BY ORGANIZATION_ID DESC NULLS FIRST, CONTAINER_ID DESC NULLS LAST";
-            rs=conn.prepareStatement(sql).executeQuery();
-            assertResultSet(rs, new Object[][]{{null,"6"},{null,"5"},{null,"4"},{null,"3"},{null,"2"},{null,"1"},{null,null},{"6","6"},{"6",null},{"5","5"},{"5",null},{"4","4"},{"4",null},{"3","3"},{"3",null},{"2","2"},{"2",null},{"1","1"},{"1",null}});
-
-            sql="SELECT ORGANIZATION_ID,CONTAINER_ID FROM "+tableName+" group by ORGANIZATION_ID, CONTAINER_ID ORDER BY ORGANIZATION_ID DESC NULLS LAST, CONTAINER_ID DESC NULLS FIRST";
-            rs=conn.prepareStatement(sql).executeQuery();
-            assertResultSet(rs, new Object[][]{{"6",null},{"6","6"},{"5",null},{"5","5"},{"4",null},{"4","4"},{"3",null},{"3","3"},{"2",null},{"2","2"},{"1",null},{"1","1"},{null,null},{null,"6"},{null,"5"},{null,"4"},{null,"3"},{null,"2"},{null,"1"}});
-
-            sql="SELECT ORGANIZATION_ID,CONTAINER_ID FROM "+tableName+" group by ORGANIZATION_ID, CONTAINER_ID ORDER BY ORGANIZATION_ID DESC NULLS LAST, CONTAINER_ID DESC NULLS LAST";
-            rs=conn.prepareStatement(sql).executeQuery();
-            assertResultSet(rs, new Object[][]{{"6","6"},{"6",null},{"5","5"},{"5",null},{"4","4"},{"4",null},{"3","3"},{"3",null},{"2","2"},{"2",null},{"1","1"},{"1",null},{null,"6"},{null,"5"},{null,"4"},{null,"3"},{null,"2"},{null,"1"},{null,null}});
-
-            //-----groupBy not orderPreserving--
-
-            //groupBy not orderPreserving orderBy asc asc
-
-            sql="SELECT ORGANIZATION_ID,SCORE FROM "+tableName+" group by ORGANIZATION_ID, SCORE ORDER BY ORGANIZATION_ID ASC NULLS FIRST, SCORE ASC NULLS FIRST";
-            rs=conn.prepareStatement(sql).executeQuery();
-            assertResultSet(rs, new Object[][]{{null,null},{null,"1"},{null,"2"},{null,"3"},{null,"4"},{null,"5"},{null,"6"},{"1",null},{"1","1"},{"2",null},{"2","2"},{"3",null},{"3","3"},{"4",null},{"4","4"},{"5",null},{"5","5"},{"6",null},{"6","6"}});
-
-            sql="SELECT ORGANIZATION_ID,SCORE FROM "+tableName+" group by ORGANIZATION_ID, SCORE ORDER BY ORGANIZATION_ID ASC NULLS FIRST, SCORE ASC NULLS LAST";
-            rs=conn.prepareStatement(sql).executeQuery();
-            assertResultSet(rs, new Object[][]{{null,"1"},{null,"2"},{null,"3"},{null,"4"},{null,"5"},{null,"6"},{null,null},{"1","1"},{"1",null},{"2","2"},{"2",null},{"3","3"},{"3",null},{"4","4"},{"4",null},{"5","5"},{"5",null},{"6","6"},{"6",null}});
-
-            sql="SELECT ORGANIZATION_ID,SCORE FROM "+tableName+" group by ORGANIZATION_ID, SCORE ORDER BY ORGANIZATION_ID ASC NULLS LAST, SCORE ASC NULLS FIRST";
-            rs=conn.prepareStatement(sql).executeQuery();
-            assertResultSet(rs, new Object[][]{{"1",null},{"1","1"},{"2",null},{"2","2"},{"3",null},{"3","3"},{"4",null},{"4","4"},{"5",null},{"5","5"},{"6",null},{"6","6"},{null,null},{null,"1"},{null,"2"},{null,"3"},{null,"4"},{null,"5"},{null,"6"}});
-
-            sql="SELECT ORGANIZATION_ID,SCORE FROM "+tableName+" group by ORGANIZATION_ID, SCORE ORDER BY ORGANIZATION_ID ASC NULLS LAST, SCORE ASC NULLS LAST";
-            rs=conn.prepareStatement(sql).executeQuery();
-            assertResultSet(rs, new Object[][]{{"1","1"},{"1",null},{"2","2"},{"2",null},{"3","3"},{"3",null},{"4","4"},{"4",null},{"5","5"},{"5",null},{"6","6"},{"6",null},{null,"1"},{null,"2"},{null,"3"},{null,"4"},{null,"5"},{null,"6"},{null,null}});
-
-            //groupBy not orderPreserving orderBy asc desc
-
-            sql="SELECT ORGANIZATION_ID,SCORE FROM "+tableName+" group by ORGANIZATION_ID, SCORE ORDER BY ORGANIZATION_ID ASC NULLS FIRST, SCORE DESC NULLS FIRST";
-            rs=conn.prepareStatement(sql).executeQuery();
-            assertResultSet(rs, new Object[][]{{null,null},{null,"6"},{null,"5"},{null,"4"},{null,"3"},{null,"2"},{null,"1"},{"1",null},{"1","1"},{"2",null},{"2","2"},{"3",null},{"3","3"},{"4",null},{"4","4"},{"5",null},{"5","5"},{"6",null},{"6","6"}});
-
-            sql="SELECT ORGANIZATION_ID,SCORE FROM "+tableName+" group by ORGANIZATION_ID, SCORE ORDER BY ORGANIZATION_ID ASC NULLS FIRST, SCORE DESC NULLS LAST";
-            rs=conn.prepareStatement(sql).executeQuery();
-            assertResultSet(rs, new Object[][]{{null,"6"},{null,"5"},{null,"4"},{null,"3"},{null,"2"},{null,"1"},{null,null},{"1","1"},{"1",null},{"2","2"},{"2",null},{"3","3"},{"3",null},{"4","4"},{"4",null},{"5","5"},{"5",null},{"6","6"},{"6",null}});
-
-            sql="SELECT ORGANIZATION_ID,SCORE FROM "+tableName+" group by ORGANIZATION_ID, SCORE ORDER BY ORGANIZATION_ID ASC NULLS LAST, SCORE DESC NULLS FIRST";
-            rs=conn.prepareStatement(sql).executeQuery();
-            assertResultSet(rs, new Object[][]{{"1",null},{"1","1"},{"2",null},{"2","2"},{"3",null},{"3","3"},{"4",null},{"4","4"},{"5",null},{"5","5"},{"6",null},{"6","6"},{null,null},{null,"6"},{null,"5"},{null,"4"},{null,"3"},{null,"2"},{null,"1"}});
-
-            sql="SELECT ORGANIZATION_ID,SCORE FROM "+tableName+" group by ORGANIZATION_ID, SCORE ORDER BY ORGANIZATION_ID ASC NULLS LAST, SCORE DESC NULLS LAST";
-            rs=conn.prepareStatement(sql).executeQuery();
-            assertResultSet(rs, new Object[][]{{"1","1"},{"1",null},{"2","2"},{"2",null},{"3","3"},{"3",null},{"4","4"},{"4",null},{"5","5"},{"5",null},{"6","6"},{"6",null},{null,"6"},{null,"5"},{null,"4"},{null,"3"},{null,"2"},{null,"1"},{null,null}});
-
-            //groupBy not orderPreserving orderBy desc asc
-
-            sql="SELECT ORGANIZATION_ID,SCORE FROM "+tableName+" group by ORGANIZATION_ID, SCORE ORDER BY ORGANIZATION_ID DESC NULLS FIRST, SCORE ASC NULLS FIRST";
-            rs=conn.prepareStatement(sql).executeQuery();
-            assertResultSet(rs, new Object[][]{{null,null},{null,"1"},{null,"2"},{null,"3"},{null,"4"},{null,"5"},{null,"6"},{"6",null},{"6","6"},{"5",null},{"5","5"},{"4",null},{"4","4"},{"3",null},{"3","3"},{"2",null},{"2","2"},{"1",null},{"1","1"}});
-
-            sql="SELECT ORGANIZATION_ID,SCORE FROM "+tableName+" group by ORGANIZATION_ID, SCORE ORDER BY ORGANIZATION_ID DESC NULLS FIRST, SCORE ASC NULLS LAST";
-            rs=conn.prepareStatement(sql).executeQuery();
-            assertResultSet(rs, new Object[][]{{null,"1"},{null,"2"},{null,"3"},{null,"4"},{null,"5"},{null,"6"},{null,null},{"6","6"},{"6",null},{"5","5"},{"5",null},{"4","4"},{"4",null},{"3","3"},{"3",null},{"2","2"},{"2",null},{"1","1"},{"1",null}});
-
-            sql="SELECT ORGANIZATION_ID,SCORE FROM "+tableName+" group by ORGANIZATION_ID, SCORE ORDER BY ORGANIZATION_ID DESC NULLS LAST, SCORE ASC NULLS FIRST";
-            rs=conn.prepareStatement(sql).executeQuery();
-            assertResultSet(rs, new Object[][]{{"6",null},{"6","6"},{"5",null},{"5","5"},{"4",null},{"4","4"},{"3",null},{"3","3"},{"2",null},{"2","2"},{"1",null},{"1","1"},{null,null},{null,"1"},{null,"2"},{null,"3"},{null,"4"},{null,"5"},{null,"6"}});
-
-            sql="SELECT ORGANIZATION_ID,SCORE FROM "+tableName+" group by ORGANIZATION_ID, SCORE ORDER BY ORGANIZATION_ID DESC NULLS LAST, SCORE ASC NULLS LAST";
-            rs=conn.prepareStatement(sql).executeQuery();
-            assertResultSet(rs, new Object[][]{{"6","6"},{"6",null},{"5","5"},{"5",null},{"4","4"},{"4",null},{"3","3"},{"3",null},{"2","2"},{"2",null},{"1","1"},{"1",null},{null,"1"},{null,"2"},{null,"3"},{null,"4"},{null,"5"},{null,"6"},{null,null}});
-
-            //groupBy not orderPreserving orderBy desc desc
-
-            sql="SELECT ORGANIZATION_ID,SCORE FROM "+tableName+" group by ORGANIZATION_ID, SCORE ORDER BY ORGANIZATION_ID DESC NULLS FIRST, SCORE DESC NULLS FIRST";
-            rs=conn.prepareStatement(sql).executeQuery();
-            assertResultSet(rs, new Object[][]{{null,null},{null,"6"},{null,"5"},{null,"4"},{null,"3"},{null,"2"},{null,"1"},{"6",null},{"6","6"},{"5",null},{"5","5"},{"4",null},{"4","4"},{"3",null},{"3","3"},{"2",null},{"2","2"},{"1",null},{"1","1"}});
-
-            sql="SELECT ORGANIZATION_ID,SCORE FROM "+tableName+" group by ORGANIZATION_ID, SCORE ORDER BY ORGANIZATION_ID DESC NULLS FIRST, SCORE DESC NULLS LAST";
-            rs=conn.prepareStatement(sql).executeQuery();
-            assertResultSet(rs, new Object[][]{{null,"6"},{null,"5"},{null,"4"},{null,"3"},{null,"2"},{null,"1"},{null,null},{"6","6"},{"6",null},{"5","5"},{"5",null},{"4","4"},{"4",null},{"3","3"},{"3",null},{"2","2"},{"2",null},{"1","1"},{"1",null}});
-
-            sql="SELECT ORGANIZATION_ID,SCORE FROM "+tableName+" group by ORGANIZATION_ID, SCORE ORDER BY ORGANIZATION_ID DESC NULLS LAST, SCORE DESC NULLS FIRST";
-            rs=conn.prepareStatement(sql).executeQuery();
-            assertResultSet(rs, new Object[][]{{"6",null},{"6","6"},{"5",null},{"5","5"},{"4",null},{"4","4"},{"3",null},{"3","3"},{"2",null},{"2","2"},{"1",null},{"1","1"},{null,null},{null,"6"},{null,"5"},{null,"4"},{null,"3"},{null,"2"},{null,"1"}});
-
-            sql="SELECT ORGANIZATION_ID,SCORE FROM "+tableName+" group by ORGANIZATION_ID, SCORE ORDER BY ORGANIZATION_ID DESC NULLS LAST, SCORE DESC NULLS LAST";
-            rs=conn.prepareStatement(sql).executeQuery();
-            assertResultSet(rs, new Object[][]{{"6","6"},{"6",null},{"5","5"},{"5",null},{"4","4"},{"4",null},{"3","3"},{"3",null},{"2","2"},{"2",null},{"1","1"},{"1",null},{null,"6"},{null,"5"},{null,"4"},{null,"3"},{null,"2"},{null,"1"},{null,null}});
-
-            //-------test only one return column----------------------------------
-
-            sql="SELECT SCORE FROM "+tableName+" group by SCORE ORDER BY SCORE ASC NULLS FIRST";
-            rs=conn.prepareStatement(sql).executeQuery();
-            assertResultSet(rs, new Object[][]{{null},{"1"},{"2"},{"3"},{"4"},{"5"},{"6"}});
-
-            sql="SELECT SCORE FROM "+tableName+" group by SCORE ORDER BY SCORE ASC NULLS LAST";
-            rs=conn.prepareStatement(sql).executeQuery();
-            assertResultSet(rs, new Object[][]{{"1"},{"2"},{"3"},{"4"},{"5"},{"6"},{null}});
-
-            sql="SELECT SCORE FROM "+tableName+" group by SCORE ORDER BY SCORE DESC NULLS FIRST";
-            rs=conn.prepareStatement(sql).executeQuery();
-            assertResultSet(rs, new Object[][]{{null},{"6"},{"5"},{"4"},{"3"},{"2"},{"1"}});
+            String query = "select * from " + tableName + " order by 2, 5";
+            ResultSet rs = conn.createStatement().executeQuery(query);
+            assertTrue(rs.next());
+            assertEquals("c",rs.getString(1));
+            assertEquals(30,rs.getInt(2));
+            assertEquals("cc",rs.getString(3));
+            assertEquals(50,rs.getInt(4));
+            assertEquals("dd",rs.getString(5));
+            assertEquals(60,rs.getInt(6));
+            assertEquals(3,rs.getInt(7));
+            assertTrue(rs.next());
+            assertEquals("b",rs.getString(1));
+            assertEquals(40,rs.getInt(2));
+            assertEquals("bb",rs.getString(3));
+            assertEquals(5,rs.getInt(4));
+            assertEquals("aa",rs.getString(5));
+            assertEquals(80,rs.getInt(6));
+            assertEquals(2,rs.getInt(7));
+            assertTrue(rs.next());
+            assertEquals("a",rs.getString(1));
+            assertEquals(40,rs.getInt(2));
+            assertEquals("aa",rs.getString(3));
+            assertEquals(10,rs.getInt(4));
+            assertEquals("bb",rs.getString(5));
+            assertEquals(20,rs.getInt(6));
+            assertEquals(1,rs.getInt(7));
+            assertFalse(rs.next());
 
-            sql="SELECT SCORE FROM "+tableName+" group by SCORE ORDER BY SCORE DESC NULLS LAST";
-            rs=conn.prepareStatement(sql).executeQuery();
-            assertResultSet(rs, new Object[][]{{"6"},{"5"},{"4"},{"3"},{"2"},{"1"},{null}});
+            query = "select * from " + tableName + " order by 7";
+            rs = conn.createStatement().executeQuery(query);
+            assertTrue(rs.next());
+            assertEquals("a",rs.getString(1));
+            assertEquals(40,rs.getInt(2));
+            assertEquals("aa",rs.getString(3));
+            assertEquals(10,rs.getInt(4));
+            assertEquals("bb",rs.getString(5));
+            assertEquals(20,rs.getInt(6));
+            assertEquals(1,rs.getInt(7));
+            assertTrue(rs.next());
+            assertEquals("b",rs.getString(1));
+            assertEquals(40,rs.getInt(2));
+            assertEquals("bb",rs.getString(3));
+            assertEquals(5,rs.getInt(4));
+            assertEquals("aa",rs.getString(5));
+            assertEquals(80,rs.getInt(6));
+            assertEquals(2,rs.getInt(7));
+            assertTrue(rs.next());
+            assertEquals("c",rs.getString(1));
+            assertEquals(30,rs.getInt(2));
+            assertEquals("cc",rs.getString(3));
+            assertEquals(50,rs.getInt(4));
+            assertEquals("dd",rs.getString(5));
+            assertEquals(60,rs.getInt(6));
+            assertEquals(3,rs.getInt(7));
+            assertFalse(rs.next());
         } finally {
-            if(conn!=null) {
-                conn.close();
-            }
+            conn.close();
         }
     }
-
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/c509d58f/phoenix-core/src/it/java/org/apache/phoenix/end2end/ParallelStatsDisabledIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ParallelStatsDisabledIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ParallelStatsDisabledIT.java
index 98939da..561aee5 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ParallelStatsDisabledIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ParallelStatsDisabledIT.java
@@ -18,12 +18,26 @@
 
 package org.apache.phoenix.end2end;
 
+import org.apache.commons.lang.StringUtils;
 import org.apache.phoenix.query.BaseTest;
+import org.apache.phoenix.util.QueryBuilder;
+import org.apache.phoenix.util.QueryUtil;
 import org.apache.phoenix.util.ReadOnlyProps;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.experimental.categories.Category;
 
+import java.sql.Connection;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+
+
 /**
  * Base class for tests whose methods run in parallel with statistics disabled.
  * You must create unique names using {@link #generateUniqueName()} for each
@@ -41,4 +55,30 @@ public abstract class ParallelStatsDisabledIT extends BaseTest {
     public static void tearDownMiniCluster() throws Exception {
         BaseTest.tearDownMiniClusterIfBeyondThreshold();
     }
+
+    protected ResultSet executeQuery(Connection conn, QueryBuilder queryBuilder) throws SQLException {
+        PreparedStatement statement = conn.prepareStatement(queryBuilder.build());
+        ResultSet rs = statement.executeQuery();
+        return rs;
+    }
+
+    protected ResultSet executeQueryThrowsException(Connection conn, QueryBuilder queryBuilder,
+            String expectedPhoenixExceptionMsg, String expectedSparkExceptionMsg) {
+        ResultSet rs = null;
+        try {
+            rs = executeQuery(conn, queryBuilder);
+            fail();
+        }
+        catch(Exception e) {
+            assertTrue(e.getMessage().contains(expectedPhoenixExceptionMsg));
+        }
+        return rs;
+    }
+
+    protected void validateQueryPlan(Connection conn, QueryBuilder queryBuilder, String expectedPhoenixPlan, String expectedSparkPlan) throws SQLException {
+        if (StringUtils.isNotBlank(expectedPhoenixPlan)) {
+            ResultSet rs = conn.createStatement().executeQuery("EXPLAIN " + queryBuilder.build());
+            assertEquals(expectedPhoenixPlan, QueryUtil.getExplainPlan(rs));
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/c509d58f/phoenix-core/src/it/java/org/apache/phoenix/end2end/salted/BaseSaltedTableIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/salted/BaseSaltedTableIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/salted/BaseSaltedTableIT.java
new file mode 100644
index 0000000..3051cd6
--- /dev/null
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/salted/BaseSaltedTableIT.java
@@ -0,0 +1,474 @@
+/*
+ * 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.salted;
+
+import static org.apache.phoenix.util.TestUtil.TABLE_WITH_SALTING;
+import static org.apache.phoenix.util.TestUtil.TEST_PROPERTIES;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+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.end2end.ParallelStatsDisabledIT;
+import org.apache.phoenix.util.PropertiesUtil;
+import org.apache.phoenix.util.QueryBuilder;
+import org.apache.phoenix.util.QueryUtil;
+import org.junit.Test;
+
+import com.google.common.collect.Lists;
+
+public abstract class BaseSaltedTableIT extends ParallelStatsDisabledIT  {
+
+    protected static String initTableValues(byte[][] splits) throws Exception {
+        String tableName = generateUniqueName();
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        Connection conn = DriverManager.getConnection(getUrl(), props);
+
+        // Rows we inserted:
+        // 1ab123abc111
+        // 1abc456abc111
+        // 1de123abc111
+        // 2abc123def222
+        // 3abc123ghi333
+        // 4abc123jkl444
+        try {
+            // Upsert with no column specifies.
+            ensureTableCreated(getUrl(), tableName, TABLE_WITH_SALTING, splits, null, null);
+            String query = "UPSERT INTO " + tableName + " VALUES(?,?,?,?,?)";
+            PreparedStatement stmt = conn.prepareStatement(query);
+            stmt.setInt(1, 1);
+            stmt.setString(2, "ab");
+            stmt.setString(3, "123");
+            stmt.setString(4, "abc");
+            stmt.setInt(5, 111);
+            stmt.execute();
+            conn.commit();
+
+            stmt.setInt(1, 1);
+            stmt.setString(2, "abc");
+            stmt.setString(3, "456");
+            stmt.setString(4, "abc");
+            stmt.setInt(5, 111);
+            stmt.execute();
+            conn.commit();
+
+            // Test upsert when statement explicitly specifies the columns to upsert into.
+            query = "UPSERT INTO " + tableName +
+                    " (a_integer, a_string, a_id, b_string, b_integer) " +
+                    " VALUES(?,?,?,?,?)";
+            stmt = conn.prepareStatement(query);
+
+            stmt.setInt(1, 1);
+            stmt.setString(2, "de");
+            stmt.setString(3, "123");
+            stmt.setString(4, "abc");
+            stmt.setInt(5, 111);
+            stmt.execute();
+            conn.commit();
+
+            stmt.setInt(1, 2);
+            stmt.setString(2, "abc");
+            stmt.setString(3, "123");
+            stmt.setString(4, "def");
+            stmt.setInt(5, 222);
+            stmt.execute();
+            conn.commit();
+
+            // Test upsert when order of column is shuffled.
+            query = "UPSERT INTO " + tableName +
+                    " (a_string, a_integer, a_id, b_string, b_integer) " +
+                    " VALUES(?,?,?,?,?)";
+            stmt = conn.prepareStatement(query);
+            stmt.setString(1, "abc");
+            stmt.setInt(2, 3);
+            stmt.setString(3, "123");
+            stmt.setString(4, "ghi");
+            stmt.setInt(5, 333);
+            stmt.execute();
+            conn.commit();
+
+            stmt.setString(1, "abc");
+            stmt.setInt(2, 4);
+            stmt.setString(3, "123");
+            stmt.setString(4, "jkl");
+            stmt.setInt(5, 444);
+            stmt.execute();
+            conn.commit();
+        } finally {
+            conn.close();
+        }
+        return tableName;
+    }
+
+    @Test
+    public void testSelectValueNoWhereClause() throws Exception {
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        Connection conn = DriverManager.getConnection(getUrl(), props);
+        try {
+            String tableName = initTableValues(null);
+            // "SELECT * FROM " + tableName;
+            QueryBuilder queryBuilder = new QueryBuilder()
+                .setSelectColumns(
+                    Lists.newArrayList("A_INTEGER", "A_STRING", "A_ID", "B_STRING", "B_INTEGER"))
+                .setFullTableName(tableName);
+            ResultSet rs = executeQuery(conn, queryBuilder);
+            assertTrue(rs.next());
+            assertEquals(1, rs.getInt(1));
+            assertEquals("ab", rs.getString(2));
+            assertEquals("123", rs.getString(3));
+            assertEquals("abc", rs.getString(4));
+            assertEquals(111, rs.getInt(5));
+
+            assertTrue(rs.next());
+            assertEquals(1, rs.getInt(1));
+            assertEquals("abc", rs.getString(2));
+            assertEquals("456", rs.getString(3));
+            assertEquals("abc", rs.getString(4));
+            assertEquals(111, rs.getInt(5));
+
+            assertTrue(rs.next());
+            assertEquals(1, rs.getInt(1));
+            assertEquals("de", rs.getString(2));
+            assertEquals("123", rs.getString(3));
+            assertEquals("abc", rs.getString(4));
+            assertEquals(111, rs.getInt(5));
+
+            assertTrue(rs.next());
+            assertEquals(2, rs.getInt(1));
+            assertEquals("abc", rs.getString(2));
+            assertEquals("123", rs.getString(3));
+            assertEquals("def", rs.getString(4));
+            assertEquals(222, rs.getInt(5));
+
+            assertTrue(rs.next());
+            assertEquals(3, rs.getInt(1));
+            assertEquals("abc", rs.getString(2));
+            assertEquals("123", rs.getString(3));
+            assertEquals("ghi", rs.getString(4));
+            assertEquals(333, rs.getInt(5));
+
+            assertTrue(rs.next());
+            assertEquals(4, rs.getInt(1));
+            assertEquals("abc", rs.getString(2));
+            assertEquals("123", rs.getString(3));
+            assertEquals("jkl", rs.getString(4));
+            assertEquals(444, rs.getInt(5));
+
+            assertFalse(rs.next());
+        } finally {
+            conn.close();
+        }
+    }
+
+    @Test
+    public void testSelectValueWithFullyQualifiedWhereClause() throws Exception {
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        Connection conn = DriverManager.getConnection(getUrl(), props);
+        try {
+            String tableName = initTableValues(null);
+            PreparedStatement stmt;
+            ResultSet rs;
+
+            // Variable length slot with bounded ranges.
+            QueryBuilder queryBuilder = new QueryBuilder()
+                .setSelectColumns(
+                        Lists.newArrayList("A_INTEGER", "A_STRING", "A_ID", "B_STRING", "B_INTEGER"))
+                .setFullTableName(tableName)
+                .setWhereClause("a_integer = 1 AND a_string >= 'ab' AND a_string < 'de' AND a_id = '123'");
+            rs = executeQuery(conn, queryBuilder);
+            assertTrue(rs.next());
+            assertEquals(1, rs.getInt(1));
+            assertEquals("ab", rs.getString(2));
+            assertEquals("123", rs.getString(3));
+            assertEquals("abc", rs.getString(4));
+            assertEquals(111, rs.getInt(5));
+            assertFalse(rs.next());
+
+            // all single slots with one value.
+            queryBuilder.setWhereClause("a_integer = 1 AND a_string = 'ab' AND a_id = '123'");
+            rs = executeQuery(conn, queryBuilder);
+            assertTrue(rs.next());
+            assertEquals(1, rs.getInt(1));
+            assertEquals("ab", rs.getString(2));
+            assertEquals("123", rs.getString(3));
+            assertEquals("abc", rs.getString(4));
+            assertEquals(111, rs.getInt(5));
+            assertFalse(rs.next());
+
+            // all single slots with multiple values.
+            queryBuilder.setWhereClause("a_integer in (2, 4) AND a_string = 'abc' AND a_id = '123'");
+            rs = executeQuery(conn, queryBuilder);
+
+            assertTrue(rs.next());
+            assertEquals(2, rs.getInt(1));
+            assertEquals("abc", rs.getString(2));
+            assertEquals("123", rs.getString(3));
+            assertEquals("def", rs.getString(4));
+            assertEquals(222, rs.getInt(5));
+
+            assertTrue(rs.next());
+            assertEquals(4, rs.getInt(1));
+            assertEquals("abc", rs.getString(2));
+            assertEquals("123", rs.getString(3));
+            assertEquals("jkl", rs.getString(4));
+            assertEquals(444, rs.getInt(5));
+            assertFalse(rs.next());
+
+            queryBuilder.setWhereClause("A_INTEGER in (1,2,3,4) AND A_STRING in ('a', 'abc', 'de') AND A_ID = '123'");
+            queryBuilder.setSelectColumns(Lists.newArrayList("A_INTEGER", "A_STRING", "A_ID"));
+            rs = executeQuery(conn, queryBuilder);
+            assertTrue(rs.next());
+            assertEquals(1, rs.getInt(1));
+            assertEquals("de", rs.getString(2));
+
+            assertTrue(rs.next());
+            assertEquals(2, rs.getInt(1));
+            assertEquals("abc", rs.getString(2));
+
+            assertTrue(rs.next());
+            assertEquals(3, rs.getInt(1));
+            assertEquals("abc", rs.getString(2));
+
+            assertTrue(rs.next());
+            assertEquals(4, rs.getInt(1));
+            assertEquals("abc", rs.getString(2));
+            assertFalse(rs.next());
+
+            // fixed length slot with bounded ranges.
+            queryBuilder.setWhereClause("A_INTEGER > 1 AND A_INTEGER < 4 AND A_STRING = 'abc' AND A_ID = '123'");
+            queryBuilder.setSelectColumns(Lists.newArrayList("A_STRING", "A_ID", "A_INTEGER"));
+            rs = executeQuery(conn, queryBuilder);
+            assertTrue(rs.next());
+            assertEquals("abc", rs.getString(1));
+            assertEquals("123", rs.getString(2));
+
+            assertTrue(rs.next());
+            assertEquals("abc", rs.getString(1));
+            assertEquals("123", rs.getString(2));
+            assertFalse(rs.next());
+
+            // fixed length slot with unbound ranges.
+            queryBuilder.setWhereClause("A_INTEGER > 1 AND A_STRING = 'abc' AND A_ID = '123'");
+            queryBuilder.setSelectColumns(Lists.newArrayList("B_STRING", "B_INTEGER", "A_INTEGER", "A_STRING", "A_ID"));
+            rs = executeQuery(conn, queryBuilder);
+            assertTrue(rs.next());
+            assertEquals("def", rs.getString(1));
+            assertEquals(222, rs.getInt(2));
+
+            assertTrue(rs.next());
+            assertEquals("ghi", rs.getString(1));
+            assertEquals(333, rs.getInt(2));
+
+            assertTrue(rs.next());
+            assertEquals("jkl", rs.getString(1));
+            assertEquals(444, rs.getInt(2));
+            assertFalse(rs.next());
+
+            // Variable length slot with unbounded ranges.
+            queryBuilder.setWhereClause("A_INTEGER = 1 AND A_STRING > 'ab' AND A_ID = '123'");
+            queryBuilder.setSelectColumns(
+                Lists.newArrayList("A_INTEGER", "A_STRING", "A_ID", "B_STRING", "B_INTEGER"));
+            rs = executeQuery(conn, queryBuilder);
+            assertTrue(rs.next());
+            assertEquals(1, rs.getInt(1));
+            assertEquals("de", rs.getString(2));
+            assertEquals("123", rs.getString(3));
+            assertEquals("abc", rs.getString(4));
+            assertEquals(111, rs.getInt(5));
+            assertFalse(rs.next());
+
+        } finally {
+            conn.close();
+        }
+    }
+
+    @Test
+    public void testSelectValueWithNotFullyQualifiedWhereClause() throws Exception {
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        Connection conn = DriverManager.getConnection(getUrl(), props);
+        try {
+            String tableName = initTableValues(null);
+
+            // Where without fully qualified key, point query.
+            String query = "SELECT * FROM " + tableName + " WHERE a_integer = ? AND a_string = ?";
+            PreparedStatement stmt = conn.prepareStatement(query);
+
+            stmt.setInt(1, 1);
+            stmt.setString(2, "abc");
+            ResultSet rs = stmt.executeQuery();
+            assertTrue(rs.next());
+            assertEquals(1, rs.getInt(1));
+            assertEquals("abc", rs.getString(2));
+            assertEquals("456", rs.getString(3));
+            assertEquals("abc", rs.getString(4));
+            assertEquals(111, rs.getInt(5));
+            assertFalse(rs.next());
+
+            // Where without fully qualified key, range query.
+            query = "SELECT * FROM " + tableName + " WHERE a_integer >= 2";
+            stmt = conn.prepareStatement(query);
+            rs = stmt.executeQuery();
+            assertTrue(rs.next());
+            assertEquals(2, rs.getInt(1));
+            assertEquals("abc", rs.getString(2));
+            assertEquals("123", rs.getString(3));
+            assertEquals("def", rs.getString(4));
+            assertEquals(222, rs.getInt(5));
+
+            assertTrue(rs.next());
+            assertEquals(3, rs.getInt(1));
+            assertEquals("abc", rs.getString(2));
+            assertEquals("123", rs.getString(3));
+            assertEquals("ghi", rs.getString(4));
+            assertEquals(333, rs.getInt(5));
+
+            assertTrue(rs.next());
+            assertEquals(4, rs.getInt(1));
+            assertEquals("abc", rs.getString(2));
+            assertEquals("123", rs.getString(3));
+            assertEquals("jkl", rs.getString(4));
+            assertEquals(444, rs.getInt(5));
+            assertFalse(rs.next());
+
+            // With point query.
+            query = "SELECT a_string FROM " + tableName + " WHERE a_string = ?";
+            stmt = conn.prepareStatement(query);
+            stmt.setString(1, "de");
+            rs = stmt.executeQuery();
+            assertTrue(rs.next());
+            assertEquals("de", rs.getString(1));
+            assertFalse(rs.next());
+
+            query = "SELECT a_id FROM " + tableName + " WHERE a_id = ?";
+            stmt = conn.prepareStatement(query);
+            stmt.setString(1, "456");
+            rs = stmt.executeQuery();
+            assertTrue(rs.next());
+            assertEquals("456", rs.getString(1));
+            assertFalse(rs.next());
+        } finally {
+            conn.close();
+        }
+    }
+
+    @Test
+    public void testSelectWithGroupBy() throws Exception {
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        Connection conn = DriverManager.getConnection(getUrl(), props);
+        try {
+            String tableName = initTableValues(null);
+
+            String query = "SELECT a_integer FROM " + tableName + " GROUP BY a_integer";
+            PreparedStatement stmt = conn.prepareStatement(query);
+            ResultSet rs = stmt.executeQuery();
+            int count = 0;
+            while (rs.next()) {
+                count++;
+            }
+            assertEquals("Group by does not return the right count.", count, 4);
+        } finally {
+            conn.close();
+        }
+    }
+
+    @Test
+    public void testLimitScan() throws Exception {
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        Connection conn = DriverManager.getConnection(getUrl(), props);
+        try {
+            String tableName = initTableValues(null);
+
+            String query = "SELECT a_integer FROM " + tableName + " WHERE a_string='abc' LIMIT 1";
+            PreparedStatement stmt = conn.prepareStatement(query);
+            ResultSet rs = stmt.executeQuery();
+            assertTrue(rs.next());
+            assertEquals(1, rs.getInt(1));
+            assertFalse(rs.next());
+        } finally {
+            conn.close();
+        }
+    }
+
+    @Test
+    public void testSelectWithOrderByRowKey() throws Exception {
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        Connection conn = DriverManager.getConnection(getUrl(), props);
+        try {
+            String tableName = initTableValues(null);
+
+            String query = "SELECT * FROM " + tableName + " ORDER  BY  a_integer, a_string, a_id";
+            PreparedStatement statement = conn.prepareStatement(query);
+            ResultSet explainPlan = statement.executeQuery("EXPLAIN " + query);
+            // Confirm that ORDER BY in row key order will be optimized out for salted table
+            assertEquals("CLIENT PARALLEL 4-WAY FULL SCAN OVER " + tableName + "\n" +
+                    "CLIENT MERGE SORT", QueryUtil.getExplainPlan(explainPlan));
+            ResultSet rs = statement.executeQuery();
+
+            assertTrue(rs.next());
+            assertEquals(1, rs.getInt(1));
+            assertEquals("ab", rs.getString(2));
+            assertEquals("123", rs.getString(3));
+            assertEquals("abc", rs.getString(4));
+            assertEquals(111, rs.getInt(5));
+
+            assertTrue(rs.next());
+            assertEquals(1, rs.getInt(1));
+            assertEquals("abc", rs.getString(2));
+            assertEquals("456", rs.getString(3));
+            assertEquals("abc", rs.getString(4));
+            assertEquals(111, rs.getInt(5));
+
+            assertTrue(rs.next());
+            assertEquals(1, rs.getInt(1));
+            assertEquals("de", rs.getString(2));
+            assertEquals("123", rs.getString(3));
+            assertEquals("abc", rs.getString(4));
+            assertEquals(111, rs.getInt(5));
+
+            assertTrue(rs.next());
+            assertEquals(2, rs.getInt(1));
+            assertEquals("abc", rs.getString(2));
+            assertEquals("123", rs.getString(3));
+            assertEquals("def", rs.getString(4));
+            assertEquals(222, rs.getInt(5));
+
+            assertTrue(rs.next());
+            assertEquals(3, rs.getInt(1));
+            assertEquals("abc", rs.getString(2));
+            assertEquals("123", rs.getString(3));
+            assertEquals("ghi", rs.getString(4));
+            assertEquals(333, rs.getInt(5));
+
+            assertTrue(rs.next());
+            assertEquals(4, rs.getInt(1));
+            assertEquals("abc", rs.getString(2));
+            assertEquals("123", rs.getString(3));
+            assertEquals("jkl", rs.getString(4));
+            assertEquals(444, rs.getInt(5));
+
+            assertFalse(rs.next());
+        } finally {
+            conn.close();
+        }
+    }
+}