You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@phoenix.apache.org by pb...@apache.org on 2018/11/27 15:18:11 UTC

[01/28] 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-cdh5.15 7f13f87c5 -> 505551251


http://git-wip-us.apache.org/repos/asf/phoenix/blob/678563f5/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/678563f5/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


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

Posted by pb...@apache.org.
http://git-wip-us.apache.org/repos/asf/phoenix/blob/678563f5/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/678563f5/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/678563f5/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();
+        }
+    }
+}


[09/28] phoenix git commit: PHOENIX-4996: Refactor PTableImpl to use Builder Pattern

Posted by pb...@apache.org.
PHOENIX-4996: Refactor PTableImpl to use Builder Pattern


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

Branch: refs/heads/4.x-cdh5.15
Commit: 1767244a04e90b9d0c39b1f149342ee02e5c9a9d
Parents: 7eb336d
Author: Chinmay Kulkarni <ch...@gmail.com>
Authored: Fri Nov 2 21:00:09 2018 +0000
Committer: Pedro Boado <pb...@apache.org>
Committed: Tue Nov 27 15:11:22 2018 +0000

----------------------------------------------------------------------
 .../apache/phoenix/compile/DeleteCompiler.java  |    5 +-
 .../apache/phoenix/compile/FromCompiler.java    |   66 +-
 .../apache/phoenix/compile/JoinCompiler.java    |   53 +-
 .../compile/TupleProjectionCompiler.java        |   60 +-
 .../apache/phoenix/compile/UnionCompiler.java   |   41 +-
 .../apache/phoenix/compile/UpsertCompiler.java  |   12 +-
 .../coprocessor/MetaDataEndpointImpl.java       |   96 +-
 .../UngroupedAggregateRegionObserver.java       |    6 +-
 .../coprocessor/WhereConstantParser.java        |    3 +-
 .../query/ConnectionlessQueryServicesImpl.java  |    9 +-
 .../apache/phoenix/schema/MetaDataClient.java   |  215 ++-
 .../apache/phoenix/schema/PMetaDataImpl.java    |   28 +-
 .../org/apache/phoenix/schema/PTableImpl.java   | 1259 +++++++++++-------
 .../org/apache/phoenix/schema/TableRef.java     |   17 +-
 .../phoenix/execute/CorrelatePlanTest.java      |   32 +-
 .../execute/LiteralResultIteratorPlanTest.java  |   33 +-
 16 files changed, 1303 insertions(+), 632 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/1767244a/phoenix-core/src/main/java/org/apache/phoenix/compile/DeleteCompiler.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/DeleteCompiler.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/DeleteCompiler.java
index 583085e..8c9a930 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/DeleteCompiler.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/DeleteCompiler.java
@@ -89,7 +89,6 @@ import org.apache.phoenix.schema.types.PLong;
 import org.apache.phoenix.transaction.PhoenixTransactionProvider.Feature;
 import org.apache.phoenix.util.ByteUtil;
 import org.apache.phoenix.util.IndexUtil;
-import org.apache.phoenix.util.MetaDataUtil;
 import org.apache.phoenix.util.ScanUtil;
 
 import com.google.common.base.Preconditions;
@@ -615,7 +614,9 @@ public class DeleteCompiler {
                     }
                 });
             }
-            PTable projectedTable = PTableImpl.makePTable(table, PTableType.PROJECTED, adjustedProjectedColumns);
+            PTable projectedTable = PTableImpl.builderWithColumns(table, adjustedProjectedColumns)
+                    .setType(PTableType.PROJECTED)
+                    .build();
             final TableRef projectedTableRef = new TableRef(projectedTable, targetTableRef.getLowerBoundTimeStamp(), targetTableRef.getTimeStamp());
 
             QueryPlan bestPlanToBe = dataPlan;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/1767244a/phoenix-core/src/main/java/org/apache/phoenix/compile/FromCompiler.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/FromCompiler.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/FromCompiler.java
index efc66a9..2701af0 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/FromCompiler.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/FromCompiler.java
@@ -32,8 +32,6 @@ import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.phoenix.coprocessor.MetaDataProtocol;
 import org.apache.phoenix.coprocessor.MetaDataProtocol.MetaDataMutationResult;
 import org.apache.phoenix.coprocessor.MetaDataProtocol.MutationCode;
-import org.apache.phoenix.exception.SQLExceptionCode;
-import org.apache.phoenix.exception.SQLExceptionInfo;
 import org.apache.phoenix.expression.Expression;
 import org.apache.phoenix.jdbc.PhoenixConnection;
 import org.apache.phoenix.parse.AliasedNode;
@@ -82,6 +80,7 @@ import org.apache.phoenix.schema.PTableImpl;
 import org.apache.phoenix.schema.PTableKey;
 import org.apache.phoenix.schema.PTableType;
 import org.apache.phoenix.schema.ProjectedColumn;
+import org.apache.phoenix.schema.RowKeySchema;
 import org.apache.phoenix.schema.SchemaNotFoundException;
 import org.apache.phoenix.schema.SortOrder;
 import org.apache.phoenix.schema.TableNotFoundException;
@@ -284,7 +283,8 @@ public class FromCompiler {
                 column.getTimestamp());
             projectedColumns.add(projectedColumn);
         }
-        PTable t = PTableImpl.makePTable(table, projectedColumns);
+        PTable t = PTableImpl.builderWithColumns(table, projectedColumns)
+                .build();
         return new SingleTableColumnResolver(connection, new TableRef(tableRef.getTableAlias(), t, tableRef.getLowerBoundTimeStamp(), tableRef.hasDynamicCols()));
     }
     
@@ -369,10 +369,30 @@ public class FromCompiler {
             if (connection.getSchema() != null) {
                 schema = schema != null ? schema : connection.getSchema();
             }
+
             // Storage scheme and encoding scheme don't matter here since the PTable is being used only for the purposes of create table.
             // The actual values of these two will be determined by the metadata client.
-            PTable theTable = new PTableImpl(connection.getTenantId(), schema, table.getName().getTableName(),
-                    scn == null ? HConstants.LATEST_TIMESTAMP : scn, families, isNamespaceMapped);
+            PName tenantId = connection.getTenantId();
+            PTableImpl.checkTenantId(tenantId);
+            String tableName = table.getName().getTableName();
+            PName name = PNameFactory.newName(SchemaUtil.getTableName(schema, tableName));
+            PTable theTable = new PTableImpl.Builder()
+                    .setTenantId(tenantId)
+                    .setName(name)
+                    .setKey(new PTableKey(tenantId, name.getString()))
+                    .setSchemaName(PNameFactory.newName(schema))
+                    .setTableName(PNameFactory.newName(tableName))
+                    .setType(PTableType.VIEW)
+                    .setViewType(PTable.ViewType.MAPPED)
+                    .setTimeStamp(scn == null ? HConstants.LATEST_TIMESTAMP : scn)
+                    .setPkColumns(Collections.<PColumn>emptyList())
+                    .setAllColumns(Collections.<PColumn>emptyList())
+                    .setRowKeySchema(RowKeySchema.EMPTY_SCHEMA)
+                    .setIndexes(Collections.<PTable>emptyList())
+                    .setFamilyAttributes(families)
+                    .setPhysicalNames(Collections.<PName>emptyList())
+                    .setNamespaceMapped(isNamespaceMapped)
+                    .build();
             theTable = this.addDynamicColumns(table.getDynamicColumns(), theTable);
             alias = null;
             tableRefs = ImmutableList.of(new TableRef(alias, theTable, timeStamp, !table.getDynamicColumns().isEmpty()));
@@ -706,7 +726,7 @@ public class FromCompiler {
             if (!dynColumns.isEmpty()) {
                 List<PColumn> allcolumns = new ArrayList<PColumn>();
                 List<PColumn> existingColumns = theTable.getColumns();
-                // Need to skip the salting column, as it's added in the makePTable call below
+                // Need to skip the salting column, as it's handled in the PTable builder call below
                 allcolumns.addAll(theTable.getBucketNum() == null ? existingColumns : existingColumns.subList(1, existingColumns.size()));
                 // Position still based on with the salting columns
                 int position = existingColumns.size();
@@ -724,7 +744,8 @@ public class FromCompiler {
                         HConstants.LATEST_TIMESTAMP));
                     position++;
                 }
-                theTable = PTableImpl.makePTable(theTable, allcolumns);
+                theTable = PTableImpl.builderWithColumns(theTable, allcolumns)
+                        .build();
             }
             return theTable;
         }
@@ -830,11 +851,32 @@ public class FromCompiler {
                     HConstants.LATEST_TIMESTAMP);
                 columns.add(column);
             }
-            PTable t = PTableImpl.makePTable(null, PName.EMPTY_NAME, PName.EMPTY_NAME, PTableType.SUBQUERY, null,
-                    MetaDataProtocol.MIN_TABLE_TIMESTAMP, PTable.INITIAL_SEQ_NUM, null, null, columns, null, null,
-                    Collections.<PTable> emptyList(), false, Collections.<PName> emptyList(), null, null, false, false,
-                    false, null, null, null, null, false, null, 0, 0L, SchemaUtil
-                            .isNamespaceMappingEnabled(PTableType.SUBQUERY, connection.getQueryServices().getProps()), null, false, ImmutableStorageScheme.ONE_CELL_PER_COLUMN, QualifierEncodingScheme.NON_ENCODED_QUALIFIERS, PTable.EncodedCQCounter.NULL_COUNTER, true);
+            PTable t = new PTableImpl.Builder()
+                    .setType(PTableType.SUBQUERY)
+                    .setTimeStamp(MetaDataProtocol.MIN_TABLE_TIMESTAMP)
+                    .setIndexDisableTimestamp(0L)
+                    .setSequenceNumber(PTable.INITIAL_SEQ_NUM)
+                    .setImmutableRows(false)
+                    .setDisableWAL(false)
+                    .setMultiTenant(false)
+                    .setStoreNulls(false)
+                    .setUpdateCacheFrequency(0)
+                    .setNamespaceMapped(SchemaUtil.isNamespaceMappingEnabled(PTableType.SUBQUERY,
+                            connection.getQueryServices().getProps()))
+                    .setAppendOnlySchema(false)
+                    .setImmutableStorageScheme(ImmutableStorageScheme.ONE_CELL_PER_COLUMN)
+                    .setQualifierEncodingScheme(QualifierEncodingScheme.NON_ENCODED_QUALIFIERS)
+                    .setBaseColumnCount(QueryConstants.BASE_TABLE_BASE_COLUMN_COUNT)
+                    .setEncodedCQCounter(PTable.EncodedCQCounter.NULL_COUNTER)
+                    .setUseStatsForParallelization(true)
+                    .setExcludedColumns(ImmutableList.<PColumn>of())
+                    .setSchemaName(PName.EMPTY_NAME)
+                    .setTableName(PName.EMPTY_NAME)
+                    .setRowKeyOrderOptimizable(false)
+                    .setIndexes(Collections.<PTable>emptyList())
+                    .setPhysicalNames(ImmutableList.<PName>of())
+                    .setColumns(columns)
+                    .build();
 
             String alias = subselectNode.getAlias();
             TableRef tableRef = new TableRef(alias, t, MetaDataProtocol.MIN_TABLE_TIMESTAMP, false);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/1767244a/phoenix-core/src/main/java/org/apache/phoenix/compile/JoinCompiler.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/JoinCompiler.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/JoinCompiler.java
index 880fa72..807e54b 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/JoinCompiler.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/JoinCompiler.java
@@ -17,6 +17,7 @@
  */
 package org.apache.phoenix.compile;
 
+import static org.apache.phoenix.query.QueryConstants.BASE_TABLE_BASE_COLUMN_COUNT;
 import static org.apache.phoenix.schema.PTable.ImmutableStorageScheme.ONE_CELL_PER_COLUMN;
 import static org.apache.phoenix.schema.PTable.QualifierEncodingScheme.NON_ENCODED_QUALIFIERS;
 
@@ -32,6 +33,7 @@ import java.util.ListIterator;
 import java.util.Map;
 import java.util.Set;
 
+import com.google.common.collect.ImmutableList;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.phoenix.exception.SQLExceptionCode;
@@ -71,8 +73,8 @@ import org.apache.phoenix.schema.ColumnRef;
 import org.apache.phoenix.schema.LocalIndexDataColumnRef;
 import org.apache.phoenix.schema.MetaDataEntityNotFoundException;
 import org.apache.phoenix.schema.PColumn;
-import org.apache.phoenix.schema.PName;
 import org.apache.phoenix.schema.PNameFactory;
+import org.apache.phoenix.schema.PName;
 import org.apache.phoenix.schema.PTable;
 import org.apache.phoenix.schema.PTable.IndexType;
 import org.apache.phoenix.schema.PTableImpl;
@@ -1270,17 +1272,44 @@ public class JoinCompiler {
         if (left.getBucketNum() != null) {
             merged.remove(0);
         }
-        return PTableImpl.makePTable(left.getTenantId(), left.getSchemaName(),
-                PNameFactory.newName(SchemaUtil.getTableName(left.getName().getString(), right.getName().getString())),
-                left.getType(), left.getIndexState(), left.getTimeStamp(), left.getSequenceNumber(), left.getPKName(),
-                left.getBucketNum(), merged, left.getParentSchemaName(), left.getParentTableName(), left.getIndexes(),
-                left.isImmutableRows(), Collections.<PName> emptyList(), null, null, PTable.DEFAULT_DISABLE_WAL,
-                left.isMultiTenant(), left.getStoreNulls(), left.getViewType(), left.getViewIndexType(), left.getViewIndexId(),
-                left.getIndexType(), left.rowKeyOrderOptimizable(), left.getTransactionProvider(),
-                left.getUpdateCacheFrequency(), left.getIndexDisableTimestamp(), left.isNamespaceMapped(), 
-                left.getAutoPartitionSeqName(), left.isAppendOnlySchema(), ONE_CELL_PER_COLUMN, NON_ENCODED_QUALIFIERS, PTable.EncodedCQCounter.NULL_COUNTER, left.useStatsForParallelization());
+        return new PTableImpl.Builder()
+                .setType(left.getType())
+                .setState(left.getIndexState())
+                .setTimeStamp(left.getTimeStamp())
+                .setIndexDisableTimestamp(left.getIndexDisableTimestamp())
+                .setSequenceNumber(left.getSequenceNumber())
+                .setImmutableRows(left.isImmutableRows())
+                .setDisableWAL(PTable.DEFAULT_DISABLE_WAL)
+                .setMultiTenant(left.isMultiTenant())
+                .setStoreNulls(left.getStoreNulls())
+                .setViewType(left.getViewType())
+                .setViewIndexType(left.getViewIndexType())
+                .setViewIndexId(left.getViewIndexId())
+                .setIndexType(left.getIndexType())
+                .setTransactionProvider(left.getTransactionProvider())
+                .setUpdateCacheFrequency(left.getUpdateCacheFrequency())
+                .setNamespaceMapped(left.isNamespaceMapped())
+                .setAutoPartitionSeqName(left.getAutoPartitionSeqName())
+                .setAppendOnlySchema(left.isAppendOnlySchema())
+                .setImmutableStorageScheme(ONE_CELL_PER_COLUMN)
+                .setQualifierEncodingScheme(NON_ENCODED_QUALIFIERS)
+                .setBaseColumnCount(BASE_TABLE_BASE_COLUMN_COUNT)
+                .setEncodedCQCounter(PTable.EncodedCQCounter.NULL_COUNTER)
+                .setUseStatsForParallelization(left.useStatsForParallelization())
+                .setExcludedColumns(ImmutableList.<PColumn>of())
+                .setTenantId(left.getTenantId())
+                .setSchemaName(left.getSchemaName())
+                .setTableName(PNameFactory.newName(SchemaUtil.getTableName(left.getName().getString(),
+                        right.getName().getString())))
+                .setPkName(left.getPKName())
+                .setRowKeyOrderOptimizable(left.rowKeyOrderOptimizable())
+                .setBucketNum(left.getBucketNum())
+                .setIndexes(left.getIndexes() == null ? Collections.<PTable>emptyList() : left.getIndexes())
+                .setParentSchemaName(left.getParentSchemaName())
+                .setParentTableName(left.getParentTableName())
+                .setPhysicalNames(ImmutableList.<PName>of())
+                .setColumns(merged)
+                .build();
     }
 
 }
-
-

http://git-wip-us.apache.org/repos/asf/phoenix/blob/1767244a/phoenix-core/src/main/java/org/apache/phoenix/compile/TupleProjectionCompiler.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/TupleProjectionCompiler.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/TupleProjectionCompiler.java
index dc85bd3..5b92a5d 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/TupleProjectionCompiler.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/TupleProjectionCompiler.java
@@ -17,6 +17,8 @@
  */
 package org.apache.phoenix.compile;
 import static org.apache.phoenix.query.QueryConstants.VALUE_COLUMN_FAMILY;
+import static org.apache.phoenix.query.QueryConstants.BASE_TABLE_BASE_COLUMN_COUNT;
+import static org.apache.phoenix.schema.PTable.ImmutableStorageScheme;
 
 import java.sql.SQLException;
 import java.util.ArrayList;
@@ -26,6 +28,7 @@ import java.util.LinkedHashSet;
 import java.util.List;
 import java.util.Set;
 
+import com.google.common.collect.ImmutableList;
 import org.apache.phoenix.parse.AliasedNode;
 import org.apache.phoenix.parse.ColumnParseNode;
 import org.apache.phoenix.parse.FamilyWildcardParseNode;
@@ -160,15 +163,13 @@ public class TupleProjectionCompiler {
                     sourceColumnRef.getColumn().isNullable(), sourceColumnRef, sourceColumnRef.getColumn().getColumnQualifierBytes());
             projectedColumns.add(column);
         }
-        
-        return PTableImpl.makePTable(table.getTenantId(), table.getSchemaName(), table.getTableName(),
-                PTableType.PROJECTED, table.getIndexState(), table.getTimeStamp(), table.getSequenceNumber(),
-                table.getPKName(), table.getBucketNum(), projectedColumns, table.getParentSchemaName(),
-                table.getParentTableName(), table.getIndexes(), table.isImmutableRows(), Collections.<PName> emptyList(),
-                table.getDefaultFamilyName(), table.getViewStatement(), table.isWALDisabled(), table.isMultiTenant(), table.getStoreNulls(), table.getViewType(),
-                table.getViewIndexType(), table.getViewIndexId(),
-                table.getIndexType(), table.rowKeyOrderOptimizable(), table.getTransactionProvider(), table.getUpdateCacheFrequency(), 
-                table.getIndexDisableTimestamp(), table.isNamespaceMapped(), table.getAutoPartitionSeqName(), table.isAppendOnlySchema(), table.getImmutableStorageScheme(), table.getEncodingScheme(), table.getEncodedCQCounter(), table.useStatsForParallelization());
+        return PTableImpl.builderWithColumns(table, projectedColumns)
+                .setType(PTableType.PROJECTED)
+                .setBaseColumnCount(BASE_TABLE_BASE_COLUMN_COUNT)
+                .setExcludedColumns(ImmutableList.<PColumn>of())
+                .setPhysicalNames(ImmutableList.<PName>of())
+                .setColumns(projectedColumns)
+                .build();
     }
     
     public static PTable createProjectedTable(TableRef tableRef, List<ColumnRef> sourceColumnRefs, boolean retainPKColumns) throws SQLException {
@@ -192,14 +193,39 @@ public class TupleProjectionCompiler {
         if (EncodedColumnsUtil.usesEncodedColumnNames(table)) {
             cqCounter = EncodedCQCounter.copy(table.getEncodedCQCounter());
         }
-        
-        return PTableImpl.makePTable(table.getTenantId(), PROJECTED_TABLE_SCHEMA, table.getName(), PTableType.PROJECTED,
-                null, table.getTimeStamp(), table.getSequenceNumber(), table.getPKName(),
-                table.getBucketNum(), projectedColumns, null, null,
-                Collections.<PTable> emptyList(), table.isImmutableRows(), Collections.<PName> emptyList(), null, null,
-                table.isWALDisabled(), table.isMultiTenant(), table.getStoreNulls(), table.getViewType(),
-                table.getViewIndexType(), table.getViewIndexId(), null, table.rowKeyOrderOptimizable(), table.getTransactionProvider(),
-                table.getUpdateCacheFrequency(), table.getIndexDisableTimestamp(), table.isNamespaceMapped(), table.getAutoPartitionSeqName(), table.isAppendOnlySchema(), table.getImmutableStorageScheme(), table.getEncodingScheme(), cqCounter, table.useStatsForParallelization());
+        return new PTableImpl.Builder()
+                .setType(PTableType.PROJECTED)
+                .setTimeStamp(table.getTimeStamp())
+                .setIndexDisableTimestamp(table.getIndexDisableTimestamp())
+                .setSequenceNumber(table.getSequenceNumber())
+                .setImmutableRows(table.isImmutableRows())
+                .setDisableWAL(table.isWALDisabled())
+                .setMultiTenant(table.isMultiTenant())
+                .setStoreNulls(table.getStoreNulls())
+                .setViewType(table.getViewType())
+                .setViewIndexType(table.getViewIndexType())
+                .setViewIndexId(table.getViewIndexId())
+                .setTransactionProvider(table.getTransactionProvider())
+                .setUpdateCacheFrequency(table.getUpdateCacheFrequency())
+                .setNamespaceMapped(table.isNamespaceMapped())
+                .setAutoPartitionSeqName(table.getAutoPartitionSeqName())
+                .setAppendOnlySchema(table.isAppendOnlySchema())
+                .setImmutableStorageScheme(table.getImmutableStorageScheme())
+                .setQualifierEncodingScheme(table.getEncodingScheme())
+                .setBaseColumnCount(BASE_TABLE_BASE_COLUMN_COUNT)
+                .setEncodedCQCounter(cqCounter)
+                .setUseStatsForParallelization(table.useStatsForParallelization())
+                .setExcludedColumns(ImmutableList.<PColumn>of())
+                .setTenantId(table.getTenantId())
+                .setSchemaName(PROJECTED_TABLE_SCHEMA)
+                .setTableName(table.getTableName())
+                .setPkName(table.getPKName())
+                .setRowKeyOrderOptimizable(table.rowKeyOrderOptimizable())
+                .setBucketNum(table.getBucketNum())
+                .setIndexes(Collections.<PTable>emptyList())
+                .setPhysicalNames(ImmutableList.<PName>of())
+                .setColumns(projectedColumns)
+                .build();
     }
 
     // For extracting column references from single select statement

http://git-wip-us.apache.org/repos/asf/phoenix/blob/1767244a/phoenix-core/src/main/java/org/apache/phoenix/compile/UnionCompiler.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/UnionCompiler.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/UnionCompiler.java
index c0c63eb..a54fbdd 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/UnionCompiler.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/UnionCompiler.java
@@ -17,10 +17,14 @@
  */
 package org.apache.phoenix.compile;
 
+import static org.apache.phoenix.query.QueryConstants.BASE_TABLE_BASE_COLUMN_COUNT;
+
 import java.sql.SQLException;
 import java.util.ArrayList;
+import java.util.Collections;
 import java.util.List;
 
+import com.google.common.collect.ImmutableList;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.phoenix.exception.SQLExceptionCode;
 import org.apache.phoenix.exception.SQLExceptionInfo;
@@ -96,15 +100,34 @@ public class UnionCompiler {
             projectedColumns.add(projectedColumn);
         }
         Long scn = statement.getConnection().getSCN();
-        PTable tempTable = PTableImpl.makePTable(statement.getConnection().getTenantId(),
-            UNION_SCHEMA_NAME, UNION_TABLE_NAME, PTableType.SUBQUERY, null,
-            HConstants.LATEST_TIMESTAMP, scn == null ? HConstants.LATEST_TIMESTAMP : scn,
-            null, null, projectedColumns, null, null, null, true, null, null, null, true,
-            true, true, null, null, null, null, false, null, 0, 0L,
-            SchemaUtil.isNamespaceMappingEnabled(PTableType.SUBQUERY,
-                statement.getConnection().getQueryServices().getProps()), null, false, ImmutableStorageScheme.ONE_CELL_PER_COLUMN, QualifierEncodingScheme.NON_ENCODED_QUALIFIERS, PTable.EncodedCQCounter.NULL_COUNTER, true);
-        TableRef tableRef = new TableRef(null, tempTable, 0, false);
-        return tableRef;
+        PTable tempTable = new PTableImpl.Builder()
+                .setType(PTableType.SUBQUERY)
+                .setTimeStamp(HConstants.LATEST_TIMESTAMP)
+                .setIndexDisableTimestamp(0L)
+                .setSequenceNumber(scn == null ? HConstants.LATEST_TIMESTAMP : scn)
+                .setImmutableRows(true)
+                .setDisableWAL(true)
+                .setMultiTenant(true)
+                .setStoreNulls(true)
+                .setUpdateCacheFrequency(0)
+                .setNamespaceMapped(SchemaUtil.isNamespaceMappingEnabled(PTableType.SUBQUERY,
+                        statement.getConnection().getQueryServices().getProps()))
+                .setAppendOnlySchema(false)
+                .setImmutableStorageScheme(ImmutableStorageScheme.ONE_CELL_PER_COLUMN)
+                .setQualifierEncodingScheme(QualifierEncodingScheme.NON_ENCODED_QUALIFIERS)
+                .setBaseColumnCount(BASE_TABLE_BASE_COLUMN_COUNT)
+                .setEncodedCQCounter(PTable.EncodedCQCounter.NULL_COUNTER)
+                .setUseStatsForParallelization(true)
+                .setExcludedColumns(ImmutableList.<PColumn>of())
+                .setTenantId(statement.getConnection().getTenantId())
+                .setSchemaName(UNION_SCHEMA_NAME)
+                .setTableName(UNION_TABLE_NAME)
+                .setRowKeyOrderOptimizable(false)
+                .setIndexes(Collections.<PTable>emptyList())
+                .setPhysicalNames(ImmutableList.<PName>of())
+                .setColumns(projectedColumns)
+                .build();
+        return new TableRef(null, tempTable, 0, false);
     }
 
     private static void compareExperssions(int i, Expression expression,

http://git-wip-us.apache.org/repos/asf/phoenix/blob/1767244a/phoenix-core/src/main/java/org/apache/phoenix/compile/UpsertCompiler.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/UpsertCompiler.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/UpsertCompiler.java
index 61be561..4ed0c9a 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/UpsertCompiler.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/UpsertCompiler.java
@@ -32,6 +32,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 
+import com.google.common.collect.ImmutableList;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.HRegionLocation;
 import org.apache.hadoop.hbase.client.Scan;
@@ -701,9 +702,11 @@ public class UpsertCompiler {
                     }
                     // Build table from projectedColumns
                     // Hack to add default column family to be used on server in case no value column is projected.
-                    PTable projectedTable = PTableImpl.makePTable(table, projectedColumns,
-                            PNameFactory.newName(SchemaUtil.getEmptyColumnFamily(table)));  
-                    
+                    PTable projectedTable = PTableImpl.builderWithColumns(table, projectedColumns)
+                            .setExcludedColumns(ImmutableList.<PColumn>of())
+                            .setDefaultFamilyName(PNameFactory.newName(SchemaUtil.getEmptyColumnFamily(table)))
+                            .setColumns(projectedColumns)
+                            .build();
                     
                     SelectStatement select = SelectStatement.create(SelectStatement.COUNT_ONE, upsert.getHint());
                     StatementContext statementContext = queryPlan.getContext();
@@ -856,7 +859,8 @@ public class UpsertCompiler {
                     }
                     updateExpressions.add(updateExpression);
                 }
-                PTable onDupKeyTable = PTableImpl.makePTable(table, updateColumns);
+                PTable onDupKeyTable = PTableImpl.builderWithColumns(table, updateColumns)
+                        .build();
                 onDupKeyBytesToBe = PhoenixIndexBuilder.serializeOnDupKeyUpdate(onDupKeyTable, updateExpressions);
             }
         }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/1767244a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java
index 424b6d6..d899e32 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java
@@ -78,6 +78,7 @@ import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.VIEW_INDEX_ID_DATA
 import static org.apache.phoenix.query.QueryConstants.DIVERGED_VIEW_BASE_COLUMN_COUNT;
 import static org.apache.phoenix.schema.PTableType.INDEX;
 import static org.apache.phoenix.schema.PTableType.TABLE;
+import static org.apache.phoenix.schema.PTableImpl.getColumnsToClone;
 import static org.apache.phoenix.util.SchemaUtil.getVarCharLength;
 import static org.apache.phoenix.util.SchemaUtil.getVarChars;
 
@@ -101,6 +102,7 @@ import java.util.NavigableMap;
 import java.util.Properties;
 import java.util.Set;
 
+import com.google.common.collect.ImmutableList;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellUtil;
@@ -215,6 +217,7 @@ import org.apache.phoenix.schema.PTableImpl;
 import org.apache.phoenix.schema.PTableKey;
 import org.apache.phoenix.schema.PTableType;
 import org.apache.phoenix.schema.ParentTableNotFoundException;
+import org.apache.phoenix.schema.RowKeySchema;
 import org.apache.phoenix.schema.SaltingUtil;
 import org.apache.phoenix.schema.SequenceAllocation;
 import org.apache.phoenix.schema.SequenceAlreadyExistsException;
@@ -689,7 +692,9 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                     }
                     indexes.add(latestIndex);
                 }
-                table = PTableImpl.makePTable(table, table.getTimeStamp(), indexes);
+                table = PTableImpl.builderWithColumns(table, getColumnsToClone(table))
+                        .setIndexes(indexes == null ? Collections.<PTable>emptyList() : indexes)
+                        .build();
             }
         }
         
@@ -939,11 +944,31 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
         int baseTableColumnCount =
                 isDiverged ? QueryConstants.DIVERGED_VIEW_BASE_COLUMN_COUNT
                         : columnsToAdd.size() - myColumns.size() + (isSalted ? 1 : 0);
+
+        // When creating a PTable for views or view indexes, use the baseTable PTable for attributes
+        // inherited from the physical base table.
+        // if a TableProperty is not valid on a view we set it to the base table value
+        // if a TableProperty is valid on a view and is not mutable on a view we set it to the base table value
+        // if a TableProperty is valid on a view and is mutable on a view we use the value set on the view
         // TODO Implement PHOENIX-4763 to set the view properties correctly instead of just
         // setting them same as the base table
-        PTableImpl pTable =
-                PTableImpl.makePTable(table, baseTable, columnsToAdd, maxTableTimestamp,
-                    baseTableColumnCount, excludedColumns);
+        PTableImpl pTable = PTableImpl.builderWithColumns(table, columnsToAdd)
+                .setImmutableRows(baseTable.isImmutableRows())
+                .setDisableWAL(baseTable.isWALDisabled())
+                .setMultiTenant(baseTable.isMultiTenant())
+                .setStoreNulls(baseTable.getStoreNulls())
+                .setTransactionProvider(baseTable.getTransactionProvider())
+                .setAutoPartitionSeqName(baseTable.getAutoPartitionSeqName())
+                .setAppendOnlySchema(baseTable.isAppendOnlySchema())
+                .setImmutableStorageScheme(baseTable.getImmutableStorageScheme() == null ?
+                        ImmutableStorageScheme.ONE_CELL_PER_COLUMN : baseTable.getImmutableStorageScheme())
+                .setQualifierEncodingScheme(baseTable.getEncodingScheme() == null ?
+                        QualifierEncodingScheme.NON_ENCODED_QUALIFIERS : baseTable.getEncodingScheme())
+                .setBaseColumnCount(baseTableColumnCount)
+                .setTimeStamp(maxTableTimestamp)
+                .setExcludedColumns(excludedColumns == null ?
+                        ImmutableList.<PColumn>of() : ImmutableList.copyOf(excludedColumns))
+                .build();
         return WhereConstantParser.addViewInfoToPColumnsIfNeeded(pTable);
     }
 
@@ -1486,11 +1511,48 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
         }
         // Avoid querying the stats table because we're holding the rowLock here. Issuing an RPC to a remote
         // server while holding this lock is a bad idea and likely to cause contention.
-        return PTableImpl.makePTable(tenantId, schemaName, tableName, tableType, indexState, timeStamp, tableSeqNum,
-                pkName, saltBucketNum, columns, parentSchemaName, parentTableName, indexes, isImmutableRows, physicalTables, defaultFamilyName,
-                viewStatement, disableWAL, multiTenant, storeNulls, viewType, viewIndexType, viewIndexId, indexType,
-                rowKeyOrderOptimizable, transactionProvider, updateCacheFrequency, baseColumnCount,
-                indexDisableTimestamp, isNamespaceMapped, autoPartitionSeq, isAppendOnlySchema, storageScheme, encodingScheme, cqCounter, useStatsForParallelization);
+        return new PTableImpl.Builder()
+                .setType(tableType)
+                .setState(indexState)
+                .setTimeStamp(timeStamp)
+                .setIndexDisableTimestamp(indexDisableTimestamp)
+                .setSequenceNumber(tableSeqNum)
+                .setImmutableRows(isImmutableRows)
+                .setViewStatement(viewStatement)
+                .setDisableWAL(disableWAL)
+                .setMultiTenant(multiTenant)
+                .setStoreNulls(storeNulls)
+                .setViewType(viewType)
+                .setViewIndexType(viewIndexType)
+                .setViewIndexId(viewIndexId)
+                .setIndexType(indexType)
+                .setTransactionProvider(transactionProvider)
+                .setUpdateCacheFrequency(updateCacheFrequency)
+                .setNamespaceMapped(isNamespaceMapped)
+                .setAutoPartitionSeqName(autoPartitionSeq)
+                .setAppendOnlySchema(isAppendOnlySchema)
+                .setImmutableStorageScheme(storageScheme == null ?
+                        ImmutableStorageScheme.ONE_CELL_PER_COLUMN : storageScheme)
+                .setQualifierEncodingScheme(encodingScheme == null ?
+                        QualifierEncodingScheme.NON_ENCODED_QUALIFIERS : encodingScheme)
+                .setBaseColumnCount(baseColumnCount)
+                .setEncodedCQCounter(cqCounter)
+                .setUseStatsForParallelization(useStatsForParallelization)
+                .setExcludedColumns(ImmutableList.<PColumn>of())
+                .setTenantId(tenantId)
+                .setSchemaName(schemaName)
+                .setTableName(tableName)
+                .setPkName(pkName)
+                .setDefaultFamilyName(defaultFamilyName)
+                .setRowKeyOrderOptimizable(rowKeyOrderOptimizable)
+                .setBucketNum(saltBucketNum)
+                .setIndexes(indexes == null ? Collections.<PTable>emptyList() : indexes)
+                .setParentSchemaName(parentSchemaName)
+                .setParentTableName(parentTableName)
+                .setPhysicalNames(physicalTables == null ?
+                        ImmutableList.<PName>of() : ImmutableList.copyOf(physicalTables))
+                .setColumns(columns)
+                .build();
     }
     private Long getViewIndexId(Cell[] tableKeyValues, PDataType viewIndexType) {
         Cell viewIndexIdKv = tableKeyValues[VIEW_INDEX_ID_INDEX];
@@ -1740,7 +1802,21 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
     }
 
     private static PTable newDeletedTableMarker(long timestamp) {
-        return new PTableImpl(timestamp);
+        try {
+            return new PTableImpl.Builder()
+                    .setType(PTableType.TABLE)
+                    .setTimeStamp(timestamp)
+                    .setPkColumns(Collections.<PColumn>emptyList())
+                    .setAllColumns(Collections.<PColumn>emptyList())
+                    .setFamilyAttributes(Collections.<PColumnFamily>emptyList())
+                    .setRowKeySchema(RowKeySchema.EMPTY_SCHEMA)
+                    .setIndexes(Collections.<PTable>emptyList())
+                    .setPhysicalNames(Collections.<PName>emptyList())
+                    .build();
+        } catch (SQLException e) {
+            // Should never happen
+            return null;
+        }
     }
 
     private static PFunction newDeletedFunctionMarker(long timestamp) {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/1767244a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/UngroupedAggregateRegionObserver.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/UngroupedAggregateRegionObserver.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/UngroupedAggregateRegionObserver.java
index a667316..73386a2 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/UngroupedAggregateRegionObserver.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/UngroupedAggregateRegionObserver.java
@@ -23,6 +23,7 @@ import static org.apache.phoenix.query.QueryConstants.SINGLE_COLUMN_FAMILY;
 import static org.apache.phoenix.query.QueryConstants.UNGROUPED_AGG_ROW_KEY;
 import static org.apache.phoenix.query.QueryServices.MUTATE_BATCH_SIZE_ATTRIB;
 import static org.apache.phoenix.query.QueryServices.MUTATE_BATCH_SIZE_BYTES_ATTRIB;
+import static org.apache.phoenix.schema.PTableImpl.getColumnsToClone;
 import static org.apache.phoenix.schema.stats.StatisticsCollectionRunTracker.COMPACTION_UPDATE_STATS_ROW_COUNT;
 import static org.apache.phoenix.schema.stats.StatisticsCollectionRunTracker.CONCURRENT_UPDATE_STATS_ROW_COUNT;
 
@@ -411,7 +412,10 @@ public class UngroupedAggregateRegionObserver extends BaseScannerRegionObserver
             logger.debug("Upgrading row key for " + region.getRegionInfo().getTable().getNameAsString());
             projectedTable = deserializeTable(descRowKeyTableBytes);
             try {
-                writeToTable = PTableImpl.makePTable(projectedTable, true);
+                writeToTable = PTableImpl.builderWithColumns(projectedTable,
+                        getColumnsToClone(projectedTable))
+                        .setRowKeyOrderOptimizable(true)
+                        .build();
             } catch (SQLException e) {
                 ServerUtil.throwIOException("Upgrade failed", e); // Impossible
             }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/1767244a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/WhereConstantParser.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/WhereConstantParser.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/WhereConstantParser.java
index 4528f55..9fc030c 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/WhereConstantParser.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/WhereConstantParser.java
@@ -94,7 +94,8 @@ public class WhereConstantParser {
                 result.add(column);
             }
         }
-        return PTableImpl.makePTable(view, result);
+        return PTableImpl.builderWithColumns(view, result)
+                .build();
     }
 
     private static PhoenixConnection getConnectionlessConnection() throws SQLException {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/1767244a/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionlessQueryServicesImpl.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionlessQueryServicesImpl.java b/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionlessQueryServicesImpl.java
index acd78ad..85fc821 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionlessQueryServicesImpl.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionlessQueryServicesImpl.java
@@ -18,6 +18,7 @@
 package org.apache.phoenix.query;
 
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.INDEX_STATE_BYTES;
+import static org.apache.phoenix.schema.PTableImpl.getColumnsToClone;
 
 import java.sql.SQLException;
 import java.util.ArrayList;
@@ -316,7 +317,8 @@ public class ConnectionlessQueryServicesImpl extends DelegateQueryServices imple
     public MetaDataMutationResult addColumn(List<Mutation> tableMetaData, PTable table, Map<String, List<Pair<String,Object>>> properties, Set<String> colFamiliesForPColumnsToBeAdded, List<PColumn> columnsToBeAdded) throws SQLException {
         List<PColumn> columns = Lists.newArrayList(table.getColumns());
         columns.addAll(columnsToBeAdded);
-        return new MetaDataMutationResult(MutationCode.TABLE_ALREADY_EXISTS, 0, PTableImpl.makePTable(table, columns));
+        return new MetaDataMutationResult(MutationCode.TABLE_ALREADY_EXISTS, 0,
+                PTableImpl.builderWithColumns(table, columns).build());
     }
 
     @Override
@@ -453,7 +455,10 @@ public class ConnectionlessQueryServicesImpl extends DelegateQueryServices imple
         String indexTableName = SchemaUtil.getTableName(schemaName, indexName);
         PName tenantId = tenantIdBytes.length == 0 ? null : PNameFactory.newName(tenantIdBytes);
         PTable index = metaData.getTableRef(new PTableKey(tenantId, indexTableName)).getTable();
-        index = PTableImpl.makePTable(index,newState == PIndexState.USABLE ? PIndexState.ACTIVE : newState == PIndexState.UNUSABLE ? PIndexState.INACTIVE : newState);
+        index = PTableImpl.builderWithColumns(index, getColumnsToClone(index))
+                .setState(newState == PIndexState.USABLE ? PIndexState.ACTIVE :
+                        newState == PIndexState.UNUSABLE ? PIndexState.INACTIVE : newState)
+                .build();
         return new MetaDataMutationResult(MutationCode.TABLE_ALREADY_EXISTS, 0, index);
     }
 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/1767244a/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java
index df0e5d4..aae9fc4 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java
@@ -104,6 +104,7 @@ import static org.apache.phoenix.schema.PTable.ImmutableStorageScheme.ONE_CELL_P
 import static org.apache.phoenix.schema.PTable.ImmutableStorageScheme.SINGLE_CELL_ARRAY_WITH_OFFSETS;
 import static org.apache.phoenix.schema.PTable.QualifierEncodingScheme.NON_ENCODED_QUALIFIERS;
 import static org.apache.phoenix.schema.PTable.ViewType.MAPPED;
+import static org.apache.phoenix.schema.PTableImpl.getColumnsToClone;
 import static org.apache.phoenix.schema.PTableType.TABLE;
 import static org.apache.phoenix.schema.PTableType.VIEW;
 import static org.apache.phoenix.schema.types.PDataType.FALSE_BYTES;
@@ -132,6 +133,7 @@ import java.util.Properties;
 import java.util.Set;
 
 import org.apache.hadoop.hbase.HColumnDescriptor;
+import com.google.common.base.Objects;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.client.ClusterConnection;
@@ -247,6 +249,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import com.google.common.base.Strings;
+import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ListMultimap;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
@@ -931,10 +934,21 @@ public class MetaDataClient {
                     + QueryConstants.CHILD_VIEW_INDEX_NAME_SEPARATOR + index.getName().getString());
                 // add the index table with a new name so that it does not conflict with the existing index table
                 // and set update cache frequency to that of the view
-                indexesToAdd.add(PTableImpl.makePTable(index, modifiedIndexName, viewStatement, view.getUpdateCacheFrequency(), view.getTenantId()));
+                if (Objects.equal(viewStatement, index.getViewStatement())) {
+                    indexesToAdd.add(index);
+                } else {
+                    indexesToAdd.add(PTableImpl.builderWithColumns(index, getColumnsToClone(index))
+                            .setTableName(modifiedIndexName)
+                            .setViewStatement(viewStatement)
+                            .setUpdateCacheFrequency(view.getUpdateCacheFrequency())
+                            .setTenantId(view.getTenantId())
+                            .build());
+                }
             }
         }
-        PTable allIndexesTable = PTableImpl.makePTable(view, view.getTimeStamp(), indexesToAdd);
+        PTable allIndexesTable = PTableImpl.builderWithColumns(view, getColumnsToClone(view))
+                .setIndexes(indexesToAdd == null ? Collections.<PTable>emptyList() : indexesToAdd)
+                .build();
         result.setTable(allIndexesTable);
         return true;
     }
@@ -2650,13 +2664,38 @@ public class MetaDataClient {
                 // TODO: what about stats for system catalog?
                 PName newSchemaName = PNameFactory.newName(schemaName);
                 // Column names and qualifiers and hardcoded for system tables.
-                PTable table = PTableImpl.makePTable(tenantId,newSchemaName, PNameFactory.newName(tableName), tableType,
-                        null, MetaDataProtocol.MIN_TABLE_TIMESTAMP, PTable.INITIAL_SEQ_NUM,
-                        PNameFactory.newName(QueryConstants.SYSTEM_TABLE_PK_NAME), null, columns.values(), null, null,
-                        Collections.<PTable>emptyList(), isImmutableRows,
-                        Collections.<PName>emptyList(), defaultFamilyName == null ? null :
-                                PNameFactory.newName(defaultFamilyName), null,
-                        Boolean.TRUE.equals(disableWAL), false, false, null, viewIndexType, null, indexType, true, null, 0, 0L, isNamespaceMapped, autoPartitionSeq, isAppendOnlySchema, ONE_CELL_PER_COLUMN, NON_ENCODED_QUALIFIERS, PTable.EncodedCQCounter.NULL_COUNTER, true);
+                PTable table = new PTableImpl.Builder()
+                        .setType(tableType)
+                        .setTimeStamp(MetaDataProtocol.MIN_TABLE_TIMESTAMP)
+                        .setIndexDisableTimestamp(0L)
+                        .setSequenceNumber(PTable.INITIAL_SEQ_NUM)
+                        .setImmutableRows(isImmutableRows)
+                        .setDisableWAL(Boolean.TRUE.equals(disableWAL))
+                        .setMultiTenant(false)
+                        .setStoreNulls(false)
+                        .setViewIndexType(viewIndexType)
+                        .setIndexType(indexType)
+                        .setUpdateCacheFrequency(0)
+                        .setNamespaceMapped(isNamespaceMapped)
+                        .setAutoPartitionSeqName(autoPartitionSeq)
+                        .setAppendOnlySchema(isAppendOnlySchema)
+                        .setImmutableStorageScheme(ONE_CELL_PER_COLUMN)
+                        .setQualifierEncodingScheme(NON_ENCODED_QUALIFIERS)
+                        .setBaseColumnCount(QueryConstants.BASE_TABLE_BASE_COLUMN_COUNT)
+                        .setEncodedCQCounter(PTable.EncodedCQCounter.NULL_COUNTER)
+                        .setUseStatsForParallelization(true)
+                        .setExcludedColumns(ImmutableList.<PColumn>of())
+                        .setTenantId(tenantId)
+                        .setSchemaName(newSchemaName)
+                        .setTableName(PNameFactory.newName(tableName))
+                        .setPkName(PNameFactory.newName(QueryConstants.SYSTEM_TABLE_PK_NAME))
+                        .setDefaultFamilyName(defaultFamilyName == null ? null :
+                                PNameFactory.newName(defaultFamilyName))
+                        .setRowKeyOrderOptimizable(true)
+                        .setIndexes(Collections.<PTable>emptyList())
+                        .setPhysicalNames(ImmutableList.<PName>of())
+                        .setColumns(columns.values())
+                        .build();
                 connection.addTable(table, MetaDataProtocol.MIN_TABLE_TIMESTAMP);
             }
             
@@ -2930,12 +2969,49 @@ public class MetaDataClient {
                  * for extra safety.
                  */
                 EncodedCQCounter cqCounterToBe = tableType == PTableType.VIEW ? NULL_COUNTER : cqCounter;
-                PTable table =  PTableImpl.makePTable(
-                        tenantId, newSchemaName, PNameFactory.newName(tableName), tableType, indexState, timestamp!=null ? timestamp : result.getMutationTime(),
-                        PTable.INITIAL_SEQ_NUM, pkName == null ? null : PNameFactory.newName(pkName), saltBucketNum, columns.values(),
-                        parent == null ? null : parent.getSchemaName(), parent == null ? null : parent.getTableName(), Collections.<PTable>emptyList(), isImmutableRows,
-                        physicalNames, defaultFamilyName == null ? null : PNameFactory.newName(defaultFamilyName), viewStatement, Boolean.TRUE.equals(disableWAL), multiTenant, storeNulls, viewType,
-                        viewIndexType, result.getViewIndexId(), indexType, rowKeyOrderOptimizable, transactionProvider, updateCacheFrequency, 0L, isNamespaceMapped, autoPartitionSeq, isAppendOnlySchema, immutableStorageScheme, encodingScheme, cqCounterToBe, useStatsForParallelizationProp);
+                PTable table = new PTableImpl.Builder()
+                        .setType(tableType)
+                        .setState(indexState)
+                        .setTimeStamp(timestamp != null ? timestamp : result.getMutationTime())
+                        .setIndexDisableTimestamp(0L)
+                        .setSequenceNumber(PTable.INITIAL_SEQ_NUM)
+                        .setImmutableRows(isImmutableRows)
+                        .setViewStatement(viewStatement)
+                        .setDisableWAL(Boolean.TRUE.equals(disableWAL))
+                        .setMultiTenant(multiTenant)
+                        .setStoreNulls(storeNulls)
+                        .setViewType(viewType)
+                        .setViewIndexType(viewIndexType)
+                        .setViewIndexId(result.getViewIndexId())
+                        .setIndexType(indexType)
+                        .setTransactionProvider(transactionProvider)
+                        .setUpdateCacheFrequency(updateCacheFrequency)
+                        .setNamespaceMapped(isNamespaceMapped)
+                        .setAutoPartitionSeqName(autoPartitionSeq)
+                        .setAppendOnlySchema(isAppendOnlySchema)
+                        .setImmutableStorageScheme(immutableStorageScheme == null ?
+                                ImmutableStorageScheme.ONE_CELL_PER_COLUMN : immutableStorageScheme)
+                        .setQualifierEncodingScheme(encodingScheme == null ?
+                                QualifierEncodingScheme.NON_ENCODED_QUALIFIERS : encodingScheme)
+                        .setBaseColumnCount(QueryConstants.BASE_TABLE_BASE_COLUMN_COUNT)
+                        .setEncodedCQCounter(cqCounterToBe)
+                        .setUseStatsForParallelization(useStatsForParallelizationProp)
+                        .setExcludedColumns(ImmutableList.<PColumn>of())
+                        .setTenantId(tenantId)
+                        .setSchemaName(newSchemaName)
+                        .setTableName(PNameFactory.newName(tableName))
+                        .setPkName(pkName == null ? null : PNameFactory.newName(pkName))
+                        .setDefaultFamilyName(defaultFamilyName == null ?
+                                null : PNameFactory.newName(defaultFamilyName))
+                        .setRowKeyOrderOptimizable(rowKeyOrderOptimizable)
+                        .setBucketNum(saltBucketNum)
+                        .setIndexes(Collections.<PTable>emptyList())
+                        .setParentSchemaName((parent == null) ? null : parent.getSchemaName())
+                        .setParentTableName((parent == null) ? null : parent.getTableName())
+                        .setPhysicalNames(physicalNames == null ?
+                                ImmutableList.<PName>of() : ImmutableList.copyOf(physicalNames))
+                        .setColumns(columns.values())
+                        .build();
                 result = new MetaDataMutationResult(code, result.getMutationTime(), table, true);
                 addTableToCache(result);
                 return table;
@@ -3131,10 +3207,29 @@ public class MetaDataClient {
                                 && (table.isMultiTenant() || hasViewIndexTable)) {
                             if (hasViewIndexTable) {
                                 byte[] viewIndexPhysicalName = MetaDataUtil.getViewIndexPhysicalName(table.getPhysicalName().getBytes());
-                                PTable viewIndexTable = new PTableImpl(null,
-                                        SchemaUtil.getSchemaNameFromFullName(viewIndexPhysicalName),
-                                        SchemaUtil.getTableNameFromFullName(viewIndexPhysicalName), ts,
-                                        table.getColumnFamilies(),table.isNamespaceMapped(), table.getImmutableStorageScheme(), table.getEncodingScheme(), table.useStatsForParallelization());
+                                String viewIndexSchemaName = SchemaUtil.getSchemaNameFromFullName(viewIndexPhysicalName);
+                                String viewIndexTableName = SchemaUtil.getTableNameFromFullName(viewIndexPhysicalName);
+                                PName viewIndexName = PNameFactory.newName(SchemaUtil.getTableName(viewIndexSchemaName, viewIndexTableName));
+
+                                PTable viewIndexTable = new PTableImpl.Builder()
+                                        .setName(viewIndexName)
+                                        .setKey(new PTableKey(tenantId, viewIndexName.getString()))
+                                        .setSchemaName(PNameFactory.newName(viewIndexSchemaName))
+                                        .setTableName(PNameFactory.newName(viewIndexTableName))
+                                        .setType(PTableType.VIEW)
+                                        .setViewType(ViewType.MAPPED)
+                                        .setTimeStamp(ts)
+                                        .setPkColumns(Collections.<PColumn>emptyList())
+                                        .setAllColumns(Collections.<PColumn>emptyList())
+                                        .setRowKeySchema(RowKeySchema.EMPTY_SCHEMA)
+                                        .setIndexes(Collections.<PTable>emptyList())
+                                        .setFamilyAttributes(table.getColumnFamilies())
+                                        .setPhysicalNames(Collections.<PName>emptyList())
+                                        .setNamespaceMapped(table.isNamespaceMapped())
+                                        .setImmutableStorageScheme(table.getImmutableStorageScheme())
+                                        .setQualifierEncodingScheme(table.getEncodingScheme())
+                                        .setUseStatsForParallelization(table.useStatsForParallelization())
+                                        .build();
                                 tableRefs.add(new TableRef(null, viewIndexTable, ts, false));
                             }
                         }
@@ -3754,13 +3849,32 @@ public class MetaDataClient {
                             long ts = (scn == null ? result.getMutationTime() : scn);
                             byte[] viewIndexPhysicalName = MetaDataUtil
                                     .getViewIndexPhysicalName(table.getPhysicalName().getBytes());
-                            PTable viewIndexTable = new PTableImpl(null,
-                                    SchemaUtil.getSchemaNameFromFullName(viewIndexPhysicalName),
-                                    SchemaUtil.getTableNameFromFullName(viewIndexPhysicalName), ts,
-                                    table.getColumnFamilies(), table.isNamespaceMapped(), table.getImmutableStorageScheme(), table.getEncodingScheme(), table.useStatsForParallelization());
+                            String viewIndexSchemaName = SchemaUtil.getSchemaNameFromFullName(viewIndexPhysicalName);
+                            String viewIndexTableName = SchemaUtil.getTableNameFromFullName(viewIndexPhysicalName);
+                            PName viewIndexName = PNameFactory.newName(SchemaUtil.getTableName(viewIndexSchemaName, viewIndexTableName));
+
+                            PTable viewIndexTable = new PTableImpl.Builder()
+                                    .setName(viewIndexName)
+                                    .setKey(new PTableKey(tenantId, viewIndexName.getString()))
+                                    .setSchemaName(PNameFactory.newName(viewIndexSchemaName))
+                                    .setTableName(PNameFactory.newName(viewIndexTableName))
+                                    .setType(PTableType.VIEW)
+                                    .setViewType(ViewType.MAPPED)
+                                    .setTimeStamp(ts)
+                                    .setPkColumns(Collections.<PColumn>emptyList())
+                                    .setAllColumns(Collections.<PColumn>emptyList())
+                                    .setRowKeySchema(RowKeySchema.EMPTY_SCHEMA)
+                                    .setIndexes(Collections.<PTable>emptyList())
+                                    .setFamilyAttributes(table.getColumnFamilies())
+                                    .setPhysicalNames(Collections.<PName>emptyList())
+                                    .setNamespaceMapped(table.isNamespaceMapped())
+                                    .setImmutableStorageScheme(table.getImmutableStorageScheme())
+                                    .setQualifierEncodingScheme(table.getEncodingScheme())
+                                    .setUseStatsForParallelization(table.useStatsForParallelization())
+                                    .build();
                             List<TableRef> tableRefs = Collections.singletonList(new TableRef(null, viewIndexTable, ts, false));
                             MutationPlan plan = new PostDDLCompiler(connection).compile(tableRefs, null, null,
-                                    Collections.<PColumn> emptyList(), ts);
+                                    Collections.<PColumn>emptyList(), ts);
                             connection.getQueryServices().updateData(plan);
                         }
                     }
@@ -4029,21 +4143,44 @@ public class MetaDataClient {
                         Map<String, List<TableRef>> tenantIdTableRefMap = Maps.newHashMap();
                         if (result.getSharedTablesToDelete() != null) {
                             for (SharedTableState sharedTableState : result.getSharedTablesToDelete()) {
-                                PTableImpl viewIndexTable =
-                                        new PTableImpl(sharedTableState.getTenantId(),
-                                                sharedTableState.getSchemaName(),
-                                                sharedTableState.getTableName(), ts,
-                                                table.getColumnFamilies(),
-                                                sharedTableState.getColumns(),
-                                                sharedTableState.getPhysicalNames(),
-                                                sharedTableState.getViewIndexType(),
-                                                sharedTableState.getViewIndexId(),
-                                                table.isMultiTenant(), table.isNamespaceMapped(),
-                                                table.getImmutableStorageScheme(),
-                                                table.getEncodingScheme(),
-                                                table.getEncodedCQCounter(),
-                                                table.useStatsForParallelization(),
-                                                table.getBucketNum());
+                                ImmutableStorageScheme storageScheme = table.getImmutableStorageScheme();
+                                QualifierEncodingScheme qualifierEncodingScheme = table.getEncodingScheme();
+                                List<PColumn> columns = sharedTableState.getColumns();
+                                if (table.getBucketNum() != null) {
+                                    columns = columns.subList(1, columns.size());
+                                }
+
+                                PTableImpl viewIndexTable = new PTableImpl.Builder()
+                                        .setPkColumns(Collections.<PColumn>emptyList())
+                                        .setAllColumns(Collections.<PColumn>emptyList())
+                                        .setRowKeySchema(RowKeySchema.EMPTY_SCHEMA)
+                                        .setIndexes(Collections.<PTable>emptyList())
+                                        .setFamilyAttributes(table.getColumnFamilies())
+                                        .setType(PTableType.INDEX)
+                                        .setTimeStamp(ts)
+                                        .setMultiTenant(table.isMultiTenant())
+                                        .setViewIndexType(sharedTableState.getViewIndexType())
+                                        .setViewIndexId(sharedTableState.getViewIndexId())
+                                        .setNamespaceMapped(table.isNamespaceMapped())
+                                        .setAppendOnlySchema(false)
+                                        .setImmutableStorageScheme(storageScheme == null ?
+                                                ImmutableStorageScheme.ONE_CELL_PER_COLUMN : storageScheme)
+                                        .setQualifierEncodingScheme(qualifierEncodingScheme == null ?
+                                                QualifierEncodingScheme.NON_ENCODED_QUALIFIERS : qualifierEncodingScheme)
+                                        .setEncodedCQCounter(table.getEncodedCQCounter())
+                                        .setUseStatsForParallelization(table.useStatsForParallelization())
+                                        .setExcludedColumns(ImmutableList.<PColumn>of())
+                                        .setTenantId(sharedTableState.getTenantId())
+                                        .setSchemaName(sharedTableState.getSchemaName())
+                                        .setTableName(sharedTableState.getTableName())
+                                        .setRowKeyOrderOptimizable(false)
+                                        .setBucketNum(table.getBucketNum())
+                                        .setIndexes(Collections.<PTable>emptyList())
+                                        .setPhysicalNames(sharedTableState.getPhysicalNames() == null ?
+                                                ImmutableList.<PName>of() :
+                                                ImmutableList.copyOf(sharedTableState.getPhysicalNames()))
+                                        .setColumns(columns)
+                                        .build();
                                 TableRef indexTableRef = new TableRef(viewIndexTable);
                                 PName indexTableTenantId = sharedTableState.getTenantId();
                                 if (indexTableTenantId==null) {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/1767244a/phoenix-core/src/main/java/org/apache/phoenix/schema/PMetaDataImpl.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/PMetaDataImpl.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/PMetaDataImpl.java
index 598cc79..3ce70ea 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/PMetaDataImpl.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/PMetaDataImpl.java
@@ -17,7 +17,10 @@
  */
 package org.apache.phoenix.schema;
 
+import static org.apache.phoenix.schema.PTableImpl.getColumnsToClone;
+
 import java.sql.SQLException;
+import java.util.Collections;
 import java.util.Iterator;
 import java.util.List;
 
@@ -27,7 +30,6 @@ import org.apache.phoenix.parse.PSchema;
 import org.apache.phoenix.query.QueryServices;
 import org.apache.phoenix.query.QueryServicesOptions;
 import org.apache.phoenix.util.ReadOnlyProps;
-import org.apache.phoenix.util.SchemaUtil;
 import org.apache.phoenix.util.TimeKeeper;
 
 import com.google.common.collect.Lists;
@@ -121,7 +123,11 @@ public class PMetaDataImpl implements PMetaData {
                 }
                 newIndexes.add(table);
                 netGain -= oldParentRef.getEstimatedSize();
-                newParentTable = PTableImpl.makePTable(oldParentRef.getTable(), table.getTimeStamp(), newIndexes);
+                newParentTable = PTableImpl.builderWithColumns(oldParentRef.getTable(),
+                        getColumnsToClone(oldParentRef.getTable()))
+                        .setIndexes(newIndexes)
+                        .setTimeStamp(table.getTimeStamp())
+                        .build();
                 newParentTableRef = tableRefFactory.makePTableRef(newParentTable, this.timeKeeper.getCurrentTime(), parentResolvedTimestamp);
                 netGain += newParentTableRef.getEstimatedSize();
             }
@@ -173,10 +179,14 @@ public class PMetaDataImpl implements PMetaData {
                     PTable index = newIndexes.get(i);
                     if (index.getName().getString().equals(tableName)) {
                         newIndexes.remove(i);
-                        PTable parentTable = PTableImpl.makePTable(
-                                parentTableRef.getTable(),
-                                tableTimeStamp == HConstants.LATEST_TIMESTAMP ? parentTableRef.getTable().getTimeStamp() : tableTimeStamp,
-                                newIndexes);
+                        PTableImpl.Builder parentTableBuilder =
+                                PTableImpl.builderWithColumns(parentTableRef.getTable(),
+                                        getColumnsToClone(parentTableRef.getTable()))
+                                .setIndexes(newIndexes == null ? Collections.<PTable>emptyList() : newIndexes);
+                        if (tableTimeStamp != HConstants.LATEST_TIMESTAMP) {
+                            parentTableBuilder.setTimeStamp(tableTimeStamp);
+                        }
+                        PTable parentTable = parentTableBuilder.build();
                         metaData.put(parentTable.getKey(), tableRefFactory.makePTableRef(parentTable, this.timeKeeper.getCurrentTime(), parentTableRef.getResolvedTimeStamp()));
                         break;
                     }
@@ -218,8 +228,10 @@ public class PMetaDataImpl implements PMetaData {
                     oldColumn.getTimestamp());
                 columns.add(newColumn);
             }
-            
-            table = PTableImpl.makePTable(table, tableTimeStamp, tableSeqNum, columns);
+            table = PTableImpl.builderWithColumns(table, columns)
+                    .setTimeStamp(tableTimeStamp)
+                    .setSequenceNumber(tableSeqNum)
+                    .build();
         }
         tables.put(table.getKey(), tableRefFactory.makePTableRef(table, this.timeKeeper.getCurrentTime(), resolvedTime));
     }


[17/28] phoenix git commit: PHOENIX-4841 staging patch commit.

Posted by pb...@apache.org.
PHOENIX-4841 staging patch commit.


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

Branch: refs/heads/4.x-cdh5.15
Commit: 1c656192f6d0ea061630c7d1ef8ab3f0970e7071
Parents: bcf2cc7
Author: Daniel Wong <da...@salesforce.com>
Authored: Wed Oct 10 00:38:11 2018 +0100
Committer: Pedro Boado <pb...@apache.org>
Committed: Tue Nov 27 15:11:54 2018 +0000

----------------------------------------------------------------------
 .../org/apache/phoenix/end2end/QueryMoreIT.java | 171 +++++++++++++++++--
 .../apache/phoenix/compile/WhereOptimizer.java  |  58 ++++++-
 .../expression/ComparisonExpression.java        |  18 +-
 .../RowValueConstructorExpressionRewriter.java  |  54 ++++++
 .../org/apache/phoenix/schema/RowKeySchema.java |   4 +
 ...wValueConstructorExpressionRewriterTest.java |  78 +++++++++
 6 files changed, 362 insertions(+), 21 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/1c656192/phoenix-core/src/it/java/org/apache/phoenix/end2end/QueryMoreIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/QueryMoreIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/QueryMoreIT.java
index 04272fa..2b1d31e 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/QueryMoreIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/QueryMoreIT.java
@@ -17,11 +17,13 @@
  */
 package org.apache.phoenix.end2end;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertTrue;
+import com.google.common.collect.Lists;
+import org.apache.hadoop.hbase.util.Pair;
+import org.apache.phoenix.jdbc.PhoenixConnection;
+import org.apache.phoenix.query.QueryServices;
+import org.apache.phoenix.util.PhoenixRuntime;
+import org.apache.phoenix.util.TestUtil;
+import org.junit.Test;
 
 import java.sql.Connection;
 import java.sql.Date;
@@ -37,18 +39,19 @@ import java.util.Map;
 import java.util.Properties;
 
 import org.apache.hadoop.hbase.util.Base64;
-import org.apache.hadoop.hbase.util.Pair;
-import org.apache.phoenix.jdbc.PhoenixConnection;
-import org.apache.phoenix.query.QueryServices;
-import org.apache.phoenix.util.PhoenixRuntime;
-import org.apache.phoenix.util.TestUtil;
-import org.junit.Test;
 
-import com.google.common.collect.Lists;
+import static org.apache.phoenix.util.PhoenixRuntime.TENANT_ID_ATTRIB;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
 
 
 public class QueryMoreIT extends ParallelStatsDisabledIT {
     
+    private final String TENANT_SPECIFIC_URL1 = getUrl() + ';' + TENANT_ID_ATTRIB + "=tenant1";
+    
     private String dataTableName;
     //queryAgainstTenantSpecificView = true, dataTableSalted = true 
     @Test
@@ -510,4 +513,148 @@ public class QueryMoreIT extends ParallelStatsDisabledIT {
             stmt.execute();
         }
     }
+
+    @Test public void testRVCWithDescAndAscendingPK() throws Exception {
+        final Connection conn = DriverManager.getConnection(getUrl());
+        String fullTableName = generateUniqueName();
+        try (Statement stmt = conn.createStatement()) {
+            stmt.execute("CREATE TABLE " + fullTableName + "(\n"
+                    + "    ORGANIZATION_ID CHAR(15) NOT NULL,\n" + "    SCORE VARCHAR NOT NULL,\n"
+                    + "    ENTITY_ID VARCHAR NOT NULL\n"
+                    + "    CONSTRAINT PAGE_SNAPSHOT_PK PRIMARY KEY (\n"
+                    + "        ORGANIZATION_ID,\n" + "        SCORE DESC,\n" + "        ENTITY_ID\n"
+                    + "    )\n" + ") MULTI_TENANT=TRUE");
+        }
+
+        conn.createStatement().execute("UPSERT INTO " + fullTableName + " VALUES ('org1','c','1')");
+        conn.createStatement().execute("UPSERT INTO " + fullTableName + " VALUES ('org1','b','3')");
+        conn.createStatement().execute("UPSERT INTO " + fullTableName + " VALUES ('org1','b','4')");
+        conn.createStatement().execute("UPSERT INTO " + fullTableName + " VALUES ('org1','a','2')");
+        conn.commit();
+
+        try (Statement stmt = conn.createStatement()) {
+            final ResultSet
+                    rs =
+                    stmt.executeQuery("SELECT score, entity_id \n" + "FROM " + fullTableName + "\n"
+                            + "WHERE organization_id = 'org1'\n"
+                            + "AND (score, entity_id) < ('b', '4')\n"
+                            + "ORDER BY score DESC, entity_id\n" + "LIMIT 3");
+            assertTrue(rs.next());
+            assertEquals("b", rs.getString(1));
+            assertEquals("3", rs.getString(2));
+            assertTrue(rs.next());
+            assertEquals("a", rs.getString(1));
+            assertEquals("2", rs.getString(2));
+            assertFalse(rs.next());
+        }
+    }
+
+    @Test
+    public void testRVCOnTenantViewThroughGlobalIdxOrderByDesc() throws Exception {
+        String fullTableName = generateUniqueName();
+        String fullViewName = generateUniqueName();
+        String tenantView = generateUniqueName();
+        String indexName = generateUniqueName();
+        // create base table and global view using global connection
+        try (Connection conn = DriverManager.getConnection(getUrl())) {
+            Statement stmt = conn.createStatement();
+            stmt.execute(
+                    "CREATE TABLE " + fullTableName + "(\n"
+                            + "    TENANT_ID CHAR(15) NOT NULL,\n"
+                            + "    KEY_PREFIX CHAR(3) NOT NULL,\n"
+                            + "    CREATED_DATE DATE,\n"
+                            + "    CREATED_BY CHAR(15),\n"
+                            + "    SYSTEM_MODSTAMP DATE\n"
+                            + "    CONSTRAINT PK PRIMARY KEY (\n"
+                            + "       TENANT_ID,"
+                            + "       KEY_PREFIX\n"
+                            + ")) MULTI_TENANT=TRUE");
+
+            stmt.execute("CREATE VIEW " + fullViewName + "(\n"
+                    + "    DATE_TIME1 DATE NOT NULL,\n"
+                    + "    TEXT2 VARCHAR,\n"
+                    + "    DOUBLE1 DECIMAL(12, 3),\n"
+                    + "    IS_BOOLEAN BOOLEAN,\n"
+                    + "    RELATIONSHIP_ID CHAR(15),\n"
+                    + "    TEXT1 VARCHAR,\n"
+                    + "    TEXT_READ_ONLY VARCHAR,\n"
+                    + "    JSON1 VARCHAR,\n"
+                    + "    IP_START_ADDRESS VARCHAR,\n"
+                    + "    CONSTRAINT PKVIEW PRIMARY KEY\n"
+                    + "    (\n"
+                    + "        DATE_TIME1, TEXT2, TEXT1\n"
+                    + "    )) AS SELECT * FROM " + fullTableName
+                    + "    WHERE KEY_PREFIX = '0CY'");
+
+            stmt.execute("CREATE INDEX " + indexName + " " + "ON " + fullViewName
+                    + " (TEXT1 DESC, TEXT2)\n"
+                    + "INCLUDE (CREATED_BY,\n"
+                    + "    RELATIONSHIP_ID,\n"
+                    + "    JSON1,\n"
+                    + "    DOUBLE1,\n"
+                    + "    IS_BOOLEAN,\n"
+                    + "    IP_START_ADDRESS,\n"
+                    + "    CREATED_DATE,\n"
+                    + "    SYSTEM_MODSTAMP,\n"
+                    + "    TEXT_READ_ONLY)");
+        }
+
+        // create and use an tenant specific view to write data
+        try (Connection viewConn = DriverManager.getConnection(TENANT_SPECIFIC_URL1)) {
+            Statement stmt = viewConn.createStatement();
+            stmt.execute("CREATE VIEW IF NOT EXISTS " + tenantView + " AS SELECT * FROM "
+                    + fullViewName);
+            viewConn.createStatement().execute("UPSERT INTO " + tenantView
+                    + "(DATE_TIME1, TEXT1, TEXT2) "
+                    + " VALUES (TO_DATE('2017-10-16 22:00:00', 'yyyy-MM-dd HH:mm:ss'), 'd', '1')");
+            viewConn.createStatement().execute("UPSERT INTO " + tenantView
+                    + "(DATE_TIME1, TEXT1, TEXT2) "
+                    + " VALUES (TO_DATE('2017-10-16 22:00:00', 'yyyy-MM-dd HH:mm:ss'), 'c', '2')");
+            viewConn.createStatement().execute("UPSERT INTO " + tenantView
+                    + "(DATE_TIME1, TEXT1, TEXT2) "
+                    + " VALUES (TO_DATE('2017-10-16 22:00:00', 'yyyy-MM-dd HH:mm:ss'), 'b', '3')");
+            viewConn.createStatement().execute("UPSERT INTO " + tenantView
+                    + "(DATE_TIME1, TEXT1, TEXT2) "
+                    + " VALUES (TO_DATE('2017-10-16 22:00:00', 'yyyy-MM-dd HH:mm:ss'), 'b', '4')");
+            viewConn.createStatement().execute("UPSERT INTO " + tenantView
+                    + "(DATE_TIME1, TEXT1, TEXT2) "
+                    + " VALUES (TO_DATE('2017-10-16 22:00:00', 'yyyy-MM-dd HH:mm:ss'), 'a', '4')");
+            viewConn.commit();
+
+            // query using desc order by so that the index is used
+            ResultSet
+                    rs =
+                    stmt.executeQuery("SELECT TEXT1, TEXT2 FROM " + tenantView
+                            + " WHERE (TEXT1, TEXT2) > ('b', '3') ORDER BY TEXT1 DESC, TEXT2");
+            assertTrue(rs.next());
+            assertEquals("d", rs.getString(1));
+            assertEquals("1", rs.getString(2));
+            assertTrue(rs.next());
+            assertEquals("c", rs.getString(1));
+            assertEquals("2", rs.getString(2));
+            assertTrue(rs.next());
+            assertEquals("b", rs.getString(1));
+            assertEquals("4", rs.getString(2));
+            assertFalse(rs.next());
+        }
+
+        // validate that running query using global view gives same results
+        try (Connection conn = DriverManager.getConnection(getUrl())) {
+            ResultSet
+                    rs =
+                    conn.createStatement().executeQuery("SELECT TEXT1, TEXT2 FROM " + fullViewName
+                            + " WHERE (TEXT1, TEXT2) > ('b', '3') ORDER BY TEXT1 DESC, TEXT2");
+            assertTrue(rs.next());
+            assertEquals("d", rs.getString(1));
+            assertEquals("1", rs.getString(2));
+            assertTrue(rs.next());
+            assertEquals("c", rs.getString(1));
+            assertEquals("2", rs.getString(2));
+            assertTrue(rs.next());
+            assertEquals("b", rs.getString(1));
+            assertEquals("4", rs.getString(2));
+            assertFalse(rs.next());
+        }
+    }
+
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/1c656192/phoenix-core/src/main/java/org/apache/phoenix/compile/WhereOptimizer.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/WhereOptimizer.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/WhereOptimizer.java
index 12e09d2..b2e4c41 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/WhereOptimizer.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/WhereOptimizer.java
@@ -227,14 +227,38 @@ public class WhereOptimizer {
             
             // Iterate through all spans of this slot
             while (true) {
-                SortOrder sortOrder =  schema.getField(slot.getPKPosition() + slotOffset).getSortOrder();
+                SortOrder sortOrder =
+                        schema.getField(slot.getPKPosition() + slotOffset).getSortOrder();
                 if (prevSortOrder == null)  {
                     prevSortOrder = sortOrder;
                 } else if (prevSortOrder != sortOrder) {
+                    //Consider the Universe of keys to be [0,7]+ on the leading column A
+                    // and [0,7]+ on trailing column B, with a padbyte of 0 for ASC and 7 for DESC
+                    //if our key range for ASC keys is leading [2,*] and trailing [3,*],
+                    //   → [x203 - x777]
+                    //for this particular plan the leading key is descending (ie index desc)
+                    // consider the data
+                    // (3,2) ORDER BY A,B→ x302 → ORDER BY A DESC,B → x472
+                    // (3,3) ORDER BY A,B→ x303 → ORDER BY A DESC,B → x473
+                    // (3,4) ORDER BY A,B→ x304 → ORDER BY A DESC,B → x474
+                    // (2,3) ORDER BY A,B→ x203 → ORDER BY A DESC,B → x573
+                    // (2,7) ORDER BY A,B→ x207 → ORDER BY A DESC,B → x577
+                    // And the logical expression (A,B) > (2,3)
+                    // In the DESC A order the selected values are not contiguous,
+                    // (2,7),(3,2),(3,3),(3,4)
+                    // In the normal ASC order by the values are all contiguous
+                    // Therefore the key cannot be extracted out and a full filter must be applied
+                    // In addition, the boundary of the scan is tricky as the values are not bound
+                    // by (2,3) it is instead bound by (2,7), this should map to, [x000,x577]
+                    // FUTURE: May be able to perform a type of skip scan for this case.
+
                     // If the sort order changes, we must clip the portion with the same sort order
                     // and invert the key ranges and swap the upper and lower bounds.
-                    List<KeyRange> leftRanges = clipLeft(schema, slot.getPKPosition() + slotOffset - clipLeftSpan, clipLeftSpan, keyRanges, ptr);
-                    keyRanges = clipRight(schema, slot.getPKPosition() + slotOffset - 1, keyRanges, leftRanges, ptr);
+                    List<KeyRange> leftRanges = clipLeft(schema, slot.getPKPosition()
+                            + slotOffset - clipLeftSpan, clipLeftSpan, keyRanges, ptr);
+                    keyRanges =
+                            clipRight(schema, slot.getPKPosition() + slotOffset - 1, keyRanges,
+                                    leftRanges, ptr);
                     if (prevSortOrder == SortOrder.DESC) {
                         leftRanges = invertKeyRanges(leftRanges);
                     }
@@ -242,6 +266,13 @@ public class WhereOptimizer {
                     cnf.add(leftRanges);
                     clipLeftSpan = 0;
                     prevSortOrder = sortOrder;
+                    // since we have to clip the portion with the same sort order, we can no longer
+                    // extract the nodes from the where clause
+                    // for eg. for the schema A VARCHAR DESC, B VARCHAR ASC and query
+                    //   WHERE (A,B) < ('a','b')
+                    // the range (* - a\xFFb) is converted to (~a-*)(*-b)
+                    // so we still need to filter on A,B
+                    stopExtracting = true;
                 }
                 clipLeftSpan++;
                 slotOffset++;
@@ -264,11 +295,12 @@ public class WhereOptimizer {
             // cardinality of this slot is low.
             /*
              *  Stop extracting nodes once we encounter:
-             *  1) An unbound range unless we're forcing a skip scan and havn't encountered
+             *  1) An unbound range unless we're forcing a skip scan and haven't encountered
              *     a multi-column span. Even if we're trying to force a skip scan, we can't
              *     execute it over a multi-column span.
              *  2) A non range key as we can extract the first one, but further ones need
              *     to be evaluated in a filter.
+             *  3) As above a non-contiguous range due to sort order
              */
             stopExtracting |= (hasUnboundedRange && !forcedSkipScan) || (hasRangeKey && forcedRangeScan);
             useSkipScan |= !stopExtracting && !forcedRangeScan && (keyRanges.size() > 1 || hasRangeKey);
@@ -2060,10 +2092,20 @@ public class WhereOptimizer {
 
                                 @Override
                                 public SortOrder getSortOrder() {
-                                    // The parts of the RVC have already been converted
-                                    // to ascending, so we don't need to consider the
-                                    // childPart sort order.
-                                    return SortOrder.ASC;
+                                    //See PHOENIX-4969: Clean up and unify code paths for RVCs with
+                                    //  respect to Optimizations for SortOrder
+                                    //Handle the different paths for InList vs Normal Comparison
+                                    //The code paths in InList assume the sortOrder is ASC for
+                                    // their optimizations
+                                    //The code paths for Comparisons on RVC rewrite equality,
+                                    // for the non-equality cases return actual sort order
+                                    //This work around should work
+                                    // but a more general approach can be taken.
+                                    if(rvcElementOp == CompareOp.EQUAL ||
+                                            rvcElementOp == CompareOp.NOT_EQUAL){
+                                        return SortOrder.ASC;
+                                    }
+                                    return childPart.getColumn().getSortOrder();
                                 }
 
                                 @Override

http://git-wip-us.apache.org/repos/asf/phoenix/blob/1c656192/phoenix-core/src/main/java/org/apache/phoenix/expression/ComparisonExpression.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/ComparisonExpression.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/ComparisonExpression.java
index 074ac0a..1810c3b 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/expression/ComparisonExpression.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/ComparisonExpression.java
@@ -22,14 +22,18 @@ import java.io.DataOutput;
 import java.io.IOException;
 import java.math.BigDecimal;
 import java.sql.SQLException;
+import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
 import java.util.List;
 
+import com.google.common.collect.ImmutableList;
 import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.hadoop.io.WritableUtils;
 import org.apache.phoenix.expression.function.ArrayElemRefExpression;
+import org.apache.phoenix.expression.function.InvertFunction;
+import org.apache.phoenix.expression.rewrite.RowValueConstructorExpressionRewriter;
 import org.apache.phoenix.expression.visitor.ExpressionVisitor;
 import org.apache.phoenix.schema.SortOrder;
 import org.apache.phoenix.schema.TypeMismatchException;
@@ -111,7 +115,7 @@ public class ComparisonExpression extends BaseCompoundExpression {
         Expression rhsExpr = children.get(1);
         PDataType lhsExprDataType = lhsExpr.getDataType();
         PDataType rhsExprDataType = rhsExpr.getDataType();
-        
+
         if ((lhsExpr instanceof RowValueConstructorExpression || rhsExpr instanceof RowValueConstructorExpression) && !(lhsExpr instanceof ArrayElemRefExpression) && !(rhsExpr instanceof ArrayElemRefExpression)) {
             if (op == CompareOp.EQUAL || op == CompareOp.NOT_EQUAL) {
                 List<Expression> andNodes = Lists.<Expression>newArrayListWithExpectedSize(Math.max(lhsExpr.getChildren().size(), rhsExpr.getChildren().size()));
@@ -128,6 +132,18 @@ public class ComparisonExpression extends BaseCompoundExpression {
             if ( ! ( lhsExpr instanceof RowValueConstructorExpression ) ) {
                 lhsExpr = new RowValueConstructorExpression(Collections.singletonList(lhsExpr), lhsExpr.isStateless());
             }
+
+            /*
+            At this point both sides should be in the same row format.
+            We add the inverts so the filtering can be done properly for mixed sort type RVCs.
+            The entire RVC has to be in ASC for the actual compare to work since compare simply does
+             a varbyte compare.  See PHOENIX-4841
+            */
+            RowValueConstructorExpressionRewriter rvcRewriter =
+                    RowValueConstructorExpressionRewriter.getSingleton();
+            lhsExpr = rvcRewriter.rewriteAllChildrenAsc((RowValueConstructorExpression) lhsExpr);
+            rhsExpr = rvcRewriter.rewriteAllChildrenAsc((RowValueConstructorExpression) rhsExpr);
+
             children = Arrays.asList(lhsExpr, rhsExpr);
         } else if(lhsExprDataType != null && rhsExprDataType != null && !lhsExprDataType.isComparableTo(rhsExprDataType)) {
             throw TypeMismatchException.newException(lhsExprDataType, rhsExprDataType,

http://git-wip-us.apache.org/repos/asf/phoenix/blob/1c656192/phoenix-core/src/main/java/org/apache/phoenix/expression/rewrite/RowValueConstructorExpressionRewriter.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/rewrite/RowValueConstructorExpressionRewriter.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/rewrite/RowValueConstructorExpressionRewriter.java
new file mode 100644
index 0000000..fc0cafd
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/rewrite/RowValueConstructorExpressionRewriter.java
@@ -0,0 +1,54 @@
+/*
+ * 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.expression.rewrite;
+
+import org.apache.phoenix.expression.CoerceExpression;
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.expression.RowValueConstructorExpression;
+import org.apache.phoenix.schema.SortOrder;
+
+import java.sql.SQLException;
+import java.util.ArrayList;
+import java.util.List;
+
+public class RowValueConstructorExpressionRewriter {
+
+    static RowValueConstructorExpressionRewriter singleton = null;
+
+    public static RowValueConstructorExpressionRewriter getSingleton() {
+        if (singleton == null) {
+            singleton = new RowValueConstructorExpressionRewriter();
+        }
+        return singleton;
+    }
+
+    public RowValueConstructorExpression rewriteAllChildrenAsc(
+            RowValueConstructorExpression rvcExpression) throws SQLException {
+        List<Expression> replacementChildren = new ArrayList<>(rvcExpression.getChildren().size());
+        for (int i = 0; i < rvcExpression.getChildren().size(); i++) {
+            Expression child = rvcExpression.getChildren().get(i);
+            if (child.getSortOrder() == SortOrder.DESC) {
+                //As The KeySlot visitor has not been setup for InvertFunction need to Use Coerce
+                child = CoerceExpression.create(child, child.getDataType(), SortOrder.ASC, null);
+            }
+            replacementChildren.add(child);
+        }
+        return rvcExpression.clone(replacementChildren);
+    }
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/1c656192/phoenix-core/src/main/java/org/apache/phoenix/schema/RowKeySchema.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/RowKeySchema.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/RowKeySchema.java
index 3210516..01be40a 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/RowKeySchema.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/RowKeySchema.java
@@ -431,6 +431,10 @@ public class RowKeySchema extends ValueSchema {
             ptr.set(upperRange, 0, ptr.getOffset() + ptr.getLength());
             upperRange = ByteUtil.copyKeyBytesIfNecessary(ptr);
         }
+        //Have to update the bounds to inclusive
+        //Consider a partial key on pk columns (INT A, INT B, ....)  and a predicate (A,B) > (3,5)
+        //This initial key as a row key would look like  (x0305 - *]
+        //If we were to clip the left to (x03 - *], we would skip values like (3,6)
         return KeyRange.getKeyRange(lowerRange, true, upperRange, true);
     }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/1c656192/phoenix-core/src/test/java/org/apache/phoenix/expression/rewrite/RowValueConstructorExpressionRewriterTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/expression/rewrite/RowValueConstructorExpressionRewriterTest.java b/phoenix-core/src/test/java/org/apache/phoenix/expression/rewrite/RowValueConstructorExpressionRewriterTest.java
new file mode 100644
index 0000000..3ea0280
--- /dev/null
+++ b/phoenix-core/src/test/java/org/apache/phoenix/expression/rewrite/RowValueConstructorExpressionRewriterTest.java
@@ -0,0 +1,78 @@
+/*
+ * 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.expression.rewrite;
+
+import com.google.common.collect.ImmutableList;
+import org.apache.phoenix.expression.CoerceExpression;
+import org.apache.phoenix.expression.Determinism;
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.expression.RowValueConstructorExpression;
+import org.apache.phoenix.schema.SortOrder;
+import org.apache.phoenix.schema.types.PFloat;
+import org.junit.Test;
+import org.mockito.Mockito;
+
+import java.sql.SQLException;
+import java.util.List;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+public class RowValueConstructorExpressionRewriterTest {
+    @Test
+    public void testRewriteAllChildrenAsc() throws SQLException {
+
+
+        Expression ascChild = Mockito.mock(Expression.class);
+        Mockito.when(ascChild.getSortOrder()).thenReturn(SortOrder.ASC);
+        Mockito.when(ascChild.getDataType()).thenReturn(PFloat.INSTANCE);
+        Mockito.when(ascChild.getDeterminism()).thenReturn(Determinism.ALWAYS);
+        Mockito.when(ascChild.requiresFinalEvaluation()).thenReturn(true);
+
+        Expression descChild = Mockito.mock(Expression.class);
+        Mockito.when(descChild.getSortOrder()).thenReturn(SortOrder.DESC);
+        Mockito.when(descChild.getDataType()).thenReturn(PFloat.INSTANCE);
+        Mockito.when(descChild.getDeterminism()).thenReturn(Determinism.ALWAYS);
+        Mockito.when(descChild.requiresFinalEvaluation()).thenReturn(true);
+
+        List<Expression> children = ImmutableList.of(ascChild,descChild);
+        RowValueConstructorExpression expression =
+                new RowValueConstructorExpression(children,false);
+
+
+        RowValueConstructorExpressionRewriter
+                rewriter =
+                RowValueConstructorExpressionRewriter.getSingleton();
+
+        RowValueConstructorExpression result = rewriter.rewriteAllChildrenAsc(expression);
+
+        assertEquals(2,result.getChildren().size());
+
+        Expression child1 = result.getChildren().get(0);
+        Expression child2 = result.getChildren().get(1);
+
+        assertEquals(SortOrder.ASC, child1.getSortOrder());
+        assertEquals(SortOrder.ASC, child2.getSortOrder());
+
+        assertEquals(ascChild, child1);
+        assertTrue(child2 instanceof CoerceExpression);
+        assertEquals(descChild, ((CoerceExpression)child2).getChild());
+
+    }
+}


[18/28] phoenix git commit: PHOENIX-5000 Make SecureUserConnectionsTest as Integration test

Posted by pb...@apache.org.
PHOENIX-5000 Make SecureUserConnectionsTest as Integration test


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

Branch: refs/heads/4.x-cdh5.15
Commit: 60c19250116d378a5f6f725d9dde9a8284d86ef5
Parents: 1c65619
Author: Karan Mehta <ka...@gmail.com>
Authored: Tue Oct 30 19:40:00 2018 +0000
Committer: Pedro Boado <pb...@apache.org>
Committed: Tue Nov 27 15:11:56 2018 +0000

----------------------------------------------------------------------
 .../phoenix/jdbc/SecureUserConnectionsIT.java   | 459 +++++++++++++++++++
 .../phoenix/jdbc/SecureUserConnectionsTest.java | 459 -------------------
 2 files changed, 459 insertions(+), 459 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/60c19250/phoenix-core/src/it/java/org/apache/phoenix/jdbc/SecureUserConnectionsIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/jdbc/SecureUserConnectionsIT.java b/phoenix-core/src/it/java/org/apache/phoenix/jdbc/SecureUserConnectionsIT.java
new file mode 100644
index 0000000..eaf981b
--- /dev/null
+++ b/phoenix-core/src/it/java/org/apache/phoenix/jdbc/SecureUserConnectionsIT.java
@@ -0,0 +1,459 @@
+/*
+ * 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.jdbc;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+
+import java.io.File;
+import java.io.IOException;
+import java.lang.reflect.Field;
+import java.security.PrivilegedExceptionAction;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Properties;
+
+import org.apache.commons.io.FileUtils;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
+import org.apache.hadoop.hbase.security.User;
+import org.apache.hadoop.minikdc.MiniKdc;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.authentication.util.KerberosName;
+import org.apache.phoenix.jdbc.PhoenixEmbeddedDriver.ConnectionInfo;
+import org.apache.phoenix.query.ConfigurationFactory;
+import org.apache.phoenix.util.InstanceResolver;
+import org.apache.phoenix.util.PhoenixRuntime;
+import org.apache.phoenix.util.ReadOnlyProps;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+/**
+ * Tests ConnectionQueryServices caching when Kerberos authentication is enabled. It's not
+ * trivial to directly test this, so we exploit the knowledge that the caching is driven by
+ * a ConcurrentHashMap. We can use a HashSet to determine when instances of ConnectionInfo
+ * collide and when they do not.
+ */
+public class SecureUserConnectionsIT {
+    private static final Log LOG = LogFactory.getLog(SecureUserConnectionsIT.class);
+    private static final int KDC_START_ATTEMPTS = 10;
+
+    private static final File TEMP_DIR = new File(getClassTempDir());
+    private static final File KEYTAB_DIR = new File(TEMP_DIR, "keytabs");
+    private static final File KDC_DIR = new File(TEMP_DIR, "kdc");
+    private static final List<File> USER_KEYTAB_FILES = new ArrayList<>();
+    private static final List<File> SERVICE_KEYTAB_FILES = new ArrayList<>();
+    private static final int NUM_USERS = 3;
+    private static final Properties EMPTY_PROPERTIES = new Properties();
+    private static final String BASE_URL = PhoenixRuntime.JDBC_PROTOCOL + ":localhost:2181";
+
+    private static MiniKdc KDC;
+
+    @BeforeClass
+    public static void setupKdc() throws Exception {
+        ensureIsEmptyDirectory(KDC_DIR);
+        ensureIsEmptyDirectory(KEYTAB_DIR);
+        // Create and start the KDC. MiniKDC appears to have a race condition in how it does
+        // port allocation (with apache-ds). See PHOENIX-3287.
+        boolean started = false;
+        for (int i = 0; !started && i < KDC_START_ATTEMPTS; i++) {
+            Properties kdcConf = MiniKdc.createConf();
+            kdcConf.put(MiniKdc.DEBUG, true);
+            KDC = new MiniKdc(kdcConf, KDC_DIR);
+            try {
+                KDC.start();
+                started = true;
+            } catch (Exception e) {
+                LOG.warn("PHOENIX-3287: Failed to start KDC, retrying..", e);
+            }
+        }
+        assertTrue("The embedded KDC failed to start successfully after " + KDC_START_ATTEMPTS
+                + " attempts.", started);
+
+        createUsers(NUM_USERS);
+        createServiceUsers(NUM_USERS);
+
+        final Configuration conf = new Configuration(false);
+        conf.set(CommonConfigurationKeys.HADOOP_SECURITY_AUTHENTICATION, "kerberos");
+        conf.set(User.HBASE_SECURITY_CONF_KEY, "kerberos");
+        conf.setBoolean(User.HBASE_SECURITY_AUTHORIZATION_CONF_KEY, true);
+        UserGroupInformation.setConfiguration(conf);
+
+        // Clear the cached singletons so we can inject our own.
+        InstanceResolver.clearSingletons();
+        // Make sure the ConnectionInfo doesn't try to pull a default Configuration
+        InstanceResolver.getSingleton(ConfigurationFactory.class, new ConfigurationFactory() {
+            @Override
+            public Configuration getConfiguration() {
+                return conf;
+            }
+            @Override
+            public Configuration getConfiguration(Configuration confToClone) {
+                Configuration copy = new Configuration(conf);
+                copy.addResource(confToClone);
+                return copy;
+            }
+        });
+        updateDefaultRealm();
+    }
+
+    private static void updateDefaultRealm() throws Exception {
+        // (at least) one other phoenix test triggers the caching of this field before the KDC is up
+        // which causes principal parsing to fail.
+        Field f = KerberosName.class.getDeclaredField("defaultRealm");
+        f.setAccessible(true);
+        // Default realm for MiniKDC
+        f.set(null, "EXAMPLE.COM");
+    }
+
+    @AfterClass
+    public static void stopKdc() throws Exception {
+        // Remove our custom ConfigurationFactory for future tests
+        InstanceResolver.clearSingletons();
+        if (null != KDC) {
+            KDC.stop();
+            KDC = null;
+        }
+    }
+
+    private static String getClassTempDir() {
+        StringBuilder sb = new StringBuilder(32);
+        sb.append(System.getProperty("user.dir")).append(File.separator);
+        sb.append("target").append(File.separator);
+        sb.append(SecureUserConnectionsIT.class.getSimpleName());
+        return sb.toString();
+    }
+
+    private static void ensureIsEmptyDirectory(File f) throws IOException {
+        if (f.exists()) {
+            if (f.isDirectory()) {
+                FileUtils.deleteDirectory(f);
+            } else {
+                assertTrue("Failed to delete keytab directory", f.delete());
+            }
+        }
+        assertTrue("Failed to create keytab directory", f.mkdirs());
+    }
+
+    private static void createUsers(int numUsers) throws Exception {
+        assertNotNull("KDC is null, was setup method called?", KDC);
+        for (int i = 1; i <= numUsers; i++) {
+            String principal = "user" + i;
+            File keytabFile = new File(KEYTAB_DIR, principal + ".keytab");
+            KDC.createPrincipal(keytabFile, principal);
+            USER_KEYTAB_FILES.add(keytabFile);
+        }
+    }
+
+    private static void createServiceUsers(int numUsers) throws Exception {
+        assertNotNull("KDC is null, was setup method called?", KDC);
+        for (int i = 1; i <= numUsers; i++) {
+            String principal = "user" + i + "/localhost";
+            File keytabFile = new File(KEYTAB_DIR, "user" + i + ".service.keytab");
+            KDC.createPrincipal(keytabFile, principal);
+            SERVICE_KEYTAB_FILES.add(keytabFile);
+        }
+    }
+
+    /**
+     * Returns the principal for a user.
+     *
+     * @param offset The "number" user to return, based on one, not zero.
+     */
+    private static String getUserPrincipal(int offset) {
+        return "user" + offset + "@" + KDC.getRealm();
+    }
+
+    private static String getServicePrincipal(int offset) {
+        return "user" + offset + "/localhost@" + KDC.getRealm();
+    }
+
+    /**
+     * Returns the keytab file for the corresponding principal with the same {@code offset}.
+     * Requires {@link #createUsers(int)} to have been called with a value greater than {@code offset}.
+     *
+     * @param offset The "number" for the principal whose keytab should be returned. One-based, not zero-based.
+     */
+    public static File getUserKeytabFile(int offset) {
+        return getKeytabFile(offset, USER_KEYTAB_FILES);
+    }
+
+    public static File getServiceKeytabFile(int offset) {
+        return getKeytabFile(offset, SERVICE_KEYTAB_FILES);
+    }
+
+    private static File getKeytabFile(int offset, List<File> keytabs) {
+        assertTrue("Invalid offset: " + offset, (offset - 1) >= 0 && (offset - 1) < keytabs.size());
+        return keytabs.get(offset - 1);
+    }
+
+    private String joinUserAuthentication(String origUrl, String principal, File keytab) {
+        StringBuilder sb = new StringBuilder(64);
+        // Knock off the trailing terminator if one exists
+        if (origUrl.charAt(origUrl.length() - 1) == PhoenixRuntime.JDBC_PROTOCOL_TERMINATOR) {
+            sb.append(origUrl, 0, origUrl.length() - 1);
+        } else {
+            sb.append(origUrl);
+        }
+
+        sb.append(PhoenixRuntime.JDBC_PROTOCOL_SEPARATOR).append(principal);
+        sb.append(PhoenixRuntime.JDBC_PROTOCOL_SEPARATOR).append(keytab.getPath());
+        return sb.append(PhoenixRuntime.JDBC_PROTOCOL_TERMINATOR).toString();
+    }
+
+    @Test
+    public void testMultipleInvocationsBySameUserAreEquivalent() throws Exception {
+        final HashSet<ConnectionInfo> connections = new HashSet<>();
+        final String princ1 = getUserPrincipal(1);
+        final File keytab1 = getUserKeytabFile(1);
+
+        UserGroupInformation ugi = UserGroupInformation.loginUserFromKeytabAndReturnUGI(princ1, keytab1.getPath());
+
+        PrivilegedExceptionAction<Void> callable = new PrivilegedExceptionAction<Void>() {
+            public Void run() throws Exception {
+                String url = joinUserAuthentication(BASE_URL, princ1, keytab1);
+                connections.add(ConnectionInfo.create(url).normalize(ReadOnlyProps.EMPTY_PROPS, EMPTY_PROPERTIES));
+                return null;
+            }
+        };
+
+        // Using the same UGI should result in two equivalent ConnectionInfo objects
+        ugi.doAs(callable);
+        assertEquals(1, connections.size());
+        verifyAllConnectionsAreKerberosBased(connections);
+
+        ugi.doAs(callable);
+        assertEquals(1, connections.size());
+        verifyAllConnectionsAreKerberosBased(connections);
+    }
+
+    @Test
+    public void testMultipleUniqueUGIInstancesAreDisjoint() throws Exception {
+        final HashSet<ConnectionInfo> connections = new HashSet<>();
+        final String princ1 = getUserPrincipal(1);
+        final File keytab1 = getUserKeytabFile(1);
+
+        UserGroupInformation ugi = UserGroupInformation.loginUserFromKeytabAndReturnUGI(princ1, keytab1.getPath());
+
+        PrivilegedExceptionAction<Void> callable = new PrivilegedExceptionAction<Void>() {
+            public Void run() throws Exception {
+                String url = joinUserAuthentication(BASE_URL, princ1, keytab1);
+                connections.add(ConnectionInfo.create(url).normalize(ReadOnlyProps.EMPTY_PROPS, EMPTY_PROPERTIES));
+                return null;
+            }
+        };
+
+        ugi.doAs(callable);
+        assertEquals(1, connections.size());
+        verifyAllConnectionsAreKerberosBased(connections);
+
+        // A second, but equivalent, call from the same "real" user but a different UGI instance
+        // is expected functionality (programmer error).
+        UserGroupInformation ugiCopy = UserGroupInformation.loginUserFromKeytabAndReturnUGI(princ1, keytab1.getPath());
+        ugiCopy.doAs(callable);
+        assertEquals(2, connections.size());
+        verifyAllConnectionsAreKerberosBased(connections);
+    }
+
+    @Test
+    public void testAlternatingLogins() throws Exception {
+        final HashSet<ConnectionInfo> connections = new HashSet<>();
+        final String princ1 = getUserPrincipal(1);
+        final File keytab1 = getUserKeytabFile(1);
+        final String princ2 = getUserPrincipal(2);
+        final File keytab2 = getUserKeytabFile(2);
+
+        UserGroupInformation ugi1 = UserGroupInformation.loginUserFromKeytabAndReturnUGI(princ1, keytab1.getPath());
+        UserGroupInformation ugi2 = UserGroupInformation.loginUserFromKeytabAndReturnUGI(princ2, keytab2.getPath());
+
+        // Using the same UGI should result in two equivalent ConnectionInfo objects
+        ugi1.doAs(new PrivilegedExceptionAction<Void>() {
+            public Void run() throws Exception {
+                String url = joinUserAuthentication(BASE_URL, princ1, keytab1);
+                connections.add(ConnectionInfo.create(url).normalize(ReadOnlyProps.EMPTY_PROPS, EMPTY_PROPERTIES));
+                return null;
+            }
+        });
+        assertEquals(1, connections.size());
+        // Sanity check
+        verifyAllConnectionsAreKerberosBased(connections);
+
+        ugi2.doAs(new PrivilegedExceptionAction<Void>() {
+            public Void run() throws Exception {
+                String url = joinUserAuthentication(BASE_URL, princ2, keytab2);
+                connections.add(ConnectionInfo.create(url).normalize(ReadOnlyProps.EMPTY_PROPS, EMPTY_PROPERTIES));
+                return null;
+            }
+        });
+        assertEquals(2, connections.size());
+        verifyAllConnectionsAreKerberosBased(connections);
+
+        ugi1.doAs(new PrivilegedExceptionAction<Void>() {
+            public Void run() throws Exception {
+                String url = joinUserAuthentication(BASE_URL, princ1, keytab1);
+                connections.add(ConnectionInfo.create(url).normalize(ReadOnlyProps.EMPTY_PROPS, EMPTY_PROPERTIES));
+                return null;
+            }
+        });
+        assertEquals(2, connections.size());
+        verifyAllConnectionsAreKerberosBased(connections);
+    }
+
+    @Test
+    public void testAlternatingDestructiveLogins() throws Exception {
+        final HashSet<ConnectionInfo> connections = new HashSet<>();
+        final String princ1 = getUserPrincipal(1);
+        final File keytab1 = getUserKeytabFile(1);
+        final String princ2 = getUserPrincipal(2);
+        final File keytab2 = getUserKeytabFile(2);
+        final String url1 = joinUserAuthentication(BASE_URL, princ1, keytab1);
+        final String url2 = joinUserAuthentication(BASE_URL, princ2, keytab2);
+
+        UserGroupInformation.loginUserFromKeytab(princ1, keytab1.getPath());
+        // Using the same UGI should result in two equivalent ConnectionInfo objects
+        connections.add(ConnectionInfo.create(url1).normalize(ReadOnlyProps.EMPTY_PROPS, EMPTY_PROPERTIES));
+        assertEquals(1, connections.size());
+        // Sanity check
+        verifyAllConnectionsAreKerberosBased(connections);
+
+        UserGroupInformation.loginUserFromKeytab(princ2, keytab2.getPath());
+        connections.add(ConnectionInfo.create(url2).normalize(ReadOnlyProps.EMPTY_PROPS, EMPTY_PROPERTIES));
+        assertEquals(2, connections.size());
+        verifyAllConnectionsAreKerberosBased(connections);
+
+        // Because the UGI instances are unique, so are the connections
+        UserGroupInformation.loginUserFromKeytab(princ1, keytab1.getPath());
+        connections.add(ConnectionInfo.create(url1).normalize(ReadOnlyProps.EMPTY_PROPS, EMPTY_PROPERTIES));
+        assertEquals(3, connections.size());
+        verifyAllConnectionsAreKerberosBased(connections);
+    }
+
+    @Test
+    public void testMultipleConnectionsAsSameUser() throws Exception {
+        final HashSet<ConnectionInfo> connections = new HashSet<>();
+        final String princ1 = getUserPrincipal(1);
+        final File keytab1 = getUserKeytabFile(1);
+        final String url = joinUserAuthentication(BASE_URL, princ1, keytab1);
+
+        UserGroupInformation.loginUserFromKeytab(princ1, keytab1.getPath());
+        // Using the same UGI should result in two equivalent ConnectionInfo objects
+        connections.add(ConnectionInfo.create(url).normalize(ReadOnlyProps.EMPTY_PROPS, EMPTY_PROPERTIES));
+        assertEquals(1, connections.size());
+        // Sanity check
+        verifyAllConnectionsAreKerberosBased(connections);
+
+        // Because the UGI instances are unique, so are the connections
+        connections.add(ConnectionInfo.create(url).normalize(ReadOnlyProps.EMPTY_PROPS, EMPTY_PROPERTIES));
+        assertEquals(1, connections.size());
+    }
+
+    @Test
+    public void testMultipleConnectionsAsSameUserWithoutLogin() throws Exception {
+        final HashSet<ConnectionInfo> connections = new HashSet<>();
+        final String princ1 = getUserPrincipal(1);
+        final File keytab1 = getUserKeytabFile(1);
+
+        // Using the same UGI should result in two equivalent ConnectionInfo objects
+        final String url = joinUserAuthentication(BASE_URL, princ1, keytab1);
+        connections.add(ConnectionInfo.create(url).normalize(ReadOnlyProps.EMPTY_PROPS, EMPTY_PROPERTIES));
+        assertEquals(1, connections.size());
+        // Sanity check
+        verifyAllConnectionsAreKerberosBased(connections);
+
+        // Because the UGI instances are unique, so are the connections
+        connections.add(ConnectionInfo.create(url).normalize(ReadOnlyProps.EMPTY_PROPS, EMPTY_PROPERTIES));
+        assertEquals(1, connections.size());
+    }
+
+    @Test
+    public void testAlternatingConnectionsWithoutLogin() throws Exception {
+        final HashSet<ConnectionInfo> connections = new HashSet<>();
+        final String princ1 = getUserPrincipal(1);
+        final File keytab1 = getUserKeytabFile(1);
+        final String princ2 = getUserPrincipal(2);
+        final File keytab2 = getUserKeytabFile(2);
+        final String url1 = joinUserAuthentication(BASE_URL, princ1, keytab1);
+        final String url2 = joinUserAuthentication(BASE_URL, princ2, keytab2);
+
+        // Using the same UGI should result in two equivalent ConnectionInfo objects
+        connections.add(ConnectionInfo.create(url1).normalize(ReadOnlyProps.EMPTY_PROPS, EMPTY_PROPERTIES));
+        assertEquals(1, connections.size());
+        // Sanity check
+        verifyAllConnectionsAreKerberosBased(connections);
+
+        // Because the UGI instances are unique, so are the connections
+        connections.add(ConnectionInfo.create(url2).normalize(ReadOnlyProps.EMPTY_PROPS, EMPTY_PROPERTIES));
+        assertEquals(2, connections.size());
+        verifyAllConnectionsAreKerberosBased(connections);
+
+        // Using the same UGI should result in two equivalent ConnectionInfo objects
+        connections.add(ConnectionInfo.create(url1).normalize(ReadOnlyProps.EMPTY_PROPS, EMPTY_PROPERTIES));
+        assertEquals(3, connections.size());
+        // Sanity check
+        verifyAllConnectionsAreKerberosBased(connections);
+    }
+
+    @Test
+    public void testHostSubstitutionInUrl() throws Exception {
+        final HashSet<ConnectionInfo> connections = new HashSet<>();
+        final String princ1 = getServicePrincipal(1);
+        final File keytab1 = getServiceKeytabFile(1);
+        final String princ2 = getServicePrincipal(2);
+        final File keytab2 = getServiceKeytabFile(2);
+        final String url1 = joinUserAuthentication(BASE_URL, princ1, keytab1);
+        final String url2 = joinUserAuthentication(BASE_URL, princ2, keytab2);
+
+        // Using the same UGI should result in two equivalent ConnectionInfo objects
+        connections.add(ConnectionInfo.create(url1).normalize(ReadOnlyProps.EMPTY_PROPS, EMPTY_PROPERTIES));
+        assertEquals(1, connections.size());
+        // Sanity check
+        verifyAllConnectionsAreKerberosBased(connections);
+
+        // Logging in as the same user again should not duplicate connections
+        connections.add(ConnectionInfo.create(url1).normalize(ReadOnlyProps.EMPTY_PROPS, EMPTY_PROPERTIES));
+        assertEquals(1, connections.size());
+        // Sanity check
+        verifyAllConnectionsAreKerberosBased(connections);
+
+        // Add a second one.
+        connections.add(ConnectionInfo.create(url2).normalize(ReadOnlyProps.EMPTY_PROPS, EMPTY_PROPERTIES));
+        assertEquals(2, connections.size());
+        verifyAllConnectionsAreKerberosBased(connections);
+
+        // Again, verify this user is not duplicated
+        connections.add(ConnectionInfo.create(url2).normalize(ReadOnlyProps.EMPTY_PROPS, EMPTY_PROPERTIES));
+        assertEquals(2, connections.size());
+        verifyAllConnectionsAreKerberosBased(connections);
+
+        // Because the UGI instances are unique, so are the connections
+        connections.add(ConnectionInfo.create(url1).normalize(ReadOnlyProps.EMPTY_PROPS, EMPTY_PROPERTIES));
+        assertEquals(3, connections.size());
+        verifyAllConnectionsAreKerberosBased(connections);
+    }
+
+    private void verifyAllConnectionsAreKerberosBased(Collection<ConnectionInfo> connections) {
+        for (ConnectionInfo cnxnInfo : connections) {
+            assertTrue("ConnectionInfo does not have kerberos credentials: " + cnxnInfo, cnxnInfo.getUser().getUGI().hasKerberosCredentials());
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/60c19250/phoenix-core/src/test/java/org/apache/phoenix/jdbc/SecureUserConnectionsTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/jdbc/SecureUserConnectionsTest.java b/phoenix-core/src/test/java/org/apache/phoenix/jdbc/SecureUserConnectionsTest.java
deleted file mode 100644
index 5a99b69..0000000
--- a/phoenix-core/src/test/java/org/apache/phoenix/jdbc/SecureUserConnectionsTest.java
+++ /dev/null
@@ -1,459 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to you under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.phoenix.jdbc;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertTrue;
-
-import java.io.File;
-import java.io.IOException;
-import java.lang.reflect.Field;
-import java.security.PrivilegedExceptionAction;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Properties;
-
-import org.apache.commons.io.FileUtils;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.CommonConfigurationKeys;
-import org.apache.hadoop.hbase.security.User;
-import org.apache.hadoop.minikdc.MiniKdc;
-import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.security.authentication.util.KerberosName;
-import org.apache.phoenix.jdbc.PhoenixEmbeddedDriver.ConnectionInfo;
-import org.apache.phoenix.query.ConfigurationFactory;
-import org.apache.phoenix.util.InstanceResolver;
-import org.apache.phoenix.util.PhoenixRuntime;
-import org.apache.phoenix.util.ReadOnlyProps;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-import org.junit.Test;
-
-/**
- * Tests ConnectionQueryServices caching when Kerberos authentication is enabled. It's not
- * trivial to directly test this, so we exploit the knowledge that the caching is driven by
- * a ConcurrentHashMap. We can use a HashSet to determine when instances of ConnectionInfo
- * collide and when they do not.
- */
-public class SecureUserConnectionsTest {
-    private static final Log LOG = LogFactory.getLog(SecureUserConnectionsTest.class); 
-    private static final int KDC_START_ATTEMPTS = 10;
-
-    private static final File TEMP_DIR = new File(getClassTempDir());
-    private static final File KEYTAB_DIR = new File(TEMP_DIR, "keytabs");
-    private static final File KDC_DIR = new File(TEMP_DIR, "kdc");
-    private static final List<File> USER_KEYTAB_FILES = new ArrayList<>();
-    private static final List<File> SERVICE_KEYTAB_FILES = new ArrayList<>();
-    private static final int NUM_USERS = 3;
-    private static final Properties EMPTY_PROPERTIES = new Properties();
-    private static final String BASE_URL = PhoenixRuntime.JDBC_PROTOCOL + ":localhost:2181";
-
-    private static MiniKdc KDC;
-
-    @BeforeClass
-    public static void setupKdc() throws Exception {
-        ensureIsEmptyDirectory(KDC_DIR);
-        ensureIsEmptyDirectory(KEYTAB_DIR);
-        // Create and start the KDC. MiniKDC appears to have a race condition in how it does
-        // port allocation (with apache-ds). See PHOENIX-3287.
-        boolean started = false;
-        for (int i = 0; !started && i < KDC_START_ATTEMPTS; i++) {
-            Properties kdcConf = MiniKdc.createConf();
-            kdcConf.put(MiniKdc.DEBUG, true);
-            KDC = new MiniKdc(kdcConf, KDC_DIR);
-            try {
-                KDC.start();
-                started = true;
-            } catch (Exception e) {
-                LOG.warn("PHOENIX-3287: Failed to start KDC, retrying..", e);
-            }
-        }
-        assertTrue("The embedded KDC failed to start successfully after " + KDC_START_ATTEMPTS
-                + " attempts.", started);
-
-        createUsers(NUM_USERS);
-        createServiceUsers(NUM_USERS);
-
-        final Configuration conf = new Configuration(false);
-        conf.set(CommonConfigurationKeys.HADOOP_SECURITY_AUTHENTICATION, "kerberos");
-        conf.set(User.HBASE_SECURITY_CONF_KEY, "kerberos");
-        conf.setBoolean(User.HBASE_SECURITY_AUTHORIZATION_CONF_KEY, true);
-        UserGroupInformation.setConfiguration(conf);
-
-        // Clear the cached singletons so we can inject our own.
-        InstanceResolver.clearSingletons();
-        // Make sure the ConnectionInfo doesn't try to pull a default Configuration
-        InstanceResolver.getSingleton(ConfigurationFactory.class, new ConfigurationFactory() {
-            @Override
-            public Configuration getConfiguration() {
-                return conf;
-            }
-            @Override
-            public Configuration getConfiguration(Configuration confToClone) {
-                Configuration copy = new Configuration(conf);
-                copy.addResource(confToClone);
-                return copy;
-            }
-        });
-        updateDefaultRealm();
-    }
-
-    private static void updateDefaultRealm() throws Exception {
-        // (at least) one other phoenix test triggers the caching of this field before the KDC is up
-        // which causes principal parsing to fail.
-        Field f = KerberosName.class.getDeclaredField("defaultRealm");
-        f.setAccessible(true);
-        // Default realm for MiniKDC
-        f.set(null, "EXAMPLE.COM");
-    }
-
-    @AfterClass
-    public static void stopKdc() throws Exception {
-        // Remove our custom ConfigurationFactory for future tests
-        InstanceResolver.clearSingletons();
-        if (null != KDC) {
-            KDC.stop();
-            KDC = null;
-        }
-    }
-
-    private static String getClassTempDir() {
-        StringBuilder sb = new StringBuilder(32);
-        sb.append(System.getProperty("user.dir")).append(File.separator);
-        sb.append("target").append(File.separator);
-        sb.append(SecureUserConnectionsTest.class.getSimpleName());
-        return sb.toString();
-    }
-
-    private static void ensureIsEmptyDirectory(File f) throws IOException {
-        if (f.exists()) {
-            if (f.isDirectory()) {
-                FileUtils.deleteDirectory(f);
-            } else {
-                assertTrue("Failed to delete keytab directory", f.delete());
-            }
-        }
-        assertTrue("Failed to create keytab directory", f.mkdirs());
-    }
-
-    private static void createUsers(int numUsers) throws Exception {
-        assertNotNull("KDC is null, was setup method called?", KDC);
-        for (int i = 1; i <= numUsers; i++) {
-            String principal = "user" + i;
-            File keytabFile = new File(KEYTAB_DIR, principal + ".keytab");
-            KDC.createPrincipal(keytabFile, principal);
-            USER_KEYTAB_FILES.add(keytabFile);
-        }
-    }
-
-    private static void createServiceUsers(int numUsers) throws Exception {
-        assertNotNull("KDC is null, was setup method called?", KDC);
-        for (int i = 1; i <= numUsers; i++) {
-            String principal = "user" + i + "/localhost";
-            File keytabFile = new File(KEYTAB_DIR, "user" + i + ".service.keytab");
-            KDC.createPrincipal(keytabFile, principal);
-            SERVICE_KEYTAB_FILES.add(keytabFile);
-        }
-    }
-
-    /**
-     * Returns the principal for a user.
-     *
-     * @param offset The "number" user to return, based on one, not zero.
-     */
-    private static String getUserPrincipal(int offset) {
-        return "user" + offset + "@" + KDC.getRealm();
-    }
-
-    private static String getServicePrincipal(int offset) {
-        return "user" + offset + "/localhost@" + KDC.getRealm();
-    }
-
-    /**
-     * Returns the keytab file for the corresponding principal with the same {@code offset}.
-     * Requires {@link #createUsers(int)} to have been called with a value greater than {@code offset}.
-     *
-     * @param offset The "number" for the principal whose keytab should be returned. One-based, not zero-based.
-     */
-    public static File getUserKeytabFile(int offset) {
-        return getKeytabFile(offset, USER_KEYTAB_FILES);
-    }
-
-    public static File getServiceKeytabFile(int offset) {
-        return getKeytabFile(offset, SERVICE_KEYTAB_FILES);
-    }
-
-    private static File getKeytabFile(int offset, List<File> keytabs) {
-        assertTrue("Invalid offset: " + offset, (offset - 1) >= 0 && (offset - 1) < keytabs.size());
-        return keytabs.get(offset - 1);
-    }
-
-    private String joinUserAuthentication(String origUrl, String principal, File keytab) {
-        StringBuilder sb = new StringBuilder(64);
-        // Knock off the trailing terminator if one exists
-        if (origUrl.charAt(origUrl.length() - 1) == PhoenixRuntime.JDBC_PROTOCOL_TERMINATOR) {
-            sb.append(origUrl, 0, origUrl.length() - 1);
-        } else {
-            sb.append(origUrl);
-        }
-
-        sb.append(PhoenixRuntime.JDBC_PROTOCOL_SEPARATOR).append(principal);
-        sb.append(PhoenixRuntime.JDBC_PROTOCOL_SEPARATOR).append(keytab.getPath());
-        return sb.append(PhoenixRuntime.JDBC_PROTOCOL_TERMINATOR).toString();
-    }
-
-    @Test
-    public void testMultipleInvocationsBySameUserAreEquivalent() throws Exception {
-        final HashSet<ConnectionInfo> connections = new HashSet<>();
-        final String princ1 = getUserPrincipal(1);
-        final File keytab1 = getUserKeytabFile(1);
-
-        UserGroupInformation ugi = UserGroupInformation.loginUserFromKeytabAndReturnUGI(princ1, keytab1.getPath());
-
-        PrivilegedExceptionAction<Void> callable = new PrivilegedExceptionAction<Void>() {
-            public Void run() throws Exception {
-                String url = joinUserAuthentication(BASE_URL, princ1, keytab1);
-                connections.add(ConnectionInfo.create(url).normalize(ReadOnlyProps.EMPTY_PROPS, EMPTY_PROPERTIES));
-                return null;
-            }
-        };
-
-        // Using the same UGI should result in two equivalent ConnectionInfo objects
-        ugi.doAs(callable);
-        assertEquals(1, connections.size());
-        verifyAllConnectionsAreKerberosBased(connections);
-
-        ugi.doAs(callable);
-        assertEquals(1, connections.size());
-        verifyAllConnectionsAreKerberosBased(connections);
-    }
-
-    @Test
-    public void testMultipleUniqueUGIInstancesAreDisjoint() throws Exception {
-        final HashSet<ConnectionInfo> connections = new HashSet<>();
-        final String princ1 = getUserPrincipal(1);
-        final File keytab1 = getUserKeytabFile(1);
-
-        UserGroupInformation ugi = UserGroupInformation.loginUserFromKeytabAndReturnUGI(princ1, keytab1.getPath());
-
-        PrivilegedExceptionAction<Void> callable = new PrivilegedExceptionAction<Void>() {
-            public Void run() throws Exception {
-                String url = joinUserAuthentication(BASE_URL, princ1, keytab1);
-                connections.add(ConnectionInfo.create(url).normalize(ReadOnlyProps.EMPTY_PROPS, EMPTY_PROPERTIES));
-                return null;
-            }
-        };
-
-        ugi.doAs(callable);
-        assertEquals(1, connections.size());
-        verifyAllConnectionsAreKerberosBased(connections);
-
-        // A second, but equivalent, call from the same "real" user but a different UGI instance
-        // is expected functionality (programmer error).
-        UserGroupInformation ugiCopy = UserGroupInformation.loginUserFromKeytabAndReturnUGI(princ1, keytab1.getPath());
-        ugiCopy.doAs(callable);
-        assertEquals(2, connections.size());
-        verifyAllConnectionsAreKerberosBased(connections);
-    }
-
-    @Test
-    public void testAlternatingLogins() throws Exception {
-        final HashSet<ConnectionInfo> connections = new HashSet<>();
-        final String princ1 = getUserPrincipal(1);
-        final File keytab1 = getUserKeytabFile(1);
-        final String princ2 = getUserPrincipal(2);
-        final File keytab2 = getUserKeytabFile(2);
-
-        UserGroupInformation ugi1 = UserGroupInformation.loginUserFromKeytabAndReturnUGI(princ1, keytab1.getPath());
-        UserGroupInformation ugi2 = UserGroupInformation.loginUserFromKeytabAndReturnUGI(princ2, keytab2.getPath());
-
-        // Using the same UGI should result in two equivalent ConnectionInfo objects
-        ugi1.doAs(new PrivilegedExceptionAction<Void>() {
-            public Void run() throws Exception {
-                String url = joinUserAuthentication(BASE_URL, princ1, keytab1);
-                connections.add(ConnectionInfo.create(url).normalize(ReadOnlyProps.EMPTY_PROPS, EMPTY_PROPERTIES));
-                return null;
-            }
-        });
-        assertEquals(1, connections.size());
-        // Sanity check
-        verifyAllConnectionsAreKerberosBased(connections);
-
-        ugi2.doAs(new PrivilegedExceptionAction<Void>() {
-            public Void run() throws Exception {
-                String url = joinUserAuthentication(BASE_URL, princ2, keytab2);
-                connections.add(ConnectionInfo.create(url).normalize(ReadOnlyProps.EMPTY_PROPS, EMPTY_PROPERTIES));
-                return null;
-            }
-        });
-        assertEquals(2, connections.size());
-        verifyAllConnectionsAreKerberosBased(connections);
-
-        ugi1.doAs(new PrivilegedExceptionAction<Void>() {
-            public Void run() throws Exception {
-                String url = joinUserAuthentication(BASE_URL, princ1, keytab1);
-                connections.add(ConnectionInfo.create(url).normalize(ReadOnlyProps.EMPTY_PROPS, EMPTY_PROPERTIES));
-                return null;
-            }
-        });
-        assertEquals(2, connections.size());
-        verifyAllConnectionsAreKerberosBased(connections);
-    }
-
-    @Test
-    public void testAlternatingDestructiveLogins() throws Exception {
-        final HashSet<ConnectionInfo> connections = new HashSet<>();
-        final String princ1 = getUserPrincipal(1);
-        final File keytab1 = getUserKeytabFile(1);
-        final String princ2 = getUserPrincipal(2);
-        final File keytab2 = getUserKeytabFile(2);
-        final String url1 = joinUserAuthentication(BASE_URL, princ1, keytab1);
-        final String url2 = joinUserAuthentication(BASE_URL, princ2, keytab2);
-
-        UserGroupInformation.loginUserFromKeytab(princ1, keytab1.getPath());
-        // Using the same UGI should result in two equivalent ConnectionInfo objects
-        connections.add(ConnectionInfo.create(url1).normalize(ReadOnlyProps.EMPTY_PROPS, EMPTY_PROPERTIES));
-        assertEquals(1, connections.size());
-        // Sanity check
-        verifyAllConnectionsAreKerberosBased(connections);
-
-        UserGroupInformation.loginUserFromKeytab(princ2, keytab2.getPath());
-        connections.add(ConnectionInfo.create(url2).normalize(ReadOnlyProps.EMPTY_PROPS, EMPTY_PROPERTIES));
-        assertEquals(2, connections.size());
-        verifyAllConnectionsAreKerberosBased(connections);
-
-        // Because the UGI instances are unique, so are the connections
-        UserGroupInformation.loginUserFromKeytab(princ1, keytab1.getPath());
-        connections.add(ConnectionInfo.create(url1).normalize(ReadOnlyProps.EMPTY_PROPS, EMPTY_PROPERTIES));
-        assertEquals(3, connections.size());
-        verifyAllConnectionsAreKerberosBased(connections);
-    }
-
-    @Test
-    public void testMultipleConnectionsAsSameUser() throws Exception {
-        final HashSet<ConnectionInfo> connections = new HashSet<>();
-        final String princ1 = getUserPrincipal(1);
-        final File keytab1 = getUserKeytabFile(1);
-        final String url = joinUserAuthentication(BASE_URL, princ1, keytab1);
-
-        UserGroupInformation.loginUserFromKeytab(princ1, keytab1.getPath());
-        // Using the same UGI should result in two equivalent ConnectionInfo objects
-        connections.add(ConnectionInfo.create(url).normalize(ReadOnlyProps.EMPTY_PROPS, EMPTY_PROPERTIES));
-        assertEquals(1, connections.size());
-        // Sanity check
-        verifyAllConnectionsAreKerberosBased(connections);
-
-        // Because the UGI instances are unique, so are the connections
-        connections.add(ConnectionInfo.create(url).normalize(ReadOnlyProps.EMPTY_PROPS, EMPTY_PROPERTIES));
-        assertEquals(1, connections.size());
-    }
-
-    @Test
-    public void testMultipleConnectionsAsSameUserWithoutLogin() throws Exception {
-        final HashSet<ConnectionInfo> connections = new HashSet<>();
-        final String princ1 = getUserPrincipal(1);
-        final File keytab1 = getUserKeytabFile(1);
-
-        // Using the same UGI should result in two equivalent ConnectionInfo objects
-        final String url = joinUserAuthentication(BASE_URL, princ1, keytab1);
-        connections.add(ConnectionInfo.create(url).normalize(ReadOnlyProps.EMPTY_PROPS, EMPTY_PROPERTIES));
-        assertEquals(1, connections.size());
-        // Sanity check
-        verifyAllConnectionsAreKerberosBased(connections);
-
-        // Because the UGI instances are unique, so are the connections
-        connections.add(ConnectionInfo.create(url).normalize(ReadOnlyProps.EMPTY_PROPS, EMPTY_PROPERTIES));
-        assertEquals(1, connections.size());
-    }
-
-    @Test
-    public void testAlternatingConnectionsWithoutLogin() throws Exception {
-        final HashSet<ConnectionInfo> connections = new HashSet<>();
-        final String princ1 = getUserPrincipal(1);
-        final File keytab1 = getUserKeytabFile(1);
-        final String princ2 = getUserPrincipal(2);
-        final File keytab2 = getUserKeytabFile(2);
-        final String url1 = joinUserAuthentication(BASE_URL, princ1, keytab1);
-        final String url2 = joinUserAuthentication(BASE_URL, princ2, keytab2);
-
-        // Using the same UGI should result in two equivalent ConnectionInfo objects
-        connections.add(ConnectionInfo.create(url1).normalize(ReadOnlyProps.EMPTY_PROPS, EMPTY_PROPERTIES));
-        assertEquals(1, connections.size());
-        // Sanity check
-        verifyAllConnectionsAreKerberosBased(connections);
-
-        // Because the UGI instances are unique, so are the connections
-        connections.add(ConnectionInfo.create(url2).normalize(ReadOnlyProps.EMPTY_PROPS, EMPTY_PROPERTIES));
-        assertEquals(2, connections.size());
-        verifyAllConnectionsAreKerberosBased(connections);
-
-        // Using the same UGI should result in two equivalent ConnectionInfo objects
-        connections.add(ConnectionInfo.create(url1).normalize(ReadOnlyProps.EMPTY_PROPS, EMPTY_PROPERTIES));
-        assertEquals(3, connections.size());
-        // Sanity check
-        verifyAllConnectionsAreKerberosBased(connections);
-    }
-
-    @Test
-    public void testHostSubstitutionInUrl() throws Exception {
-        final HashSet<ConnectionInfo> connections = new HashSet<>();
-        final String princ1 = getServicePrincipal(1);
-        final File keytab1 = getServiceKeytabFile(1);
-        final String princ2 = getServicePrincipal(2);
-        final File keytab2 = getServiceKeytabFile(2);
-        final String url1 = joinUserAuthentication(BASE_URL, princ1, keytab1);
-        final String url2 = joinUserAuthentication(BASE_URL, princ2, keytab2);
-
-        // Using the same UGI should result in two equivalent ConnectionInfo objects
-        connections.add(ConnectionInfo.create(url1).normalize(ReadOnlyProps.EMPTY_PROPS, EMPTY_PROPERTIES));
-        assertEquals(1, connections.size());
-        // Sanity check
-        verifyAllConnectionsAreKerberosBased(connections);
-
-        // Logging in as the same user again should not duplicate connections
-        connections.add(ConnectionInfo.create(url1).normalize(ReadOnlyProps.EMPTY_PROPS, EMPTY_PROPERTIES));
-        assertEquals(1, connections.size());
-        // Sanity check
-        verifyAllConnectionsAreKerberosBased(connections);
-
-        // Add a second one.
-        connections.add(ConnectionInfo.create(url2).normalize(ReadOnlyProps.EMPTY_PROPS, EMPTY_PROPERTIES));
-        assertEquals(2, connections.size());
-        verifyAllConnectionsAreKerberosBased(connections);
-
-        // Again, verify this user is not duplicated
-        connections.add(ConnectionInfo.create(url2).normalize(ReadOnlyProps.EMPTY_PROPS, EMPTY_PROPERTIES));
-        assertEquals(2, connections.size());
-        verifyAllConnectionsAreKerberosBased(connections);
-
-        // Because the UGI instances are unique, so are the connections
-        connections.add(ConnectionInfo.create(url1).normalize(ReadOnlyProps.EMPTY_PROPS, EMPTY_PROPERTIES));
-        assertEquals(3, connections.size());
-        verifyAllConnectionsAreKerberosBased(connections);
-    }
-
-    private void verifyAllConnectionsAreKerberosBased(Collection<ConnectionInfo> connections) {
-        for (ConnectionInfo cnxnInfo : connections) {
-            assertTrue("ConnectionInfo does not have kerberos credentials: " + cnxnInfo, cnxnInfo.getUser().getUGI().hasKerberosCredentials());
-        }
-    }
-}


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

Posted by pb...@apache.org.
http://git-wip-us.apache.org/repos/asf/phoenix/blob/678563f5/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/678563f5/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/678563f5/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/678563f5/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/678563f5/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/678563f5/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/678563f5/phoenix-spark/pom.xml
----------------------------------------------------------------------
diff --git a/phoenix-spark/pom.xml b/phoenix-spark/pom.xml
index 6b9d58b..858895a 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/678563f5/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/678563f5/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/678563f5/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/678563f5/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/678563f5/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'") {


[15/28] phoenix git commit: PHOENIX-5008: CQSI.init should not bubble up RetriableUpgradeException to client in case of an UpgradeRequiredException

Posted by pb...@apache.org.
PHOENIX-5008: CQSI.init should not bubble up RetriableUpgradeException to client in case of an UpgradeRequiredException


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

Branch: refs/heads/4.x-cdh5.15
Commit: f33f7d7c92ab75520b15fa158c7feccfb7041cae
Parents: 7afa954
Author: Chinmay Kulkarni <ch...@gmail.com>
Authored: Sat Nov 10 03:22:57 2018 +0000
Committer: Pedro Boado <pb...@apache.org>
Committed: Tue Nov 27 15:11:48 2018 +0000

----------------------------------------------------------------------
 .../SystemCatalogCreationOnConnectionIT.java    | 97 +++++++++++++++++---
 .../query/ConnectionQueryServicesImpl.java      |  4 +-
 2 files changed, 84 insertions(+), 17 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/f33f7d7c/phoenix-core/src/it/java/org/apache/phoenix/end2end/SystemCatalogCreationOnConnectionIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SystemCatalogCreationOnConnectionIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SystemCatalogCreationOnConnectionIT.java
index a1685c44..eadd391 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SystemCatalogCreationOnConnectionIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SystemCatalogCreationOnConnectionIT.java
@@ -21,9 +21,11 @@ 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.query.BaseTest.generateUniqueName;
 
 import java.io.IOException;
 import java.sql.Connection;
+import java.sql.DriverManager;
 import java.sql.SQLException;
 import java.util.Arrays;
 import java.util.HashMap;
@@ -42,6 +44,7 @@ import org.apache.phoenix.coprocessor.MetaDataProtocol;
 import org.apache.phoenix.exception.SQLExceptionCode;
 import org.apache.phoenix.exception.UpgradeRequiredException;
 import org.apache.phoenix.jdbc.PhoenixConnection;
+import org.apache.phoenix.jdbc.PhoenixDriver;
 import org.apache.phoenix.jdbc.PhoenixEmbeddedDriver;
 import org.apache.phoenix.jdbc.PhoenixTestDriver;
 import org.apache.phoenix.query.ConnectionQueryServices;
@@ -69,6 +72,12 @@ public class SystemCatalogCreationOnConnectionIT {
     private static final String PHOENIX_SYSTEM_CATALOG = "SYSTEM.CATALOG";
     private static final String EXECUTE_UPGRADE_COMMAND = "EXECUTE UPGRADE";
     private static final String MODIFIED_MAX_VERSIONS ="5";
+    private static final String CREATE_TABLE_STMT = "CREATE TABLE %s"
+            + " (k1 VARCHAR NOT NULL, k2 VARCHAR, CONSTRAINT PK PRIMARY KEY(K1,K2))";
+    private static final String SELECT_STMT = "SELECT * FROM %s";
+    private static final String DELETE_STMT = "DELETE FROM %s";
+    private static final String CREATE_INDEX_STMT = "CREATE INDEX DUMMY_IDX ON %s (K1) INCLUDE (K2)";
+    private static final String UPSERT_STMT = "UPSERT INTO %s VALUES ('A', 'B')";
 
     private static final Set<String> PHOENIX_SYSTEM_TABLES = new HashSet<>(Arrays.asList(
       "SYSTEM.CATALOG", "SYSTEM.SEQUENCE", "SYSTEM.STATS", "SYSTEM.FUNCTION",
@@ -167,12 +176,8 @@ public class SystemCatalogCreationOnConnectionIT {
         UpgradeUtil.doNotUpgradeOnFirstConnection(propsDoNotUpgradePropSet);
         SystemCatalogCreationOnConnectionIT.PhoenixSysCatCreationTestingDriver driver =
           new SystemCatalogCreationOnConnectionIT.PhoenixSysCatCreationTestingDriver(ReadOnlyProps.EMPTY_PROPS);
-        try {
-            driver.getConnectionQueryServices(getJdbcUrl(), propsDoNotUpgradePropSet);
-            fail("Client should not be able to create SYSTEM.CATALOG since we set the doNotUpgrade property");
-        } catch (Exception e) {
-            assertTrue(e instanceof UpgradeRequiredException);
-        }
+
+        driver.getConnectionQueryServices(getJdbcUrl(), propsDoNotUpgradePropSet);
         hbaseTables = getHBaseTables();
         assertFalse(hbaseTables.contains(PHOENIX_SYSTEM_CATALOG) || hbaseTables.contains(PHOENIX_NAMESPACE_MAPPED_SYSTEM_CATALOG));
         assertTrue(hbaseTables.size() == 0);
@@ -428,6 +433,70 @@ public class SystemCatalogCreationOnConnectionIT {
         assertEquals(Integer.parseInt(MODIFIED_MAX_VERSIONS), verifyModificationTableMetadata(driver, PHOENIX_SYSTEM_CATALOG));
     }
 
+    // Test the case when an end-user uses the vanilla PhoenixDriver to create a connection and a
+    // requirement for upgrade is detected. In this case, the user should get a connection on which
+    // they are only able to run "EXECUTE UPGRADE"
+    @Test
+    public void testExecuteUpgradeSameConnWithPhoenixDriver() throws Exception {
+        // Register the vanilla PhoenixDriver
+        DriverManager.registerDriver(PhoenixDriver.INSTANCE);
+        startMiniClusterWithToggleNamespaceMapping(Boolean.FALSE.toString());
+        Properties propsDoNotUpgradePropSet = new Properties();
+        // Set doNotUpgradeProperty to true
+        UpgradeUtil.doNotUpgradeOnFirstConnection(propsDoNotUpgradePropSet);
+
+        Connection conn = DriverManager.getConnection(getJdbcUrl(), propsDoNotUpgradePropSet);
+        hbaseTables = getHBaseTables();
+        assertFalse(hbaseTables.contains(PHOENIX_SYSTEM_CATALOG)
+                || hbaseTables.contains(PHOENIX_NAMESPACE_MAPPED_SYSTEM_CATALOG));
+        assertTrue(hbaseTables.size() == 0);
+
+        // Test that we are unable to run any other queries using this connection until we upgrade
+        final String tableName = generateUniqueName();
+        try {
+            conn.createStatement().execute(String.format(CREATE_TABLE_STMT, tableName));
+            fail("CREATE TABLE should have failed with UpgradeRequiredException");
+        } catch (UpgradeRequiredException expected) {
+
+        }
+        try {
+            conn.createStatement().execute(String.format(SELECT_STMT, tableName));
+            fail("SELECT should have failed with UpgradeRequiredException");
+        } catch (UpgradeRequiredException expected) {
+
+        }
+        try {
+            conn.createStatement().execute(String.format(DELETE_STMT, tableName));
+            fail("DELETE should have failed with UpgradeRequiredException");
+        } catch (UpgradeRequiredException expected) {
+
+        }
+        try {
+            conn.createStatement().execute(String.format(CREATE_INDEX_STMT, tableName));
+            fail("CREATE INDEX should have failed with UpgradeRequiredException");
+        } catch (UpgradeRequiredException expected) {
+
+        }
+        try {
+            conn.createStatement().execute(String.format(UPSERT_STMT, tableName));
+            fail("UPSERT VALUES should have failed with UpgradeRequiredException");
+        } catch (UpgradeRequiredException expected) {
+
+        }
+
+        // Now run the upgrade command. All SYSTEM tables should be created
+        conn.createStatement().execute("EXECUTE UPGRADE");
+        hbaseTables = getHBaseTables();
+        assertEquals(PHOENIX_SYSTEM_TABLES, hbaseTables);
+
+        // Now we can run any other query/mutation using this connection object
+        conn.createStatement().execute(String.format(CREATE_TABLE_STMT, tableName));
+        conn.createStatement().execute(String.format(SELECT_STMT, tableName));
+        conn.createStatement().execute(String.format(DELETE_STMT, tableName));
+        conn.createStatement().execute(String.format(CREATE_INDEX_STMT, tableName));
+        conn.createStatement().execute(String.format(UPSERT_STMT, tableName));
+    }
+
     /**
      * Return all created HBase tables
      * @return Set of HBase table name strings
@@ -435,7 +504,7 @@ public class SystemCatalogCreationOnConnectionIT {
      */
     private Set<String> getHBaseTables() throws IOException {
         Set<String> tables = new HashSet<>();
-        for (TableName tn : testUtil.getHBaseAdmin().listTableNames()) {
+        for (TableName tn : testUtil.getAdmin().listTableNames()) {
             tables.add(tn.getNameAsString());
         }
         return tables;
@@ -520,19 +589,17 @@ public class SystemCatalogCreationOnConnectionIT {
         ReadOnlyProps readOnlyProps = new ReadOnlyProps(props);
         SystemCatalogCreationOnConnectionIT.PhoenixSysCatCreationTestingDriver driver =
           new SystemCatalogCreationOnConnectionIT.PhoenixSysCatCreationTestingDriver(readOnlyProps);
-        try {
-            driver.getConnectionQueryServices(getJdbcUrl(), new Properties());
-            fail("Client should not be able to create SYSTEM.CATALOG since we set the isAutoUpgradeEnabled property to false");
-        } catch (Exception e) {
-            assertTrue(e instanceof UpgradeRequiredException);
-        }
+
+        // We should be able to get a connection, however upgradeRequired should be set so that we
+        // are not allowed to run any query/mutation until "EXECUTE UPGRADE" has been run
+        Connection conn = driver.getConnectionQueryServices(getJdbcUrl(), new Properties())
+                .connect(getJdbcUrl(), new Properties());
         hbaseTables = getHBaseTables();
         assertFalse(hbaseTables.contains(PHOENIX_SYSTEM_CATALOG) || hbaseTables.contains(PHOENIX_NAMESPACE_MAPPED_SYSTEM_CATALOG));
         assertTrue(hbaseTables.size() == 0);
         assertEquals(1, countUpgradeAttempts);
 
-        // We use the same ConnectionQueryServices instance to run "EXECUTE UPGRADE"
-        Connection conn = driver.getConnectionQueryServices(getJdbcUrl(), new Properties()).connect(getJdbcUrl(), new Properties());
+        // We use the same connection to run "EXECUTE UPGRADE"
         try {
             conn.createStatement().execute(EXECUTE_UPGRADE_COMMAND);
         } finally {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/f33f7d7c/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java b/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
index eddaf14..b468bc2 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
@@ -2919,8 +2919,8 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                                     upgradeSystemTables(url, props);
                                 } else {
                                     // We expect the user to manually run the "EXECUTE UPGRADE" command first.
-                                    // This exception will get caught below as a RetriableUpgradeException
-                                    throw new UpgradeRequiredException();
+                                    logger.error("Upgrade is required. Must run 'EXECUTE UPGRADE' "
+                                            + "before any other command");
                                 }
                             }
                             scheduleRenewLeaseTasks();


[25/28] phoenix git commit: PHOENIX-5000 Make SecureUserConnectionsTest as Integration test (Addendum)

Posted by pb...@apache.org.
PHOENIX-5000 Make SecureUserConnectionsTest as Integration test (Addendum)


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

Branch: refs/heads/4.x-cdh5.15
Commit: bb17957ca2938093dd94bed6052cde92e28d176a
Parents: d2e4a73
Author: Karan Mehta <ka...@gmail.com>
Authored: Mon Nov 19 22:48:32 2018 +0000
Committer: Pedro Boado <pb...@apache.org>
Committed: Tue Nov 27 15:12:15 2018 +0000

----------------------------------------------------------------------
 .../it/java/org/apache/phoenix/jdbc/SecureUserConnectionsIT.java  | 3 +++
 1 file changed, 3 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/bb17957c/phoenix-core/src/it/java/org/apache/phoenix/jdbc/SecureUserConnectionsIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/jdbc/SecureUserConnectionsIT.java b/phoenix-core/src/it/java/org/apache/phoenix/jdbc/SecureUserConnectionsIT.java
index eaf981b..1ab54d2 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/jdbc/SecureUserConnectionsIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/jdbc/SecureUserConnectionsIT.java
@@ -39,6 +39,7 @@ import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.minikdc.MiniKdc;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.authentication.util.KerberosName;
+import org.apache.phoenix.end2end.NeedsOwnMiniClusterTest;
 import org.apache.phoenix.jdbc.PhoenixEmbeddedDriver.ConnectionInfo;
 import org.apache.phoenix.query.ConfigurationFactory;
 import org.apache.phoenix.util.InstanceResolver;
@@ -47,6 +48,7 @@ import org.apache.phoenix.util.ReadOnlyProps;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
+import org.junit.experimental.categories.Category;
 
 /**
  * Tests ConnectionQueryServices caching when Kerberos authentication is enabled. It's not
@@ -54,6 +56,7 @@ import org.junit.Test;
  * a ConcurrentHashMap. We can use a HashSet to determine when instances of ConnectionInfo
  * collide and when they do not.
  */
+@Category(NeedsOwnMiniClusterTest.class)
 public class SecureUserConnectionsIT {
     private static final Log LOG = LogFactory.getLog(SecureUserConnectionsIT.class);
     private static final int KDC_START_ATTEMPTS = 10;


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

Posted by pb...@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/678563f5
Tree: http://git-wip-us.apache.org/repos/asf/phoenix/tree/678563f5
Diff: http://git-wip-us.apache.org/repos/asf/phoenix/diff/678563f5

Branch: refs/heads/4.x-cdh5.15
Commit: 678563f5dc1fbaa37ef890ab135fb301dcf20ad6
Parents: 7f13f87
Author: Thomas D'Silva <td...@apache.org>
Authored: Fri Oct 19 06:00:01 2018 +0100
Committer: pboado <pe...@gmail.com>
Committed: Mon Nov 26 10:52:48 2018 +0000

----------------------------------------------------------------------
 .../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 +-
 20 files changed, 4649 insertions(+), 2269 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/678563f5/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));
         }
     }
 }
+


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

Posted by pb...@apache.org.
http://git-wip-us.apache.org/repos/asf/phoenix/blob/678563f5/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();
+            }
+        }
+    }
+}


[11/28] phoenix git commit: PHOENIX-5012 Don't derive IndexToolIT from ParallelStatsEnabled

Posted by pb...@apache.org.
PHOENIX-5012 Don't derive IndexToolIT from ParallelStatsEnabled


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

Branch: refs/heads/4.x-cdh5.15
Commit: b296ddc19a1533e105e01597a3b761a37922d261
Parents: 1da0ad7
Author: James Taylor <ja...@apache.org>
Authored: Sat Nov 10 19:04:48 2018 +0000
Committer: Pedro Boado <pb...@apache.org>
Committed: Tue Nov 27 15:11:36 2018 +0000

----------------------------------------------------------------------
 .../src/it/java/org/apache/phoenix/end2end/IndexToolIT.java  | 8 +++++---
 1 file changed, 5 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/b296ddc1/phoenix-core/src/it/java/org/apache/phoenix/end2end/IndexToolIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/IndexToolIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/IndexToolIT.java
index c99f145..e096bb5 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/IndexToolIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/IndexToolIT.java
@@ -58,7 +58,6 @@ import org.apache.phoenix.util.SchemaUtil;
 import org.apache.phoenix.util.TestUtil;
 import org.junit.BeforeClass;
 import org.junit.Test;
-import org.junit.experimental.categories.Category;
 import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
 import org.junit.runners.Parameterized.Parameters;
@@ -67,8 +66,7 @@ import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
 
 @RunWith(Parameterized.class)
-@Category(NeedsOwnMiniClusterTest.class)
-public class IndexToolIT extends ParallelStatsEnabledIT {
+public class IndexToolIT extends BaseUniqueNamesOwnClusterIT {
 
     private final boolean localIndex;
     private final boolean transactional;
@@ -99,9 +97,13 @@ public class IndexToolIT extends ParallelStatsEnabledIT {
     @BeforeClass
     public static void setup() throws Exception {
         Map<String, String> serverProps = Maps.newHashMapWithExpectedSize(2);
+        serverProps.put(QueryServices.STATS_GUIDEPOST_WIDTH_BYTES_ATTRIB, Long.toString(20));
+        serverProps.put(QueryServices.MAX_SERVER_METADATA_CACHE_TIME_TO_LIVE_MS_ATTRIB, Long.toString(5));
         serverProps.put(QueryServices.EXTRA_JDBC_ARGUMENTS_ATTRIB,
             QueryServicesOptions.DEFAULT_EXTRA_JDBC_ARGUMENTS);
         Map<String, String> clientProps = Maps.newHashMapWithExpectedSize(2);
+        clientProps.put(QueryServices.USE_STATS_FOR_PARALLELIZATION, Boolean.toString(true));
+        clientProps.put(QueryServices.STATS_UPDATE_FREQ_MS_ATTRIB, Long.toString(5));
         clientProps.put(QueryServices.TRANSACTIONS_ENABLED, Boolean.TRUE.toString());
         clientProps.put(QueryServices.FORCE_ROW_KEY_ORDER_ATTRIB, Boolean.TRUE.toString());
         setUpTestDriver(new ReadOnlyProps(serverProps.entrySet().iterator()),


[12/28] phoenix git commit: PHOENIX-5013 Increase timeout for Tephra discovery service

Posted by pb...@apache.org.
PHOENIX-5013 Increase timeout for Tephra discovery service


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

Branch: refs/heads/4.x-cdh5.15
Commit: a0e98599b8ffeca26c1d316d59585ccc7df6daa9
Parents: b296ddc
Author: James Taylor <ja...@apache.org>
Authored: Sat Nov 10 19:07:02 2018 +0000
Committer: Pedro Boado <pb...@apache.org>
Committed: Tue Nov 27 15:11:39 2018 +0000

----------------------------------------------------------------------
 .../apache/phoenix/query/QueryServicesTestImpl.java   |  6 +++---
 .../test/java/org/apache/phoenix/util/TestUtil.java   | 14 ++++++++++++++
 2 files changed, 17 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/a0e98599/phoenix-core/src/test/java/org/apache/phoenix/query/QueryServicesTestImpl.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/query/QueryServicesTestImpl.java b/phoenix-core/src/test/java/org/apache/phoenix/query/QueryServicesTestImpl.java
index 841abb6..49fb8e8 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/query/QueryServicesTestImpl.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/query/QueryServicesTestImpl.java
@@ -25,8 +25,8 @@ import org.apache.hadoop.hbase.regionserver.wal.IndexedWALEditCodec;
 import org.apache.phoenix.transaction.OmidTransactionProvider;
 import org.apache.phoenix.util.PhoenixRuntime;
 import org.apache.phoenix.util.ReadOnlyProps;
+import org.apache.phoenix.util.TestUtil;
 import org.apache.tephra.TxConstants;
-import org.apache.twill.internal.utils.Networks;
 
 
 /**
@@ -129,12 +129,12 @@ public final class QueryServicesTestImpl extends BaseQueryServicesImpl {
                 .set(TxConstants.Manager.CFG_DO_PERSIST, false)
                 .set(TxConstants.Service.CFG_DATA_TX_CLIENT_RETRY_STRATEGY, "n-times")
                 .set(TxConstants.Service.CFG_DATA_TX_CLIENT_ATTEMPTS, 1)
-                .set(TxConstants.Service.CFG_DATA_TX_BIND_PORT, Networks.getRandomPort())
+                .set(TxConstants.Service.CFG_DATA_TX_BIND_PORT, TestUtil.getRandomPort())
                 .set(TxConstants.Manager.CFG_TX_SNAPSHOT_DIR, Files.createTempDir().getAbsolutePath())
                 .set(TxConstants.Manager.CFG_TX_TIMEOUT, DEFAULT_TXN_TIMEOUT_SECONDS)
                 .set(TxConstants.Manager.CFG_TX_SNAPSHOT_INTERVAL, 5L)
                 // setup default test configs for Omid
-                .set(OmidTransactionProvider.OMID_TSO_PORT, Networks.getRandomPort())
+                .set(OmidTransactionProvider.OMID_TSO_PORT, TestUtil.getRandomPort())
                 ;
     }
     

http://git-wip-us.apache.org/repos/asf/phoenix/blob/a0e98599/phoenix-core/src/test/java/org/apache/phoenix/util/TestUtil.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/util/TestUtil.java b/phoenix-core/src/test/java/org/apache/phoenix/util/TestUtil.java
index f0a26b9..f3faa0c 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/util/TestUtil.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/util/TestUtil.java
@@ -36,6 +36,7 @@ import static org.junit.Assert.fail;
 import java.io.File;
 import java.io.IOException;
 import java.math.BigDecimal;
+import java.net.ServerSocket;
 import java.sql.Connection;
 import java.sql.Date;
 import java.sql.DriverManager;
@@ -1105,4 +1106,17 @@ public class TestUtil {
         }
         return filteredData;
     }
+    
+    /**
+     * Find a random free port in localhost for binding.
+     * @return A port number or -1 for failure.
+     */
+    public static int getRandomPort() {
+        try (ServerSocket socket = new ServerSocket(0)) {
+            socket.setReuseAddress(true);
+            return socket.getLocalPort();
+        } catch (IOException e) {
+            return -1;
+        }
+    }
 }


[10/28] phoenix git commit: Revert "PHOENIX-4971 Drop index will execute successfully using Incorrect name of parent tables"

Posted by pb...@apache.org.
Revert "PHOENIX-4971 Drop index will execute successfully using Incorrect name of parent tables"

This reverts commit 7b5482367eb010b5b2db285ff8bc4b345863c477.


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

Branch: refs/heads/4.x-cdh5.15
Commit: 1da0ad70ee2d0c904d3d210c0f7584f03c102303
Parents: 1767244
Author: Thomas D'Silva <td...@apache.org>
Authored: Wed Nov 7 19:09:31 2018 +0000
Committer: Pedro Boado <pb...@apache.org>
Committed: Tue Nov 27 15:11:26 2018 +0000

----------------------------------------------------------------------
 .../phoenix/end2end/index/DropMetadataIT.java   | 24 +-------------------
 .../phoenix/exception/SQLExceptionCode.java     |  2 --
 .../apache/phoenix/schema/MetaDataClient.java   | 15 ------------
 3 files changed, 1 insertion(+), 40 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/1da0ad70/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/DropMetadataIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/DropMetadataIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/DropMetadataIT.java
index a285526..b92ed8d 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/DropMetadataIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/DropMetadataIT.java
@@ -18,13 +18,9 @@
 package org.apache.phoenix.end2end.index;
 
 import static org.apache.phoenix.util.TestUtil.HBASE_NATIVE_SCHEMA_NAME;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
 
 import java.sql.Connection;
 import java.sql.DriverManager;
-import java.sql.SQLException;
-
 import java.util.Properties;
 
 import org.apache.hadoop.hbase.HColumnDescriptor;
@@ -33,7 +29,6 @@ import org.apache.hadoop.hbase.client.HBaseAdmin;
 import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.phoenix.end2end.ParallelStatsDisabledIT;
-import org.apache.phoenix.exception.SQLExceptionCode;
 import org.apache.phoenix.jdbc.PhoenixConnection;
 import org.apache.phoenix.query.QueryServices;
 import org.apache.phoenix.util.PropertiesUtil;
@@ -61,24 +56,7 @@ public class DropMetadataIT extends ParallelStatsDisabledIT {
         String url = QueryUtil.getConnectionUrl(props, config, PRINCIPAL);
         return DriverManager.getConnection(url, props);
     }
-
-    @Test
-    public void testDropIndexTableHasSameNameWithDataTable() {
-        String tableName = generateUniqueName();
-        String indexName = "IDX_" + tableName;
-        try (Connection conn = DriverManager.getConnection(getUrl())) {
-            String createTable = "CREATE TABLE " + tableName + "  (id varchar not null primary key, col integer)";
-            conn.createStatement().execute(createTable);
-            String createIndex = "CREATE INDEX " + indexName + " on " + tableName + "(col)";
-            conn.createStatement().execute(createIndex);
-            String dropIndex = "DROP INDEX " + indexName + " on " + indexName;
-            conn.createStatement().execute(dropIndex);
-            fail("should not execute successfully");
-        } catch (SQLException e) {
-            assertTrue(SQLExceptionCode.PARENT_TABLE_NOT_FOUND.getErrorCode() == e.getErrorCode());
-        }
-    }
-
+    
     @Test
     public void testDropViewKeepsHTable() throws Exception {
         Connection conn = getConnection();

http://git-wip-us.apache.org/repos/asf/phoenix/blob/1da0ad70/phoenix-core/src/main/java/org/apache/phoenix/exception/SQLExceptionCode.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/exception/SQLExceptionCode.java b/phoenix-core/src/main/java/org/apache/phoenix/exception/SQLExceptionCode.java
index 5bffed5..d557714 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/exception/SQLExceptionCode.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/exception/SQLExceptionCode.java
@@ -185,8 +185,6 @@ public enum SQLExceptionCode {
      INVALID_REPLAY_AT(533, "42910", "Value of REPLAY_AT cannot be less than zero."),
      UNEQUAL_SCN_AND_BUILD_INDEX_AT(534, "42911", "If both specified, values of CURRENT_SCN and BUILD_INDEX_AT must be equal."),
      ONLY_INDEX_UPDATABLE_AT_SCN(535, "42912", "Only an index may be updated when the BUILD_INDEX_AT property is specified"),
-     PARENT_TABLE_NOT_FOUND(536, "42913", "Can't drop the index because the parent table in the DROP statement is incorrect."),
-
      /**
      * HBase and Phoenix specific implementation defined sub-classes.
      * Column family related exceptions.

http://git-wip-us.apache.org/repos/asf/phoenix/blob/1da0ad70/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java
index aae9fc4..3b68247 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java
@@ -23,7 +23,6 @@ import static org.apache.hadoop.hbase.HColumnDescriptor.TTL;
 import static org.apache.phoenix.coprocessor.BaseScannerRegionObserver.ANALYZE_TABLE;
 import static org.apache.phoenix.coprocessor.BaseScannerRegionObserver.RUN_UPDATE_STATS_ASYNC_ATTRIB;
 import static org.apache.phoenix.exception.SQLExceptionCode.INSUFFICIENT_MULTI_TENANT_COLUMNS;
-import static org.apache.phoenix.exception.SQLExceptionCode.PARENT_TABLE_NOT_FOUND;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.APPEND_ONLY_SCHEMA;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.ARG_POSITION;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.ARRAY_SIZE;
@@ -3125,20 +3124,6 @@ public class MetaDataClient {
     }
     MutationState dropTable(String schemaName, String tableName, String parentTableName, PTableType tableType,
             boolean ifExists, boolean cascade, boolean skipAddingParentColumns) throws SQLException {
-        // Checking the parent table whether exists
-        String fullTableName = SchemaUtil.getTableName(schemaName, tableName);
-        try {
-            PTable ptable = connection.getTable(new PTableKey(connection.getTenantId(), fullTableName));
-            if (parentTableName != null &&!parentTableName.equals(ptable.getParentTableName().getString())) {
-                throw new SQLExceptionInfo.Builder(PARENT_TABLE_NOT_FOUND)
-                        .setSchemaName(schemaName).setTableName(tableName).build().buildException();
-            }
-        } catch (TableNotFoundException e) {
-            if (!ifExists) {
-                throw e;
-            }
-        }
-
         connection.rollback();
         boolean wasAutoCommit = connection.getAutoCommit();
         PName tenantId = connection.getTenantId();


[22/28] phoenix git commit: PHOENIX-5005 Server-side delete / upsert-select potentially blocked after a split

Posted by pb...@apache.org.
PHOENIX-5005 Server-side delete / upsert-select potentially blocked after a split


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

Branch: refs/heads/4.x-cdh5.15
Commit: b20b21d101bf95e42c21350b778ebd5352be37f8
Parents: dd81989
Author: Vincent Poon <vi...@apache.org>
Authored: Thu Nov 8 23:38:20 2018 +0000
Committer: Pedro Boado <pb...@apache.org>
Committed: Tue Nov 27 15:12:08 2018 +0000

----------------------------------------------------------------------
 .../UngroupedAggregateRegionObserver.java       | 43 ++++++++++++--------
 1 file changed, 26 insertions(+), 17 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/b20b21d1/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/UngroupedAggregateRegionObserver.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/UngroupedAggregateRegionObserver.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/UngroupedAggregateRegionObserver.java
index 73386a2..26e338f 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/UngroupedAggregateRegionObserver.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/UngroupedAggregateRegionObserver.java
@@ -262,7 +262,7 @@ public class UngroupedAggregateRegionObserver extends BaseScannerRegionObserver
       // flush happen which decrease the memstore size and then writes allowed on the region.
       for (int i = 0; blockingMemstoreSize > 0 && region.getMemstoreSize() > blockingMemstoreSize && i < 30; i++) {
           try {
-              checkForRegionClosing();
+              checkForRegionClosingOrSplitting();
               Thread.sleep(100);
           } catch (InterruptedException e) {
               Thread.currentThread().interrupt();
@@ -311,7 +311,7 @@ public class UngroupedAggregateRegionObserver extends BaseScannerRegionObserver
      * a high chance that flush might not proceed and memstore won't be freed up.
      * @throws IOException
      */
-    private void checkForRegionClosing() throws IOException {
+    private void checkForRegionClosingOrSplitting() throws IOException {
         synchronized (lock) {
             if(isRegionClosingOrSplitting) {
                 lock.notifyAll();
@@ -1333,13 +1333,31 @@ public class UngroupedAggregateRegionObserver extends BaseScannerRegionObserver
     @Override
     public void preSplit(ObserverContext<RegionCoprocessorEnvironment> c, byte[] splitRow)
             throws IOException {
-        // Don't allow splitting if operations need read and write to same region are going on in the
-        // the coprocessors to avoid dead lock scenario. See PHOENIX-3111.
+        waitForScansToFinish(c);
+    }
+
+    // Don't allow splitting/closing if operations need read and write to same region are going on in the
+    // the coprocessors to avoid dead lock scenario. See PHOENIX-3111.
+    private void waitForScansToFinish(ObserverContext<RegionCoprocessorEnvironment> c) throws IOException {
+        int maxWaitTime = c.getEnvironment().getConfiguration().getInt(HConstants.HBASE_CLIENT_OPERATION_TIMEOUT,
+            HConstants.DEFAULT_HBASE_CLIENT_OPERATION_TIMEOUT);
+        long start = EnvironmentEdgeManager.currentTimeMillis();
         synchronized (lock) {
             isRegionClosingOrSplitting = true;
-            if (scansReferenceCount > 0) {
-                throw new IOException("Operations like local index building/delete/upsert select"
-                        + " might be going on so not allowing to split.");
+            while (scansReferenceCount > 0) {
+                try {
+                    lock.wait(1000);
+                    if (EnvironmentEdgeManager.currentTimeMillis() - start >= maxWaitTime) {
+                        isRegionClosingOrSplitting = false; // must reset in case split is not retried
+                        throw new IOException(String.format(
+                            "Operations like local index building/delete/upsert select"
+                                    + " might be going on so not allowing to split/close. scansReferenceCount=%s region=%s",
+                            scansReferenceCount,
+                            c.getEnvironment().getRegionInfo().getRegionNameAsString()));
+                    }
+                } catch (InterruptedException e) {
+                    Thread.currentThread().interrupt();
+                }
             }
         }
     }
@@ -1360,16 +1378,7 @@ public class UngroupedAggregateRegionObserver extends BaseScannerRegionObserver
     @Override
     public void preClose(ObserverContext<RegionCoprocessorEnvironment> c, boolean abortRequested)
             throws IOException {
-        synchronized (lock) {
-            isRegionClosingOrSplitting = true;
-            while (scansReferenceCount > 0) {
-                try {
-                    lock.wait(1000);
-                } catch (InterruptedException e) {
-                    Thread.currentThread().interrupt();
-                }
-            }
-        }
+        waitForScansToFinish(c);
     }
 
     @Override


[24/28] phoenix git commit: PHOENIX-5029 Increase parallelism of tests to decrease test time

Posted by pb...@apache.org.
PHOENIX-5029 Increase parallelism of tests to decrease test time


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

Branch: refs/heads/4.x-cdh5.15
Commit: d2e4a737e87faa2b7148404e73ae047236bd2dbc
Parents: 1a09ebf
Author: James Taylor <ja...@apache.org>
Authored: Sat Nov 17 23:18:39 2018 +0000
Committer: Pedro Boado <pb...@apache.org>
Committed: Tue Nov 27 15:12:13 2018 +0000

----------------------------------------------------------------------
 pom.xml | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/d2e4a737/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index b6577ec..bcd8130 100644
--- a/pom.xml
+++ b/pom.xml
@@ -165,7 +165,7 @@
 
     <!-- Plugin options -->
     <numForkedUT>8</numForkedUT>
-    <numForkedIT>4</numForkedIT>
+    <numForkedIT>7</numForkedIT>
     <it.failIfNoSpecifiedTests>false</it.failIfNoSpecifiedTests>
     <surefire.failIfNoSpecifiedTests>false</surefire.failIfNoSpecifiedTests>
 


[16/28] phoenix git commit: PHOENIX-5008 (Addendum): CQSI.init should not bubble up RetriableUpgradeException to client in case of an UpgradeRequiredException

Posted by pb...@apache.org.
PHOENIX-5008 (Addendum): CQSI.init should not bubble up RetriableUpgradeException to client in case of an UpgradeRequiredException


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

Branch: refs/heads/4.x-cdh5.15
Commit: bcf2cc7f69a4a107229a01e514c9f6ec7fe4d534
Parents: f33f7d7
Author: Chinmay Kulkarni <ch...@gmail.com>
Authored: Wed Nov 14 01:11:53 2018 +0000
Committer: Pedro Boado <pb...@apache.org>
Committed: Tue Nov 27 15:11:50 2018 +0000

----------------------------------------------------------------------
 .../phoenix/end2end/SystemCatalogCreationOnConnectionIT.java       | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/bcf2cc7f/phoenix-core/src/it/java/org/apache/phoenix/end2end/SystemCatalogCreationOnConnectionIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SystemCatalogCreationOnConnectionIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SystemCatalogCreationOnConnectionIT.java
index eadd391..7a5f80c 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SystemCatalogCreationOnConnectionIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SystemCatalogCreationOnConnectionIT.java
@@ -504,7 +504,7 @@ public class SystemCatalogCreationOnConnectionIT {
      */
     private Set<String> getHBaseTables() throws IOException {
         Set<String> tables = new HashSet<>();
-        for (TableName tn : testUtil.getAdmin().listTableNames()) {
+        for (TableName tn : testUtil.getHBaseAdmin().listTableNames()) {
             tables.add(tn.getNameAsString());
         }
         return tables;


[23/28] phoenix git commit: PHOENIX-5028 Delay acquisition of port and increase Tephra test discovery timeouts

Posted by pb...@apache.org.
PHOENIX-5028 Delay acquisition of port and increase Tephra test discovery timeouts


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

Branch: refs/heads/4.x-cdh5.15
Commit: 1a09ebf9d57c0dd50947cc33f1ec8415b54e6e9b
Parents: b20b21d
Author: James Taylor <ja...@apache.org>
Authored: Sat Nov 17 23:13:59 2018 +0000
Committer: Pedro Boado <pb...@apache.org>
Committed: Tue Nov 27 15:12:10 2018 +0000

----------------------------------------------------------------------
 .../end2end/ConnectionQueryServicesTestImpl.java   |  4 +++-
 .../transaction/OmidTransactionProvider.java       |  2 +-
 .../transaction/PhoenixTransactionProvider.java    |  2 +-
 .../transaction/TephraTransactionProvider.java     | 17 ++++++++++-------
 .../phoenix/query/QueryServicesTestImpl.java       |  3 ---
 5 files changed, 15 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/1a09ebf9/phoenix-core/src/it/java/org/apache/phoenix/end2end/ConnectionQueryServicesTestImpl.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ConnectionQueryServicesTestImpl.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ConnectionQueryServicesTestImpl.java
index 6ebaa65..969e0f4 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ConnectionQueryServicesTestImpl.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ConnectionQueryServicesTestImpl.java
@@ -35,6 +35,7 @@ import org.apache.phoenix.transaction.PhoenixTransactionService;
 import org.apache.phoenix.transaction.TransactionFactory;
 import org.apache.phoenix.transaction.TransactionFactory.Provider;
 import org.apache.phoenix.util.SQLCloseables;
+import org.apache.phoenix.util.TestUtil;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -102,7 +103,8 @@ public class ConnectionQueryServicesTestImpl extends ConnectionQueryServicesImpl
     public synchronized PhoenixTransactionClient initTransactionClient(Provider provider) throws SQLException {
         PhoenixTransactionService txService = txServices[provider.ordinal()];
         if (txService == null) {
-            txService = txServices[provider.ordinal()] = provider.getTransactionProvider().getTransactionService(config, connectionInfo);
+            int port = TestUtil.getRandomPort();
+            txService = txServices[provider.ordinal()] = provider.getTransactionProvider().getTransactionService(config, connectionInfo, port);
         }
         return super.initTransactionClient(provider);
     }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/1a09ebf9/phoenix-core/src/main/java/org/apache/phoenix/transaction/OmidTransactionProvider.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/transaction/OmidTransactionProvider.java b/phoenix-core/src/main/java/org/apache/phoenix/transaction/OmidTransactionProvider.java
index c53215c..bace2bc 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/transaction/OmidTransactionProvider.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/transaction/OmidTransactionProvider.java
@@ -72,7 +72,7 @@ public class OmidTransactionProvider implements PhoenixTransactionProvider {
     }
 
     @Override
-    public PhoenixTransactionService getTransactionService(Configuration config, ConnectionInfo connectionInfo) throws  SQLException{
+    public PhoenixTransactionService getTransactionService(Configuration config, ConnectionInfo connectionInfo, int port) throws  SQLException{
         return new OmidTransactionService();
     }
 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/1a09ebf9/phoenix-core/src/main/java/org/apache/phoenix/transaction/PhoenixTransactionProvider.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/transaction/PhoenixTransactionProvider.java b/phoenix-core/src/main/java/org/apache/phoenix/transaction/PhoenixTransactionProvider.java
index b7f660e..3af554b 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/transaction/PhoenixTransactionProvider.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/transaction/PhoenixTransactionProvider.java
@@ -50,7 +50,7 @@ public interface PhoenixTransactionProvider {
     public PhoenixTransactionContext getTransactionContext(PhoenixConnection connection) throws SQLException;
 
     public PhoenixTransactionClient getTransactionClient(Configuration config, ConnectionInfo connectionInfo) throws SQLException;
-    public PhoenixTransactionService getTransactionService(Configuration config, ConnectionInfo connectionInfo) throws  SQLException;
+    public PhoenixTransactionService getTransactionService(Configuration config, ConnectionInfo connectionInfo, int port) throws  SQLException;
     public Class<? extends RegionObserver> getCoprocessor();
     public Class<? extends RegionObserver> getGCCoprocessor();
 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/1a09ebf9/phoenix-core/src/main/java/org/apache/phoenix/transaction/TephraTransactionProvider.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/transaction/TephraTransactionProvider.java b/phoenix-core/src/main/java/org/apache/phoenix/transaction/TephraTransactionProvider.java
index 70937cf..7025e5b 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/transaction/TephraTransactionProvider.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/transaction/TephraTransactionProvider.java
@@ -89,16 +89,17 @@ public class TephraTransactionProvider implements PhoenixTransactionProvider {
         }
 
         int timeOut = config.getInt(HConstants.ZK_SESSION_TIMEOUT, HConstants.DEFAULT_ZK_SESSION_TIMEOUT);
+        int retryTimeOut = config.getInt(TxConstants.Service.CFG_DATA_TX_CLIENT_DISCOVERY_TIMEOUT_SEC, 
+                TxConstants.Service.DEFAULT_DATA_TX_CLIENT_DISCOVERY_TIMEOUT_SEC);
         // Create instance of the tephra zookeeper client
         ZKClientService zkClientService  = ZKClientServices.delegate(
             ZKClients.reWatchOnExpire(
                 ZKClients.retryOnFailure(
                      new TephraZKClientService(zkQuorumServersString, timeOut, null,
                              ArrayListMultimap.<String, byte[]>create()), 
-                         RetryStrategies.exponentialDelay(500, 2000, TimeUnit.MILLISECONDS))
+                         RetryStrategies.exponentialDelay(500, retryTimeOut, TimeUnit.MILLISECONDS))
                      )
                 );
-        //txZKClientService.startAndWait();
         ZKDiscoveryService zkDiscoveryService = new ZKDiscoveryService(zkClientService);
         PooledClientProvider pooledClientProvider = new PooledClientProvider(
                 config, zkDiscoveryService);
@@ -110,7 +111,10 @@ public class TephraTransactionProvider implements PhoenixTransactionProvider {
     }
 
     @Override
-    public PhoenixTransactionService getTransactionService(Configuration config, ConnectionInfo connInfo) {
+    public PhoenixTransactionService getTransactionService(Configuration config, ConnectionInfo connInfo, int port) {
+        config.setInt(TxConstants.Service.CFG_DATA_TX_BIND_PORT, port);
+        int retryTimeOut = config.getInt(TxConstants.Service.CFG_DATA_TX_CLIENT_DISCOVERY_TIMEOUT_SEC, 
+                TxConstants.Service.DEFAULT_DATA_TX_CLIENT_DISCOVERY_TIMEOUT_SEC);
         ZKClientService zkClient = ZKClientServices.delegate(
           ZKClients.reWatchOnExpire(
             ZKClients.retryOnFailure(
@@ -118,17 +122,16 @@ public class TephraTransactionProvider implements PhoenixTransactionProvider {
                 .setSessionTimeout(config.getInt(HConstants.ZK_SESSION_TIMEOUT,
                         HConstants.DEFAULT_ZK_SESSION_TIMEOUT))
                 .build(),
-              RetryStrategies.exponentialDelay(500, 2000, TimeUnit.MILLISECONDS)
+              RetryStrategies.exponentialDelay(500, retryTimeOut, TimeUnit.MILLISECONDS)
             )
           )
         );
 
-        //zkClient.startAndWait();
         DiscoveryService discovery = new ZKDiscoveryService(zkClient);
-        TransactionManager txManager = new TransactionManager(config, new HDFSTransactionStateStorage(config, new SnapshotCodecProvider(config), new TxMetricsCollector()), new TxMetricsCollector());
+        TransactionManager txManager = new TransactionManager(config, new HDFSTransactionStateStorage(config, 
+                new SnapshotCodecProvider(config), new TxMetricsCollector()), new TxMetricsCollector());
         TransactionService txService = new TransactionService(config, zkClient, discovery, Providers.of(txManager));
         TephraTransactionService service = new TephraTransactionService(zkClient, txService);
-        //txService.startAndWait();            
         service.start();
         return service;
     }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/1a09ebf9/phoenix-core/src/test/java/org/apache/phoenix/query/QueryServicesTestImpl.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/query/QueryServicesTestImpl.java b/phoenix-core/src/test/java/org/apache/phoenix/query/QueryServicesTestImpl.java
index eae951a..2049390 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/query/QueryServicesTestImpl.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/query/QueryServicesTestImpl.java
@@ -129,13 +129,10 @@ public final class QueryServicesTestImpl extends BaseQueryServicesImpl {
                 .set(TxConstants.Manager.CFG_DO_PERSIST, false)
                 .set(TxConstants.Service.CFG_DATA_TX_CLIENT_RETRY_STRATEGY, "n-times")
                 .set(TxConstants.Service.CFG_DATA_TX_CLIENT_ATTEMPTS, 1)
-                .set(TxConstants.Service.CFG_DATA_TX_BIND_PORT, TestUtil.getRandomPort())
                 .set(TxConstants.Service.CFG_DATA_TX_CLIENT_DISCOVERY_TIMEOUT_SEC, 60)
                 .set(TxConstants.Manager.CFG_TX_SNAPSHOT_DIR, Files.createTempDir().getAbsolutePath())
                 .set(TxConstants.Manager.CFG_TX_TIMEOUT, DEFAULT_TXN_TIMEOUT_SECONDS)
                 .set(TxConstants.Manager.CFG_TX_SNAPSHOT_INTERVAL, 5L)
-                // setup default test configs for Omid
-                .set(OmidTransactionProvider.OMID_TSO_PORT, TestUtil.getRandomPort())
                 ;
     }
     


[27/28] phoenix git commit: PHOENIX-5026 Addendum; test-fix.

Posted by pb...@apache.org.
PHOENIX-5026 Addendum; test-fix.


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

Branch: refs/heads/4.x-cdh5.15
Commit: 027d21e2a87aadaae030d9a06fc25ec8a59e4267
Parents: f6b7594
Author: Lars Hofhansl <la...@apache.org>
Authored: Thu Nov 22 21:11:19 2018 +0000
Committer: Pedro Boado <pb...@apache.org>
Committed: Tue Nov 27 15:12:22 2018 +0000

----------------------------------------------------------------------
 .../java/org/apache/phoenix/end2end/UpsertSelectAutoCommitIT.java  | 2 ++
 1 file changed, 2 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/027d21e2/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpsertSelectAutoCommitIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpsertSelectAutoCommitIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpsertSelectAutoCommitIT.java
index c56296c..6fad376 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpsertSelectAutoCommitIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpsertSelectAutoCommitIT.java
@@ -192,6 +192,8 @@ public class UpsertSelectAutoCommitIT extends ParallelStatsDisabledIT {
             int upsertCount = stmt.executeUpdate();
             assertEquals((int)Math.pow(2, i), upsertCount);
         }
+        // cleanup after ourselves
+        conn.createStatement().execute("DROP SEQUENCE keys");
         admin.close();
         conn.close();
     }


[20/28] phoenix git commit: PHOENIX-5024 - Cleanup anonymous inner classes in PostDDLCompiler

Posted by pb...@apache.org.
PHOENIX-5024 - Cleanup anonymous inner classes in PostDDLCompiler


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

Branch: refs/heads/4.x-cdh5.15
Commit: 590f88bdc0d93771e0659f0e20f67da0d99e001d
Parents: b28a241
Author: Geoffrey Jacoby <gj...@apache.org>
Authored: Fri Nov 16 17:55:49 2018 +0000
Committer: Pedro Boado <pb...@apache.org>
Committed: Tue Nov 27 15:12:02 2018 +0000

----------------------------------------------------------------------
 .../apache/phoenix/compile/PostDDLCompiler.java | 478 ++++++++++---------
 1 file changed, 258 insertions(+), 220 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/590f88bd/phoenix-core/src/main/java/org/apache/phoenix/compile/PostDDLCompiler.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/PostDDLCompiler.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/PostDDLCompiler.java
index 709534e..a74c5f1 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/PostDDLCompiler.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/PostDDLCompiler.java
@@ -87,248 +87,286 @@ public class PostDDLCompiler {
             final long timestamp) throws SQLException {
         PhoenixStatement statement = new PhoenixStatement(connection);
         final StatementContext context = new StatementContext(
-                statement, 
-                new ColumnResolver() {
+                statement,
+            new MultipleTableRefColumnResolver(tableRefs),
+                scan,
+                new SequenceManager(statement));
+        return new PostDDLMutationPlan(context, tableRefs, timestamp, emptyCF, deleteList, projectCFs);
+    }
 
-                    @Override
-                    public List<TableRef> getTables() {
-                        return tableRefs;
-                    }
+    private static class MultipleTableRefColumnResolver implements ColumnResolver {
 
-                    @Override
-                    public TableRef resolveTable(String schemaName, String tableName) throws SQLException {
-                        throw new UnsupportedOperationException();
-                    }
+        private final List<TableRef> tableRefs;
 
-                    @Override
-                    public ColumnRef resolveColumn(String schemaName, String tableName, String colName)
-                            throws SQLException {
-                        throw new UnsupportedOperationException();
-                    }
+        public MultipleTableRefColumnResolver(List<TableRef> tableRefs) {
+            this.tableRefs = tableRefs;
+        }
 
-					@Override
-					public List<PFunction> getFunctions() {
-						return Collections.<PFunction>emptyList();
-					}
-
-					@Override
-					public PFunction resolveFunction(String functionName)
-							throws SQLException {
-						throw new FunctionNotFoundException(functionName);
-					}
-
-					@Override
-					public boolean hasUDFs() {
-						return false;
-					}
-
-					@Override
-					public PSchema resolveSchema(String schemaName) throws SQLException {
-						throw new SchemaNotFoundException(schemaName);
-					}
-
-					@Override
-					public List<PSchema> getSchemas() {
-						throw new UnsupportedOperationException();
-					}
-                    
-                },
-                scan,
-                new SequenceManager(statement));
-        return new BaseMutationPlan(context, Operation.UPSERT /* FIXME */) {
-            
-            @Override
-            public MutationState execute() throws SQLException {
-                if (tableRefs.isEmpty()) {
-                    return new MutationState(0, 1000, connection);
-                }
-                boolean wasAutoCommit = connection.getAutoCommit();
-                try {
-                    connection.setAutoCommit(true);
-                    SQLException sqlE = null;
-                    /*
-                     * Handles:
-                     * 1) deletion of all rows for a DROP TABLE and subsequently deletion of all rows for a DROP INDEX;
-                     * 2) deletion of all column values for a ALTER TABLE DROP COLUMN
-                     * 3) updating the necessary rows to have an empty KV
-                     * 4) updating table stats
-                     */
-                    long totalMutationCount = 0;
-                    for (final TableRef tableRef : tableRefs) {
-                        Scan scan = ScanUtil.newScan(context.getScan());
-                        SelectStatement select = SelectStatement.COUNT_ONE;
-                        // We need to use this tableRef
-                        ColumnResolver resolver = new ColumnResolver() {
-                            @Override
-                            public List<TableRef> getTables() {
-                                return Collections.singletonList(tableRef);
-                            }
-                            
-                            @Override
-                            public java.util.List<PFunction> getFunctions() {
-                                return Collections.emptyList();
-                            };
-                            
-                            @Override
-                            public TableRef resolveTable(String schemaName, String tableName)
-                                    throws SQLException {
-                                throw new UnsupportedOperationException();
-                            }
-                            @Override
-                            public ColumnRef resolveColumn(String schemaName, String tableName, String colName) throws SQLException {
-                                PColumn column = tableName != null
-                                        ? tableRef.getTable().getColumnFamily(tableName).getPColumnForColumnName(colName)
-                                        : tableRef.getTable().getColumnForColumnName(colName);
-                                return new ColumnRef(tableRef, column.getPosition());
-                            }
-                            
-                            @Override
-                            public PFunction resolveFunction(String functionName) throws SQLException {
-                                throw new UnsupportedOperationException();
-                            };
-
-                            @Override
-                            public boolean hasUDFs() {
-                                return false;
-                            }
+        @Override
+        public List<TableRef> getTables() {
+            return tableRefs;
+        }
 
-                            @Override
-                            public List<PSchema> getSchemas() {
-                                throw new UnsupportedOperationException();
-                            }
+        @Override
+        public TableRef resolveTable(String schemaName, String tableName) throws SQLException {
+            throw new UnsupportedOperationException();
+        }
 
-                            @Override
-                            public PSchema resolveSchema(String schemaName) throws SQLException {
-                                throw new SchemaNotFoundException(schemaName);
-                            }
-                        };
-                        PhoenixStatement statement = new PhoenixStatement(connection);
-                        StatementContext context = new StatementContext(statement, resolver, scan, new SequenceManager(statement));
-                        long ts = timestamp;
-                        // FIXME: DDL operations aren't transactional, so we're basing the timestamp on a server timestamp.
-                        // Not sure what the fix should be. We don't need conflict detection nor filtering of invalid transactions
-                        // in this case, so maybe this is ok.
-                        if (ts!=HConstants.LATEST_TIMESTAMP && tableRef.getTable().isTransactional()) {
-                            ts = TransactionUtil.convertToNanoseconds(ts);
-                        }
-                        ScanUtil.setTimeRange(scan, scan.getTimeRange().getMin(), ts);
-                        if (emptyCF != null) {
-                            scan.setAttribute(BaseScannerRegionObserver.EMPTY_CF, emptyCF);
-                            scan.setAttribute(BaseScannerRegionObserver.EMPTY_COLUMN_QUALIFIER, EncodedColumnsUtil.getEmptyKeyValueInfo(tableRef.getTable()).getFirst());
-                        }
-                        ServerCache cache = null;
-                        try {
-                            if (deleteList != null) {
-                                if (deleteList.isEmpty()) {
-                                    scan.setAttribute(BaseScannerRegionObserver.DELETE_AGG, QueryConstants.TRUE);
-                                    // In the case of a row deletion, add index metadata so mutable secondary indexing works
-                                    /* TODO: we currently manually run a scan to delete the index data here
-                                    ImmutableBytesWritable ptr = context.getTempPtr();
-                                    tableRef.getTable().getIndexMaintainers(ptr);
-                                    if (ptr.getLength() > 0) {
-                                        IndexMetaDataCacheClient client = new IndexMetaDataCacheClient(connection, tableRef);
-                                        cache = client.addIndexMetadataCache(context.getScanRanges(), ptr);
-                                        byte[] uuidValue = cache.getId();
-                                        scan.setAttribute(PhoenixIndexCodec.INDEX_UUID, uuidValue);
-                                    }
-                                    */
-                                } else {
-                                    // In the case of the empty key value column family changing, do not send the index
-                                    // metadata, as we're currently managing this from the client. It's possible for the
-                                    // data empty column family to stay the same, while the index empty column family
-                                    // changes.
-                                    PColumn column = deleteList.get(0);
-                                    byte[] cq = column.getColumnQualifierBytes();
-                                    if (emptyCF == null) {
-                                        scan.addColumn(column.getFamilyName().getBytes(), cq);
-                                    }
-                                    scan.setAttribute(BaseScannerRegionObserver.DELETE_CF, column.getFamilyName().getBytes());
-                                    scan.setAttribute(BaseScannerRegionObserver.DELETE_CQ, cq);
-                                }
-                            }
-                            List<byte[]> columnFamilies = Lists.newArrayListWithExpectedSize(tableRef.getTable().getColumnFamilies().size());
-                            if (projectCFs == null) {
-                                for (PColumnFamily family : tableRef.getTable().getColumnFamilies()) {
-                                    columnFamilies.add(family.getName().getBytes());
+        @Override
+        public ColumnRef resolveColumn(String schemaName, String tableName, String colName)
+                throws SQLException {
+            throw new UnsupportedOperationException();
+        }
+
+        @Override
+        public List<PFunction> getFunctions() {
+            return Collections.<PFunction>emptyList();
+        }
+
+        @Override
+        public PFunction resolveFunction(String functionName)
+            throws SQLException {
+            throw new FunctionNotFoundException(functionName);
+        }
+
+        @Override
+        public boolean hasUDFs() {
+            return false;
+        }
+
+        @Override
+        public PSchema resolveSchema(String schemaName) throws SQLException {
+            throw new SchemaNotFoundException(schemaName);
+        }
+
+        @Override
+        public List<PSchema> getSchemas() {
+            throw new UnsupportedOperationException();
+        }
+
+    }
+
+    private class PostDDLMutationPlan extends BaseMutationPlan {
+
+        private final StatementContext context;
+        private final List<TableRef> tableRefs;
+        private final long timestamp;
+        private final byte[] emptyCF;
+        private final List<PColumn> deleteList;
+        private final List<byte[]> projectCFs;
+
+        public PostDDLMutationPlan(StatementContext context, List<TableRef> tableRefs, long timestamp, byte[] emptyCF, List<PColumn> deleteList, List<byte[]> projectCFs) {
+            super(context, Operation.UPSERT);
+            this.context = context;
+            this.tableRefs = tableRefs;
+            this.timestamp = timestamp;
+            this.emptyCF = emptyCF;
+            this.deleteList = deleteList;
+            this.projectCFs = projectCFs;
+        }
+
+        @Override
+        public MutationState execute() throws SQLException {
+            if (tableRefs.isEmpty()) {
+                return new MutationState(0, 1000, connection);
+            }
+            boolean wasAutoCommit = connection.getAutoCommit();
+            try {
+                connection.setAutoCommit(true);
+                SQLException sqlE = null;
+                /*
+                 * Handles:
+                 * 1) deletion of all rows for a DROP TABLE and subsequently deletion of all rows for a DROP INDEX;
+                 * 2) deletion of all column values for a ALTER TABLE DROP COLUMN
+                 * 3) updating the necessary rows to have an empty KV
+                 * 4) updating table stats
+                 */
+                long totalMutationCount = 0;
+                for (final TableRef tableRef : tableRefs) {
+                    Scan scan = ScanUtil.newScan(context.getScan());
+                    SelectStatement select = SelectStatement.COUNT_ONE;
+                    // We need to use this tableRef
+                    ColumnResolver resolver = new SingleTableRefColumnResolver(tableRef);
+                    PhoenixStatement statement = new PhoenixStatement(connection);
+                    StatementContext context = new StatementContext(statement, resolver, scan, new SequenceManager(statement));
+                    long ts = timestamp;
+                    // FIXME: DDL operations aren't transactional, so we're basing the timestamp on a server timestamp.
+                    // Not sure what the fix should be. We don't need conflict detection nor filtering of invalid transactions
+                    // in this case, so maybe this is ok.
+                    if (ts!= HConstants.LATEST_TIMESTAMP && tableRef.getTable().isTransactional()) {
+                        ts = TransactionUtil.convertToNanoseconds(ts);
+                    }
+                    ScanUtil.setTimeRange(scan, scan.getTimeRange().getMin(), ts);
+                    if (emptyCF != null) {
+                        scan.setAttribute(BaseScannerRegionObserver.EMPTY_CF, emptyCF);
+                        scan.setAttribute(BaseScannerRegionObserver.EMPTY_COLUMN_QUALIFIER, EncodedColumnsUtil.getEmptyKeyValueInfo(tableRef.getTable()).getFirst());
+                    }
+                    ServerCache cache = null;
+                    try {
+                        if (deleteList != null) {
+                            if (deleteList.isEmpty()) {
+                                scan.setAttribute(BaseScannerRegionObserver.DELETE_AGG, QueryConstants.TRUE);
+                                // In the case of a row deletion, add index metadata so mutable secondary indexing works
+                                /* TODO: we currently manually run a scan to delete the index data here
+                                ImmutableBytesWritable ptr = context.getTempPtr();
+                                tableRef.getTable().getIndexMaintainers(ptr);
+                                if (ptr.getLength() > 0) {
+                                    IndexMetaDataCacheClient client = new IndexMetaDataCacheClient(connection, tableRef);
+                                    cache = client.addIndexMetadataCache(context.getScanRanges(), ptr);
+                                    byte[] uuidValue = cache.getId();
+                                    scan.setAttribute(PhoenixIndexCodec.INDEX_UUID, uuidValue);
                                 }
+                                */
                             } else {
-                                for (byte[] projectCF : projectCFs) {
-                                    columnFamilies.add(projectCF);
+                                // In the case of the empty key value column family changing, do not send the index
+                                // metadata, as we're currently managing this from the client. It's possible for the
+                                // data empty column family to stay the same, while the index empty column family
+                                // changes.
+                                PColumn column = deleteList.get(0);
+                                byte[] cq = column.getColumnQualifierBytes();
+                                if (emptyCF == null) {
+                                    scan.addColumn(column.getFamilyName().getBytes(), cq);
                                 }
+                                scan.setAttribute(BaseScannerRegionObserver.DELETE_CF, column.getFamilyName().getBytes());
+                                scan.setAttribute(BaseScannerRegionObserver.DELETE_CQ, cq);
                             }
-                            // Need to project all column families into the scan, since we haven't yet created our empty key value
-                            RowProjector projector = ProjectionCompiler.compile(context, SelectStatement.COUNT_ONE, GroupBy.EMPTY_GROUP_BY);
-                            context.getAggregationManager().compile(context, GroupBy.EMPTY_GROUP_BY);
-                            // Explicitly project these column families and don't project the empty key value,
-                            // since at this point we haven't added the empty key value everywhere.
-                            if (columnFamilies != null) {
-                                scan.getFamilyMap().clear();
-                                for (byte[] family : columnFamilies) {
-                                    scan.addFamily(family);
-                                }
-                                projector = new RowProjector(projector,false);
+                        }
+                        List<byte[]> columnFamilies = Lists.newArrayListWithExpectedSize(tableRef.getTable().getColumnFamilies().size());
+                        if (projectCFs == null) {
+                            for (PColumnFamily family : tableRef.getTable().getColumnFamilies()) {
+                                columnFamilies.add(family.getName().getBytes());
                             }
-                            // Ignore exceptions due to not being able to resolve any view columns,
-                            // as this just means the view is invalid. Continue on and try to perform
-                            // any other Post DDL operations.
-                            try {
-                                // Since dropping a VIEW does not affect the underlying data, we do
-                                // not need to pass through the view statement here.
-                                WhereCompiler.compile(context, select); // Push where clause into scan
-                            } catch (ColumnFamilyNotFoundException e) {
-                                continue;
-                            } catch (ColumnNotFoundException e) {
-                                continue;
-                            } catch (AmbiguousColumnException e) {
-                                continue;
+                        } else {
+                            for (byte[] projectCF : projectCFs) {
+                                columnFamilies.add(projectCF);
+                            }
+                        }
+                        // Need to project all column families into the scan, since we haven't yet created our empty key value
+                        RowProjector projector = ProjectionCompiler.compile(context, SelectStatement.COUNT_ONE, GroupBy.EMPTY_GROUP_BY);
+                        context.getAggregationManager().compile(context, GroupBy.EMPTY_GROUP_BY);
+                        // Explicitly project these column families and don't project the empty key value,
+                        // since at this point we haven't added the empty key value everywhere.
+                        if (columnFamilies != null) {
+                            scan.getFamilyMap().clear();
+                            for (byte[] family : columnFamilies) {
+                                scan.addFamily(family);
                             }
-                            QueryPlan plan = new AggregatePlan(context, select, tableRef, projector, null, null,
-                                    OrderBy.EMPTY_ORDER_BY, null, GroupBy.EMPTY_GROUP_BY, null, null);
+                            projector = new RowProjector(projector,false);
+                        }
+                        // Ignore exceptions due to not being able to resolve any view columns,
+                        // as this just means the view is invalid. Continue on and try to perform
+                        // any other Post DDL operations.
+                        try {
+                            // Since dropping a VIEW does not affect the underlying data, we do
+                            // not need to pass through the view statement here.
+                            WhereCompiler.compile(context, select); // Push where clause into scan
+                        } catch (ColumnFamilyNotFoundException e) {
+                            continue;
+                        } catch (ColumnNotFoundException e) {
+                            continue;
+                        } catch (AmbiguousColumnException e) {
+                            continue;
+                        }
+                        QueryPlan plan = new AggregatePlan(context, select, tableRef, projector, null, null,
+                                OrderBy.EMPTY_ORDER_BY, null, GroupBy.EMPTY_GROUP_BY, null, null);
+                        try {
+                            ResultIterator iterator = plan.iterator();
                             try {
-                                ResultIterator iterator = plan.iterator();
+                                Tuple row = iterator.next();
+                                ImmutableBytesWritable ptr = context.getTempPtr();
+                                totalMutationCount += (Long)projector.getColumnProjector(0).getValue(row, PLong.INSTANCE, ptr);
+                            } catch (SQLException e) {
+                                sqlE = e;
+                            } finally {
                                 try {
-                                    Tuple row = iterator.next();
-                                    ImmutableBytesWritable ptr = context.getTempPtr();
-                                    totalMutationCount += (Long)projector.getColumnProjector(0).getValue(row, PLong.INSTANCE, ptr);
+                                    iterator.close();
                                 } catch (SQLException e) {
-                                    sqlE = e;
+                                    if (sqlE == null) {
+                                        sqlE = e;
+                                    } else {
+                                        sqlE.setNextException(e);
+                                    }
                                 } finally {
-                                    try {
-                                        iterator.close();
-                                    } catch (SQLException e) {
-                                        if (sqlE == null) {
-                                            sqlE = e;
-                                        } else {
-                                            sqlE.setNextException(e);
-                                        }
-                                    } finally {
-                                        if (sqlE != null) {
-                                            throw sqlE;
-                                        }
+                                    if (sqlE != null) {
+                                        throw sqlE;
                                     }
                                 }
-                            } catch (TableNotFoundException e) {
-                                // Ignore and continue, as HBase throws when table hasn't been written to
-                                // FIXME: Remove if this is fixed in 0.96
-                            }
-                        } finally {
-                            if (cache != null) { // Remove server cache if there is one
-                                cache.close();
                             }
+                        } catch (TableNotFoundException e) {
+                            // Ignore and continue, as HBase throws when table hasn't been written to
+                            // FIXME: Remove if this is fixed in 0.96
                         }
-                        
-                    }
-                    final long count = totalMutationCount;
-                    return new MutationState(1, 1000, connection) {
-                        @Override
-                        public long getUpdateCount() {
-                            return count;
+                    } finally {
+                        if (cache != null) { // Remove server cache if there is one
+                            cache.close();
                         }
-                    };
-                } finally {
-                    if (!wasAutoCommit) connection.setAutoCommit(wasAutoCommit);
+                    }
+
                 }
+                final long count = totalMutationCount;
+                return new MutationState(1, 1000, connection) {
+                    @Override
+                    public long getUpdateCount() {
+                        return count;
+                    }
+                };
+            } finally {
+                if (!wasAutoCommit) connection.setAutoCommit(wasAutoCommit);
+            }
+        }
+
+        private class SingleTableRefColumnResolver implements ColumnResolver {
+            private final TableRef tableRef;
+
+            public SingleTableRefColumnResolver(TableRef tableRef) {
+                this.tableRef = tableRef;
+            }
+
+            @Override
+            public List<TableRef> getTables() {
+                return Collections.singletonList(tableRef);
+            }
+
+            @Override
+            public List<PFunction> getFunctions() {
+                return Collections.emptyList();
+            }
+
+            ;
+
+            @Override
+            public TableRef resolveTable(String schemaName, String tableName)
+                    throws SQLException {
+                throw new UnsupportedOperationException();
+            }
+
+            @Override
+            public ColumnRef resolveColumn(String schemaName, String tableName, String colName) throws SQLException {
+                PColumn column = tableName != null
+                        ? tableRef.getTable().getColumnFamily(tableName).getPColumnForColumnName(colName)
+                        : tableRef.getTable().getColumnForColumnName(colName);
+                return new ColumnRef(tableRef, column.getPosition());
+            }
+
+            @Override
+            public PFunction resolveFunction(String functionName) throws SQLException {
+                throw new UnsupportedOperationException();
+            }
+
+            @Override
+            public boolean hasUDFs() {
+                return false;
+            }
+
+            @Override
+            public List<PSchema> getSchemas() {
+                throw new UnsupportedOperationException();
+            }
+
+            @Override
+            public PSchema resolveSchema(String schemaName) throws SQLException {
+                throw new SchemaNotFoundException(schemaName);
             }
-        };
+        }
     }
 }
\ No newline at end of file


[13/28] phoenix git commit: PHOENIX-5010 Don't build client guidepost cache when phoenix.stats.collection.enabled is disabled

Posted by pb...@apache.org.
PHOENIX-5010 Don't build client guidepost cache when phoenix.stats.collection.enabled is disabled


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

Branch: refs/heads/4.x-cdh5.15
Commit: 21c3a7c2e9cd4d4f59623dd987c6602304ac9335
Parents: a0e9859
Author: Ankit Singhal <an...@gmail.com>
Authored: Tue Nov 13 19:36:26 2018 +0000
Committer: Pedro Boado <pb...@apache.org>
Committed: Tue Nov 27 15:11:41 2018 +0000

----------------------------------------------------------------------
 .../org/apache/phoenix/query/GuidePostsCache.java | 18 +++++++++++++++++-
 1 file changed, 17 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/21c3a7c2/phoenix-core/src/main/java/org/apache/phoenix/query/GuidePostsCache.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/GuidePostsCache.java b/phoenix-core/src/main/java/org/apache/phoenix/query/GuidePostsCache.java
index d27be1b..1d9fa36 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/query/GuidePostsCache.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/GuidePostsCache.java
@@ -16,6 +16,10 @@
  */
 package org.apache.phoenix.query;
 
+import static org.apache.phoenix.query.QueryServices.STATS_COLLECTION_ENABLED;
+import static org.apache.phoenix.query.QueryServices.STATS_ENABLED_ATTRIB;
+import static org.apache.phoenix.query.QueryServicesOptions.DEFAULT_STATS_COLLECTION_ENABLED;
+
 import java.io.IOException;
 import java.util.List;
 import java.util.Objects;
@@ -66,6 +70,8 @@ public class GuidePostsCache {
         final long maxTableStatsCacheSize = config.getLong(
                 QueryServices.STATS_MAX_CACHE_SIZE,
                 QueryServicesOptions.DEFAULT_STATS_MAX_CACHE_SIZE);
+		final boolean isStatsEnabled = config.getBoolean(STATS_COLLECTION_ENABLED, DEFAULT_STATS_COLLECTION_ENABLED)
+				&& config.getBoolean(STATS_ENABLED_ATTRIB, true);
         cache = CacheBuilder.newBuilder()
                 // Expire entries a given amount of time after they were written
                 .expireAfterWrite(statsUpdateFrequency, TimeUnit.MILLISECONDS)
@@ -80,7 +86,7 @@ public class GuidePostsCache {
                 // Log removals at TRACE for debugging
                 .removalListener(new PhoenixStatsCacheRemovalListener())
                 // Automatically load the cache when entries are missing
-                .build(new StatsLoader());
+                .build(isStatsEnabled ? new StatsLoader() : new EmptyStatsLoader());
     }
 
     /**
@@ -129,6 +135,16 @@ public class GuidePostsCache {
     }
 
     /**
+     * Empty stats loader if stats are disabled
+     */
+	protected class EmptyStatsLoader extends CacheLoader<GuidePostsKey, GuidePostsInfo> {
+		@Override
+		public GuidePostsInfo load(GuidePostsKey statsKey) throws Exception {
+			return GuidePostsInfo.NO_GUIDEPOST;
+		}
+	}
+
+    /**
      * Returns the underlying cache. Try to use the provided methods instead of accessing the cache
      * directly.
      */


[26/28] phoenix git commit: PHOENIX-5026 Add client setting to disable server side mutations.

Posted by pb...@apache.org.
PHOENIX-5026 Add client setting to disable server side mutations.


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

Branch: refs/heads/4.x-cdh5.15
Commit: f6b75942701dbf90d7dc3d69be6265130e69ff94
Parents: bb17957
Author: Lars Hofhansl <la...@apache.org>
Authored: Thu Nov 22 03:53:14 2018 +0000
Committer: Pedro Boado <pb...@apache.org>
Committed: Tue Nov 27 15:12:18 2018 +0000

----------------------------------------------------------------------
 .../org/apache/phoenix/end2end/DeleteIT.java    |  62 ++++++++---
 .../end2end/UpsertSelectAutoCommitIT.java       |  26 +++--
 .../apache/phoenix/end2end/UpsertSelectIT.java  | 103 +++++++++++++------
 .../apache/phoenix/compile/DeleteCompiler.java  |   6 +-
 .../apache/phoenix/compile/UpsertCompiler.java  |   6 +-
 .../org/apache/phoenix/query/QueryServices.java |   3 +
 .../phoenix/query/QueryServicesOptions.java     |   3 +
 7 files changed, 159 insertions(+), 50 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/f6b75942/phoenix-core/src/it/java/org/apache/phoenix/end2end/DeleteIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/DeleteIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/DeleteIT.java
index 5e65927..39210fa 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/DeleteIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/DeleteIT.java
@@ -40,12 +40,26 @@ import org.apache.phoenix.query.QueryServices;
 import org.apache.phoenix.util.PropertiesUtil;
 import org.apache.phoenix.util.QueryUtil;
 import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameters;
 
-
+@RunWith(Parameterized.class)
 public class DeleteIT extends ParallelStatsDisabledIT {
     private static final int NUMBER_OF_ROWS = 20;
     private static final int NTH_ROW_NULL = 5;
-    
+
+    private final String allowServerSideMutations;
+
+    public DeleteIT(String allowServerSideMutations) {
+        this.allowServerSideMutations = allowServerSideMutations;
+    }
+
+    @Parameters(name="DeleteIT_allowServerSideMutations={0}") // name is used by failsafe as file name in reports
+    public static Object[] data() {
+        return new Object[] {"true", "false"};
+    }
+
     private static String initTableValues(Connection conn) throws SQLException {
         String tableName = generateUniqueName();
         ensureTableCreated(getUrl(), tableName, "IntIntKeyTest");
@@ -75,7 +89,9 @@ public class DeleteIT extends ParallelStatsDisabledIT {
     }
     
     private void testDeleteFilter(boolean autoCommit) throws Exception {
-        Connection conn = DriverManager.getConnection(getUrl());
+        Properties props = new Properties();
+        props.setProperty(QueryServices.ENABLE_SERVER_SIDE_MUTATIONS, allowServerSideMutations);
+        Connection conn = DriverManager.getConnection(getUrl(), props);
         String tableName = initTableValues(conn);
 
         assertTableCount(conn, tableName, NUMBER_OF_ROWS);
@@ -102,7 +118,9 @@ public class DeleteIT extends ParallelStatsDisabledIT {
     }
 
     private void testDeleteByFilterAndRow(boolean autoCommit) throws SQLException {
-        Connection conn = DriverManager.getConnection(getUrl());
+        Properties props = new Properties();
+        props.setProperty(QueryServices.ENABLE_SERVER_SIDE_MUTATIONS, allowServerSideMutations);
+        Connection conn = DriverManager.getConnection(getUrl(), props);
         String tableName = initTableValues(conn);
 
         assertTableCount(conn, tableName, NUMBER_OF_ROWS);
@@ -167,7 +185,9 @@ public class DeleteIT extends ParallelStatsDisabledIT {
     }
 
     private void testDeleteRange(boolean autoCommit, boolean createIndex, boolean local) throws Exception {
-        Connection conn = DriverManager.getConnection(getUrl());
+        Properties props = new Properties();
+        props.setProperty(QueryServices.ENABLE_SERVER_SIDE_MUTATIONS, allowServerSideMutations);
+        Connection conn = DriverManager.getConnection(getUrl(), props);
         String tableName = initTableValues(conn);
         String indexName = generateUniqueName();
         String localIndexName = generateUniqueName();
@@ -298,7 +318,9 @@ public class DeleteIT extends ParallelStatsDisabledIT {
     private void testDeleteAllFromTableWithIndex(boolean autoCommit, boolean isSalted, boolean localIndex) throws Exception {
         Connection con = null;
         try {
-            con = DriverManager.getConnection(getUrl());
+            Properties props = new Properties();
+            props.setProperty(QueryServices.ENABLE_SERVER_SIDE_MUTATIONS, allowServerSideMutations);
+            con = DriverManager.getConnection(getUrl(), props);
             con.setAutoCommit(autoCommit);
 
             Statement stm = con.createStatement();
@@ -390,7 +412,9 @@ public class DeleteIT extends ParallelStatsDisabledIT {
         Connection con = null;
         try {
             boolean autoCommit = false;
-            con = DriverManager.getConnection(getUrl());
+            Properties props = new Properties();
+            props.setProperty(QueryServices.ENABLE_SERVER_SIDE_MUTATIONS, allowServerSideMutations);
+            con = DriverManager.getConnection(getUrl(), props);
             con.setAutoCommit(autoCommit);
 
             Statement stm = con.createStatement();
@@ -465,7 +489,9 @@ public class DeleteIT extends ParallelStatsDisabledIT {
         Connection con = null;
         try {
             boolean autoCommit = false;
-            con = DriverManager.getConnection(getUrl());
+            Properties props = new Properties();
+            props.setProperty(QueryServices.ENABLE_SERVER_SIDE_MUTATIONS, allowServerSideMutations);
+            con = DriverManager.getConnection(getUrl(), props);
             con.setAutoCommit(autoCommit);
 
             Statement stm = con.createStatement();
@@ -588,7 +614,9 @@ public class DeleteIT extends ParallelStatsDisabledIT {
     private void testDeleteAllFromTable(boolean autoCommit) throws SQLException {
         Connection con = null;
         try {
-            con = DriverManager.getConnection(getUrl());
+            Properties props = new Properties();
+            props.setProperty(QueryServices.ENABLE_SERVER_SIDE_MUTATIONS, allowServerSideMutations);
+            con = DriverManager.getConnection(getUrl(), props);
             con.setAutoCommit(autoCommit);
 
             String tableName = generateUniqueName();
@@ -649,7 +677,9 @@ public class DeleteIT extends ParallelStatsDisabledIT {
     }
     
     private void testDeleteForTableWithRowTimestampCol(boolean autoCommit, String tableName) throws Exception {
-        try (Connection conn = DriverManager.getConnection(getUrl())) {
+        Properties props = new Properties();
+        props.setProperty(QueryServices.ENABLE_SERVER_SIDE_MUTATIONS, allowServerSideMutations);
+        try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
             conn.setAutoCommit(autoCommit);
             Statement stm = conn.createStatement();
             stm.execute("CREATE TABLE IF NOT EXISTS " + tableName +
@@ -733,7 +763,9 @@ public class DeleteIT extends ParallelStatsDisabledIT {
                 + "CREATE INDEX IF NOT EXISTS index_column_varchar_id ON " + tableName + "(varchar_id);"
                 + "CREATE INDEX IF NOT EXISTS index_column_double_id ON " + tableName + "(double_id);" + "UPSERT INTO "
                 + tableName + " VALUES (9000000,0.5,'Sample text extra');" ;
-        try (Connection conn = DriverManager.getConnection(getUrl())) {
+        Properties props = new Properties();
+        props.setProperty(QueryServices.ENABLE_SERVER_SIDE_MUTATIONS, allowServerSideMutations);
+        try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
             conn.setAutoCommit(true);
             Statement stm = conn.createStatement();
             for (String sql : commands.split(";")) {
@@ -755,7 +787,9 @@ public class DeleteIT extends ParallelStatsDisabledIT {
 
         String ddl = "CREATE TABLE IF NOT EXISTS " + tableName + " (pk1 DECIMAL NOT NULL, v1 VARCHAR CONSTRAINT PK PRIMARY KEY (pk1))";
         int numRecords = 1010;
-        try (Connection conn = DriverManager.getConnection(getUrl())) {
+        Properties props = new Properties();
+        props.setProperty(QueryServices.ENABLE_SERVER_SIDE_MUTATIONS, allowServerSideMutations);
+        try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
             conn.createStatement().execute(ddl);
             Statement stmt = conn.createStatement();
             for (int i = 0; i < numRecords ; i++) {
@@ -788,7 +822,9 @@ public class DeleteIT extends ParallelStatsDisabledIT {
                         + " (pk1 DECIMAL NOT NULL, v1 VARCHAR, v2 VARCHAR CONSTRAINT PK PRIMARY KEY (pk1))";
         String idx1 = "CREATE INDEX " + indexName1 + " ON " + tableName + "(v1)";
         String idx2 = "CREATE INDEX " + indexName2 + " ON " + tableName + "(v1, v2)";
-        try (Connection conn = DriverManager.getConnection(getUrl())) {
+        Properties props = new Properties();
+        props.setProperty(QueryServices.ENABLE_SERVER_SIDE_MUTATIONS, allowServerSideMutations);
+        try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
             conn.createStatement().execute(ddl);
             conn.createStatement().execute(idx1);
             conn.createStatement().execute(idx2);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/f6b75942/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpsertSelectAutoCommitIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpsertSelectAutoCommitIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpsertSelectAutoCommitIT.java
index 6210852..c56296c 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpsertSelectAutoCommitIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpsertSelectAutoCommitIT.java
@@ -23,35 +23,44 @@ 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.sql.Statement;
 import java.util.Properties;
 
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.HBaseAdmin;
-import org.apache.phoenix.exception.SQLExceptionCode;
 import org.apache.phoenix.jdbc.PhoenixConnection;
 import org.apache.phoenix.query.QueryServices;
 import org.apache.phoenix.util.PropertiesUtil;
 import org.apache.phoenix.util.TestUtil;
 import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameters;
 
-
+@RunWith(Parameterized.class)
 public class UpsertSelectAutoCommitIT extends ParallelStatsDisabledIT {
 
-    public UpsertSelectAutoCommitIT() {
+    private final String allowServerSideMutations;
+
+    public UpsertSelectAutoCommitIT(String allowServerSideMutations) {
+        this.allowServerSideMutations = allowServerSideMutations;
+    }
+
+    @Parameters(name="UpsertSelectAutoCommitIT_allowServerSideMutations={0}") // name is used by failsafe as file name in reports
+    public static Object[] data() {
+        return new Object[] {"true", "false"};
     }
 
     @Test
     public void testAutoCommitUpsertSelect() throws Exception {
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        props.setProperty(QueryServices.ENABLE_SERVER_SIDE_MUTATIONS, allowServerSideMutations);
         Connection conn = DriverManager.getConnection(getUrl(), props);
         conn.setAutoCommit(true);
         String atable = generateUniqueName();
@@ -104,7 +113,9 @@ public class UpsertSelectAutoCommitIT extends ParallelStatsDisabledIT {
 
     @Test
     public void testDynamicUpsertSelect() throws Exception {
-        Connection conn = DriverManager.getConnection(getUrl());
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        props.setProperty(QueryServices.ENABLE_SERVER_SIDE_MUTATIONS, allowServerSideMutations);
+        Connection conn = DriverManager.getConnection(getUrl(), props);
         String tableName = generateUniqueName();
         String cursorDDL = " CREATE TABLE IF NOT EXISTS " + tableName
             + " (ORGANIZATION_ID VARCHAR(15) NOT NULL, \n"
@@ -161,6 +172,7 @@ public class UpsertSelectAutoCommitIT extends ParallelStatsDisabledIT {
         props.setProperty(QueryServices.MUTATE_BATCH_SIZE_BYTES_ATTRIB, Integer.toString(512));
         props.setProperty(QueryServices.SCAN_CACHE_SIZE_ATTRIB, Integer.toString(3));
         props.setProperty(QueryServices.SCAN_RESULT_CHUNK_SIZE, Integer.toString(3));
+        props.setProperty(QueryServices.ENABLE_SERVER_SIDE_MUTATIONS, allowServerSideMutations);
         Connection conn = DriverManager.getConnection(getUrl(), props);
         conn.setAutoCommit(true);
         conn.createStatement().execute("CREATE SEQUENCE keys CACHE 1000");
@@ -189,6 +201,7 @@ public class UpsertSelectAutoCommitIT extends ParallelStatsDisabledIT {
         Properties connectionProperties = new Properties();
         connectionProperties.setProperty(QueryServices.MAX_MUTATION_SIZE_ATTRIB, "3");
         connectionProperties.setProperty(QueryServices.MAX_MUTATION_SIZE_BYTES_ATTRIB, "50000");
+        connectionProperties.setProperty(QueryServices.ENABLE_SERVER_SIDE_MUTATIONS, allowServerSideMutations);
         PhoenixConnection connection =
                 (PhoenixConnection) DriverManager.getConnection(getUrl(), connectionProperties);
         connection.setAutoCommit(true);
@@ -214,6 +227,7 @@ public class UpsertSelectAutoCommitIT extends ParallelStatsDisabledIT {
         props.setProperty(QueryServices.MUTATE_BATCH_SIZE_ATTRIB, Integer.toString(3));
         props.setProperty(QueryServices.SCAN_CACHE_SIZE_ATTRIB, Integer.toString(3));
         props.setProperty(QueryServices.SCAN_RESULT_CHUNK_SIZE, Integer.toString(3));
+        props.setProperty(QueryServices.ENABLE_SERVER_SIDE_MUTATIONS, allowServerSideMutations);
         Connection conn = DriverManager.getConnection(getUrl(), props);
         conn.setAutoCommit(false);
         String tableName = generateUniqueName();

http://git-wip-us.apache.org/repos/asf/phoenix/blob/f6b75942/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpsertSelectIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpsertSelectIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpsertSelectIT.java
index 5db1fdd..b5aa8be 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpsertSelectIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpsertSelectIT.java
@@ -58,8 +58,22 @@ import org.apache.phoenix.util.PropertiesUtil;
 import org.apache.phoenix.util.QueryUtil;
 import org.apache.phoenix.util.TestUtil;
 import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameters;
 
+@RunWith(Parameterized.class)
 public class UpsertSelectIT extends ParallelStatsDisabledIT {
+    private final String allowServerSideMutations;
+
+    public UpsertSelectIT(String allowServerSideMutations) {
+        this.allowServerSideMutations = allowServerSideMutations;
+    }
+
+    @Parameters(name="UpsertSelecttIT_allowServerSideMutations={0}") // name is used by failsafe as file name in reports
+    public static Object[] data() {
+        return new Object[] {"true", "false"};
+    }
 	
     @Test
     public void testUpsertSelectWithNoIndex() throws Exception {
@@ -85,6 +99,7 @@ public class UpsertSelectIT extends ParallelStatsDisabledIT {
         String tenantId = getOrganizationId();
         byte[][] splits = getDefaultSplits(tenantId);
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        props.setProperty(QueryServices.ENABLE_SERVER_SIDE_MUTATIONS, allowServerSideMutations);
         String aTable = initATableValues(tenantId, saltTable ? null : splits, null, null, getUrl(), saltTable ? "salt_buckets = 2" : null);
 
         String customEntityTable = generateUniqueName();
@@ -234,6 +249,7 @@ public class UpsertSelectIT extends ParallelStatsDisabledIT {
         String ptsdbTable = generateUniqueName();
         ensureTableCreated(getUrl(), ptsdbTable, PTSDB_NAME);
         Properties props = new Properties();
+        props.setProperty(QueryServices.ENABLE_SERVER_SIDE_MUTATIONS, allowServerSideMutations);
         Connection conn = DriverManager.getConnection(getUrl(), props);
         conn.setAutoCommit(false);
         String upsert = "UPSERT INTO " + ptsdbTable + "(\"DATE\", val, host) " +
@@ -404,6 +420,7 @@ public class UpsertSelectIT extends ParallelStatsDisabledIT {
         String ptsdbTable = generateUniqueName();
         ensureTableCreated(getUrl(), ptsdbTable, PTSDB_NAME);
         Properties props = new Properties();
+        props.setProperty(QueryServices.ENABLE_SERVER_SIDE_MUTATIONS, allowServerSideMutations);
         Connection conn = DriverManager.getConnection(getUrl(), props);
         conn.setAutoCommit(autoCommit);
         String upsert = "UPSERT INTO " + ptsdbTable + "(\"DATE\", val, host) " +
@@ -475,6 +492,7 @@ public class UpsertSelectIT extends ParallelStatsDisabledIT {
         String tableName = generateUniqueName();
         ensureTableCreated(getUrl(), tableName, "IntKeyTest", splits, null);
         Properties props = new Properties();
+        props.setProperty(QueryServices.ENABLE_SERVER_SIDE_MUTATIONS, allowServerSideMutations);
         Connection conn = DriverManager.getConnection(getUrl(), props);
         String upsert = "UPSERT INTO " + tableName + " VALUES(1)";
         PreparedStatement upsertStmt = conn.prepareStatement(upsert);
@@ -509,6 +527,7 @@ public class UpsertSelectIT extends ParallelStatsDisabledIT {
         String tableName = generateUniqueName();
         createTestTable(getUrl(), "create table " + tableName + " (i integer not null primary key desc, j integer)", splits, null);
         Properties props = new Properties();
+        props.setProperty(QueryServices.ENABLE_SERVER_SIDE_MUTATIONS, allowServerSideMutations);
         Connection conn = DriverManager.getConnection(getUrl(), props);
         String upsert = "UPSERT INTO " + tableName + " VALUES(1, 1)";
         PreparedStatement upsertStmt = conn.prepareStatement(upsert);
@@ -566,6 +585,7 @@ public class UpsertSelectIT extends ParallelStatsDisabledIT {
         String tableName = generateUniqueName();
         createTestTable(getUrl(), "create table " + tableName + " (i integer not null primary key desc, j integer)", splits, null);
         Properties props = new Properties();
+        props.setProperty(QueryServices.ENABLE_SERVER_SIDE_MUTATIONS, allowServerSideMutations);
         Connection conn = DriverManager.getConnection(getUrl(), props);
         String upsert = "UPSERT INTO " + tableName + " VALUES(1, 1)";
         PreparedStatement upsertStmt = conn.prepareStatement(upsert);
@@ -603,6 +623,7 @@ public class UpsertSelectIT extends ParallelStatsDisabledIT {
         String tableName = generateUniqueName();
         ensureTableCreated(getUrl(), tableName, "IntKeyTest", splits, null, null);
         Properties props = new Properties();
+        props.setProperty(QueryServices.ENABLE_SERVER_SIDE_MUTATIONS, allowServerSideMutations);
         Connection conn = DriverManager.getConnection(getUrl(), props);
         String upsert = "UPSERT INTO " + tableName + " VALUES(?)";
         PreparedStatement upsertStmt = conn.prepareStatement(upsert);
@@ -640,6 +661,7 @@ public class UpsertSelectIT extends ParallelStatsDisabledIT {
     @Test
     public void testUpsertSelectWithLimit() throws Exception {
         Properties props = new Properties();
+        props.setProperty(QueryServices.ENABLE_SERVER_SIDE_MUTATIONS, allowServerSideMutations);
         Connection conn = DriverManager.getConnection(getUrl(), props);
         String tableName = generateUniqueName();
         conn.createStatement().execute("create table " + tableName + " (id varchar(10) not null primary key, val varchar(10), ts timestamp)");
@@ -703,6 +725,7 @@ public class UpsertSelectIT extends ParallelStatsDisabledIT {
     @Test
     public void testUpsertSelectWithSequence() throws Exception {
         Properties props = new Properties();
+        props.setProperty(QueryServices.ENABLE_SERVER_SIDE_MUTATIONS, allowServerSideMutations);
         Connection conn = DriverManager.getConnection(getUrl(), props);
         String t1 = generateUniqueName();
         String t2 = generateUniqueName();
@@ -745,6 +768,7 @@ public class UpsertSelectIT extends ParallelStatsDisabledIT {
     public void testUpsertSelectWithSequenceAndOrderByWithSalting() throws Exception {
         int numOfRecords = 200;
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        props.setProperty(QueryServices.ENABLE_SERVER_SIDE_MUTATIONS, allowServerSideMutations);
         Connection conn = DriverManager.getConnection(getUrl(), props);
         String t1 = generateUniqueName();
         String t2 = generateUniqueName();
@@ -788,6 +812,8 @@ public class UpsertSelectIT extends ParallelStatsDisabledIT {
             assertEquals(seq - 1, rs2.getLong("k1"));
             seq++;
         }
+        // cleanup afrer ourselves
+        conn.createStatement().execute("drop sequence s");
         conn.close();
 
     }
@@ -806,7 +832,9 @@ public class UpsertSelectIT extends ParallelStatsDisabledIT {
         // The timestamp of the put will be the value of the row_timestamp column.
         long rowTimestamp = EnvironmentEdgeManager.currentTimeMillis();
         Date rowTimestampDate = new Date(rowTimestamp);
-        try (Connection conn = DriverManager.getConnection(getUrl())) {
+        Properties props = new Properties();
+        props.setProperty(QueryServices.ENABLE_SERVER_SIDE_MUTATIONS, allowServerSideMutations);
+        try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
             PreparedStatement stmt = conn.prepareStatement("UPSERT INTO " + t1 + " (PK1, PK2, KV1) VALUES(?, ?, ?)");
             stmt.setString(1, "PK1");
             stmt.setDate(2, rowTimestampDate);
@@ -833,7 +861,6 @@ public class UpsertSelectIT extends ParallelStatsDisabledIT {
         }
         
         // Verify that you can't see the data in T2 if the connection is at a timestamp  lower than the row timestamp.
-        Properties props = new Properties();
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(rowTimestamp-1));
         try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
             PreparedStatement stmt = conn.prepareStatement("SELECT * FROM " + t2 + " WHERE PK1 = ? AND PK2 = ?");
@@ -873,14 +900,16 @@ public class UpsertSelectIT extends ParallelStatsDisabledIT {
     @Test
     public void testUpsertSelectSameTableWithRowTimestampColumn() throws Exception {
         String tableName = generateUniqueName();
-        try (Connection conn = DriverManager.getConnection(getUrl())) {
+        Properties props = new Properties();
+        props.setProperty(QueryServices.ENABLE_SERVER_SIDE_MUTATIONS, allowServerSideMutations);
+        try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
             conn.createStatement().execute("CREATE TABLE " + tableName + " (PK1 INTEGER NOT NULL, PK2 DATE NOT NULL, KV1 VARCHAR CONSTRAINT PK PRIMARY KEY(PK1, PK2 ROW_TIMESTAMP)) ");
         }
 
         // The timestamp of the put will be the value of the row_timestamp column.
         long rowTimestamp = 100;
         Date rowTimestampDate = new Date(rowTimestamp);
-        try (Connection conn = DriverManager.getConnection(getUrl())) {
+        try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
             PreparedStatement stmt = conn.prepareStatement("UPSERT INTO  " + tableName + " (PK1, PK2, KV1) VALUES(?, ?, ?)");
             stmt.setInt(1, 1);
             stmt.setDate(2, rowTimestampDate);
@@ -889,18 +918,18 @@ public class UpsertSelectIT extends ParallelStatsDisabledIT {
             conn.commit();
         }
         String seq = generateUniqueName();
-        try (Connection conn = DriverManager.getConnection(getUrl())) {
+        try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
             conn.createStatement().execute("CREATE SEQUENCE " + seq);
         }
         // Upsert select data into table. The connection needs to be at a timestamp beyond the row timestamp. Otherwise 
         // it won't see the data from table.
-        try (Connection conn = DriverManager.getConnection(getUrl())) {
+        try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
             conn.createStatement().executeUpdate("UPSERT INTO  " + tableName + "  SELECT NEXT VALUE FOR " + seq + ", PK2 FROM  " + tableName);
             conn.commit();
         }
         
         // Upsert select using sequences.
-        try (Connection conn = DriverManager.getConnection(getUrl())) {
+        try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
             conn.setAutoCommit(true);
             for (int i = 0; i < 10; i++) {
                 int count = conn.createStatement().executeUpdate("UPSERT INTO  " + tableName + "  SELECT NEXT VALUE FOR " + seq + ", PK2 FROM  " + tableName);
@@ -920,7 +949,9 @@ public class UpsertSelectIT extends ParallelStatsDisabledIT {
             conn.createStatement().execute("CREATE TABLE " + table3 + " (T3PK1 VARCHAR NOT NULL, T3PK2 DATE NOT NULL, T3KV1 VARCHAR, T3KV2 VARCHAR CONSTRAINT PK PRIMARY KEY(T3PK1, T3PK2 DESC ROW_TIMESTAMP)) ");
         }
         long startTime = EnvironmentEdgeManager.currentTimeMillis();
-        try (Connection conn = DriverManager.getConnection(getUrl())) {
+        Properties props = new Properties();
+        props.setProperty(QueryServices.ENABLE_SERVER_SIDE_MUTATIONS, allowServerSideMutations);
+        try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
             // Upsert values where row_timestamp column PK2 is not set and the column names are specified
             // This should upsert data with the value for PK2 as server timestamp
             PreparedStatement stmt = conn.prepareStatement("UPSERT INTO  " + table1 + " (T1PK1, T1KV1, T1KV2) VALUES (?, ?, ?)");
@@ -932,7 +963,7 @@ public class UpsertSelectIT extends ParallelStatsDisabledIT {
         }
         long endTime = EnvironmentEdgeManager.currentTimeMillis();
         
-        try (Connection conn = DriverManager.getConnection(getUrl())) {
+        try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
             // Now query for data that was upserted above. If the row key was generated correctly then we should be able to see
             // the data in this query.
             PreparedStatement stmt = conn.prepareStatement("SELECT T1KV1, T1KV2 FROM " + table1 + " WHERE T1PK1 = ? AND T1PK2 >= ? AND T1PK2 <= ?");
@@ -947,7 +978,7 @@ public class UpsertSelectIT extends ParallelStatsDisabledIT {
         }
         
         startTime = EnvironmentEdgeManager.currentTimeMillis();
-        try (Connection conn = DriverManager.getConnection(getUrl())) {
+        try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
             // Upsert select into table2 by not selecting the row timestamp column. In this case, the rowtimestamp column would end up being set to the server timestamp
             PreparedStatement stmt = conn.prepareStatement("UPSERT INTO  " + table2 + " (T2PK1, T2KV1, T2KV2) SELECT T1PK1, T1KV1, T1KV2 FROM " + table1);
             stmt.executeUpdate();
@@ -955,7 +986,7 @@ public class UpsertSelectIT extends ParallelStatsDisabledIT {
         }
         endTime = EnvironmentEdgeManager.currentTimeMillis();
         
-        try (Connection conn = DriverManager.getConnection(getUrl())) {
+        try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
             // Now query for data that was upserted above. If the row key was generated correctly then we should be able to see
             // the data in this query.
             PreparedStatement stmt = conn.prepareStatement("SELECT T2KV1, T2KV2 FROM " + table2 + " WHERE T2PK1 = ? AND T2PK2 >= ?  AND T2PK2 <= ?");
@@ -970,7 +1001,7 @@ public class UpsertSelectIT extends ParallelStatsDisabledIT {
         }
         
         startTime = EnvironmentEdgeManager.currentTimeMillis();
-        try (Connection conn = DriverManager.getConnection(getUrl())) {
+        try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
             // Upsert select into table3 by not selecting the row timestamp column. In this case, the rowtimestamp column would end up being set to the server timestamp
             PreparedStatement stmt = conn.prepareStatement("UPSERT INTO  " + table3 + " (T3PK1, T3KV1, T3KV2) SELECT T2PK1, T2KV1, T2KV2 FROM " + table2);
             stmt.executeUpdate();
@@ -978,7 +1009,7 @@ public class UpsertSelectIT extends ParallelStatsDisabledIT {
         }
         endTime = EnvironmentEdgeManager.currentTimeMillis();
         
-        try (Connection conn = DriverManager.getConnection(getUrl())) {
+        try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
             // Now query for data that was upserted above. If the row key was generated correctly then we should be able to see
             // the data in this query.
             PreparedStatement stmt = conn.prepareStatement("SELECT T3KV1, T3KV2 FROM " + table3 + " WHERE T3PK1 = ? AND T3PK2 >= ? AND T3PK2 <= ?");
@@ -997,7 +1028,9 @@ public class UpsertSelectIT extends ParallelStatsDisabledIT {
     public void testUpsertSelectAutoCommitWithRowTimestampColumn() throws Exception {
         String tableName1 = generateUniqueName();
         String tableName2 = generateUniqueName();
-        try (Connection conn = DriverManager.getConnection(getUrl())) {
+        Properties props = new Properties();
+        props.setProperty(QueryServices.ENABLE_SERVER_SIDE_MUTATIONS, allowServerSideMutations);
+        try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
             conn.createStatement().execute("CREATE TABLE " + tableName1 + " (PK1 INTEGER NOT NULL, PK2 DATE NOT NULL, PK3 INTEGER NOT NULL, KV1 VARCHAR CONSTRAINT PK PRIMARY KEY(PK1, PK2 ROW_TIMESTAMP, PK3)) ");
             conn.createStatement().execute("CREATE TABLE " + tableName2 + " (PK1 INTEGER NOT NULL, PK2 DATE NOT NULL, PK3 INTEGER NOT NULL, KV1 VARCHAR CONSTRAINT PK PRIMARY KEY(PK1, PK2 DESC ROW_TIMESTAMP, PK3)) ");
         }
@@ -1007,7 +1040,7 @@ public class UpsertSelectIT extends ParallelStatsDisabledIT {
             // Upsert data with the row timestamp value set
             long rowTimestamp1 = 100;
             Date rowTimestampDate = new Date(rowTimestamp1);
-            try (Connection conn = DriverManager.getConnection(getUrl())) {
+            try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
                 PreparedStatement stmt = conn.prepareStatement("UPSERT INTO  " + tableName + " (PK1, PK2, PK3, KV1) VALUES(?, ?, ?, ?)");
                 stmt.setInt(1, 1);
                 stmt.setDate(2, rowTimestampDate);
@@ -1018,7 +1051,7 @@ public class UpsertSelectIT extends ParallelStatsDisabledIT {
             }
 
             long startTime = EnvironmentEdgeManager.currentTimeMillis();
-            try (Connection conn = DriverManager.getConnection(getUrl())) {
+            try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
                 conn.setAutoCommit(true);
                 // Upsert select in the same table with the row_timestamp column PK2 not specified. 
                 // This will end up creating a new row whose timestamp is the server time stamp 
@@ -1027,7 +1060,7 @@ public class UpsertSelectIT extends ParallelStatsDisabledIT {
             }
             long endTime = EnvironmentEdgeManager.currentTimeMillis();
             
-            try (Connection conn = DriverManager.getConnection(getUrl())) {
+            try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
                 // Verify the row that was upserted above
                 PreparedStatement stmt = conn.prepareStatement("SELECT * FROM  " + tableName + " WHERE PK1 = ? AND PK2 >= ? AND PK2<= ? AND PK3 = ?");
                 stmt.setInt(1, 1);
@@ -1046,13 +1079,13 @@ public class UpsertSelectIT extends ParallelStatsDisabledIT {
                 assertEquals(2, rs.getInt(1));
 
             }
-            try (Connection conn = DriverManager.getConnection(getUrl())) {
+            try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
                 conn.setAutoCommit(true);
                 // Upsert select in the same table with the row_timestamp column PK2 specified. This will not end up creating a new row
                 // because the destination pk columns, including the row timestamp column PK2, are the same as the source column.
                 conn.createStatement().executeUpdate("UPSERT INTO  " + tableName + " (PK1, PK2, PK3, KV1) SELECT PK1, PK2, PK3, KV1 FROM  " + tableName);
             }
-            try (Connection conn = DriverManager.getConnection(getUrl())) {
+            try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
                 // Verify that two rows were created. One with rowtimestamp1 and the other with rowtimestamp2
                 ResultSet rs = conn.createStatement().executeQuery("SELECT COUNT(*) FROM " + tableName);
                 assertTrue(rs.next());
@@ -1071,7 +1104,9 @@ public class UpsertSelectIT extends ParallelStatsDisabledIT {
         String baseTableIdx = generateUniqueName();
         String tenantViewIdx = generateUniqueName();
 
-        try (Connection conn = DriverManager.getConnection(getUrl())) {
+        Properties props = new Properties();
+        props.setProperty(QueryServices.ENABLE_SERVER_SIDE_MUTATIONS, allowServerSideMutations);
+        try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
             conn.createStatement().execute("CREATE IMMUTABLE TABLE " + baseTable + " (TENANT_ID CHAR(15) NOT NULL, PK2 DATE NOT NULL, PK3 INTEGER NOT NULL, KV1 VARCHAR, KV2 VARCHAR, KV3 VARCHAR CONSTRAINT PK PRIMARY KEY(TENANT_ID, PK2 ROW_TIMESTAMP, PK3)) MULTI_TENANT = true, SALT_BUCKETS = 8");
             conn.createStatement().execute("CREATE INDEX " + baseTableIdx + " ON " + baseTable + " (PK2, KV3) INCLUDE (KV1)");
             conn.createStatement().execute("CREATE VIEW " + globalView + " AS SELECT * FROM " + baseTable + " WHERE KV1 = 'KV1'");
@@ -1085,7 +1120,7 @@ public class UpsertSelectIT extends ParallelStatsDisabledIT {
 
         // upsert data into base table without specifying the row timestamp column PK2
         long startTime = EnvironmentEdgeManager.currentTimeMillis();
-        try (Connection conn = DriverManager.getConnection(getUrl())) {
+        try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
             // Upsert select in the same table with the row_timestamp column PK2 not specified. This will end up
             // creating a new row whose timestamp is the latest timestamp (which will be used
             // for the row key too)
@@ -1101,7 +1136,7 @@ public class UpsertSelectIT extends ParallelStatsDisabledIT {
         long endTime = EnvironmentEdgeManager.currentTimeMillis();
 
         // Verify that we can see data when querying through base table, global view and index on the base table
-        try (Connection conn = DriverManager.getConnection(getUrl())) {
+        try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
             // Query the base table
             PreparedStatement stmt = conn.prepareStatement("SELECT * FROM  " + baseTable + " WHERE TENANT_ID = ? AND PK2 >= ? AND PK2 <= ? AND PK3 = ?");
             stmt.setString(1, tenantId);
@@ -1195,7 +1230,7 @@ public class UpsertSelectIT extends ParallelStatsDisabledIT {
 
         // Verify that the data upserted using the tenant view can now be queried using base table and the base table index
         Date upsertedDate;
-        try (Connection conn = DriverManager.getConnection(getUrl())) {
+        try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
             // Query the base table
             PreparedStatement stmt = conn.prepareStatement("SELECT * FROM  " + baseTable + " WHERE TENANT_ID = ? AND PK2 >= ? AND PK2 <= ? AND PK3 = ? ");
             stmt.setString(1, tenantId);
@@ -1276,11 +1311,13 @@ public class UpsertSelectIT extends ParallelStatsDisabledIT {
     public void testDisallowNegativeValuesForRowTsColumn() throws Exception {
         String tableName = generateUniqueName();
         String tableName2 = generateUniqueName();
-        try (Connection conn = DriverManager.getConnection(getUrl())) {
+        Properties props = new Properties();
+        props.setProperty(QueryServices.ENABLE_SERVER_SIDE_MUTATIONS, allowServerSideMutations);
+        try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
             conn.createStatement().execute("CREATE TABLE " + tableName + " (PK1 BIGINT NOT NULL PRIMARY KEY ROW_TIMESTAMP, KV1 VARCHAR)");
             conn.createStatement().execute("CREATE TABLE " + tableName2 + " (PK1 BIGINT NOT NULL PRIMARY KEY ROW_TIMESTAMP, KV1 VARCHAR)");
         }
-        try (Connection conn = DriverManager.getConnection(getUrl())) {
+        try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
             long upsertedTs = 100;
             PreparedStatement stmt = conn.prepareStatement("UPSERT INTO " + tableName +  " VALUES (?, ?)");
             stmt.setLong(1, upsertedTs);
@@ -1288,7 +1325,7 @@ public class UpsertSelectIT extends ParallelStatsDisabledIT {
             stmt.executeUpdate();
             conn.commit();
         }
-        try (Connection conn = DriverManager.getConnection(getUrl())) {
+        try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
             PreparedStatement stmt = conn.prepareStatement("UPSERT INTO " + tableName2 +  " SELECT (PK1 - 500), KV1 FROM " + tableName);
             stmt.executeUpdate();
             fail();
@@ -1300,6 +1337,7 @@ public class UpsertSelectIT extends ParallelStatsDisabledIT {
     @Test
     public void testUpsertSelectWithFixedWidthNullByteSizeArray() throws Exception {
         Properties props = new Properties();
+        props.setProperty(QueryServices.ENABLE_SERVER_SIDE_MUTATIONS, allowServerSideMutations);
         Connection conn = DriverManager.getConnection(getUrl(), props);
         String t1 = generateUniqueName();
         conn.createStatement().execute(
@@ -1357,6 +1395,7 @@ public class UpsertSelectIT extends ParallelStatsDisabledIT {
 
     private void testUpsertSelectWithMultiByteChars(boolean autoCommit) throws Exception {
         Properties props = new Properties();
+        props.setProperty(QueryServices.ENABLE_SERVER_SIDE_MUTATIONS, allowServerSideMutations);
         Connection conn = DriverManager.getConnection(getUrl(), props);
         conn.setAutoCommit(autoCommit);
         String t1 = generateUniqueName();
@@ -1401,6 +1440,7 @@ public class UpsertSelectIT extends ParallelStatsDisabledIT {
         props.setProperty(QueryServices.MUTATE_BATCH_SIZE_BYTES_ATTRIB, Integer.toString(512));
         props.setProperty(QueryServices.SCAN_CACHE_SIZE_ATTRIB, Integer.toString(3));
         props.setProperty(QueryServices.SCAN_RESULT_CHUNK_SIZE, Integer.toString(3));
+        props.setProperty(QueryServices.ENABLE_SERVER_SIDE_MUTATIONS, allowServerSideMutations);
         Connection conn = DriverManager.getConnection(getUrl(), props);
         conn.setAutoCommit(false);
         String t1 = generateUniqueName();
@@ -1428,7 +1468,9 @@ public class UpsertSelectIT extends ParallelStatsDisabledIT {
     public void testLongCodecUsedForRowTimestamp() throws Exception {
         String tableName = generateUniqueName();
         String indexName = generateUniqueName();
-        try (Connection conn = DriverManager.getConnection(getUrl())) {
+        Properties props = new Properties();
+        props.setProperty(QueryServices.ENABLE_SERVER_SIDE_MUTATIONS, allowServerSideMutations);
+        try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
             conn.createStatement().execute("CREATE IMMUTABLE TABLE " + tableName
                     + " (k1 TIMESTAMP not null, k2 bigint not null, v bigint, constraint pk primary key (k1 row_timestamp, k2)) SALT_BUCKETS = 9");
             conn.createStatement().execute(
@@ -1492,7 +1534,9 @@ public class UpsertSelectIT extends ParallelStatsDisabledIT {
     public void testLengthLimitedVarchar() throws Exception {
         String tableName1 = generateUniqueName();
         String tableName2 = generateUniqueName();
-        try (Connection conn = DriverManager.getConnection(getUrl())) {
+        Properties props = new Properties();
+        props.setProperty(QueryServices.ENABLE_SERVER_SIDE_MUTATIONS, allowServerSideMutations);
+        try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
             conn.setAutoCommit(true);
             conn.createStatement().execute("create table " + tableName1 + "(name varchar(160) primary key, id varchar(120), address varchar(160))"); 
             conn.createStatement().execute("create table " + tableName2 + "(name varchar(160) primary key, id varchar(10), address  varchar(10))");
@@ -1507,8 +1551,9 @@ public class UpsertSelectIT extends ParallelStatsDisabledIT {
         }
     }
     
-    private static Connection getTenantConnection(String tenantId) throws Exception {
+    private Connection getTenantConnection(String tenantId) throws Exception {
         Properties props = PropertiesUtil.deepCopy(TestUtil.TEST_PROPERTIES);
+        props.setProperty(QueryServices.ENABLE_SERVER_SIDE_MUTATIONS, allowServerSideMutations);
         props.setProperty(TENANT_ID_ATTRIB, tenantId);
         return DriverManager.getConnection(getUrl(), props);
     }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/f6b75942/phoenix-core/src/main/java/org/apache/phoenix/compile/DeleteCompiler.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/DeleteCompiler.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/DeleteCompiler.java
index 8c9a930..ed21374 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/DeleteCompiler.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/DeleteCompiler.java
@@ -437,6 +437,9 @@ public class DeleteCompiler {
         final boolean hasPostProcessing = delete.getLimit() != null;
         final ConnectionQueryServices services = connection.getQueryServices();
         List<QueryPlan> queryPlans;
+        boolean allowServerMutations =
+                services.getProps().getBoolean(QueryServices.ENABLE_SERVER_SIDE_MUTATIONS,
+                        QueryServicesOptions.DEFAULT_ENABLE_SERVER_SIDE_MUTATIONS);
         NamedTableNode tableNode = delete.getTable();
         String tableName = tableNode.getName().getTableName();
         String schemaName = tableNode.getName().getSchemaName();
@@ -550,7 +553,8 @@ public class DeleteCompiler {
         }
         
         runOnServer &= queryPlans.get(0).getTableRef().getTable().getType() != PTableType.INDEX;
-        
+        runOnServer &= allowServerMutations;
+
         // We need to have all indexed columns available in all immutable indexes in order
         // to generate the delete markers from the query. We also cannot have any filters
         // except for our SkipScanFilter for point lookups.

http://git-wip-us.apache.org/repos/asf/phoenix/blob/f6b75942/phoenix-core/src/main/java/org/apache/phoenix/compile/UpsertCompiler.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/UpsertCompiler.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/UpsertCompiler.java
index 4ed0c9a..410ac22 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/UpsertCompiler.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/UpsertCompiler.java
@@ -353,6 +353,9 @@ public class UpsertCompiler {
         boolean serverUpsertSelectEnabled =
                 services.getProps().getBoolean(QueryServices.ENABLE_SERVER_UPSERT_SELECT,
                         QueryServicesOptions.DEFAULT_ENABLE_SERVER_UPSERT_SELECT);
+        boolean allowServerMutations =
+                services.getProps().getBoolean(QueryServices.ENABLE_SERVER_SIDE_MUTATIONS,
+                        QueryServicesOptions.DEFAULT_ENABLE_SERVER_SIDE_MUTATIONS);
         UpsertingParallelIteratorFactory parallelIteratorFactoryToBe = null;
         boolean useServerTimestampToBe = false;
         
@@ -549,6 +552,7 @@ public class UpsertCompiler {
                         && !(table.isImmutableRows() && !table.getIndexes().isEmpty())
                         && !select.isJoin() && !hasWhereSubquery && table.getRowTimestampColPos() == -1;
             }
+            runOnServer &= allowServerMutations;
             // If we may be able to run on the server, add a hint that favors using the data table
             // if all else is equal.
             // TODO: it'd be nice if we could figure out in advance if the PK is potentially changing,
@@ -1322,7 +1326,7 @@ public class UpsertCompiler {
         public MutationState execute() throws SQLException {
             ResultIterator iterator = queryPlan.iterator();
             if (parallelIteratorFactory == null) {
-                return upsertSelect(new StatementContext(statement), tableRef, projector, iterator, columnIndexes, pkSlotIndexes, useServerTimestamp, false);
+                return upsertSelect(new StatementContext(statement, queryPlan.getContext().getScan()), tableRef, projector, iterator, columnIndexes, pkSlotIndexes, useServerTimestamp, false);
             }
             try {
                 parallelIteratorFactory.setRowProjector(projector);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/f6b75942/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServices.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServices.java b/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServices.java
index a8bbc22..78b72a1 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServices.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServices.java
@@ -307,6 +307,9 @@ public interface QueryServices extends SQLCloseable {
     // whether to enable server side RS -> RS calls for upsert select statements
     public static final String ENABLE_SERVER_UPSERT_SELECT ="phoenix.client.enable.server.upsert.select";
 
+    // whether to trigger mutations on the server at all (UPSERT/DELETE or DELETE FROM)
+    public static final String ENABLE_SERVER_SIDE_MUTATIONS ="phoenix.client.enable.server.mutations";
+
     //Update Cache Frequency default config attribute
     public static final String DEFAULT_UPDATE_CACHE_FREQUENCY_ATRRIB  = "phoenix.default.update.cache.frequency";
 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/f6b75942/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServicesOptions.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServicesOptions.java b/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServicesOptions.java
index d2f5bce..4e507d2 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServicesOptions.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServicesOptions.java
@@ -362,6 +362,9 @@ public class QueryServicesOptions {
     // RS -> RS calls for upsert select statements are disabled by default
     public static final boolean DEFAULT_ENABLE_SERVER_UPSERT_SELECT = false;
 
+    // By default generally allow server trigger mutations
+    public static final boolean DEFAULT_ENABLE_SERVER_SIDE_MUTATIONS = true;
+
     public static final boolean DEFAULT_COST_BASED_OPTIMIZER_ENABLED = false;
     public static final String DEFAULT_LOGGING_LEVEL = LogLevel.OFF.name();
     public static final String DEFAULT_LOG_SAMPLE_RATE = "1.0";


[19/28] phoenix git commit: PHOENIX-5013 Increase timeout for Tephra discovery service

Posted by pb...@apache.org.
PHOENIX-5013 Increase timeout for Tephra discovery service


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

Branch: refs/heads/4.x-cdh5.15
Commit: b28a241c8b38414ee4cba6a3fc1a74a5cf8cdd39
Parents: 60c1925
Author: Thomas D'Silva <td...@apache.org>
Authored: Thu Nov 15 20:33:26 2018 +0000
Committer: Pedro Boado <pb...@apache.org>
Committed: Tue Nov 27 15:11:59 2018 +0000

----------------------------------------------------------------------
 .../test/java/org/apache/phoenix/query/QueryServicesTestImpl.java   | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/b28a241c/phoenix-core/src/test/java/org/apache/phoenix/query/QueryServicesTestImpl.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/query/QueryServicesTestImpl.java b/phoenix-core/src/test/java/org/apache/phoenix/query/QueryServicesTestImpl.java
index 49fb8e8..eae951a 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/query/QueryServicesTestImpl.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/query/QueryServicesTestImpl.java
@@ -130,6 +130,7 @@ public final class QueryServicesTestImpl extends BaseQueryServicesImpl {
                 .set(TxConstants.Service.CFG_DATA_TX_CLIENT_RETRY_STRATEGY, "n-times")
                 .set(TxConstants.Service.CFG_DATA_TX_CLIENT_ATTEMPTS, 1)
                 .set(TxConstants.Service.CFG_DATA_TX_BIND_PORT, TestUtil.getRandomPort())
+                .set(TxConstants.Service.CFG_DATA_TX_CLIENT_DISCOVERY_TIMEOUT_SEC, 60)
                 .set(TxConstants.Manager.CFG_TX_SNAPSHOT_DIR, Files.createTempDir().getAbsolutePath())
                 .set(TxConstants.Manager.CFG_TX_TIMEOUT, DEFAULT_TXN_TIMEOUT_SECONDS)
                 .set(TxConstants.Manager.CFG_TX_SNAPSHOT_INTERVAL, 5L)


[28/28] phoenix git commit: PHOENIX-5026; another test addendum.

Posted by pb...@apache.org.
PHOENIX-5026; another test addendum.


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

Branch: refs/heads/4.x-cdh5.15
Commit: 5055512515c7c5cf3dc359a50c0b5bb07398f4aa
Parents: 027d21e
Author: Lars Hofhansl <la...@apache.org>
Authored: Sun Nov 25 00:23:38 2018 +0000
Committer: Pedro Boado <pb...@apache.org>
Committed: Tue Nov 27 15:12:24 2018 +0000

----------------------------------------------------------------------
 .../phoenix/end2end/UpsertSelectAutoCommitIT.java     | 14 ++++++--------
 1 file changed, 6 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/50555125/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpsertSelectAutoCommitIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpsertSelectAutoCommitIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpsertSelectAutoCommitIT.java
index 6fad376..4078578 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpsertSelectAutoCommitIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpsertSelectAutoCommitIT.java
@@ -175,16 +175,16 @@ public class UpsertSelectAutoCommitIT extends ParallelStatsDisabledIT {
         props.setProperty(QueryServices.ENABLE_SERVER_SIDE_MUTATIONS, allowServerSideMutations);
         Connection conn = DriverManager.getConnection(getUrl(), props);
         conn.setAutoCommit(true);
-        conn.createStatement().execute("CREATE SEQUENCE keys CACHE 1000");
         String tableName = generateUniqueName();
+        conn.createStatement().execute("CREATE SEQUENCE " + tableName + "_seq CACHE 1000");
         conn.createStatement().execute("CREATE TABLE " + tableName
                 + " (pk INTEGER PRIMARY KEY, val INTEGER) UPDATE_CACHE_FREQUENCY=3600000");
 
         conn.createStatement().execute(
-            "UPSERT INTO " + tableName + " VALUES (NEXT VALUE FOR keys,1)");
+            "UPSERT INTO " + tableName + " VALUES (NEXT VALUE FOR "+ tableName + "_seq,1)");
         PreparedStatement stmt =
                 conn.prepareStatement("UPSERT INTO " + tableName
-                        + " SELECT NEXT VALUE FOR keys, val FROM " + tableName);
+                        + " SELECT NEXT VALUE FOR "+ tableName + "_seq, val FROM " + tableName);
         HBaseAdmin admin =
                 driver.getConnectionQueryServices(getUrl(), TestUtil.TEST_PROPERTIES).getAdmin();
         for (int i=0; i<12; i++) {
@@ -192,8 +192,6 @@ public class UpsertSelectAutoCommitIT extends ParallelStatsDisabledIT {
             int upsertCount = stmt.executeUpdate();
             assertEquals((int)Math.pow(2, i), upsertCount);
         }
-        // cleanup after ourselves
-        conn.createStatement().execute("DROP SEQUENCE keys");
         admin.close();
         conn.close();
     }
@@ -234,17 +232,17 @@ public class UpsertSelectAutoCommitIT extends ParallelStatsDisabledIT {
         conn.setAutoCommit(false);
         String tableName = generateUniqueName();
 
-        conn.createStatement().execute("CREATE SEQUENCE "+ tableName);
+        conn.createStatement().execute("CREATE SEQUENCE "+ tableName + "_seq");
         conn.createStatement().execute(
                 "CREATE TABLE " + tableName + " (pk INTEGER PRIMARY KEY, val INTEGER)");
 
         conn.createStatement().execute(
-                "UPSERT INTO " + tableName + " VALUES (NEXT VALUE FOR keys,1)");
+                "UPSERT INTO " + tableName + " VALUES (NEXT VALUE FOR "+ tableName + "_seq, 1)");
         conn.commit();
         for (int i=0; i<6; i++) {
             Statement stmt = conn.createStatement();
             int upsertCount = stmt.executeUpdate(
-                    "UPSERT INTO " + tableName + " SELECT NEXT VALUE FOR keys, val FROM "
+                    "UPSERT INTO " + tableName + " SELECT NEXT VALUE FOR "+ tableName + "_seq, val FROM "
                             + tableName);
             conn.commit();
             assertEquals((int)Math.pow(2, i), upsertCount);


[08/28] phoenix git commit: PHOENIX-4996: Refactor PTableImpl to use Builder Pattern

Posted by pb...@apache.org.
http://git-wip-us.apache.org/repos/asf/phoenix/blob/1767244a/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableImpl.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableImpl.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableImpl.java
index 9f06e04..7939b97 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableImpl.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableImpl.java
@@ -36,6 +36,7 @@ import java.util.Map.Entry;
 
 import javax.annotation.Nonnull;
 
+import com.google.common.annotations.VisibleForTesting;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.client.Delete;
@@ -69,7 +70,6 @@ import org.apache.phoenix.schema.types.PChar;
 import org.apache.phoenix.schema.types.PDataType;
 import org.apache.phoenix.schema.types.PDouble;
 import org.apache.phoenix.schema.types.PFloat;
-import org.apache.phoenix.schema.types.PLong;
 import org.apache.phoenix.schema.types.PVarchar;
 import org.apache.phoenix.transaction.TransactionFactory;
 import org.apache.phoenix.util.ByteUtil;
@@ -102,164 +102,661 @@ import com.google.common.collect.Maps;
 public class PTableImpl implements PTable {
     private static final Integer NO_SALTING = -1;
 
-    private PTableKey key;
-    private PName name;
-    private PName schemaName = PName.EMPTY_NAME;
-    private PName tableName = PName.EMPTY_NAME;
-    private PName tenantId;
-    private PTableType type;
-    private PIndexState state;
-    private long sequenceNumber;
-    private long timeStamp;
-    private long indexDisableTimestamp;
+    private IndexMaintainer indexMaintainer;
+    private ImmutableBytesWritable indexMaintainersPtr;
+
+    private final PTableKey key;
+    private final PName name;
+    private final PName schemaName;
+    private final PName tableName;
+    private final PName tenantId;
+    private final PTableType type;
+    private final PIndexState state;
+    private final long sequenceNumber;
+    private final long timeStamp;
+    private final long indexDisableTimestamp;
     // Have MultiMap for String->PColumn (may need family qualifier)
-    private List<PColumn> pkColumns;
-    private List<PColumn> allColumns;
+    private final List<PColumn> pkColumns;
+    private final List<PColumn> allColumns;
     // columns that were inherited from a parent table but that were dropped in the view
-    private List<PColumn> excludedColumns;
-    private List<PColumnFamily> families;
-    private Map<byte[], PColumnFamily> familyByBytes;
-    private Map<String, PColumnFamily> familyByString;
-    private ListMultimap<String, PColumn> columnsByName;
-    private Map<KVColumnFamilyQualifier, PColumn> kvColumnsByQualifiers;
-    private PName pkName;
-    private Integer bucketNum;
-    private RowKeySchema rowKeySchema;
+    private final List<PColumn> excludedColumns;
+    private final List<PColumnFamily> families;
+    private final Map<byte[], PColumnFamily> familyByBytes;
+    private final Map<String, PColumnFamily> familyByString;
+    private final ListMultimap<String, PColumn> columnsByName;
+    private final Map<KVColumnFamilyQualifier, PColumn> kvColumnsByQualifiers;
+    private final PName pkName;
+    private final Integer bucketNum;
+    private final RowKeySchema rowKeySchema;
     // Indexes associated with this table.
-    private List<PTable> indexes;
+    private final List<PTable> indexes;
     // Data table name that the index is created on.
-    private PName parentName;
-    private PName parentSchemaName;
-    private PName parentTableName;
-    private List<PName> physicalNames;
-    private boolean isImmutableRows;
-    private IndexMaintainer indexMaintainer;
-    private ImmutableBytesWritable indexMaintainersPtr;
-    private PName defaultFamilyName;
-    private String viewStatement;
-    private boolean disableWAL;
-    private boolean multiTenant;
-    private boolean storeNulls;
-    private TransactionFactory.Provider transactionProvider;
-    private ViewType viewType;
-    private PDataType viewIndexType;
-    private Long viewIndexId;
-    private int estimatedSize;
-    private IndexType indexType;
-    private int baseColumnCount;
-    private boolean rowKeyOrderOptimizable; // TODO: remove when required that tables have been upgrade for PHOENIX-2067
-    private boolean hasColumnsRequiringUpgrade; // TODO: remove when required that tables have been upgrade for PHOENIX-2067
-    private int rowTimestampColPos;
-    private long updateCacheFrequency;
-    private boolean isNamespaceMapped;
-    private String autoPartitionSeqName;
-    private boolean isAppendOnlySchema;
-    private ImmutableStorageScheme immutableStorageScheme;
-    private QualifierEncodingScheme qualifierEncodingScheme;
-    private EncodedCQCounter encodedCQCounter;
-    private Boolean useStatsForParallelization;
-
-    public PTableImpl() {
-        this.indexes = Collections.emptyList();
-        this.physicalNames = Collections.emptyList();
-        this.rowKeySchema = RowKeySchema.EMPTY_SCHEMA;
-    }
-    
-    // Constructor used at table creation time
-    public PTableImpl(PName tenantId, String schemaName, String tableName, long timestamp, List<PColumnFamily> families, boolean isNamespaceMapped) {
-        Preconditions.checkArgument(tenantId==null || tenantId.getBytes().length > 0); // tenantId should be null or not empty
-        this.tenantId = tenantId;
-        this.name = PNameFactory.newName(SchemaUtil.getTableName(schemaName, tableName));
-        this.key = new PTableKey(tenantId, name.getString());
-        this.schemaName = PNameFactory.newName(schemaName);
-        this.tableName = PNameFactory.newName(tableName);
-        this.type = PTableType.VIEW;
-        this.viewType = ViewType.MAPPED;
-        this.timeStamp = timestamp;
-        this.pkColumns = this.allColumns = Collections.emptyList();
-        this.rowKeySchema = RowKeySchema.EMPTY_SCHEMA;
-        this.indexes = Collections.emptyList();
-        this.familyByBytes = Maps.newHashMapWithExpectedSize(families.size());
-        this.familyByString = Maps.newHashMapWithExpectedSize(families.size());
-        for (PColumnFamily family : families) {
-            familyByBytes.put(family.getName().getBytes(), family);
-            familyByString.put(family.getName().getString(), family);
-        }
-        this.families = families;
-        this.physicalNames = Collections.emptyList();
-        this.isNamespaceMapped = isNamespaceMapped;
-    }
-    
-    public PTableImpl(PName tenantId, String schemaName, String tableName, long timestamp, List<PColumnFamily> families, boolean isNamespaceMapped, ImmutableStorageScheme storageScheme, QualifierEncodingScheme encodingScheme, Boolean useStatsForParallelization) { // For base table of mapped VIEW
-        Preconditions.checkArgument(tenantId==null || tenantId.getBytes().length > 0); // tenantId should be null or not empty
-        this.tenantId = tenantId;
-        this.name = PNameFactory.newName(SchemaUtil.getTableName(schemaName, tableName));
-        this.key = new PTableKey(tenantId, name.getString());
-        this.schemaName = PNameFactory.newName(schemaName);
-        this.tableName = PNameFactory.newName(tableName);
-        this.type = PTableType.VIEW;
-        this.viewType = ViewType.MAPPED;
-        this.timeStamp = timestamp;
-        this.pkColumns = this.allColumns = Collections.emptyList();
-        this.rowKeySchema = RowKeySchema.EMPTY_SCHEMA;
-        this.indexes = Collections.emptyList();
-        this.familyByBytes = Maps.newHashMapWithExpectedSize(families.size());
-        this.familyByString = Maps.newHashMapWithExpectedSize(families.size());
-        for (PColumnFamily family : families) {
-            familyByBytes.put(family.getName().getBytes(), family);
-            familyByString.put(family.getName().getString(), family);
-        }
-        this.families = families;
-        this.physicalNames = Collections.emptyList();
-        this.isNamespaceMapped = isNamespaceMapped;
-        this.immutableStorageScheme = storageScheme;
-        this.qualifierEncodingScheme = encodingScheme;
-        this.useStatsForParallelization = useStatsForParallelization;
-    }
-    
-    // For indexes stored in shared physical tables
-    public PTableImpl(PName tenantId, PName schemaName, PName tableName, long timestamp, List<PColumnFamily> families, 
-            List<PColumn> columns, List<PName> physicalNames, PDataType viewIndexType, Long viewIndexId, boolean multiTenant, boolean isNamespaceMpped, ImmutableStorageScheme storageScheme, QualifierEncodingScheme qualifierEncodingScheme,
-            EncodedCQCounter encodedCQCounter, Boolean useStatsForParallelization, Integer bucketNum) throws SQLException {
-        this.pkColumns = this.allColumns = Collections.emptyList();
-        this.rowKeySchema = RowKeySchema.EMPTY_SCHEMA;
-        this.indexes = Collections.emptyList();
-        this.familyByBytes = Maps.newHashMapWithExpectedSize(families.size());
-        this.familyByString = Maps.newHashMapWithExpectedSize(families.size());
-        for (PColumnFamily family : families) {
-            familyByBytes.put(family.getName().getBytes(), family);
-            familyByString.put(family.getName().getString(), family);
-        }
-        this.families = families;
-        if (bucketNum!=null) {
-            columns = columns.subList(1, columns.size());
-        }
-        init(tenantId, this.schemaName, this.tableName, PTableType.INDEX, state, timeStamp, sequenceNumber, pkName, bucketNum, columns,
-            this.schemaName, parentTableName, indexes, isImmutableRows, physicalNames, defaultFamilyName,
-            null, disableWAL, multiTenant, storeNulls, viewType, viewIndexType, viewIndexId, indexType, baseColumnCount, rowKeyOrderOptimizable,
-            transactionProvider, updateCacheFrequency, indexDisableTimestamp, isNamespaceMpped, null,
-            false, storageScheme, qualifierEncodingScheme, encodedCQCounter, useStatsForParallelization, null);
-    }
-
-    public PTableImpl(long timeStamp) { // For delete marker
-        this(timeStamp, false);
-    }
-
-    public PTableImpl(long timeStamp, boolean isIndex) { // For index delete marker
-        if (isIndex) {
-            this.type = PTableType.INDEX;
-            this.state = PIndexState.INACTIVE;
-        } else {
-            this.type = PTableType.TABLE;
+    private final PName parentName;
+    private final PName parentSchemaName;
+    private final PName parentTableName;
+    private final List<PName> physicalNames;
+    private final boolean isImmutableRows;
+    private final PName defaultFamilyName;
+    private final String viewStatement;
+    private final boolean disableWAL;
+    private final boolean multiTenant;
+    private final boolean storeNulls;
+    private final TransactionFactory.Provider transactionProvider;
+    private final ViewType viewType;
+    private final PDataType viewIndexType;
+    private final Long viewIndexId;
+    private final int estimatedSize;
+    private final IndexType indexType;
+    private final int baseColumnCount;
+    private final boolean rowKeyOrderOptimizable; // TODO: remove when required that tables have been upgrade for PHOENIX-2067
+    private final boolean hasColumnsRequiringUpgrade; // TODO: remove when required that tables have been upgrade for PHOENIX-2067
+    private final int rowTimestampColPos;
+    private final long updateCacheFrequency;
+    private final boolean isNamespaceMapped;
+    private final String autoPartitionSeqName;
+    private final boolean isAppendOnlySchema;
+    private final ImmutableStorageScheme immutableStorageScheme;
+    private final QualifierEncodingScheme qualifierEncodingScheme;
+    private final EncodedCQCounter encodedCQCounter;
+    private final Boolean useStatsForParallelization;
+
+    public static class Builder {
+        private PTableKey key;
+        private PName name;
+        private PName schemaName = PName.EMPTY_NAME;
+        private PName tableName = PName.EMPTY_NAME;
+        private PName tenantId;
+        private PTableType type;
+        private PIndexState state;
+        private long sequenceNumber;
+        private long timeStamp;
+        private long indexDisableTimestamp;
+        private List<PColumn> pkColumns;
+        private List<PColumn> allColumns;
+        private List<PColumn> excludedColumns;
+        private List<PColumnFamily> families;
+        private Map<byte[], PColumnFamily> familyByBytes;
+        private Map<String, PColumnFamily> familyByString;
+        private ListMultimap<String, PColumn> columnsByName;
+        private Map<KVColumnFamilyQualifier, PColumn> kvColumnsByQualifiers;
+        private PName pkName;
+        private Integer bucketNum;
+        private RowKeySchema rowKeySchema;
+        private List<PTable> indexes;
+        private PName parentName;
+        private PName parentSchemaName;
+        private PName parentTableName;
+        private List<PName> physicalNames;
+        private boolean isImmutableRows;
+        private IndexMaintainer indexMaintainer;
+        private ImmutableBytesWritable indexMaintainersPtr;
+        private PName defaultFamilyName;
+        private String viewStatement;
+        private boolean disableWAL;
+        private boolean multiTenant;
+        private boolean storeNulls;
+        private TransactionFactory.Provider transactionProvider;
+        private ViewType viewType;
+        private PDataType viewIndexType;
+        private Long viewIndexId;
+        private int estimatedSize;
+        private IndexType indexType;
+        private int baseColumnCount;
+        private boolean rowKeyOrderOptimizable;
+        private boolean hasColumnsRequiringUpgrade;
+        private int rowTimestampColPos;
+        private long updateCacheFrequency;
+        private boolean isNamespaceMapped;
+        private String autoPartitionSeqName;
+        private boolean isAppendOnlySchema;
+        private ImmutableStorageScheme immutableStorageScheme;
+        private QualifierEncodingScheme qualifierEncodingScheme;
+        private EncodedCQCounter encodedCQCounter;
+        private Boolean useStatsForParallelization;
+        // Optionally set columns for the builder, but not for the actual PTable
+        private Collection<PColumn> columns;
+
+        public Builder setKey(PTableKey key) {
+            this.key = key;
+            return this;
+        }
+
+        public Builder setName(PName name) {
+            this.name = name;
+            return this;
+        }
+
+        public Builder setSchemaName(PName schemaName) {
+            this.schemaName = schemaName;
+            return this;
+        }
+
+        public Builder setTableName(PName tableName) {
+            this.tableName = tableName;
+            return this;
+        }
+
+        public Builder setTenantId(PName tenantId) {
+            this.tenantId = tenantId;
+            return this;
+        }
+
+        public Builder setType(PTableType type) {
+            this.type = type;
+            return this;
+        }
+
+        public Builder setState(PIndexState state) {
+            this.state = state;
+            return this;
+        }
+
+        public Builder setSequenceNumber(long sequenceNumber) {
+            this.sequenceNumber = sequenceNumber;
+            return this;
+        }
+
+        public Builder setTimeStamp(long timeStamp) {
+            this.timeStamp = timeStamp;
+            return this;
+        }
+
+        public Builder setIndexDisableTimestamp(long indexDisableTimestamp) {
+            this.indexDisableTimestamp = indexDisableTimestamp;
+            return this;
+        }
+
+        public Builder setPkColumns(List<PColumn> pkColumns) {
+            this.pkColumns = pkColumns;
+            return this;
+        }
+
+        public Builder setAllColumns(List<PColumn> allColumns) {
+            this.allColumns = allColumns;
+            return this;
+        }
+
+        public Builder setExcludedColumns(List<PColumn> excludedColumns) {
+            this.excludedColumns = excludedColumns;
+            return this;
+        }
+
+        public Builder setFamilyAttributes(List<PColumnFamily> families) {
+            this.familyByBytes = Maps.newHashMapWithExpectedSize(families.size());
+            this.familyByString = Maps.newHashMapWithExpectedSize(families.size());
+            for (PColumnFamily family : families) {
+                familyByBytes.put(family.getName().getBytes(), family);
+                familyByString.put(family.getName().getString(), family);
+            }
+            this.families = families;
+            return this;
+        }
+
+        public Builder setFamilies(List<PColumnFamily> families) {
+            this.families = families;
+            return this;
+        }
+
+        public Builder setFamilyByBytes(Map<byte[], PColumnFamily> familyByBytes) {
+            this.familyByBytes = familyByBytes;
+            return this;
+        }
+
+        public Builder setFamilyByString(Map<String, PColumnFamily> familyByString) {
+            this.familyByString = familyByString;
+            return this;
+        }
+
+        public Builder setColumnsByName(ListMultimap<String, PColumn> columnsByName) {
+            this.columnsByName = columnsByName;
+            return this;
+        }
+
+        public Builder setKvColumnsByQualifiers(Map<KVColumnFamilyQualifier, PColumn> kvColumnsByQualifiers) {
+            this.kvColumnsByQualifiers = kvColumnsByQualifiers;
+            return this;
+        }
+
+        public Builder setPkName(PName pkName) {
+            this.pkName = pkName;
+            return this;
+        }
+
+        public Builder setBucketNum(Integer bucketNum) {
+            this.bucketNum = bucketNum;
+            return this;
+        }
+
+        public Builder setRowKeySchema(RowKeySchema rowKeySchema) {
+            this.rowKeySchema = rowKeySchema;
+            return this;
+        }
+
+        public Builder setIndexes(List<PTable> indexes) {
+            this.indexes = indexes;
+            return this;
+        }
+
+        public Builder setParentName(PName parentName) {
+            this.parentName = parentName;
+            return this;
+        }
+
+        public Builder setParentSchemaName(PName parentSchemaName) {
+            this.parentSchemaName = parentSchemaName;
+            return this;
+        }
+
+        public Builder setParentTableName(PName parentTableName) {
+            this.parentTableName = parentTableName;
+            return this;
+        }
+
+        public Builder setPhysicalNames(List<PName> physicalNames) {
+            this.physicalNames = physicalNames;
+            return this;
+        }
+
+        public Builder setImmutableRows(boolean immutableRows) {
+            isImmutableRows = immutableRows;
+            return this;
+        }
+
+        public Builder setIndexMaintainer(IndexMaintainer indexMaintainer) {
+            this.indexMaintainer = indexMaintainer;
+            return this;
+        }
+
+        public Builder setIndexMaintainersPtr(ImmutableBytesWritable indexMaintainersPtr) {
+            this.indexMaintainersPtr = indexMaintainersPtr;
+            return this;
+        }
+
+        public Builder setDefaultFamilyName(PName defaultFamilyName) {
+            this.defaultFamilyName = defaultFamilyName;
+            return this;
+        }
+
+        public Builder setViewStatement(String viewStatement) {
+            this.viewStatement = viewStatement;
+            return this;
+        }
+
+        public Builder setDisableWAL(boolean disableWAL) {
+            this.disableWAL = disableWAL;
+            return this;
+        }
+
+        public Builder setMultiTenant(boolean multiTenant) {
+            this.multiTenant = multiTenant;
+            return this;
+        }
+
+        public Builder setStoreNulls(boolean storeNulls) {
+            this.storeNulls = storeNulls;
+            return this;
+        }
+
+        public Builder setTransactionProvider(TransactionFactory.Provider transactionProvider) {
+            this.transactionProvider = transactionProvider;
+            return this;
+        }
+
+        public Builder setViewType(ViewType viewType) {
+            this.viewType = viewType;
+            return this;
+        }
+
+        public Builder setViewIndexType(PDataType viewIndexType) {
+            this.viewIndexType = viewIndexType;
+            return this;
+        }
+
+        public Builder setViewIndexId(Long viewIndexId) {
+            this.viewIndexId = viewIndexId;
+            return this;
+        }
+
+        public Builder setEstimatedSize(int estimatedSize) {
+            this.estimatedSize = estimatedSize;
+            return this;
+        }
+
+        public Builder setIndexType(IndexType indexType) {
+            this.indexType = indexType;
+            return this;
+        }
+
+        public Builder setBaseColumnCount(int baseColumnCount) {
+            this.baseColumnCount = baseColumnCount;
+            return this;
+        }
+
+        public Builder setRowKeyOrderOptimizable(boolean rowKeyOrderOptimizable) {
+            this.rowKeyOrderOptimizable = rowKeyOrderOptimizable;
+            return this;
+        }
+
+        public Builder setHasColumnsRequiringUpgrade(boolean hasColumnsRequiringUpgrade) {
+            this.hasColumnsRequiringUpgrade = hasColumnsRequiringUpgrade;
+            return this;
+        }
+
+        public Builder setRowTimestampColPos(int rowTimestampColPos) {
+            this.rowTimestampColPos = rowTimestampColPos;
+            return this;
+        }
+
+        public Builder setUpdateCacheFrequency(long updateCacheFrequency) {
+            this.updateCacheFrequency = updateCacheFrequency;
+            return this;
+        }
+
+        public Builder setNamespaceMapped(boolean namespaceMapped) {
+            isNamespaceMapped = namespaceMapped;
+            return this;
+        }
+
+        public Builder setAutoPartitionSeqName(String autoPartitionSeqName) {
+            this.autoPartitionSeqName = autoPartitionSeqName;
+            return this;
+        }
+
+        public Builder setAppendOnlySchema(boolean appendOnlySchema) {
+            isAppendOnlySchema = appendOnlySchema;
+            return this;
+        }
+
+        public Builder setImmutableStorageScheme(ImmutableStorageScheme immutableStorageScheme) {
+            this.immutableStorageScheme = immutableStorageScheme;
+            return this;
+        }
+
+        public Builder setQualifierEncodingScheme(QualifierEncodingScheme qualifierEncodingScheme) {
+            this.qualifierEncodingScheme = qualifierEncodingScheme;
+            return this;
+        }
+
+        public Builder setEncodedCQCounter(EncodedCQCounter encodedCQCounter) {
+            this.encodedCQCounter = encodedCQCounter;
+            return this;
+        }
+
+        public Builder setUseStatsForParallelization(Boolean useStatsForParallelization) {
+            this.useStatsForParallelization = useStatsForParallelization;
+            return this;
+        }
+
+        /**
+         * Note: When set in the builder, we must call {@link Builder#initDerivedAttributes()}
+         * before building the PTable in order to correctly populate other attributes of the PTable
+         * @param columns PColumns to be set in the builder
+         * @return PTableImpl.Builder object
+         */
+        public Builder setColumns(Collection<PColumn> columns) {
+            this.columns = columns;
+            return this;
+        }
+
+        /**
+         * Populate derivable attributes of the PTable
+         * @return PTableImpl.Builder object
+         * @throws SQLException
+         */
+        private Builder initDerivedAttributes() throws SQLException {
+            checkTenantId(this.tenantId);
+            Preconditions.checkNotNull(this.schemaName);
+            Preconditions.checkNotNull(this.tableName);
+            Preconditions.checkNotNull(this.columns);
+            Preconditions.checkNotNull(this.indexes);
+            Preconditions.checkNotNull(this.physicalNames);
+            Preconditions.checkNotNull(this.hasColumnsRequiringUpgrade);
+            Preconditions.checkNotNull(this.rowKeyOrderOptimizable);
+
+            PName fullName = PNameFactory.newName(SchemaUtil.getTableName(
+                    this.schemaName.getString(), this.tableName.getString()));
+            int estimatedSize = SizedUtil.OBJECT_SIZE * 2 + 23 * SizedUtil.POINTER_SIZE +
+                    4 * SizedUtil.INT_SIZE + 2 * SizedUtil.LONG_SIZE + 2 * SizedUtil.INT_OBJECT_SIZE +
+                    PNameFactory.getEstimatedSize(this.tenantId) +
+                    PNameFactory.getEstimatedSize(this.schemaName) +
+                    PNameFactory.getEstimatedSize(this.tableName) +
+                    PNameFactory.getEstimatedSize(this.pkName) +
+                    PNameFactory.getEstimatedSize(this.parentTableName) +
+                    PNameFactory.getEstimatedSize(this.defaultFamilyName);
+            int numPKColumns = 0;
+            List<PColumn> pkColumns;
+            PColumn[] allColumns;
+            if (this.bucketNum != null) {
+                // Add salt column to allColumns and pkColumns, but don't add to
+                // columnsByName, since it should not be addressable via name.
+                allColumns = new PColumn[this.columns.size()+1];
+                allColumns[SALTING_COLUMN.getPosition()] = SALTING_COLUMN;
+                pkColumns = Lists.newArrayListWithExpectedSize(this.columns.size()+1);
+                ++numPKColumns;
+            } else {
+                allColumns = new PColumn[this.columns.size()];
+                pkColumns = Lists.newArrayListWithExpectedSize(this.columns.size());
+            }
+            // Must do this as with the new method of storing diffs, we just care about
+            // ordinal position relative order and not the true ordinal value itself.
+            List<PColumn> sortedColumns = Lists.newArrayList(this.columns);
+            Collections.sort(sortedColumns, new Comparator<PColumn>() {
+                @Override
+                public int compare(PColumn o1, PColumn o2) {
+                    return Integer.valueOf(o1.getPosition()).compareTo(o2.getPosition());
+                }
+            });
+
+            int position = 0;
+            if (this.bucketNum != null) {
+                position = 1;
+            }
+            ListMultimap<String, PColumn> populateColumnsByName =
+                    ArrayListMultimap.create(this.columns.size(), 1);
+            Map<KVColumnFamilyQualifier, PColumn> populateKvColumnsByQualifiers =
+                    Maps.newHashMapWithExpectedSize(this.columns.size());
+            for (PColumn column : sortedColumns) {
+                allColumns[position] = column;
+                position++;
+                PName familyName = column.getFamilyName();
+                if (familyName == null) {
+                    ++numPKColumns;
+                }
+                String columnName = column.getName().getString();
+                if (populateColumnsByName.put(columnName, column)) {
+                    int count = 0;
+                    for (PColumn dupColumn : populateColumnsByName.get(columnName)) {
+                        if (Objects.equal(familyName, dupColumn.getFamilyName())) {
+                            count++;
+                            if (count > 1) {
+                                throw new ColumnAlreadyExistsException(this.schemaName.getString(),
+                                        fullName.getString(), columnName);
+                            }
+                        }
+                    }
+                }
+                byte[] cq = column.getColumnQualifierBytes();
+                String cf = column.getFamilyName() != null ?
+                        column.getFamilyName().getString() : null;
+                if (cf != null && cq != null) {
+                    KVColumnFamilyQualifier info = new KVColumnFamilyQualifier(cf, cq);
+                    if (populateKvColumnsByQualifiers.get(info) != null) {
+                        throw new ColumnAlreadyExistsException(this.schemaName.getString(),
+                                fullName.getString(), columnName);
+                    }
+                    populateKvColumnsByQualifiers.put(info, column);
+                }
+            }
+            estimatedSize += SizedUtil.sizeOfMap(allColumns.length, SizedUtil.POINTER_SIZE,
+                    SizedUtil.sizeOfArrayList(1)); // for multi-map
+            estimatedSize += SizedUtil.sizeOfMap(numPKColumns) +
+                    SizedUtil.sizeOfMap(allColumns.length);
+
+            RowKeySchemaBuilder builder = new RowKeySchemaBuilder(numPKColumns);
+            // Two pass so that column order in column families matches overall column order
+            // and to ensure that column family order is constant
+            int maxExpectedSize = allColumns.length - numPKColumns;
+            // Maintain iteration order so that column families are ordered as they are listed
+            Map<PName, List<PColumn>> familyMap = Maps.newLinkedHashMap();
+            PColumn rowTimestampCol = null;
+            boolean hasColsRequiringUpgrade = false;
+            for (PColumn column : allColumns) {
+                PName familyName = column.getFamilyName();
+                if (familyName == null) {
+                    hasColsRequiringUpgrade |=
+                            (column.getSortOrder() == SortOrder.DESC
+                                    && (!column.getDataType().isFixedWidth()
+                                    || column.getDataType() == PChar.INSTANCE
+                                    || column.getDataType() == PFloat.INSTANCE
+                                    || column.getDataType() == PDouble.INSTANCE
+                                    || column.getDataType() == PBinary.INSTANCE) )
+                                    || (column.getSortOrder() == SortOrder.ASC
+                                        && column.getDataType() == PBinary.INSTANCE
+                                        && column.getMaxLength() != null
+                                        && column.getMaxLength() > 1);
+                    pkColumns.add(column);
+                    if (column.isRowTimestamp()) {
+                        rowTimestampCol = column;
+                    }
+                }
+                if (familyName == null) {
+                    estimatedSize += column.getEstimatedSize(); // PK columns
+                    builder.addField(column, column.isNullable(), column.getSortOrder());
+                } else {
+                    List<PColumn> columnsInFamily = familyMap.get(familyName);
+                    if (columnsInFamily == null) {
+                        columnsInFamily = Lists.newArrayListWithExpectedSize(maxExpectedSize);
+                        familyMap.put(familyName, columnsInFamily);
+                    }
+                    columnsInFamily.add(column);
+                }
+            }
+            int rowTimestampColPos;
+            if (rowTimestampCol != null) {
+                rowTimestampColPos = pkColumns.indexOf(rowTimestampCol);
+            } else {
+                rowTimestampColPos = -1;
+            }
+
+            Iterator<Map.Entry<PName,List<PColumn>>> iterator = familyMap.entrySet().iterator();
+            PColumnFamily[] families = new PColumnFamily[familyMap.size()];
+            ImmutableMap.Builder<String, PColumnFamily> familyByString = ImmutableMap.builder();
+            ImmutableSortedMap.Builder<byte[], PColumnFamily> familyByBytes = ImmutableSortedMap
+                    .orderedBy(Bytes.BYTES_COMPARATOR);
+            for (int i = 0; i < families.length; i++) {
+                Map.Entry<PName,List<PColumn>> entry = iterator.next();
+                PColumnFamily family = new PColumnFamilyImpl(entry.getKey(), entry.getValue());
+                families[i] = family;
+                familyByString.put(family.getName().getString(), family);
+                familyByBytes.put(family.getName().getBytes(), family);
+                estimatedSize += family.getEstimatedSize();
+            }
+            estimatedSize += SizedUtil.sizeOfArrayList(families.length);
+            estimatedSize += SizedUtil.sizeOfMap(families.length) * 2;
+            for (PTable index : this.indexes) {
+                estimatedSize += index.getEstimatedSize();
+            }
+
+            estimatedSize += PNameFactory.getEstimatedSize(this.parentName);
+            for (PName physicalName : this.physicalNames) {
+                estimatedSize += physicalName.getEstimatedSize();
+            }
+            // Populate the derived fields and return the builder
+            return this.setName(fullName)
+                    .setKey(new PTableKey(this.tenantId, fullName.getString()))
+                    .setParentName(this.parentTableName == null ? null :
+                            PNameFactory.newName(SchemaUtil.getTableName(
+                                    this.parentSchemaName != null ?
+                                            this.parentSchemaName.getString() : null,
+                                    this.parentTableName.getString())))
+                    .setColumnsByName(populateColumnsByName)
+                    .setKvColumnsByQualifiers(populateKvColumnsByQualifiers)
+                    .setAllColumns(ImmutableList.copyOf(allColumns))
+                    .setHasColumnsRequiringUpgrade(hasColsRequiringUpgrade
+                            | this.hasColumnsRequiringUpgrade)
+                    .setPkColumns(ImmutableList.copyOf(pkColumns))
+                    .setRowTimestampColPos(rowTimestampColPos)
+                    // after hasDescVarLengthColumns is calculated
+                    .setRowKeySchema(builder.rowKeyOrderOptimizable(
+                            this.rowKeyOrderOptimizable || !this.hasColumnsRequiringUpgrade)
+                            .build())
+                    .setFamilies(ImmutableList.copyOf(families))
+                    .setFamilyByBytes(familyByBytes.build())
+                    .setFamilyByString(familyByString.build())
+                    .setEstimatedSize(estimatedSize + this.rowKeySchema.getEstimatedSize());
         }
-        this.timeStamp = timeStamp;
-        this.pkColumns = this.allColumns = Collections.emptyList();
-        this.families = Collections.emptyList();
-        this.familyByBytes = Collections.emptyMap();
-        this.familyByString = Collections.emptyMap();
-        this.rowKeySchema = RowKeySchema.EMPTY_SCHEMA;
-        this.indexes = Collections.emptyList();
-        this.physicalNames = Collections.emptyList();;
+
+        public PTableImpl build() throws SQLException {
+            // Note that we call initDerivedAttributes to populate derivable attributes if
+            // this.columns is set in the PTableImpl.Builder object
+            return (this.columns == null) ? new PTableImpl(this) :
+                    new PTableImpl(this.initDerivedAttributes());
+        }
+
+    }
+
+    @VisibleForTesting
+    PTableImpl() {
+        this(new PTableImpl.Builder()
+                .setIndexes(Collections.<PTable>emptyList())
+                .setPhysicalNames(Collections.<PName>emptyList())
+                .setRowKeySchema(RowKeySchema.EMPTY_SCHEMA));
+    }
+
+    // Private constructor used by the builder
+    private PTableImpl(Builder builder) {
+        this.key = builder.key;
+        this.name = builder.name;
+        this.schemaName = builder.schemaName;
+        this.tableName = builder.tableName;
+        this.tenantId = builder.tenantId;
+        this.type = builder.type;
+        this.state = builder.state;
+        this.sequenceNumber = builder.sequenceNumber;
+        this.timeStamp = builder.timeStamp;
+        this.indexDisableTimestamp = builder.indexDisableTimestamp;
+        this.pkColumns = builder.pkColumns;
+        this.allColumns = builder.allColumns;
+        this.excludedColumns = builder.excludedColumns;
+        this.families = builder.families;
+        this.familyByBytes = builder.familyByBytes;
+        this.familyByString = builder.familyByString;
+        this.columnsByName = builder.columnsByName;
+        this.kvColumnsByQualifiers = builder.kvColumnsByQualifiers;
+        this.pkName = builder.pkName;
+        this.bucketNum = builder.bucketNum;
+        this.rowKeySchema = builder.rowKeySchema;
+        this.indexes = builder.indexes;
+        this.parentName = builder.parentName;
+        this.parentSchemaName = builder.parentSchemaName;
+        this.parentTableName = builder.parentTableName;
+        this.physicalNames = builder.physicalNames;
+        this.isImmutableRows = builder.isImmutableRows;
+        this.indexMaintainer = builder.indexMaintainer;
+        this.indexMaintainersPtr = builder.indexMaintainersPtr;
+        this.defaultFamilyName = builder.defaultFamilyName;
+        this.viewStatement = builder.viewStatement;
+        this.disableWAL = builder.disableWAL;
+        this.multiTenant = builder.multiTenant;
+        this.storeNulls = builder.storeNulls;
+        this.transactionProvider = builder.transactionProvider;
+        this.viewType = builder.viewType;
+        this.viewIndexType = builder.viewIndexType;
+        this.viewIndexId = builder.viewIndexId;
+        this.estimatedSize = builder.estimatedSize;
+        this.indexType = builder.indexType;
+        this.baseColumnCount = builder.baseColumnCount;
+        this.rowKeyOrderOptimizable = builder.rowKeyOrderOptimizable;
+        this.hasColumnsRequiringUpgrade = builder.hasColumnsRequiringUpgrade;
+        this.rowTimestampColPos = builder.rowTimestampColPos;
+        this.updateCacheFrequency = builder.updateCacheFrequency;
+        this.isNamespaceMapped = builder.isNamespaceMapped;
+        this.autoPartitionSeqName = builder.autoPartitionSeqName;
+        this.isAppendOnlySchema = builder.isAppendOnlySchema;
+        this.immutableStorageScheme = builder.immutableStorageScheme;
+        this.qualifierEncodingScheme = builder.qualifierEncodingScheme;
+        this.encodedCQCounter = builder.encodedCQCounter;
+        this.useStatsForParallelization = builder.useStatsForParallelization;
     }
 
     // When cloning table, ignore the salt column as it will be added back in the constructor
@@ -267,153 +764,65 @@ public class PTableImpl implements PTable {
         return table.getBucketNum() == null ? table.getColumns() : table.getColumns().subList(1, table.getColumns().size());
     }
 
-    public static PTableImpl makePTable(PTable table, long timeStamp, List<PTable> indexes) throws SQLException {
-        return new PTableImpl(table, table.rowKeyOrderOptimizable(), table.getIndexState(), timeStamp,
-                table.getSequenceNumber(), getColumnsToClone(table), table.getDefaultFamilyName(), table.getType(),
-                table.getBaseColumnCount(), table.getSchemaName(), table.getTableName(), table.getViewStatement(),
-                table.getUpdateCacheFrequency(), table.getTenantId(), indexes);
-    }
-
-    public static PTable makePTable(PTable index, PName indexName, String viewStatement, long updateCacheFrequency,
-            PName tenantId) throws SQLException {
-        return Objects.equal(viewStatement, index.getViewStatement()) ? index
-                : new PTableImpl(index, index.rowKeyOrderOptimizable(), index.getIndexState(), index.getTimeStamp(),
-                        index.getSequenceNumber(), index.getColumns(), index.getDefaultFamilyName(), index.getType(),
-                        index.getBaseColumnCount(), index.getSchemaName(), indexName,
-                        viewStatement, updateCacheFrequency, tenantId,
-                        index.getIndexes());
-    }
-    
-    public static PTableImpl makePTable(PTable table, Collection<PColumn> columns) throws SQLException {
-        return new PTableImpl(table, table.rowKeyOrderOptimizable(), table.getIndexState(), table.getTimeStamp(),
-                table.getSequenceNumber(), columns, table.getDefaultFamilyName(), table.getType(),
-                table.getBaseColumnCount(), table.getSchemaName(), table.getTableName(), table.getViewStatement(),
-                table.getUpdateCacheFrequency(), table.getTenantId(), table.getIndexes());
+    /**
+     * Get a PTableImpl.Builder from an existing PTable and set the builder columns
+     * @param table Original PTable
+     * @param columns Columns to set in the builder for the new PTable to be constructed
+     * @return PTable builder object based on an existing PTable
+     */
+    public static PTableImpl.Builder builderWithColumns(PTable table, Collection<PColumn> columns) {
+        return builderFromExisting(table).setColumns(columns);
     }
 
     /**
-     * Used to create a PTable for views or view indexes, the basePTable is for attributes we inherit from the physical table
+     * Get a PTableImpl.Builder from an existing PTable
+     * @param table Original PTable
      */
-    public static PTableImpl makePTable(PTable view, PTable baseTable, Collection<PColumn> columns, long timestamp, int baseTableColumnCount, Collection<PColumn> excludedColumns) throws SQLException {
-        // if a TableProperty is not valid on a view we set it to the base table value
-        // if a TableProperty is valid on a view and is not mutable on a view we set it to the base table value
-        // if a TableProperty is valid on a view and is mutable on a view we use the value set on the view 
-        return new PTableImpl(
-            view.getTenantId(), view.getSchemaName(), view.getTableName(), view.getType(), view.getIndexState(), timestamp,
-            view.getSequenceNumber(), view.getPKName(), view.getBucketNum(), columns, view.getParentSchemaName(), view.getParentTableName(),
-            view.getIndexes(), baseTable.isImmutableRows(), view.getPhysicalNames(), view.getDefaultFamilyName(), view.getViewStatement(),
-            baseTable.isWALDisabled(), baseTable.isMultiTenant(), baseTable.getStoreNulls(), view.getViewType(),
-            view.getViewIndexType(), view.getViewIndexId(), view.getIndexType(),
-            baseTableColumnCount, view.rowKeyOrderOptimizable(), baseTable.getTransactionProvider(), view.getUpdateCacheFrequency(),
-            view.getIndexDisableTimestamp(), view.isNamespaceMapped(), baseTable.getAutoPartitionSeqName(), baseTable.isAppendOnlySchema(),
-            baseTable.getImmutableStorageScheme(), baseTable.getEncodingScheme(), view.getEncodedCQCounter(), view.useStatsForParallelization(), excludedColumns);
+    private static PTableImpl.Builder builderFromExisting(PTable table) {
+        return new PTableImpl.Builder()
+                .setType(table.getType())
+                .setState(table.getIndexState())
+                .setTimeStamp(table.getTimeStamp())
+                .setIndexDisableTimestamp(table.getIndexDisableTimestamp())
+                .setSequenceNumber(table.getSequenceNumber())
+                .setImmutableRows(table.isImmutableRows())
+                .setViewStatement(table.getViewStatement())
+                .setDisableWAL(table.isWALDisabled())
+                .setMultiTenant(table.isMultiTenant())
+                .setStoreNulls(table.getStoreNulls())
+                .setViewType(table.getViewType())
+                .setViewIndexType(table.getViewIndexType())
+                .setViewIndexId(table.getViewIndexId())
+                .setIndexType(table.getIndexType())
+                .setTransactionProvider(table.getTransactionProvider())
+                .setUpdateCacheFrequency(table.getUpdateCacheFrequency())
+                .setNamespaceMapped(table.isNamespaceMapped())
+                .setAutoPartitionSeqName(table.getAutoPartitionSeqName())
+                .setAppendOnlySchema(table.isAppendOnlySchema())
+                .setImmutableStorageScheme(table.getImmutableStorageScheme() == null ?
+                        ImmutableStorageScheme.ONE_CELL_PER_COLUMN : table.getImmutableStorageScheme())
+                .setQualifierEncodingScheme(table.getEncodingScheme() == null ?
+                        QualifierEncodingScheme.NON_ENCODED_QUALIFIERS : table.getEncodingScheme())
+                .setBaseColumnCount(table.getBaseColumnCount())
+                .setEncodedCQCounter(table.getEncodedCQCounter())
+                .setUseStatsForParallelization(table.useStatsForParallelization())
+                .setExcludedColumns(table.getExcludedColumns() == null ?
+                        ImmutableList.<PColumn>of() : ImmutableList.copyOf(table.getExcludedColumns()))
+                .setTenantId(table.getTenantId())
+                .setSchemaName(table.getSchemaName())
+                .setTableName(table.getTableName())
+                .setPkName(table.getPKName())
+                .setDefaultFamilyName(table.getDefaultFamilyName())
+                .setRowKeyOrderOptimizable(table.rowKeyOrderOptimizable())
+                .setBucketNum(table.getBucketNum())
+                .setIndexes(table.getIndexes() == null ?
+                        Collections.<PTable>emptyList() : table.getIndexes())
+                .setParentSchemaName(table.getParentSchemaName())
+                .setParentTableName(table.getParentTableName())
+                .setPhysicalNames(table.getPhysicalNames() == null ?
+                        ImmutableList.<PName>of() : ImmutableList.copyOf(table.getPhysicalNames()));
     }
-    
-    public static PTableImpl makePTable(PTable table, PTableType type, Collection<PColumn> columns) throws SQLException {
-        return new PTableImpl(table, table.rowKeyOrderOptimizable(), table.getIndexState(), table.getTimeStamp(),
-                table.getSequenceNumber(), columns, table.getDefaultFamilyName(), type,
-                table.getBaseColumnCount(), table.getSchemaName(), table.getTableName(), table.getViewStatement(),
-                table.getUpdateCacheFrequency(), table.getTenantId(), table.getIndexes());
-    }
-
-    public static PTableImpl makePTable(PTable table, Collection<PColumn> columns, PName defaultFamily)
-            throws SQLException {
-        return new PTableImpl(table, table.rowKeyOrderOptimizable(), table.getIndexState(), table.getTimeStamp(),
-                table.getSequenceNumber(), columns, defaultFamily, table.getType(),
-                table.getBaseColumnCount(), table.getSchemaName(), table.getTableName(), table.getViewStatement(),
-                table.getUpdateCacheFrequency(), table.getTenantId(), table.getIndexes());
-    }
-
-    public static PTableImpl makePTable(PTable table, long timeStamp, long sequenceNumber, Collection<PColumn> columns)
-            throws SQLException {
-        return new PTableImpl(table, table.rowKeyOrderOptimizable(), table.getIndexState(), timeStamp,
-                sequenceNumber, columns, table.getDefaultFamilyName(), table.getType(),
-                table.getBaseColumnCount(), table.getSchemaName(), table.getTableName(), table.getViewStatement(),
-                table.getUpdateCacheFrequency(), table.getTenantId(), table.getIndexes());
-    }
-
-    public static PTableImpl makePTable(PTable table, PIndexState state) throws SQLException {
-        return new PTableImpl(table, table.rowKeyOrderOptimizable(), state, table.getTimeStamp(),
-                table.getSequenceNumber(), getColumnsToClone(table), table.getDefaultFamilyName(), table.getType(),
-                table.getBaseColumnCount(), table.getSchemaName(), table.getTableName(), table.getViewStatement(),
-                table.getUpdateCacheFrequency(), table.getTenantId(), table.getIndexes());
-    }
-
-    public static PTableImpl makePTable(PTable table, boolean rowKeyOrderOptimizable) throws SQLException {
-        return new PTableImpl(table, rowKeyOrderOptimizable, table.getIndexState(), table.getTimeStamp(),
-                table.getSequenceNumber(), getColumnsToClone(table), table.getDefaultFamilyName(), table.getType(),
-                table.getBaseColumnCount(), table.getSchemaName(), table.getTableName(), table.getViewStatement(),
-                table.getUpdateCacheFrequency(), table.getTenantId(), table.getIndexes());
-    }
-
-    public static PTableImpl makePTable(PName tenantId, PName schemaName, PName tableName, PTableType type,
-            PIndexState state, long timeStamp, long sequenceNumber, PName pkName, Integer bucketNum,
-            Collection<PColumn> columns, PName dataSchemaName, PName dataTableName, List<PTable> indexes,
-            boolean isImmutableRows, List<PName> physicalNames, PName defaultFamilyName, String viewExpression,
-            boolean disableWAL, boolean multiTenant, boolean storeNulls, ViewType viewType, PDataType viewIndexType, Long viewIndexId,
-            IndexType indexType, boolean rowKeyOrderOptimizable, TransactionFactory.Provider transactionProvider,
-            long updateCacheFrequency, long indexDisableTimestamp, boolean isNamespaceMapped,
-            String autoPartitionSeqName, boolean isAppendOnlySchema, ImmutableStorageScheme storageScheme,
-            QualifierEncodingScheme qualifierEncodingScheme, EncodedCQCounter encodedCQCounter,
-            Boolean useStatsForParallelization) throws SQLException {
-        return new PTableImpl(tenantId, schemaName, tableName, type, state, timeStamp, sequenceNumber, pkName,
-                bucketNum, columns, dataSchemaName, dataTableName, indexes, isImmutableRows, physicalNames,
-                defaultFamilyName, viewExpression, disableWAL, multiTenant, storeNulls, viewType, viewIndexType, viewIndexId,
-                indexType, QueryConstants.BASE_TABLE_BASE_COLUMN_COUNT, rowKeyOrderOptimizable, transactionProvider,
-                updateCacheFrequency, indexDisableTimestamp, isNamespaceMapped, autoPartitionSeqName,
-                isAppendOnlySchema, storageScheme, qualifierEncodingScheme, encodedCQCounter,
-                useStatsForParallelization, null);
-    }
-
-    public static PTableImpl makePTable(PName tenantId, PName schemaName, PName tableName, PTableType type,
-            PIndexState state, long timeStamp, long sequenceNumber, PName pkName, Integer bucketNum,
-            Collection<PColumn> columns, PName dataSchemaName, PName dataTableName, List<PTable> indexes,
-            boolean isImmutableRows, List<PName> physicalNames, PName defaultFamilyName, String viewExpression,
-            boolean disableWAL, boolean multiTenant, boolean storeNulls, ViewType viewType, PDataType viewIndexType, Long viewIndexId,
-            IndexType indexType, boolean rowKeyOrderOptimizable, TransactionFactory.Provider transactionProvider,
-            long updateCacheFrequency, int baseColumnCount, long indexDisableTimestamp, boolean isNamespaceMapped,
-            String autoPartitionSeqName, boolean isAppendOnlySchema, ImmutableStorageScheme storageScheme,
-            QualifierEncodingScheme qualifierEncodingScheme, EncodedCQCounter encodedCQCounter,
-            Boolean useStatsForParallelization) throws SQLException {
-        return new PTableImpl(tenantId, schemaName, tableName, type, state, timeStamp, sequenceNumber, pkName,
-                bucketNum, columns, dataSchemaName, dataTableName, indexes, isImmutableRows, physicalNames,
-                defaultFamilyName, viewExpression, disableWAL, multiTenant, storeNulls, viewType,viewIndexType,  viewIndexId,
-                indexType, baseColumnCount, rowKeyOrderOptimizable, transactionProvider, updateCacheFrequency,
-                indexDisableTimestamp, isNamespaceMapped, autoPartitionSeqName, isAppendOnlySchema, storageScheme,
-                qualifierEncodingScheme, encodedCQCounter, useStatsForParallelization, null);
-    }
-
-    private PTableImpl(PTable table, boolean rowKeyOrderOptimizable, PIndexState state, long timeStamp,
-            long sequenceNumber, Collection<PColumn> columns, PName defaultFamily, PTableType type,
-            int baseTableColumnCount, PName schemaName, PName tableName, String viewStatement,
-            long updateCacheFrequency, PName tenantId, List<PTable> indexes) throws SQLException {
-        init(tenantId, schemaName, tableName, type, state, timeStamp, sequenceNumber, table.getPKName(),
-                table.getBucketNum(), columns, table.getParentSchemaName(), table.getParentTableName(), indexes,
-                table.isImmutableRows(), table.getPhysicalNames(), defaultFamily, viewStatement, table.isWALDisabled(),
-                table.isMultiTenant(), table.getStoreNulls(), table.getViewType(), table.getViewIndexType(), table.getViewIndexId(),
-                table.getIndexType(), baseTableColumnCount, rowKeyOrderOptimizable, table.getTransactionProvider(),
-                updateCacheFrequency, table.getIndexDisableTimestamp(), table.isNamespaceMapped(),
-                table.getAutoPartitionSeqName(), table.isAppendOnlySchema(), table.getImmutableStorageScheme(),
-                table.getEncodingScheme(), table.getEncodedCQCounter(), table.useStatsForParallelization(), null);
-    }
-
-    private PTableImpl(PName tenantId, PName schemaName, PName tableName, PTableType type, PIndexState state,
-            long timeStamp, long sequenceNumber, PName pkName, Integer bucketNum, Collection<PColumn> columns,
-            PName parentSchemaName, PName parentTableName, List<PTable> indexes, boolean isImmutableRows,
-            List<PName> physicalNames, PName defaultFamilyName, String viewExpression, boolean disableWAL, boolean multiTenant,
-            boolean storeNulls, ViewType viewType, PDataType viewIndexType, Long viewIndexId, IndexType indexType,
-            int baseColumnCount, boolean rowKeyOrderOptimizable, TransactionFactory.Provider transactionProvider, long updateCacheFrequency,
-            long indexDisableTimestamp, boolean isNamespaceMapped, String autoPartitionSeqName, boolean isAppendOnlySchema, ImmutableStorageScheme storageScheme, 
-            QualifierEncodingScheme qualifierEncodingScheme, EncodedCQCounter encodedCQCounter,
-            Boolean useStatsForParallelization, Collection<PColumn> excludedColumns)
-            throws SQLException {
-        init(tenantId, schemaName, tableName, type, state, timeStamp, sequenceNumber, pkName, bucketNum, columns,
-                parentSchemaName, parentTableName, indexes, isImmutableRows, physicalNames, defaultFamilyName,
-                viewExpression, disableWAL, multiTenant, storeNulls, viewType, viewIndexType, viewIndexId, indexType, baseColumnCount, rowKeyOrderOptimizable,
-                transactionProvider, updateCacheFrequency, indexDisableTimestamp, isNamespaceMapped, autoPartitionSeqName, isAppendOnlySchema, storageScheme, 
-                qualifierEncodingScheme, encodedCQCounter, useStatsForParallelization, excludedColumns);
-    }
-    
+
     @Override
     public long getUpdateCacheFrequency() {
         return updateCacheFrequency;
@@ -434,208 +843,14 @@ public class PTableImpl implements PTable {
         return viewType;
     }
 
-
     @Override
     public int getEstimatedSize() {
         return estimatedSize;
     }
 
-    private void init(PName tenantId, PName schemaName, PName tableName, PTableType type, PIndexState state, long timeStamp, long sequenceNumber,
-            PName pkName, Integer bucketNum, Collection<PColumn> columns, PName parentSchemaName, PName parentTableName,
-            List<PTable> indexes, boolean isImmutableRows, List<PName> physicalNames, PName defaultFamilyName, String viewExpression, boolean disableWAL,
-            boolean multiTenant, boolean storeNulls, ViewType viewType,PDataType viewIndexType,  Long viewIndexId,
-            IndexType indexType , int baseColumnCount, boolean rowKeyOrderOptimizable, TransactionFactory.Provider transactionProvider, long updateCacheFrequency, long indexDisableTimestamp, 
-            boolean isNamespaceMapped, String autoPartitionSeqName, boolean isAppendOnlySchema, ImmutableStorageScheme storageScheme, QualifierEncodingScheme qualifierEncodingScheme, 
-            EncodedCQCounter encodedCQCounter, Boolean useStatsForParallelization, Collection<PColumn> excludedColumns) throws SQLException {
-        Preconditions.checkNotNull(schemaName);
-        Preconditions.checkArgument(tenantId==null || tenantId.getBytes().length > 0); // tenantId should be null or not empty
-        int estimatedSize = SizedUtil.OBJECT_SIZE * 2 + 23 * SizedUtil.POINTER_SIZE + 4 * SizedUtil.INT_SIZE + 2 * SizedUtil.LONG_SIZE + 2 * SizedUtil.INT_OBJECT_SIZE +
-              PNameFactory.getEstimatedSize(tenantId) +
-              PNameFactory.getEstimatedSize(schemaName) +
-              PNameFactory.getEstimatedSize(tableName) +
-              PNameFactory.getEstimatedSize(pkName) +
-              PNameFactory.getEstimatedSize(parentTableName) +
-              PNameFactory.getEstimatedSize(defaultFamilyName);
-        this.tenantId = tenantId;
-        this.schemaName = schemaName;
-        this.tableName = tableName;
-        this.name = PNameFactory.newName(SchemaUtil.getTableName(schemaName.getString(), tableName.getString()));
-        this.key = new PTableKey(tenantId, name.getString());
-        this.type = type;
-        this.state = state;
-        this.timeStamp = timeStamp;
-        this.indexDisableTimestamp = indexDisableTimestamp;
-        this.sequenceNumber = sequenceNumber;
-        this.pkName = pkName;
-        this.isImmutableRows = isImmutableRows;
-        this.defaultFamilyName = defaultFamilyName;
-        this.viewStatement = viewExpression;
-        this.disableWAL = disableWAL;
-        this.multiTenant = multiTenant;
-        this.storeNulls = storeNulls;
-        this.viewType = viewType;
-        this.viewIndexType = viewIndexType;
-        this.viewIndexId = viewIndexId;
-        this.indexType = indexType;
-        this.transactionProvider = transactionProvider;
-        this.rowKeyOrderOptimizable = rowKeyOrderOptimizable;
-        this.updateCacheFrequency = updateCacheFrequency;
-        this.isNamespaceMapped = isNamespaceMapped;
-        this.autoPartitionSeqName = autoPartitionSeqName;
-        this.isAppendOnlySchema = isAppendOnlySchema;
-        // null check for backward compatibility and sanity. If any of the two below is null, then it means the table is a non-encoded table.
-        this.immutableStorageScheme = storageScheme == null ? ImmutableStorageScheme.ONE_CELL_PER_COLUMN : storageScheme;
-        this.qualifierEncodingScheme = qualifierEncodingScheme == null ? QualifierEncodingScheme.NON_ENCODED_QUALIFIERS : qualifierEncodingScheme;
-        List<PColumn> pkColumns;
-        PColumn[] allColumns;
-        
-        this.columnsByName = ArrayListMultimap.create(columns.size(), 1);
-        this.kvColumnsByQualifiers = Maps.newHashMapWithExpectedSize(columns.size());
-        int numPKColumns = 0;
-        if (bucketNum != null) {
-            // Add salt column to allColumns and pkColumns, but don't add to
-            // columnsByName, since it should not be addressable via name.
-            allColumns = new PColumn[columns.size()+1];
-            allColumns[SALTING_COLUMN.getPosition()] = SALTING_COLUMN;
-            pkColumns = Lists.newArrayListWithExpectedSize(columns.size()+1);
-            ++numPKColumns;
-        } else {
-            allColumns = new PColumn[columns.size()];
-            pkColumns = Lists.newArrayListWithExpectedSize(columns.size());
-        }
-        // Must do this as with the new method of storing diffs, we just care about ordinal position
-        // relative order and not the true ordinal value itself.
-        List<PColumn> sortedColumns = Lists.newArrayList(columns);
-        Collections.sort(sortedColumns, new Comparator<PColumn>() {
-            @Override
-            public int compare(PColumn o1, PColumn o2) {
-                return Integer.valueOf(o1.getPosition()).compareTo(o2.getPosition());
-            }
-        });
-
-        int position = 0;
-        if (bucketNum != null) {
-            position = 1;
-        }
-        for (PColumn column : sortedColumns) {
-            allColumns[position] = column;
-            position++;
-            PName familyName = column.getFamilyName();
-            if (familyName == null) {
-                ++numPKColumns;
-            }
-            String columnName = column.getName().getString();
-            if (columnsByName.put(columnName, column)) {
-                int count = 0;
-                for (PColumn dupColumn : columnsByName.get(columnName)) {
-                    if (Objects.equal(familyName, dupColumn.getFamilyName())) {
-                        count++;
-                        if (count > 1) {
-                            throw new ColumnAlreadyExistsException(schemaName.getString(), name.getString(), columnName);
-                        }
-                    }
-                }
-            }
-            byte[] cq = column.getColumnQualifierBytes();
-            String cf = column.getFamilyName() != null ? column.getFamilyName().getString() : null;
-            if (cf != null && cq != null) {
-                KVColumnFamilyQualifier info = new KVColumnFamilyQualifier(cf, cq);
-                if (kvColumnsByQualifiers.get(info) != null) {
-                    throw new ColumnAlreadyExistsException(schemaName.getString(),
-                            name.getString(), columnName);
-                }
-                kvColumnsByQualifiers.put(info, column);
-            }
-        }
-        estimatedSize += SizedUtil.sizeOfMap(allColumns.length, SizedUtil.POINTER_SIZE, SizedUtil.sizeOfArrayList(1)); // for multi-map
-
-        this.bucketNum = bucketNum;
-        this.allColumns = ImmutableList.copyOf(allColumns);
-        estimatedSize += SizedUtil.sizeOfMap(numPKColumns) + SizedUtil.sizeOfMap(allColumns.length);
-
-        RowKeySchemaBuilder builder = new RowKeySchemaBuilder(numPKColumns);
-        // Two pass so that column order in column families matches overall column order
-        // and to ensure that column family order is constant
-        int maxExpectedSize = allColumns.length - numPKColumns;
-        // Maintain iteration order so that column families are ordered as they are listed
-        Map<PName, List<PColumn>> familyMap = Maps.newLinkedHashMap();
-        PColumn rowTimestampCol = null;
-        for (PColumn column : allColumns) {
-            PName familyName = column.getFamilyName();
-            if (familyName == null) {
-                hasColumnsRequiringUpgrade |= 
-                        ( column.getSortOrder() == SortOrder.DESC 
-                            && (!column.getDataType().isFixedWidth() 
-                                || column.getDataType() == PChar.INSTANCE 
-                                || column.getDataType() == PFloat.INSTANCE 
-                                || column.getDataType() == PDouble.INSTANCE 
-                                || column.getDataType() == PBinary.INSTANCE) )
-                        || (column.getSortOrder() == SortOrder.ASC && column.getDataType() == PBinary.INSTANCE && column.getMaxLength() != null && column.getMaxLength() > 1);
-                pkColumns.add(column);
-                if (column.isRowTimestamp()) {
-                    rowTimestampCol = column;
-                }
-            }
-            if (familyName == null) {
-                estimatedSize += column.getEstimatedSize(); // PK columns
-                builder.addField(column, column.isNullable(), column.getSortOrder());
-            } else {
-                List<PColumn> columnsInFamily = familyMap.get(familyName);
-                if (columnsInFamily == null) {
-                    columnsInFamily = Lists.newArrayListWithExpectedSize(maxExpectedSize);
-                    familyMap.put(familyName, columnsInFamily);
-                }
-                columnsInFamily.add(column);
-            }
-        }
-        this.pkColumns = ImmutableList.copyOf(pkColumns);
-        if (rowTimestampCol != null) {
-            this.rowTimestampColPos = this.pkColumns.indexOf(rowTimestampCol);
-        } else {
-            this.rowTimestampColPos = -1;
-        }
-        
-        builder.rowKeyOrderOptimizable(this.rowKeyOrderOptimizable()); // after hasDescVarLengthColumns is calculated
-        this.rowKeySchema = builder.build();
-        estimatedSize += rowKeySchema.getEstimatedSize();
-        Iterator<Map.Entry<PName,List<PColumn>>> iterator = familyMap.entrySet().iterator();
-        PColumnFamily[] families = new PColumnFamily[familyMap.size()];
-        ImmutableMap.Builder<String, PColumnFamily> familyByString = ImmutableMap.builder();
-        ImmutableSortedMap.Builder<byte[], PColumnFamily> familyByBytes = ImmutableSortedMap
-                .orderedBy(Bytes.BYTES_COMPARATOR);
-        for (int i = 0; i < families.length; i++) {
-            Map.Entry<PName,List<PColumn>> entry = iterator.next();
-            PColumnFamily family = new PColumnFamilyImpl(entry.getKey(), entry.getValue());
-            families[i] = family;
-            familyByString.put(family.getName().getString(), family);
-            familyByBytes.put(family.getName().getBytes(), family);
-            estimatedSize += family.getEstimatedSize();
-        }
-        this.families = ImmutableList.copyOf(families);
-        this.familyByBytes = familyByBytes.build();
-        this.familyByString = familyByString.build();
-        estimatedSize += SizedUtil.sizeOfArrayList(families.length);
-        estimatedSize += SizedUtil.sizeOfMap(families.length) * 2;
-        this.indexes = indexes == null ? Collections.<PTable>emptyList() : indexes;
-        for (PTable index : this.indexes) {
-            estimatedSize += index.getEstimatedSize();
-        }
-
-        this.parentSchemaName = parentSchemaName;
-        this.parentTableName = parentTableName;
-        this.parentName = parentTableName == null ? null : PNameFactory.newName(SchemaUtil.getTableName(
-            parentSchemaName!=null ? parentSchemaName.getString() : null, parentTableName.getString()));
-        estimatedSize += PNameFactory.getEstimatedSize(this.parentName);
-
-        this.physicalNames = physicalNames == null ? ImmutableList.<PName>of() : ImmutableList.copyOf(physicalNames);
-        for (PName name : this.physicalNames) {
-            estimatedSize += name.getEstimatedSize();
-        }
-        this.estimatedSize = estimatedSize;
-        this.baseColumnCount = baseColumnCount;
-        this.encodedCQCounter = encodedCQCounter;
-        this.useStatsForParallelization = useStatsForParallelization;
-        this.excludedColumns = excludedColumns == null ? ImmutableList.<PColumn>of() : ImmutableList.copyOf(excludedColumns);
+    public static void checkTenantId(PName tenantId) {
+        // tenantId should be null or not empty
+        Preconditions.checkArgument(tenantId == null || tenantId.getBytes().length > 0);
     }
 
     @Override
@@ -1340,9 +1555,9 @@ public class PTableImpl implements PTable {
         if (table.hasIsNamespaceMapped()) {
             isNamespaceMapped = table.getIsNamespaceMapped();
         }
-        String autoParititonSeqName = null;
+        String autoPartitionSeqName = null;
         if (table.hasAutoParititonSeqName()) {
-            autoParititonSeqName = table.getAutoParititonSeqName();
+            autoPartitionSeqName = table.getAutoParititonSeqName();
         }
         boolean isAppendOnlySchema = false;
         if (table.hasIsAppendOnlySchema()) {
@@ -1358,7 +1573,7 @@ public class PTableImpl implements PTable {
         if (table.hasEncodingScheme()) {
             qualifierEncodingScheme = QualifierEncodingScheme.fromSerializedValue(table.getEncodingScheme().toByteArray()[0]);
         }
-        EncodedCQCounter encodedColumnQualifierCounter = null;
+        EncodedCQCounter encodedColumnQualifierCounter;
         if ((!EncodedColumnsUtil.usesEncodedColumnNames(qualifierEncodingScheme) || tableType == PTableType.VIEW)) {
             encodedColumnQualifierCounter = PTable.EncodedCQCounter.NULL_COUNTER;
         }
@@ -1375,14 +1590,50 @@ public class PTableImpl implements PTable {
             useStatsForParallelization = table.getUseStatsForParallelization();
         }
         try {
-            PTableImpl result = new PTableImpl();
-            result.init(tenantId, schemaName, tableName, tableType, indexState, timeStamp, sequenceNumber, pkName,
-                (bucketNum == NO_SALTING) ? null : bucketNum, columns, parentSchemaName, parentTableName, indexes,
-                        isImmutableRows, physicalNames, defaultFamilyName, viewStatement, disableWAL,
-                        multiTenant, storeNulls, viewType, viewIndexType, viewIndexId, indexType, baseColumnCount, rowKeyOrderOptimizable,
-                        transactionProvider, updateCacheFrequency, indexDisableTimestamp, isNamespaceMapped, autoParititonSeqName, 
-                        isAppendOnlySchema, storageScheme, qualifierEncodingScheme, encodedColumnQualifierCounter, useStatsForParallelization, null);
-            return result;
+            return new PTableImpl.Builder()
+                    .setType(tableType)
+                    .setState(indexState)
+                    .setTimeStamp(timeStamp)
+                    .setIndexDisableTimestamp(indexDisableTimestamp)
+                    .setSequenceNumber(sequenceNumber)
+                    .setImmutableRows(isImmutableRows)
+                    .setViewStatement(viewStatement)
+                    .setDisableWAL(disableWAL)
+                    .setMultiTenant(multiTenant)
+                    .setStoreNulls(storeNulls)
+                    .setViewType(viewType)
+                    .setViewIndexType(viewIndexType)
+                    .setViewIndexId(viewIndexId)
+                    .setIndexType(indexType)
+                    .setTransactionProvider(transactionProvider)
+                    .setUpdateCacheFrequency(updateCacheFrequency)
+                    .setNamespaceMapped(isNamespaceMapped)
+                    .setAutoPartitionSeqName(autoPartitionSeqName)
+                    .setAppendOnlySchema(isAppendOnlySchema)
+                    // null check for backward compatibility and sanity. If any of the two below is null,
+                    // then it means the table is a non-encoded table.
+                    .setImmutableStorageScheme(storageScheme == null ?
+                            ImmutableStorageScheme.ONE_CELL_PER_COLUMN : storageScheme)
+                    .setQualifierEncodingScheme(qualifierEncodingScheme == null ?
+                            QualifierEncodingScheme.NON_ENCODED_QUALIFIERS : qualifierEncodingScheme)
+                    .setBaseColumnCount(baseColumnCount)
+                    .setEncodedCQCounter(encodedColumnQualifierCounter)
+                    .setUseStatsForParallelization(useStatsForParallelization)
+                    .setExcludedColumns(ImmutableList.<PColumn>of())
+                    .setTenantId(tenantId)
+                    .setSchemaName(schemaName)
+                    .setTableName(tableName)
+                    .setPkName(pkName)
+                    .setDefaultFamilyName(defaultFamilyName)
+                    .setRowKeyOrderOptimizable(rowKeyOrderOptimizable)
+                    .setBucketNum((bucketNum == NO_SALTING) ? null : bucketNum)
+                    .setIndexes(indexes == null ? Collections.<PTable>emptyList() : indexes)
+                    .setParentSchemaName(parentSchemaName)
+                    .setParentTableName(parentTableName)
+                    .setPhysicalNames(physicalNames == null ?
+                            ImmutableList.<PName>of() : ImmutableList.copyOf(physicalNames))
+                    .setColumns(columns)
+                    .build();
         } catch (SQLException e) {
             throw new RuntimeException(e); // Impossible
         }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/1767244a/phoenix-core/src/main/java/org/apache/phoenix/schema/TableRef.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/TableRef.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/TableRef.java
index 37cae22..b40c0b8 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/TableRef.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/TableRef.java
@@ -17,6 +17,8 @@
  */
 package org.apache.phoenix.schema;
 
+import java.sql.SQLException;
+import java.util.Collections;
 import java.util.Objects;
 
 import org.apache.phoenix.compile.TupleProjectionCompiler;
@@ -28,7 +30,7 @@ import org.apache.phoenix.util.SchemaUtil;
 
 
 public class TableRef {
-    public static final TableRef EMPTY_TABLE_REF = new TableRef(new PTableImpl());
+    public static final TableRef EMPTY_TABLE_REF = createEmptyTableRef();
     
     private PTable table;
     private long upperBoundTimeStamp;
@@ -37,6 +39,19 @@ public class TableRef {
     private final boolean hasDynamicCols;
     private final long currentTime;
 
+    private static TableRef createEmptyTableRef() {
+        try {
+            return new TableRef(new PTableImpl.Builder()
+                    .setIndexes(Collections.<PTable>emptyList())
+                    .setPhysicalNames(Collections.<PName>emptyList())
+                    .setRowKeySchema(RowKeySchema.EMPTY_SCHEMA)
+                    .build());
+        } catch (SQLException e) {
+            // Should never happen
+            return null;
+        }
+    }
+
     public TableRef(TableRef tableRef) {
         this(tableRef.alias, tableRef.table, tableRef.upperBoundTimeStamp, tableRef.lowerBoundTimeStamp, tableRef.hasDynamicCols);
     }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/1767244a/phoenix-core/src/test/java/org/apache/phoenix/execute/CorrelatePlanTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/execute/CorrelatePlanTest.java b/phoenix-core/src/test/java/org/apache/phoenix/execute/CorrelatePlanTest.java
index 26caa0d..34f82d2 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/execute/CorrelatePlanTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/execute/CorrelatePlanTest.java
@@ -17,6 +17,7 @@
  */
 package org.apache.phoenix.execute;
 
+import static org.apache.phoenix.query.QueryConstants.BASE_TABLE_BASE_COLUMN_COUNT;
 import static org.apache.phoenix.query.QueryConstants.VALUE_COLUMN_FAMILY;
 import static org.apache.phoenix.util.PhoenixRuntime.CONNECTIONLESS;
 import static org.apache.phoenix.util.PhoenixRuntime.JDBC_PROTOCOL;
@@ -30,6 +31,7 @@ import java.util.Arrays;
 import java.util.Collections;
 import java.util.List;
 
+import com.google.common.collect.ImmutableList;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.client.Scan;
@@ -258,11 +260,31 @@ public class CorrelatePlanTest {
                     i, expr.getSortOrder(), null, null, false, name, false, false, colName.getBytes(), HConstants.LATEST_TIMESTAMP));
         }
         try {
-            PTable pTable = PTableImpl.makePTable(null, PName.EMPTY_NAME, PName.EMPTY_NAME,
-                    PTableType.SUBQUERY, null, MetaDataProtocol.MIN_TABLE_TIMESTAMP, PTable.INITIAL_SEQ_NUM,
-                    null, null, columns, null, null, Collections.<PTable>emptyList(),
-                    false, Collections.<PName>emptyList(), null, null, false, false, false, null, null,
-                    null, null, true, null, 0, 0L, Boolean.FALSE, null, false, ImmutableStorageScheme.ONE_CELL_PER_COLUMN, QualifierEncodingScheme.NON_ENCODED_QUALIFIERS, EncodedCQCounter.NULL_COUNTER, true);
+            PTable pTable =  new PTableImpl.Builder()
+                    .setType(PTableType.SUBQUERY)
+                    .setTimeStamp(MetaDataProtocol.MIN_TABLE_TIMESTAMP)
+                    .setIndexDisableTimestamp(0L)
+                    .setSequenceNumber(PTable.INITIAL_SEQ_NUM)
+                    .setImmutableRows(false)
+                    .setDisableWAL(false)
+                    .setMultiTenant(false)
+                    .setStoreNulls(false)
+                    .setUpdateCacheFrequency(0)
+                    .setNamespaceMapped(Boolean.FALSE)
+                    .setAppendOnlySchema(false)
+                    .setImmutableStorageScheme(ImmutableStorageScheme.ONE_CELL_PER_COLUMN)
+                    .setQualifierEncodingScheme(QualifierEncodingScheme.NON_ENCODED_QUALIFIERS)
+                    .setBaseColumnCount(BASE_TABLE_BASE_COLUMN_COUNT)
+                    .setEncodedCQCounter(EncodedCQCounter.NULL_COUNTER)
+                    .setUseStatsForParallelization(true)
+                    .setExcludedColumns(ImmutableList.<PColumn>of())
+                    .setSchemaName(PName.EMPTY_NAME)
+                    .setTableName(PName.EMPTY_NAME)
+                    .setRowKeyOrderOptimizable(true)
+                    .setIndexes(Collections.<PTable>emptyList())
+                    .setPhysicalNames(ImmutableList.<PName>of())
+                    .setColumns(columns)
+                    .build();
             TableRef sourceTable = new TableRef(pTable);
             List<ColumnRef> sourceColumnRefs = Lists.<ColumnRef> newArrayList();
             for (PColumn column : sourceTable.getTable().getColumns()) {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/1767244a/phoenix-core/src/test/java/org/apache/phoenix/execute/LiteralResultIteratorPlanTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/execute/LiteralResultIteratorPlanTest.java b/phoenix-core/src/test/java/org/apache/phoenix/execute/LiteralResultIteratorPlanTest.java
index f400d0b..110bd26 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/execute/LiteralResultIteratorPlanTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/execute/LiteralResultIteratorPlanTest.java
@@ -17,6 +17,7 @@
  */
 package org.apache.phoenix.execute;
 
+import static org.apache.phoenix.query.QueryConstants.BASE_TABLE_BASE_COLUMN_COUNT;
 import static org.apache.phoenix.query.QueryConstants.VALUE_COLUMN_FAMILY;
 import static org.apache.phoenix.util.PhoenixRuntime.CONNECTIONLESS;
 import static org.apache.phoenix.util.PhoenixRuntime.JDBC_PROTOCOL;
@@ -30,6 +31,7 @@ import java.sql.SQLException;
 import java.util.Collections;
 import java.util.List;
 
+import com.google.common.collect.ImmutableList;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.client.Scan;
@@ -181,12 +183,33 @@ public class LiteralResultIteratorPlanTest {
                 HConstants.LATEST_TIMESTAMP));
         }
         try {
-            PTable pTable = PTableImpl.makePTable(null, PName.EMPTY_NAME, PName.EMPTY_NAME, PTableType.SUBQUERY, null,
-                    MetaDataProtocol.MIN_TABLE_TIMESTAMP, PTable.INITIAL_SEQ_NUM, null, null, columns, null, null,
-                    Collections.<PTable> emptyList(), false, Collections.<PName> emptyList(), null, null, false, false,
-                    false, null, null, null, null, true, null, 0, 0L, false, null, false, ImmutableStorageScheme.ONE_CELL_PER_COLUMN, QualifierEncodingScheme.NON_ENCODED_QUALIFIERS, EncodedCQCounter.NULL_COUNTER, true);
+            PTable pTable = new PTableImpl.Builder()
+                    .setType(PTableType.SUBQUERY)
+                    .setTimeStamp(MetaDataProtocol.MIN_TABLE_TIMESTAMP)
+                    .setIndexDisableTimestamp(0L)
+                    .setSequenceNumber(PTable.INITIAL_SEQ_NUM)
+                    .setImmutableRows(false)
+                    .setDisableWAL(false)
+                    .setMultiTenant(false)
+                    .setStoreNulls(false)
+                    .setUpdateCacheFrequency(0)
+                    .setNamespaceMapped(false)
+                    .setAppendOnlySchema(false)
+                    .setImmutableStorageScheme(ImmutableStorageScheme.ONE_CELL_PER_COLUMN)
+                    .setQualifierEncodingScheme(QualifierEncodingScheme.NON_ENCODED_QUALIFIERS)
+                    .setBaseColumnCount(BASE_TABLE_BASE_COLUMN_COUNT)
+                    .setEncodedCQCounter(EncodedCQCounter.NULL_COUNTER)
+                    .setUseStatsForParallelization(true)
+                    .setExcludedColumns(ImmutableList.<PColumn>of())
+                    .setSchemaName(PName.EMPTY_NAME)
+                    .setTableName(PName.EMPTY_NAME)
+                    .setRowKeyOrderOptimizable(true)
+                    .setIndexes(Collections.<PTable>emptyList())
+                    .setPhysicalNames(ImmutableList.<PName>of())
+                    .setColumns(columns)
+                    .build();
             TableRef sourceTable = new TableRef(pTable);
-            List<ColumnRef> sourceColumnRefs = Lists.<ColumnRef> newArrayList();
+            List<ColumnRef> sourceColumnRefs = Lists.newArrayList();
             for (PColumn column : sourceTable.getTable().getColumns()) {
                 sourceColumnRefs.add(new ColumnRef(sourceTable, column.getPosition()));
             }


[07/28] phoenix git commit: PHOENIX-4981 Add tests for ORDER BY, GROUP BY and salted tables using phoenix-spark ( changes for spark 1.6 - CDH 5.15 )

Posted by pb...@apache.org.
PHOENIX-4981 Add tests for ORDER BY, GROUP BY and salted tables using phoenix-spark ( changes for spark 1.6 - CDH 5.15 )


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

Branch: refs/heads/4.x-cdh5.15
Commit: 7eb336de12a350608c9e24f2c6d70eb35d2a0d52
Parents: 678563f
Author: Pedro Boado <pb...@apache.org>
Authored: Mon Nov 26 12:50:00 2018 +0000
Committer: Pedro Boado <pb...@apache.org>
Committed: Tue Nov 27 15:10:31 2018 +0000

----------------------------------------------------------------------
 .gitignore                                      |  3 +
 .../org/apache/phoenix/spark/AggregateIT.java   | 23 ++++++++
 .../org/apache/phoenix/spark/OrderByIT.java     | 61 ++++++++++++--------
 .../org/apache/phoenix/spark/SparkUtil.java     | 32 ++++++----
 .../apache/phoenix/spark/PhoenixSparkIT.scala   |  2 +-
 5 files changed, 87 insertions(+), 34 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/7eb336de/.gitignore
----------------------------------------------------------------------
diff --git a/.gitignore b/.gitignore
index 2f47957..485e5b0 100644
--- a/.gitignore
+++ b/.gitignore
@@ -28,3 +28,6 @@ RESULTS/
 CSV_EXPORT/
 .DS_Store
 
+# jenv stuff
+.java-version
+

http://git-wip-us.apache.org/repos/asf/phoenix/blob/7eb336de/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
index e4b96a3..72197c3 100644
--- a/phoenix-spark/src/it/java/org/apache/phoenix/spark/AggregateIT.java
+++ b/phoenix-spark/src/it/java/org/apache/phoenix/spark/AggregateIT.java
@@ -28,9 +28,32 @@ import java.sql.SQLException;
 
 import org.apache.phoenix.end2end.BaseAggregateIT;
 import org.apache.phoenix.util.QueryBuilder;
+import org.junit.Ignore;
+import org.junit.Test;
 
 public class AggregateIT extends BaseAggregateIT {
 
+    @Ignore("Not passing on CDH 4.15")
+    @Test
+    @Override
+    public void testExpressionInGroupBy() throws Exception {
+        super.testExpressionInGroupBy();
+    }
+
+    @Ignore("Not passing on CDH 4.15")
+    @Test
+    @Override
+    public void testGroupByCase() throws Exception {
+        super.testGroupByCase();
+    }
+
+    @Ignore("Not passing on CDH 4.15")
+    @Test
+    @Override
+    public void testGroupByDescColumnWithNullsLastBug3452() throws Exception {
+        super.testGroupByDescColumnWithNullsLastBug3452();
+    }
+
     @Override
     protected ResultSet executeQueryThrowsException(Connection conn, QueryBuilder queryBuilder,
         String expectedPhoenixExceptionMsg, String expectedSparkExceptionMsg) {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/7eb336de/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
index bdffaf5..83578ba 100644
--- a/phoenix-spark/src/it/java/org/apache/phoenix/spark/OrderByIT.java
+++ b/phoenix-spark/src/it/java/org/apache/phoenix/spark/OrderByIT.java
@@ -18,7 +18,7 @@ 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.DataFrame;
 import org.apache.spark.sql.Row;
 import org.apache.spark.sql.SQLContext;
 import org.junit.Ignore;
@@ -31,6 +31,28 @@ import scala.collection.JavaConverters;
 
 public class OrderByIT extends BaseOrderByIT {
 
+    @Ignore(" || operator not supported in order by Spark 1.6 ")
+    @Test
+    @Override
+    public void testDescMultiOrderByExpr() throws Exception {
+        super.testDescMultiOrderByExpr();
+    }
+
+    @Ignore("NULLS FIRST|LAST not supported in Spark 1.6")
+    @Test
+    @Override
+    public void testNullsLastWithDesc() throws Exception {
+        super.testNullsLastWithDesc();
+    }
+
+    @Ignore("NULLS FIRST|LAST not supported in Spark 1.6")
+    @Test
+    @Override
+    public void testOrderByReverseOptimizationWithNullsLast() throws Exception {
+        super.testOrderByReverseOptimizationWithNullsLast();
+    }
+
+
     @Override
     protected ResultSet executeQueryThrowsException(Connection conn, QueryBuilder queryBuilder,
                                                     String expectedPhoenixExceptionMsg, String expectedSparkExceptionMsg) {
@@ -107,18 +129,16 @@ public class OrderByIT extends BaseOrderByIT {
             // 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 =
+            DataFrame phoenixDataSet =
                     new PhoenixRDD(SparkUtil.getSparkContext(), tableName1,
-                            JavaConverters.collectionAsScalaIterableConverter(table1Columns)
-                                    .asScala().toSeq(),
+                            JavaConverters.asScalaBufferConverter(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(),
+                            JavaConverters.asScalaBufferConverter(table2Columns).asScala().toSeq(),
                             Option.apply((String) null), Option.apply(getUrl()), config, false,
                             null).toDataFrame(sqlContext);
             phoenixDataSet.registerTempTable(tableName2);
@@ -126,7 +146,7 @@ public class OrderByIT extends BaseOrderByIT {
             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 =
+            DataFrame dataset =
                     sqlContext.sql(query);
             List<Row> rows = dataset.collectAsList();
             ResultSet rs = new SparkResultSet(rows, dataset.columns());
@@ -173,6 +193,7 @@ public class OrderByIT extends BaseOrderByIT {
         }
     }
 
+    @Ignore("Not passing on CDH 4.15")
     @Test
     public void testOrderByWithUnionAll() throws Exception {
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
@@ -230,18 +251,16 @@ public class OrderByIT extends BaseOrderByIT {
 
             List<String> table1Columns = Lists.newArrayList("A_STRING", "CF1.A", "CF1.B", "COL1", "CF2.C", "CF2.D");
             SQLContext sqlContext = SparkUtil.getSqlContext();
-            Dataset phoenixDataSet =
+            DataFrame phoenixDataSet =
                     new PhoenixRDD(SparkUtil.getSparkContext(), tableName1,
-                            JavaConverters.collectionAsScalaIterableConverter(table1Columns)
-                                    .asScala().toSeq(),
+                            JavaConverters.asScalaBufferConverter(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(),
+                            JavaConverters.asScalaBufferConverter(table2Columns).asScala().toSeq(),
                             Option.apply((String) null), Option.apply(getUrl()), config, false,
                             null).toDataFrame(sqlContext);
             phoenixDataSet.registerTempTable(tableName2);
@@ -249,7 +268,7 @@ public class OrderByIT extends BaseOrderByIT {
             String query =
                     "select a_string, `cf2.d` from " + tableName1 + " union all select * from "
                             + tableName2 + " order by `cf2.d`";
-            Dataset<Row> dataset =
+            DataFrame dataset =
                     sqlContext.sql(query);
             List<Row> rows = dataset.collectAsList();
             ResultSet rs = new SparkResultSet(rows, dataset.columns());
@@ -312,17 +331,14 @@ public class OrderByIT extends BaseOrderByIT {
             conn.commit();
 
             SQLContext sqlContext = SparkUtil.getSqlContext();
-            Dataset phoenixDataSet =
+            DataFrame phoenixDataSet =
                     new PhoenixRDD(SparkUtil.getSparkContext(), tableName,
-                            JavaConverters
-                                    .collectionAsScalaIterableConverter(
-                                        Lists.newArrayList("col1", "col2", "col4"))
-                                    .asScala().toSeq(),
+                            JavaConverters.asScalaBufferConverter(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 =
+            DataFrame dataset =
                     sqlContext.sql("SELECT col1+col2, col4, a_string FROM " + tableName
                             + " ORDER BY col1+col2, col4");
             List<Row> rows = dataset.collectAsList();
@@ -384,15 +400,14 @@ public class OrderByIT extends BaseOrderByIT {
                         "COL2");
 
             SQLContext sqlContext = SparkUtil.getSqlContext();
-            Dataset phoenixDataSet =
+            DataFrame phoenixDataSet =
                     new PhoenixRDD(SparkUtil.getSparkContext(), tableName,
-                            JavaConverters.collectionAsScalaIterableConverter(columns).asScala()
-                                    .toSeq(),
+                            JavaConverters.asScalaBufferConverter(columns).asScala().toSeq(),
                             Option.apply((String) null), Option.apply(url), config, false, null)
                                     .toDataFrame(sqlContext);
 
             phoenixDataSet.registerTempTable(tableName);
-            Dataset<Row> dataset =
+            DataFrame 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();

http://git-wip-us.apache.org/repos/asf/phoenix/blob/7eb336de/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
index 6285209..db2fe1a 100644
--- a/phoenix-spark/src/it/java/org/apache/phoenix/spark/SparkUtil.java
+++ b/phoenix-spark/src/it/java/org/apache/phoenix/spark/SparkUtil.java
@@ -23,11 +23,11 @@ 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.SparkConf;
 import org.apache.spark.SparkContext;
-import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.DataFrame;
 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;
@@ -42,15 +42,28 @@ 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 final String CASE_SENSITIVE_COLUMNS = "spark.sql.caseSensitive";
+
+    private static SparkContext sparkContext = null;
+    private static SQLContext sqlContext = null;
 
     public static SparkContext getSparkContext() {
-        return SparkSession.builder().appName(APP_NAME).master(NUM_EXECUTORS)
-                .config(UI_SHOW_CONSOLE_PROGRESS, false).getOrCreate().sparkContext();
+        if (sparkContext == null) {
+            SparkConf conf = new SparkConf(true);
+            conf.setAppName(APP_NAME);
+            conf.setMaster(NUM_EXECUTORS);
+            conf.set(UI_SHOW_CONSOLE_PROGRESS, "false");
+            conf.set(CASE_SENSITIVE_COLUMNS, "false");
+            sparkContext = new SparkContext(conf);
+        }
+        return sparkContext;
     }
 
     public static SQLContext getSqlContext() {
-        return SparkSession.builder().appName(APP_NAME).master(NUM_EXECUTORS)
-                .config(UI_SHOW_CONSOLE_PROGRESS, false).getOrCreate().sqlContext();
+        if (sqlContext == null) {
+            sqlContext = new SQLContext(getSparkContext());
+        }
+        return sqlContext;
     }
 
     public static ResultSet executeQuery(Connection conn, QueryBuilder queryBuilder, String url, Configuration config)
@@ -69,15 +82,14 @@ public class SparkUtil {
 
         // 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 =
+        DataFrame phoenixDataSet =
                 new PhoenixRDD(SparkUtil.getSparkContext(), queryBuilder.getFullTableName(),
-                        JavaConverters.collectionAsScalaIterableConverter(queryBuilder.getRequiredColumns()).asScala()
-                                .toSeq(),
+                        JavaConverters.asScalaBufferConverter(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());
+        DataFrame dataset = sqlContext.sql(queryBuilder.build());
         SparkPlan plan = dataset.queryExecution().executedPlan();
         List<Row> rows = dataset.collectAsList();
         queryBuilder.setOrderByClause(prevOrderBy);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/7eb336de/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 d1e38fa..fb4bb64 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
@@ -277,7 +277,7 @@ class PhoenixSparkIT extends AbstractPhoenixSparkIT {
     df.schema("COL1").dataType shouldEqual StringType
   }
 
-  test("Spark SQL can use Phoenix as a data source with PrunedFilteredScan") {
+  ignore("Spark SQL can use Phoenix as a data source with PrunedFilteredScan") {
     val sqlContext = new SQLContext(sc)
     val df = sqlContext.load("org.apache.phoenix.spark", Map("table" -> "TABLE1",
       "zkUrl" -> quorumAddress))


[14/28] phoenix git commit: PHOENIX-5017 Fix testRecreateViewWhoseParentWasDropped test flapper

Posted by pb...@apache.org.
PHOENIX-5017 Fix testRecreateViewWhoseParentWasDropped test flapper


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

Branch: refs/heads/4.x-cdh5.15
Commit: 7afa9549df2e5f14f963a5c61d0cce006fb4a014
Parents: 21c3a7c
Author: Thomas D'Silva <td...@apache.org>
Authored: Tue Nov 13 23:42:19 2018 +0000
Committer: Pedro Boado <pb...@apache.org>
Committed: Tue Nov 27 15:11:45 2018 +0000

----------------------------------------------------------------------
 .../phoenix/coprocessor/MetaDataEndpointImpl.java       | 12 ++++++++----
 1 file changed, 8 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/7afa9549/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java
index d899e32..5562340 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java
@@ -2035,8 +2035,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                 }
             }
             
-            // check if the table was dropped, but had child views that were have not yet
-            // been cleaned up by compaction
+            // check if the table was dropped, but had child views that were have not yet been cleaned up
             if (!Bytes.toString(schemaName).equals(QueryConstants.SYSTEM_SCHEMA_NAME)) {
                 dropChildViews(env, tenantIdBytes, schemaName, tableName);
             }
@@ -2434,8 +2433,13 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                     MetaDataClient client = new MetaDataClient(connection);
                     org.apache.phoenix.parse.TableName viewTableName = org.apache.phoenix.parse.TableName
                             .create(Bytes.toString(viewSchemaName), Bytes.toString(viewName));
-                    client.dropTable(
-                            new DropTableStatement(viewTableName, PTableType.VIEW, false, true, true));
+                    try {
+                        client.dropTable(
+                                new DropTableStatement(viewTableName, PTableType.VIEW, false, true, true));
+                    }
+                    catch (TableNotFoundException e) {
+                        logger.info("Ignoring view "+viewTableName+" as it has already been dropped");
+                    }
                 }
             }
         }


[21/28] phoenix git commit: PHOENIX-4955 - PhoenixIndexImportDirectMapper undercounts failed records

Posted by pb...@apache.org.
PHOENIX-4955 - PhoenixIndexImportDirectMapper undercounts failed records


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

Branch: refs/heads/4.x-cdh5.15
Commit: dd81989fab80cb283678218ada0c0359930731c8
Parents: 590f88b
Author: Geoffrey Jacoby <gj...@apache.org>
Authored: Fri Nov 16 21:57:45 2018 +0000
Committer: Pedro Boado <pb...@apache.org>
Committed: Tue Nov 27 15:12:05 2018 +0000

----------------------------------------------------------------------
 .../mapreduce/index/PhoenixIndexImportDirectMapper.java  | 11 +++++++----
 1 file changed, 7 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/dd81989f/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/index/PhoenixIndexImportDirectMapper.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/index/PhoenixIndexImportDirectMapper.java b/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/index/PhoenixIndexImportDirectMapper.java
index eb4bc0e..e2ac491 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/index/PhoenixIndexImportDirectMapper.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/index/PhoenixIndexImportDirectMapper.java
@@ -68,6 +68,8 @@ public class PhoenixIndexImportDirectMapper extends
     private long batchSizeBytes;
 
     private MutationState mutationState;
+    private int currentBatchCount = 0;
+
 
     @Override
     protected void setup(final Context context) throws IOException, InterruptedException {
@@ -113,6 +115,7 @@ public class PhoenixIndexImportDirectMapper extends
             throws IOException, InterruptedException {
 
         try {
+            currentBatchCount++;
             final List<Object> values = record.getValues();
             indxWritable.setValues(values);
             indxWritable.write(this.pStatement);
@@ -125,9 +128,8 @@ public class PhoenixIndexImportDirectMapper extends
             }
             // Keep accumulating Mutations till batch size
             mutationState.join(currentMutationState);
-
             // Write Mutation Batch
-            if (context.getCounter(PhoenixJobCounters.INPUT_RECORDS).getValue() % batchSize == 0) {
+            if (currentBatchCount % batchSize == 0) {
                 writeBatch(mutationState, context);
                 mutationState = null;
             }
@@ -136,7 +138,7 @@ public class PhoenixIndexImportDirectMapper extends
             context.progress();
         } catch (SQLException e) {
             LOG.error(" Error {}  while read/write of a record ", e.getMessage());
-            context.getCounter(PhoenixJobCounters.FAILED_RECORDS).increment(1);
+            context.getCounter(PhoenixJobCounters.FAILED_RECORDS).increment(currentBatchCount);
             throw new RuntimeException(e);
         }
         context.getCounter(PhoenixJobCounters.INPUT_RECORDS).increment(1);
@@ -157,6 +159,7 @@ public class PhoenixIndexImportDirectMapper extends
                 mutationPair.getSecond().size());
         }
         connection.rollback();
+        currentBatchCount = 0;
     }
 
     @Override
@@ -173,7 +176,7 @@ public class PhoenixIndexImportDirectMapper extends
             super.cleanup(context);
         } catch (SQLException e) {
             LOG.error(" Error {}  while read/write of a record ", e.getMessage());
-            context.getCounter(PhoenixJobCounters.FAILED_RECORDS).increment(1);
+            context.getCounter(PhoenixJobCounters.FAILED_RECORDS).increment(currentBatchCount);
             throw new RuntimeException(e);
         } finally {
             if (connection != null) {


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

Posted by pb...@apache.org.
http://git-wip-us.apache.org/repos/asf/phoenix/blob/678563f5/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