[10/12] phoenix git commit: Branch rebased to 08d9c7154194de69ce688ad78357bffb8d34c92f of 4.x-HBase-0.98

2016-10-19 Thread samarth
Branch rebased to 08d9c7154194de69ce688ad78357bffb8d34c92f of 4.x-HBase-0.98


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

Branch: refs/heads/encodecolumns2
Commit: 23a879890e1e00a9bf6ab4f239b59e50b8762909
Parents: 153612a
Author: Samarth 
Authored: Wed Oct 5 00:11:07 2016 -0700
Committer: Samarth 
Committed: Fri Oct 7 13:06:36 2016 -0700

--
 .../apache/phoenix/end2end/AlterTableIT.java|  250 +++-
 .../phoenix/end2end/AlterTableWithViewsIT.java  |4 +-
 .../apache/phoenix/end2end/CreateTableIT.java   |   22 +
 .../phoenix/end2end/PhoenixRuntimeIT.java   |4 +-
 .../phoenix/end2end/RowValueConstructorIT.java  |2 +-
 .../phoenix/end2end/StatsCollectorIT.java   |8 +-
 .../apache/phoenix/end2end/UpsertValuesIT.java  |   45 +-
 .../phoenix/end2end/index/DropMetadataIT.java   |   13 +-
 .../end2end/index/IndexExpressionIT.java|   21 +-
 .../apache/phoenix/end2end/index/IndexIT.java   |   26 +-
 .../phoenix/end2end/index/IndexTestUtil.java|   13 +-
 .../phoenix/compile/CreateTableCompiler.java|2 +-
 .../phoenix/compile/ExpressionCompiler.java |   18 +-
 .../apache/phoenix/compile/FromCompiler.java|   54 +-
 .../apache/phoenix/compile/JoinCompiler.java|8 +-
 .../phoenix/compile/ListJarsQueryPlan.java  |3 +-
 .../apache/phoenix/compile/PostDDLCompiler.java |   11 +-
 .../phoenix/compile/ProjectionCompiler.java |   10 +-
 .../apache/phoenix/compile/QueryCompiler.java   |2 +-
 .../apache/phoenix/compile/TraceQueryPlan.java  |2 +-
 .../compile/TupleProjectionCompiler.java|   22 +-
 .../apache/phoenix/compile/UnionCompiler.java   |6 +-
 .../apache/phoenix/compile/WhereCompiler.java   |   36 +-
 .../coprocessor/BaseScannerRegionObserver.java  |   42 +-
 .../coprocessor/DelegateRegionScanner.java  |5 +
 .../GroupedAggregateRegionObserver.java |   36 +-
 .../coprocessor/HashJoinRegionScanner.java  |2 +-
 .../coprocessor/MetaDataEndpointImpl.java   |  171 ++-
 .../phoenix/coprocessor/ScanRegionObserver.java |   12 +-
 .../coprocessor/SequenceRegionObserver.java |1 +
 .../UngroupedAggregateRegionObserver.java   |   28 +-
 .../coprocessor/generated/PTableProtos.java | 1379 --
 .../apache/phoenix/execute/BaseQueryPlan.java   |   25 +-
 .../apache/phoenix/execute/MutationState.java   |   12 +-
 .../phoenix/execute/SortMergeJoinPlan.java  |1 +
 .../apache/phoenix/execute/TupleProjector.java  |6 +-
 .../expression/ArrayColumnExpression.java   |  110 ++
 .../expression/ArrayConstructorExpression.java  |2 +-
 .../phoenix/expression/ExpressionType.java  |3 +-
 .../expression/KeyValueColumnExpression.java|   16 +-
 .../phoenix/expression/LiteralExpression.java   |   11 +-
 .../expression/ProjectedColumnExpression.java   |1 +
 .../visitor/CloneExpressionVisitor.java |6 +
 .../expression/visitor/ExpressionVisitor.java   |2 +
 ...lumnWithKeyValueColumnExpressionVisitor.java |   37 +
 .../StatelessTraverseAllExpressionVisitor.java  |7 +-
 .../StatelessTraverseNoExpressionVisitor.java   |7 +-
 .../phoenix/filter/ColumnProjectionFilter.java  |   24 +-
 .../filter/MultiKeyValueComparisonFilter.java   |   10 +-
 .../SingleCQKeyValueComparisonFilter.java   |3 +-
 .../filter/SingleKeyValueComparisonFilter.java  |   10 +-
 .../apache/phoenix/hbase/index/ValueGetter.java |1 +
 .../example/CoveredColumnIndexCodec.java|1 -
 .../apache/phoenix/index/IndexMaintainer.java   |  307 +++-
 .../apache/phoenix/index/PhoenixIndexCodec.java |2 +-
 .../index/PhoenixTransactionalIndexer.java  |   16 +-
 .../phoenix/iterate/BaseResultIterators.java|   83 +-
 .../iterate/LookAheadResultIterator.java|2 +-
 .../phoenix/iterate/MappedByteBufferQueue.java  |1 +
 .../phoenix/iterate/OrderedResultIterator.java  |3 +-
 .../iterate/RegionScannerResultIterator.java|   14 +-
 .../phoenix/jdbc/PhoenixDatabaseMetaData.java   |   12 +-
 .../apache/phoenix/jdbc/PhoenixResultSet.java   |2 +-
 .../apache/phoenix/join/HashCacheFactory.java   |1 +
 .../mapreduce/FormatToBytesWritableMapper.java  |   22 +-
 .../mapreduce/FormatToKeyValueReducer.java  |   30 +-
 .../query/ConnectionQueryServicesImpl.java  |2 +-
 .../query/ConnectionlessQueryServicesImpl.java  |1 -
 .../apache/phoenix/query/QueryConstants.java|   57 +-
 .../phoenix/query/QueryServicesOptions.java |1 -
 .../org/apache/phoenix/schema/ColumnRef.java|   10 +-
 .../apache/phoenix/schema/DelegateColumn.java   |5 +
 .../apache/phoenix/schema/DelegateTable.java|   20 +-
 .../apache/phoenix/schema/KeyValueSchema.java   |

[12/12] phoenix git commit: Fix test failures

2016-10-19 Thread samarth
Fix test failures


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

Branch: refs/heads/encodecolumns2
Commit: c1958d07e4b8777d5ab7ff30f36450d19dce69a7
Parents: 23a8798
Author: Samarth 
Authored: Wed Oct 19 11:57:48 2016 -0700
Committer: Samarth 
Committed: Wed Oct 19 11:57:48 2016 -0700

--
 .../AlterMultiTenantTableWithViewsIT.java   |  25 +-
 .../apache/phoenix/end2end/AlterTableIT.java|  98 +---
 .../phoenix/end2end/AlterTableWithViewsIT.java  | 139 ++-
 .../phoenix/end2end/StatsCollectorIT.java   |   8 +-
 .../apache/phoenix/end2end/StoreNullsIT.java|  41 +++-
 .../phoenix/end2end/index/DropMetadataIT.java   |   2 +-
 .../end2end/index/IndexExpressionIT.java|   7 +-
 .../end2end/index/MutableIndexFailureIT.java|   2 +
 .../phoenix/compile/CreateTableCompiler.java|  13 +-
 .../phoenix/compile/ExpressionCompiler.java |   4 +-
 .../compile/PostLocalIndexDDLCompiler.java  |   7 +-
 .../apache/phoenix/compile/WhereCompiler.java   |  21 --
 .../GroupedAggregateRegionObserver.java |  14 +-
 .../coprocessor/MetaDataEndpointImpl.java   |  95 +---
 .../phoenix/coprocessor/ScanRegionObserver.java |  10 +-
 .../UngroupedAggregateRegionObserver.java   |   4 +-
 .../expression/ArrayColumnExpression.java   | 116 +
 .../expression/KeyValueColumnExpression.java|   2 +-
 ...lumnWithKeyValueColumnExpressionVisitor.java |  37 ---
 .../filter/MultiKeyValueComparisonFilter.java   |   5 -
 .../filter/SingleKeyValueComparisonFilter.java  |   6 -
 .../apache/phoenix/index/IndexMaintainer.java   | 108 +
 .../apache/phoenix/index/PhoenixIndexCodec.java |   2 +-
 .../index/PhoenixIndexFailurePolicy.java|   5 +-
 .../phoenix/iterate/BaseResultIterators.java|   1 -
 .../iterate/RegionScannerResultIterator.java|   4 +-
 .../apache/phoenix/schema/MetaDataClient.java   | 127 +-
 .../tuple/EncodedColumnQualiferCellsList.java   |  71 --
 .../tuple/PositionBasedMultiKeyValueTuple.java  |   4 +-
 .../schema/tuple/PositionBasedResultTuple.java  |   1 +
 .../apache/phoenix/util/EncodedColumnsUtil.java |   7 +-
 .../java/org/apache/phoenix/util/IndexUtil.java | 240 ++-
 .../org/apache/phoenix/util/MetaDataUtil.java   |   6 +
 .../java/org/apache/phoenix/util/ScanUtil.java  |   4 +-
 .../phoenix/index/IndexMaintainerTest.java  |   2 +-
 35 files changed, 696 insertions(+), 542 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/phoenix/blob/c1958d07/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterMultiTenantTableWithViewsIT.java
--
diff --git 
a/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterMultiTenantTableWithViewsIT.java
 
b/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterMultiTenantTableWithViewsIT.java
index adadca7..8275f3f 100644
--- 
a/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterMultiTenantTableWithViewsIT.java
+++ 
b/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterMultiTenantTableWithViewsIT.java
@@ -17,6 +17,7 @@
  */
 package org.apache.phoenix.end2end;
 
+import static 
org.apache.phoenix.exception.SQLExceptionCode.CANNOT_MUTATE_TABLE;
 import static 
org.apache.phoenix.query.QueryConstants.BASE_TABLE_BASE_COLUMN_COUNT;
 import static 
org.apache.phoenix.query.QueryConstants.DIVERGED_VIEW_BASE_COLUMN_COUNT;
 import static org.junit.Assert.assertEquals;
@@ -476,14 +477,14 @@ public class AlterMultiTenantTableWithViewsIT extends 
ParallelStatsDisabledIT {
 // For a diverged view, only base table's pk column will be added 
and that too at the end.
 assertTableDefinition(conn, divergedView, PTableType.VIEW, 
baseTable, 2, 7, DIVERGED_VIEW_BASE_COLUMN_COUNT, "TENANT_ID", "PK1", "V1", 
"V3", "VIEW_COL1", "VIEW_COL2", "PK2");
 
-// Add existing column VIEW_COL2 to the base table
+// Adding existing column VIEW_COL2 to the base table isn't 
allowed.
 alterBaseTable = "ALTER TABLE " + baseTable + " ADD VIEW_COL2 
CHAR(256)";
-conn.createStatement().execute(alterBaseTable);
-
-// For the non-diverged view, adding the column VIEW_COL2 will end 
up changing its ordinal position in the view.
-assertTableDefinition(conn, view1, PTableType.VIEW, baseTable, 2, 
9, 8, "TENANT_ID", "PK1", "V1", "V2", "V3", "KV", "PK2", "VIEW_COL2", 
"VIEW_COL1");
-// For the diverged view, adding the column VIEW_COL2 will not 
change its ordinal position in the view. It also won't change the base column 
count or the sequence

[06/12] phoenix git commit: Branch rebased to 08d9c7154194de69ce688ad78357bffb8d34c92f of 4.x-HBase-0.98

2016-10-19 Thread samarth
http://git-wip-us.apache.org/repos/asf/phoenix/blob/23a87989/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/FormatToKeyValueReducer.java
--
diff --git 
a/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/FormatToKeyValueReducer.java
 
b/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/FormatToKeyValueReducer.java
index 15d6d2f..c5f690b 100644
--- 
a/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/FormatToKeyValueReducer.java
+++ 
b/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/FormatToKeyValueReducer.java
@@ -44,6 +44,7 @@ import org.apache.phoenix.query.QueryConstants;
 import org.apache.phoenix.schema.PColumn;
 import org.apache.phoenix.schema.PTable;
 import org.apache.phoenix.util.Closeables;
+import org.apache.phoenix.util.EncodedColumnsUtil;
 import org.apache.phoenix.util.PhoenixRuntime;
 import org.apache.phoenix.util.QueryUtil;
 import org.apache.phoenix.util.SchemaUtil;
@@ -89,7 +90,7 @@ public class FormatToKeyValueReducer
 }
 
 private void initColumnsMap(PhoenixConnection conn) throws SQLException {
-Map indexMap = new TreeMap(Bytes.BYTES_COMPARATOR);
+Map indexMap = new TreeMap<>(Bytes.BYTES_COMPARATOR);
 columnIndexes = new HashMap<>();
 int columnIndex = 0;
 for (int index = 0; index < logicalNames.size(); index++) {
@@ -98,12 +99,16 @@ public class FormatToKeyValueReducer
 for (int i = 0; i < cls.size(); i++) {
 PColumn c = cls.get(i);
 byte[] family = new byte[0];
-if (c.getFamilyName() != null) {
+byte[] cq;
+if (!SchemaUtil.isPKColumn(c)) {
 family = c.getFamilyName().getBytes();
+cq = EncodedColumnsUtil.getColumnQualifier(c, table);
+} else {
+// TODO: samarth verify if this is the right thing to do 
here.
+cq = c.getName().getBytes();
 }
-byte[] name = c.getName().getBytes();
-byte[] cfn = Bytes.add(family, 
QueryConstants.NAMESPACE_SEPARATOR_BYTES, name);
-Pair pair = new Pair(family, name);
+byte[] cfn = Bytes.add(family, 
QueryConstants.NAMESPACE_SEPARATOR_BYTES, cq);
+Pair pair = new Pair<>(family, cq);
 if (!indexMap.containsKey(cfn)) {
 indexMap.put(cfn, new Integer(columnIndex));
 columnIndexes.put(new Integer(columnIndex), pair);
@@ -111,8 +116,8 @@ public class FormatToKeyValueReducer
 }
 }
 byte[] emptyColumnFamily = SchemaUtil.getEmptyColumnFamily(table);
-Pair pair = new Pair(emptyColumnFamily, 
QueryConstants
-.EMPTY_COLUMN_BYTES);
+byte[] emptyKeyValue = 
EncodedColumnsUtil.getEmptyKeyValueInfo(table).getFirst();
+Pair pair = new Pair<>(emptyColumnFamily, 
emptyKeyValue);
 columnIndexes.put(new Integer(columnIndex), pair);
 columnIndex++;
 }
@@ -123,18 +128,17 @@ public class FormatToKeyValueReducer
   Reducer.Context context)
 throws IOException, InterruptedException {
 TreeSet map = new TreeSet(KeyValue.COMPARATOR);
-ImmutableBytesWritable rowKey = key.getRowkey();
 for (ImmutableBytesWritable aggregatedArray : values) {
 DataInputStream input = new DataInputStream(new 
ByteArrayInputStream(aggregatedArray.get()));
 while (input.available() != 0) {
 byte type = input.readByte();
 int index = WritableUtils.readVInt(input);
 ImmutableBytesWritable family;
-ImmutableBytesWritable name;
+ImmutableBytesWritable cq;
 ImmutableBytesWritable value = 
QueryConstants.EMPTY_COLUMN_VALUE_BYTES_PTR;
 Pair pair = columnIndexes.get(index);
 family = new ImmutableBytesWritable(pair.getFirst());
-name = new ImmutableBytesWritable(pair.getSecond());
+cq = new ImmutableBytesWritable(pair.getSecond());
 int len = WritableUtils.readVInt(input);
 if (len > 0) {
 byte[] array = new byte[len];
@@ -145,10 +149,10 @@ public class FormatToKeyValueReducer
 KeyValue.Type kvType = KeyValue.Type.codeToType(type);
 switch (kvType) {
 case Put: // not null value
-kv = builder.buildPut(key.getRowkey(), family, name, 
value);
+kv = builder.buildPut(key.getRowkey(), family, cq, 
value);
 break;
 case DeleteColumn: // null value
-kv = builder.buildDeleteColumns(key.getRowkey(), 
family, name);
+kv = builder.buildDe

[11/12] phoenix git commit: Fix test failures

2016-10-19 Thread samarth
http://git-wip-us.apache.org/repos/asf/phoenix/blob/c1958d07/phoenix-core/src/main/java/org/apache/phoenix/index/IndexMaintainer.java
--
diff --git 
a/phoenix-core/src/main/java/org/apache/phoenix/index/IndexMaintainer.java 
b/phoenix-core/src/main/java/org/apache/phoenix/index/IndexMaintainer.java
index b1dd5f4..edf27eb 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/index/IndexMaintainer.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/index/IndexMaintainer.java
@@ -23,6 +23,7 @@ import java.io.DataInputStream;
 import java.io.DataOutput;
 import java.io.DataOutputStream;
 import java.io.IOException;
+import java.nio.ByteBuffer;
 import java.sql.SQLException;
 import java.util.ArrayList;
 import java.util.Arrays;
@@ -60,7 +61,6 @@ import org.apache.phoenix.expression.ExpressionType;
 import org.apache.phoenix.expression.KeyValueColumnExpression;
 import org.apache.phoenix.expression.LiteralExpression;
 import org.apache.phoenix.expression.visitor.KeyValueExpressionVisitor;
-import 
org.apache.phoenix.expression.visitor.ReplaceArrayColumnWithKeyValueColumnExpressionVisitor;
 import org.apache.phoenix.hbase.index.ValueGetter;
 import org.apache.phoenix.hbase.index.covered.update.ColumnReference;
 import org.apache.phoenix.hbase.index.util.ImmutableBytesPtr;
@@ -337,13 +337,7 @@ public class IndexMaintainer implements Writable, 
Iterable {
 this.isMultiTenant = dataTable.isMultiTenant();
 this.viewIndexId = index.getViewIndexId() == null ? null : 
MetaDataUtil.getViewIndexIdDataType().toBytes(index.getViewIndexId());
 this.isLocalIndex = index.getIndexType() == IndexType.LOCAL;
-/* 
- * There is nothing to prevent new indexes on existing tables to have 
encoded column names.
- * Except, due to backward compatibility reasons, we aren't able to 
change IndexMaintainer and the state
- * that is serialized in it. Because of this we are forced to have the 
indexes inherit the
- * storage scheme of the parent data tables. 
- */
-this.usesEncodedColumnNames = 
EncodedColumnsUtil.usesEncodedColumnNames(dataTable);
+this.usesEncodedColumnNames = 
EncodedColumnsUtil.usesEncodedColumnNames(index);
 byte[] indexTableName = index.getPhysicalName().getBytes();
 // Use this for the nDataSaltBuckets as we need this for local indexes
 // TODO: persist nDataSaltBuckets separately, but maintain b/w compat.
@@ -526,7 +520,7 @@ public class IndexMaintainer implements Writable, 
Iterable {
 initCachedState();
 }
 
-public byte[] buildRowKey(ValueGetter valueGetter, ImmutableBytesWritable 
rowKeyPtr, byte[] regionStartKey, byte[] regionEndKey, boolean 
convertArrayColToKeyValueCol)  {
+public byte[] buildRowKey(ValueGetter valueGetter, ImmutableBytesWritable 
rowKeyPtr, byte[] regionStartKey, byte[] regionEndKey)  {
 ImmutableBytesWritable ptr = new ImmutableBytesWritable();
 boolean prependRegionStartKey = isLocalIndex && regionStartKey != null;
 boolean isIndexSalted = !isLocalIndex && nIndexSaltBuckets > 0;
@@ -595,9 +589,6 @@ public class IndexMaintainer implements Writable, 
Iterable {
 SortOrder dataSortOrder;
 if (dataPkPosition[i] == EXPRESSION_NOT_PRESENT) {
Expression expression = expressionIterator.next();
-   if (convertArrayColToKeyValueCol) {
-   expression = expression.accept(new 
ReplaceArrayColumnWithKeyValueColumnExpressionVisitor());
-   }
dataColumnType = expression.getDataType();
dataSortOrder = expression.getSortOrder();
 isNullable = expression.isNullable();
@@ -930,11 +921,11 @@ public class IndexMaintainer implements Writable, 
Iterable {
 return indexRowKeySchema;
 }
 
-public Put buildUpdateMutation(KeyValueBuilder kvBuilder, ValueGetter 
valueGetter, ImmutableBytesWritable dataRowKeyPtr, long ts, byte[] 
regionStartKey, byte[] regionEndKey, boolean convertArrayColToKeyValueCol) 
throws IOException {
-Put put = null;
+public Put buildUpdateMutation(KeyValueBuilder kvBuilder, ValueGetter 
valueGetter, ImmutableBytesWritable dataRowKeyPtr, long ts, byte[] 
regionStartKey, byte[] regionEndKey) throws IOException {
+   byte[] indexRowKey = this.buildRowKey(valueGetter, dataRowKeyPtr, 
regionStartKey, regionEndKey);
+   Put put = null;
 // New row being inserted: add the empty key value
 if (valueGetter.getLatestValue(dataEmptyKeyValueRef) == null) {
-byte[] indexRowKey = this.buildRowKey(valueGetter, dataRowKeyPtr, 
regionStartKey, regionEndKey, convertArrayColToKeyValueCol);
 put = new Put(indexRowKey);
 // add the keyvalue for the empty row
 put.add(kvBuilder.buildPut(new Immutabl

[02/12] phoenix git commit: PHOENIX-3363 Join-related IT had problematic usage of generating new table names

2016-10-19 Thread samarth
http://git-wip-us.apache.org/repos/asf/phoenix/blob/153612aa/phoenix-core/src/it/java/org/apache/phoenix/end2end/SubqueryIT.java
--
diff --git 
a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SubqueryIT.java 
b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SubqueryIT.java
index d7b3b29..684c3c2 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SubqueryIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SubqueryIT.java
@@ -272,8 +272,11 @@ public class SubqueryIT extends BaseJoinIT {
 public void testNonCorrelatedSubquery() throws Exception {
 Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
 Connection conn = DriverManager.getConnection(getUrl(), props);
+String tableName1 = getTableName(conn, JOIN_ITEM_TABLE_FULL_NAME);
+String tableName4 = getTableName(conn, JOIN_ORDER_TABLE_FULL_NAME);
+String tableName5 = getTableName(conn, JOIN_COITEM_TABLE_FULL_NAME);
 try {
-String query = "SELECT \"item_id\", name FROM " + 
getTableName(conn,JOIN_ITEM_TABLE_FULL_NAME) + " WHERE \"item_id\" >= ALL 
(SELECT \"item_id\" FROM " + getTableName(conn,JOIN_ORDER_TABLE_FULL_NAME) + ") 
ORDER BY name";
+String query = "SELECT \"item_id\", name FROM " + tableName1 + " 
WHERE \"item_id\" >= ALL (SELECT \"item_id\" FROM " + tableName4 + ") ORDER BY 
name";
 PreparedStatement statement = conn.prepareStatement(query);
 ResultSet rs = statement.executeQuery();
 assertTrue (rs.next());
@@ -285,7 +288,7 @@ public class SubqueryIT extends BaseJoinIT {
 
 assertFalse(rs.next());
 
-query = "SELECT \"item_id\", name FROM " + 
getTableName(conn,JOIN_ITEM_TABLE_FULL_NAME) + " WHERE \"item_id\" < ANY 
(SELECT \"item_id\" FROM " + getTableName(conn,JOIN_ORDER_TABLE_FULL_NAME) + 
")";
+query = "SELECT \"item_id\", name FROM " + tableName1 + " WHERE 
\"item_id\" < ANY (SELECT \"item_id\" FROM " + tableName4 + ")";
 statement = conn.prepareStatement(query);
 rs = statement.executeQuery();
 assertTrue (rs.next());
@@ -306,7 +309,7 @@ public class SubqueryIT extends BaseJoinIT {
 
 assertFalse(rs.next());
 
-query = "SELECT \"item_id\", name FROM " + 
getTableName(conn,JOIN_ITEM_TABLE_FULL_NAME) + " WHERE \"item_id\" < (SELECT 
max(\"item_id\") FROM " + getTableName(conn,JOIN_ORDER_TABLE_FULL_NAME) + ")";
+query = "SELECT \"item_id\", name FROM " + tableName1 + " WHERE 
\"item_id\" < (SELECT max(\"item_id\") FROM " + tableName4 + ")";
 statement = conn.prepareStatement(query);
 rs = statement.executeQuery();
 assertTrue (rs.next());
@@ -327,7 +330,7 @@ public class SubqueryIT extends BaseJoinIT {
 
 assertFalse(rs.next());
 
-query = "SELECT * FROM " + getTableName(conn, 
JOIN_COITEM_TABLE_FULL_NAME) + " WHERE (item_id, item_name) != ALL (SELECT 
\"item_id\", name FROM " + getTableName(conn,JOIN_ITEM_TABLE_FULL_NAME) + ")";
+query = "SELECT * FROM " + tableName5 + " WHERE (item_id, 
item_name) != ALL (SELECT \"item_id\", name FROM " + tableName1 + ")";
 statement = conn.prepareStatement(query);
 rs = statement.executeQuery();
 assertTrue (rs.next());
@@ -338,7 +341,7 @@ public class SubqueryIT extends BaseJoinIT {
 
 assertFalse(rs.next());
 
-query = "SELECT * FROM " + getTableName(conn, 
JOIN_COITEM_TABLE_FULL_NAME) + " WHERE EXISTS (SELECT \"item_id\", name FROM " 
+ getTableName(conn,JOIN_ITEM_TABLE_FULL_NAME) + ")";
+query = "SELECT * FROM " + tableName5 + " WHERE EXISTS (SELECT 
\"item_id\", name FROM " + tableName1 + ")";
 statement = conn.prepareStatement(query);
 rs = statement.executeQuery();
 assertTrue (rs.next());
@@ -364,7 +367,7 @@ public class SubqueryIT extends BaseJoinIT {
 
 assertFalse(rs.next());
 
-query = "SELECT \"item_id\", name FROM " + 
getTableName(conn,JOIN_ITEM_TABLE_FULL_NAME) + " WHERE \"item_id\" < (SELECT 
\"item_id\" FROM " + getTableName(conn,JOIN_ORDER_TABLE_FULL_NAME) + ")";
+query = "SELECT \"item_id\", name FROM " + tableName1 + " WHERE 
\"item_id\" < (SELECT \"item_id\" FROM " + tableName4 + ")";
 statement = conn.prepareStatement(query);
 try {
 rs = statement.executeQuery();
@@ -380,8 +383,12 @@ public class SubqueryIT extends BaseJoinIT {
 public void testInSubquery() throws Exception {
 Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
 Connection conn = DriverManager.getConnection(getUrl(), props);
+String tableName1 = getTableName(conn, JOIN_ITEM_TABLE_FULL_NAME);
+String tableName2 = getTableNam

[09/12] phoenix git commit: Branch rebased to 08d9c7154194de69ce688ad78357bffb8d34c92f of 4.x-HBase-0.98

2016-10-19 Thread samarth
http://git-wip-us.apache.org/repos/asf/phoenix/blob/23a87989/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 bc3466c..e884439 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
@@ -39,6 +39,7 @@ import org.apache.phoenix.schema.PTableImpl;
 import org.apache.phoenix.schema.PTableType;
 import org.apache.phoenix.schema.SortOrder;
 import org.apache.phoenix.schema.TableRef;
+import org.apache.phoenix.schema.PTable.StorageScheme;
 import org.apache.phoenix.schema.types.PDataType;
 import org.apache.phoenix.util.SchemaUtil;
 
@@ -88,17 +89,18 @@ public class UnionCompiler {
 UNION_FAMILY_NAME, targetTypes.get(i).getType(), 
targetTypes.get(i).getMaxLength(),
 targetTypes.get(i).getScale(), 
colProj.getExpression().isNullable(), i,
 targetTypes.get(i).getSortOrder(), 500, null, false,
-colProj.getExpression().toString(), false, false);
+colProj.getExpression().toString(), false, false, null);
 projectedColumns.add(projectedColumn);
 }
 Long scn = statement.getConnection().getSCN();
+// TODO: samarth this is likely just an in memory reference for 
compilation purposes. Probably ok to pass non-encoded scheme and null counter.
 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, false, false, 0, 0L,
 SchemaUtil.isNamespaceMappingEnabled(PTableType.SUBQUERY,
-statement.getConnection().getQueryServices().getProps()), 
null, false);
+statement.getConnection().getQueryServices().getProps()), 
null, false, StorageScheme.NON_ENCODED_COLUMN_NAMES, 
PTable.EncodedCQCounter.NULL_COUNTER);
 TableRef tableRef = new TableRef(null, tempTable, 0, false);
 return tableRef;
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/23a87989/phoenix-core/src/main/java/org/apache/phoenix/compile/WhereCompiler.java
--
diff --git 
a/phoenix-core/src/main/java/org/apache/phoenix/compile/WhereCompiler.java 
b/phoenix-core/src/main/java/org/apache/phoenix/compile/WhereCompiler.java
index 13963d7..63ad9c0 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/WhereCompiler.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/WhereCompiler.java
@@ -29,6 +29,8 @@ import org.apache.hadoop.hbase.filter.Filter;
 import org.apache.phoenix.exception.SQLExceptionCode;
 import org.apache.phoenix.exception.SQLExceptionInfo;
 import org.apache.phoenix.expression.AndExpression;
+import org.apache.phoenix.expression.ArrayColumnExpression;
+import org.apache.phoenix.expression.ColumnExpression;
 import org.apache.phoenix.expression.Determinism;
 import org.apache.phoenix.expression.Expression;
 import org.apache.phoenix.expression.KeyValueColumnExpression;
@@ -51,11 +53,13 @@ import org.apache.phoenix.schema.ColumnNotFoundException;
 import org.apache.phoenix.schema.ColumnRef;
 import org.apache.phoenix.schema.PTable;
 import org.apache.phoenix.schema.PTable.IndexType;
+import org.apache.phoenix.schema.PTable.StorageScheme;
 import org.apache.phoenix.schema.PTableType;
 import org.apache.phoenix.schema.TableRef;
 import org.apache.phoenix.schema.TypeMismatchException;
 import org.apache.phoenix.schema.types.PBoolean;
 import org.apache.phoenix.util.ByteUtil;
+import org.apache.phoenix.util.EncodedColumnsUtil;
 import org.apache.phoenix.util.ScanUtil;
 import org.apache.phoenix.util.SchemaUtil;
 
@@ -168,12 +172,14 @@ public class WhereCompiler {
 public Expression visit(ColumnParseNode node) throws SQLException {
 ColumnRef ref = resolveColumn(node);
 TableRef tableRef = ref.getTableRef();
+ColumnExpression newColumnExpression = 
ref.newColumnExpression(node.isTableNameCaseSensitive(), 
node.isCaseSensitive());
 if (tableRef.equals(context.getCurrentTable()) && 
!SchemaUtil.isPKColumn(ref.getColumn())) {
+byte[] cq = tableRef.getTable().getStorageScheme() == 
StorageScheme.COLUMNS_STORED_IN_SINGLE_CELL 
+   ? ref.getColumn().getFamilyName().getBytes() : 
EncodedColumnsUtil.getColumnQualifier(ref.getColumn(), tableRef.getTable());
 // track the where condition column

[04/12] phoenix git commit: Branch rebased to 08d9c7154194de69ce688ad78357bffb8d34c92f of 4.x-HBase-0.98

2016-10-19 Thread samarth
http://git-wip-us.apache.org/repos/asf/phoenix/blob/23a87989/phoenix-core/src/main/java/org/apache/phoenix/schema/tuple/ResultTuple.java
--
diff --git 
a/phoenix-core/src/main/java/org/apache/phoenix/schema/tuple/ResultTuple.java 
b/phoenix-core/src/main/java/org/apache/phoenix/schema/tuple/ResultTuple.java
index c28a2bf..845b113 100644
--- 
a/phoenix-core/src/main/java/org/apache/phoenix/schema/tuple/ResultTuple.java
+++ 
b/phoenix-core/src/main/java/org/apache/phoenix/schema/tuple/ResultTuple.java
@@ -17,33 +17,44 @@
  */
 package org.apache.phoenix.schema.tuple;
 
+import static 
org.apache.phoenix.query.QueryConstants.ENCODED_CQ_COUNTER_INITIAL_VALUE;
+
+import java.util.Collections;
+
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.phoenix.hbase.index.util.GenericKeyValueBuilder;
+import org.apache.phoenix.query.QueryConstants;
+import org.apache.phoenix.schema.SortOrder;
+import org.apache.phoenix.schema.types.PInteger;
 import org.apache.phoenix.util.KeyValueUtil;
 
-
+/**
+ * 
+ * Wrapper around {@link Result} that implements Phoenix's {@link Tuple} 
interface.
+ *
+ */
 public class ResultTuple extends BaseTuple {
-private Result result;
+private final Result result;
+public static final ResultTuple EMPTY_TUPLE = new 
ResultTuple(Result.create(Collections.emptyList()));
 
+//TODO: samarth see if we can get rid of this constructor altogether.
 public ResultTuple(Result result) {
 this.result = result;
 }
 
-public ResultTuple() {
-}
+//public ResultTuple(Result result, boolean useQualifierAsIndex) {
+//this.result = result;
+//this.useQualifierAsIndex = useQualifierAsIndex;
+//}
 
 public Result getResult() {
 return this.result;
 }
 
-public void setResult(Result result) {
-this.result = result;
-}
-
 @Override
 public void getKey(ImmutableBytesWritable ptr) {
 ptr.set(result.getRow());
@@ -56,6 +67,12 @@ public class ResultTuple extends BaseTuple {
 
 @Override
 public KeyValue getValue(byte[] family, byte[] qualifier) {
+//if (useQualifierAsIndex) {
+//int index = PInteger.INSTANCE.getCodec().decodeInt(qualifier, 0, 
SortOrder.ASC);
+////TODO: samarth this seems like a hack here at this place. Think 
more. Maybe we should use a new tuple here?
+//index = index >= ENCODED_CQ_COUNTER_INITIAL_VALUE ? (index - 
ENCODED_CQ_COUNTER_INITIAL_VALUE) : index;
+//return 
org.apache.hadoop.hbase.KeyValueUtil.ensureKeyValue(result.rawCells()[index]);
+//}
 Cell cell = 
KeyValueUtil.getColumnLatest(GenericKeyValueBuilder.INSTANCE, 
   result.rawCells(), family, qualifier);
 return org.apache.hadoop.hbase.KeyValueUtil.ensureKeyValue(cell);
@@ -104,4 +121,4 @@ public class ResultTuple extends BaseTuple {
 ptr.set(kv.getValueArray(), kv.getValueOffset(), kv.getValueLength());
 return true;
 }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/phoenix/blob/23a87989/phoenix-core/src/main/java/org/apache/phoenix/schema/tuple/Tuple.java
--
diff --git 
a/phoenix-core/src/main/java/org/apache/phoenix/schema/tuple/Tuple.java 
b/phoenix-core/src/main/java/org/apache/phoenix/schema/tuple/Tuple.java
index 61b2a4f..e4a887b 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/tuple/Tuple.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/tuple/Tuple.java
@@ -17,6 +17,8 @@
  */
 package org.apache.phoenix.schema.tuple;
 
+import java.util.List;
+
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 
@@ -87,4 +89,6 @@ public interface Tuple {
  * @return the current or next sequence value
  */
 public long getSequenceValue(int index);
+
+public void setKeyValues(List values);
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/23a87989/phoenix-core/src/main/java/org/apache/phoenix/schema/tuple/UnboundedSkipNullCellsList.java
--
diff --git 
a/phoenix-core/src/main/java/org/apache/phoenix/schema/tuple/UnboundedSkipNullCellsList.java
 
b/phoenix-core/src/main/java/org/apache/phoenix/schema/tuple/UnboundedSkipNullCellsList.java
new file mode 100644
index 000..02a85a5
--- /dev/null
+++ 
b/phoenix-core/src/main/java/org/apache/phoenix/schema/tuple/UnboundedSkipNullCellsList.java
@@ -0,0 +1,488 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional info

[05/12] phoenix git commit: Branch rebased to 08d9c7154194de69ce688ad78357bffb8d34c92f of 4.x-HBase-0.98

2016-10-19 Thread samarth
http://git-wip-us.apache.org/repos/asf/phoenix/blob/23a87989/phoenix-core/src/main/java/org/apache/phoenix/schema/PName.java
--
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/PName.java 
b/phoenix-core/src/main/java/org/apache/phoenix/schema/PName.java
index 0e1337c..8df6a95 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/PName.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/PName.java
@@ -83,6 +83,32 @@ public interface PName {
 return 0;
 }
 };
+public static PName ENCODED_EMPTY_COLUMN_NAME = new PName() {
+@Override
+public String getString() {
+return String.valueOf(QueryConstants.ENCODED_EMPTY_COLUMN_NAME);
+}
+
+@Override
+public byte[] getBytes() {
+return QueryConstants.ENCODED_EMPTY_COLUMN_BYTES;
+}
+
+@Override
+public String toString() {
+return getString();
+}
+
+@Override
+public ImmutableBytesPtr getBytesPtr() {
+return QueryConstants.ENCODED_EMPTY_COLUMN_BYTES_PTR;
+}
+
+@Override
+public int getEstimatedSize() {
+return 0;
+}
+};
 /**
  * Get the client-side, normalized name as referenced
  * in a SQL statement.

http://git-wip-us.apache.org/repos/asf/phoenix/blob/23a87989/phoenix-core/src/main/java/org/apache/phoenix/schema/PTable.java
--
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/PTable.java 
b/phoenix-core/src/main/java/org/apache/phoenix/schema/PTable.java
index b585323..2c7e4bb 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/PTable.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/PTable.java
@@ -17,7 +17,15 @@
  */
 package org.apache.phoenix.schema;
 
+import static 
org.apache.phoenix.query.QueryConstants.ENCODED_CQ_COUNTER_INITIAL_VALUE;
+
+import java.util.Collections;
+import java.util.HashMap;
 import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+
+import javax.annotation.Nullable;
 
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -129,7 +137,7 @@ public interface PTable extends PMetaDataEntity {
  * Link from a view to its parent table
  */
 PARENT_TABLE((byte)3);
-
+
 private final byte[] byteValue;
 private final byte serializedValue;
 
@@ -153,6 +161,35 @@ public interface PTable extends PMetaDataEntity {
 return LinkType.values()[serializedValue-1];
 }
 }
+
+public enum StorageScheme {
+ENCODED_COLUMN_NAMES((byte)1),
+NON_ENCODED_COLUMN_NAMES((byte)2),
+COLUMNS_STORED_IN_SINGLE_CELL((byte)3);
+
+private final byte[] byteValue;
+private final byte serializedValue;
+
+StorageScheme(byte serializedValue) {
+this.serializedValue = serializedValue;
+this.byteValue = Bytes.toBytes(this.name());
+}
+
+public byte[] getBytes() {
+return byteValue;
+}
+
+public byte getSerializedValue() {
+return this.serializedValue;
+}
+
+public static StorageScheme fromSerializedValue(byte serializedValue) {
+if (serializedValue < 1 || serializedValue > 
StorageScheme.values().length) {
+return null;
+}
+return StorageScheme.values()[serializedValue-1];
+}
+}
 
 long getTimeStamp();
 long getSequenceNumber();
@@ -208,7 +245,16 @@ public interface PTable extends PMetaDataEntity {
  * can be found
  * @throws AmbiguousColumnException if multiple columns are found with the 
given name
  */
-PColumn getColumn(String name) throws ColumnNotFoundException, 
AmbiguousColumnException;
+PColumn getPColumnForColumnName(String name) throws 
ColumnNotFoundException, AmbiguousColumnException;
+
+/**
+ * Get the column with the given column qualifier.
+ * @param column qualifier bytes
+ * @return the PColumn with the given column qualifier
+ * @throws ColumnNotFoundException if no column with the given column 
qualifier can be found
+ * @throws AmbiguousColumnException if multiple columns are found with the 
given column qualifier
+ */
+PColumn getPColumnForColumnQualifier(byte[] cf, byte[] cq) throws 
ColumnNotFoundException, AmbiguousColumnException; 
 
 /**
  * Get the PK column with the given name.
@@ -343,7 +389,6 @@ public interface PTable extends PMetaDataEntity {
  */
 int getRowTimestampColPos();
 long getUpdateCacheFrequency();
-
 boolean isNamespaceMapped();
 
 /**
@@ -357,4 +402,77 @@ public interface PTable extends PMetaDataEntity {
  * you are also not allowed to del

[01/12] phoenix git commit: PHOENIX-3296 ArrayConstructor expression does not serialize arrays with leading nulls correctly [Forced Update!]

2016-10-19 Thread samarth
Repository: phoenix
Updated Branches:
  refs/heads/encodecolumns2 ada45e91a -> c1958d07e (forced update)


PHOENIX-3296 ArrayConstructor expression does not serialize arrays with leading 
nulls correctly


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

Branch: refs/heads/encodecolumns2
Commit: 67027d618f3817eea9ce215dc3e2999397995b83
Parents: 08d9c71
Author: Thomas D'Silva 
Authored: Wed Oct 5 20:58:03 2016 -0700
Committer: Thomas D'Silva 
Committed: Thu Oct 6 21:08:22 2016 -0700

--
 .../expression/ArrayConstructorExpression.java  |  5 +-
 .../ArrayConstructorExpressionTest.java | 63 
 2 files changed, 67 insertions(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/phoenix/blob/67027d61/phoenix-core/src/main/java/org/apache/phoenix/expression/ArrayConstructorExpression.java
--
diff --git 
a/phoenix-core/src/main/java/org/apache/phoenix/expression/ArrayConstructorExpression.java
 
b/phoenix-core/src/main/java/org/apache/phoenix/expression/ArrayConstructorExpression.java
index d8df29a..c2f4dd2 100644
--- 
a/phoenix-core/src/main/java/org/apache/phoenix/expression/ArrayConstructorExpression.java
+++ 
b/phoenix-core/src/main/java/org/apache/phoenix/expression/ArrayConstructorExpression.java
@@ -31,6 +31,7 @@ import org.apache.phoenix.util.TrustedByteArrayOutputStream;
 public class ArrayConstructorExpression extends BaseCompoundExpression {
 private PDataType baseType;
 private int position = -1;
+private int nNulls = 0;
 private Object[] elements;
 private final ImmutableBytesWritable valuePtr = new 
ImmutableBytesWritable();
 private int estimatedSize = 0;
@@ -71,6 +72,7 @@ public class ArrayConstructorExpression extends 
BaseCompoundExpression {
 public void reset() {
 super.reset();
 position = 0;
+nNulls = 0;
 Arrays.fill(elements, null);
 valuePtr.set(ByteUtil.EMPTY_BYTE_ARRAY);
 }
@@ -85,7 +87,7 @@ public class ArrayConstructorExpression extends 
BaseCompoundExpression {
 DataOutputStream oStream = new DataOutputStream(byteStream);
 try {
 int noOfElements =  children.size();
-int nNulls = 0;
+nNulls = 0;
 for (int i = position >= 0 ? position : 0; i < elements.length; 
i++) {
 Expression child = children.get(i);
 if (!child.evaluate(tuple, ptr)) {
@@ -115,6 +117,7 @@ public class ArrayConstructorExpression extends 
BaseCompoundExpression {
 offsetPos[i] = byteStream.size();
 oStream.write(ptr.get(), ptr.getOffset(), 
ptr.getLength());
 
oStream.write(PArrayDataType.getSeparatorByte(rowKeyOrderOptimizable, 
getSortOrder()));
+nNulls=0;
 }
 } else { // No nulls for fixed length
 oStream.write(ptr.get(), ptr.getOffset(), 
ptr.getLength());

http://git-wip-us.apache.org/repos/asf/phoenix/blob/67027d61/phoenix-core/src/test/java/org/apache/phoenix/expression/ArrayConstructorExpressionTest.java
--
diff --git 
a/phoenix-core/src/test/java/org/apache/phoenix/expression/ArrayConstructorExpressionTest.java
 
b/phoenix-core/src/test/java/org/apache/phoenix/expression/ArrayConstructorExpressionTest.java
new file mode 100644
index 000..e99a71c
--- /dev/null
+++ 
b/phoenix-core/src/test/java/org/apache/phoenix/expression/ArrayConstructorExpressionTest.java
@@ -0,0 +1,63 @@
+/*
+ * 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;
+
+import static org.junit.Assert.assertArrayEquals;
+
+import java.util.List;
+
+import org.apache.phoenix.expression.function.ArrayEl

[08/12] phoenix git commit: Branch rebased to 08d9c7154194de69ce688ad78357bffb8d34c92f of 4.x-HBase-0.98

2016-10-19 Thread samarth
http://git-wip-us.apache.org/repos/asf/phoenix/blob/23a87989/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/generated/PTableProtos.java
--
diff --git 
a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/generated/PTableProtos.java
 
b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/generated/PTableProtos.java
index b8b8b2f..2f0c00b 100644
--- 
a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/generated/PTableProtos.java
+++ 
b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/generated/PTableProtos.java
@@ -269,6 +269,16 @@ public final class PTableProtos {
  * optional bool isDynamic = 14;
  */
 boolean getIsDynamic();
+
+// optional int32 columnQualifier = 15;
+/**
+ * optional int32 columnQualifier = 15;
+ */
+boolean hasColumnQualifier();
+/**
+ * optional int32 columnQualifier = 15;
+ */
+int getColumnQualifier();
   }
   /**
* Protobuf type {@code PColumn}
@@ -391,6 +401,11 @@ public final class PTableProtos {
   isDynamic_ = input.readBool();
   break;
 }
+case 120: {
+  bitField0_ |= 0x4000;
+  columnQualifier_ = input.readInt32();
+  break;
+}
   }
 }
   } catch (com.google.protobuf.InvalidProtocolBufferException e) {
@@ -709,6 +724,22 @@ public final class PTableProtos {
   return isDynamic_;
 }
 
+// optional int32 columnQualifier = 15;
+public static final int COLUMNQUALIFIER_FIELD_NUMBER = 15;
+private int columnQualifier_;
+/**
+ * optional int32 columnQualifier = 15;
+ */
+public boolean hasColumnQualifier() {
+  return ((bitField0_ & 0x4000) == 0x4000);
+}
+/**
+ * optional int32 columnQualifier = 15;
+ */
+public int getColumnQualifier() {
+  return columnQualifier_;
+}
+
 private void initFields() {
   columnNameBytes_ = com.google.protobuf.ByteString.EMPTY;
   familyNameBytes_ = com.google.protobuf.ByteString.EMPTY;
@@ -724,6 +755,7 @@ public final class PTableProtos {
   expression_ = "";
   isRowTimestamp_ = false;
   isDynamic_ = false;
+  columnQualifier_ = 0;
 }
 private byte memoizedIsInitialized = -1;
 public final boolean isInitialized() {
@@ -799,6 +831,9 @@ public final class PTableProtos {
   if (((bitField0_ & 0x2000) == 0x2000)) {
 output.writeBool(14, isDynamic_);
   }
+  if (((bitField0_ & 0x4000) == 0x4000)) {
+output.writeInt32(15, columnQualifier_);
+  }
   getUnknownFields().writeTo(output);
 }
 
@@ -864,6 +899,10 @@ public final class PTableProtos {
 size += com.google.protobuf.CodedOutputStream
   .computeBoolSize(14, isDynamic_);
   }
+  if (((bitField0_ & 0x4000) == 0x4000)) {
+size += com.google.protobuf.CodedOutputStream
+  .computeInt32Size(15, columnQualifier_);
+  }
   size += getUnknownFields().getSerializedSize();
   memoizedSerializedSize = size;
   return size;
@@ -957,6 +996,11 @@ public final class PTableProtos {
 result = result && (getIsDynamic()
 == other.getIsDynamic());
   }
+  result = result && (hasColumnQualifier() == other.hasColumnQualifier());
+  if (hasColumnQualifier()) {
+result = result && (getColumnQualifier()
+== other.getColumnQualifier());
+  }
   result = result &&
   getUnknownFields().equals(other.getUnknownFields());
   return result;
@@ -1026,6 +1070,10 @@ public final class PTableProtos {
 hash = (37 * hash) + ISDYNAMIC_FIELD_NUMBER;
 hash = (53 * hash) + hashBoolean(getIsDynamic());
   }
+  if (hasColumnQualifier()) {
+hash = (37 * hash) + COLUMNQUALIFIER_FIELD_NUMBER;
+hash = (53 * hash) + getColumnQualifier();
+  }
   hash = (29 * hash) + getUnknownFields().hashCode();
   memoizedHashCode = hash;
   return hash;
@@ -1163,6 +1211,8 @@ public final class PTableProtos {
 bitField0_ = (bitField0_ & ~0x1000);
 isDynamic_ = false;
 bitField0_ = (bitField0_ & ~0x2000);
+columnQualifier_ = 0;
+bitField0_ = (bitField0_ & ~0x4000);
 return this;
   }
 
@@ -1247,6 +1297,10 @@ public final class PTableProtos {
   to_bitField0_ |= 0x2000;
 }
 result.isDynamic_ = isDynamic_;
+if (((from_bitField0_ & 0x4000) == 0x4000)) {
+  to_bitField0_ |= 0x4000;
+}
+result.columnQualifier_ = columnQualifier_;
 result.bitField0_ = to_bitField0_;
 onBuilt();
 return result;
@@ -1309,6 +1363,9 @@ public final class PTableProtos {
 if (other.hasIsDynamic()) {
   setIsDynamic(other.getIsDynamic());
 }
+if (other.hasColumnQualifier()

[07/12] phoenix git commit: Branch rebased to 08d9c7154194de69ce688ad78357bffb8d34c92f of 4.x-HBase-0.98

2016-10-19 Thread samarth
http://git-wip-us.apache.org/repos/asf/phoenix/blob/23a87989/phoenix-core/src/main/java/org/apache/phoenix/expression/visitor/CloneExpressionVisitor.java
--
diff --git 
a/phoenix-core/src/main/java/org/apache/phoenix/expression/visitor/CloneExpressionVisitor.java
 
b/phoenix-core/src/main/java/org/apache/phoenix/expression/visitor/CloneExpressionVisitor.java
index 00ece40..15a9f74 100644
--- 
a/phoenix-core/src/main/java/org/apache/phoenix/expression/visitor/CloneExpressionVisitor.java
+++ 
b/phoenix-core/src/main/java/org/apache/phoenix/expression/visitor/CloneExpressionVisitor.java
@@ -26,6 +26,7 @@ import 
org.apache.phoenix.expression.ArrayConstructorExpression;
 import org.apache.phoenix.expression.CaseExpression;
 import org.apache.phoenix.expression.CoerceExpression;
 import org.apache.phoenix.expression.ComparisonExpression;
+import org.apache.phoenix.expression.ArrayColumnExpression;
 import org.apache.phoenix.expression.CorrelateVariableFieldAccessExpression;
 import org.apache.phoenix.expression.DivideExpression;
 import org.apache.phoenix.expression.Expression;
@@ -80,6 +81,11 @@ public abstract class CloneExpressionVisitor extends 
TraverseAllExpressionVisito
 public Expression visit(KeyValueColumnExpression node) {
 return node;
 }
+
+@Override
+public Expression visit(ArrayColumnExpression node) {
+return node;
+}
 
 @Override
 public Expression visit(ProjectedColumnExpression node) {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/23a87989/phoenix-core/src/main/java/org/apache/phoenix/expression/visitor/ExpressionVisitor.java
--
diff --git 
a/phoenix-core/src/main/java/org/apache/phoenix/expression/visitor/ExpressionVisitor.java
 
b/phoenix-core/src/main/java/org/apache/phoenix/expression/visitor/ExpressionVisitor.java
index 31f340d..100f099 100644
--- 
a/phoenix-core/src/main/java/org/apache/phoenix/expression/visitor/ExpressionVisitor.java
+++ 
b/phoenix-core/src/main/java/org/apache/phoenix/expression/visitor/ExpressionVisitor.java
@@ -27,6 +27,7 @@ import 
org.apache.phoenix.expression.ArrayConstructorExpression;
 import org.apache.phoenix.expression.CaseExpression;
 import org.apache.phoenix.expression.CoerceExpression;
 import org.apache.phoenix.expression.ComparisonExpression;
+import org.apache.phoenix.expression.ArrayColumnExpression;
 import org.apache.phoenix.expression.CorrelateVariableFieldAccessExpression;
 import org.apache.phoenix.expression.DivideExpression;
 import org.apache.phoenix.expression.Expression;
@@ -113,6 +114,7 @@ public interface ExpressionVisitor {
 public E visit(LiteralExpression node);
 public E visit(RowKeyColumnExpression node);
 public E visit(KeyValueColumnExpression node);
+public E visit(ArrayColumnExpression node);
 public E visit(ProjectedColumnExpression node);
 public E visit(SequenceValueExpression node);
 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/23a87989/phoenix-core/src/main/java/org/apache/phoenix/expression/visitor/ReplaceArrayColumnWithKeyValueColumnExpressionVisitor.java
--
diff --git 
a/phoenix-core/src/main/java/org/apache/phoenix/expression/visitor/ReplaceArrayColumnWithKeyValueColumnExpressionVisitor.java
 
b/phoenix-core/src/main/java/org/apache/phoenix/expression/visitor/ReplaceArrayColumnWithKeyValueColumnExpressionVisitor.java
new file mode 100644
index 000..7ca6d9e
--- /dev/null
+++ 
b/phoenix-core/src/main/java/org/apache/phoenix/expression/visitor/ReplaceArrayColumnWithKeyValueColumnExpressionVisitor.java
@@ -0,0 +1,37 @@
+/*
+ * 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.visitor;
+
+import java.util.List;
+
+import org.apache.phoenix.expression.ArrayColumnExpression;
+import org.apache.phoenix.expression.Expression;
+
+public class ReplaceArrayColumnWithKeyValueColumnExpressionVisitor extends 
CloneExpressionVisitor {
+
+@Override
+public boolean isCloneNode(Expression node, List children) {
+return !children.equals(node.get

[03/12] phoenix git commit: PHOENIX-3363 Join-related IT had problematic usage of generating new table names

2016-10-19 Thread samarth
PHOENIX-3363 Join-related IT had problematic usage of generating new table names


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

Branch: refs/heads/encodecolumns2
Commit: 153612aa9e6e4ef3542bfb1659e3ec1b1ad17a71
Parents: 67027d6
Author: maryannxue 
Authored: Fri Oct 7 11:58:56 2016 -0700
Committer: maryannxue 
Committed: Fri Oct 7 11:58:56 2016 -0700

--
 .../apache/phoenix/end2end/SortMergeJoinIT.java | 324 ---
 .../org/apache/phoenix/end2end/SubqueryIT.java  |  99 +++---
 .../end2end/SubqueryUsingSortMergeJoinIT.java   |  54 ++--
 3 files changed, 296 insertions(+), 181 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/phoenix/blob/153612aa/phoenix-core/src/it/java/org/apache/phoenix/end2end/SortMergeJoinIT.java
--
diff --git 
a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SortMergeJoinIT.java 
b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SortMergeJoinIT.java
index eade3cb..b387ee8 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SortMergeJoinIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SortMergeJoinIT.java
@@ -187,7 +187,9 @@ public class SortMergeJoinIT extends BaseJoinIT {
 public void testDefaultJoin() throws Exception {
 Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
 Connection conn = DriverManager.getConnection(getUrl(), props);
-String query = "SELECT /*+ USE_SORT_MERGE_JOIN*/ item.\"item_id\", 
item.name, supp.\"supplier_id\", supp.name FROM " + getTableName(conn, 
JOIN_ITEM_TABLE_FULL_NAME) + " item JOIN " + getTableName(conn, 
JOIN_SUPPLIER_TABLE_FULL_NAME) + " supp ON item.\"supplier_id\" = 
supp.\"supplier_id\" ORDER BY \"item_id\"";
+String tableName1 = getTableName(conn, JOIN_ITEM_TABLE_FULL_NAME);
+String tableName2 = getTableName(conn, JOIN_SUPPLIER_TABLE_FULL_NAME);
+String query = "SELECT /*+ USE_SORT_MERGE_JOIN*/ item.\"item_id\", 
item.name, supp.\"supplier_id\", supp.name FROM " + tableName1 + " item JOIN " 
+ tableName2 + " supp ON item.\"supplier_id\" = supp.\"supplier_id\" ORDER BY 
\"item_id\"";
 try {
 PreparedStatement statement = conn.prepareStatement(query);
 ResultSet rs = statement.executeQuery();
@@ -232,7 +234,9 @@ public class SortMergeJoinIT extends BaseJoinIT {
 public void testInnerJoin() throws Exception {
 Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
 Connection conn = DriverManager.getConnection(getUrl(), props);
-String query = "SELECT /*+ USE_SORT_MERGE_JOIN*/ item.\"item_id\", 
item.name, supp.\"supplier_id\", supp.name, next value for " + seqName + " FROM 
" + getTableName(conn, JOIN_ITEM_TABLE_FULL_NAME) + " item INNER JOIN " + 
getTableName(conn, JOIN_SUPPLIER_TABLE_FULL_NAME) + " supp ON 
item.\"supplier_id\" = supp.\"supplier_id\" ORDER BY \"item_id\"";
+String tableName1 = getTableName(conn, JOIN_ITEM_TABLE_FULL_NAME);
+String tableName2 = getTableName(conn, JOIN_SUPPLIER_TABLE_FULL_NAME);
+String query = "SELECT /*+ USE_SORT_MERGE_JOIN*/ item.\"item_id\", 
item.name, supp.\"supplier_id\", supp.name, next value for " + seqName + " FROM 
" + tableName1 + " item INNER JOIN " + tableName2 + " supp ON 
item.\"supplier_id\" = supp.\"supplier_id\" ORDER BY \"item_id\"";
 try {
 PreparedStatement statement = conn.prepareStatement(query);
 ResultSet rs = statement.executeQuery();
@@ -283,10 +287,12 @@ public class SortMergeJoinIT extends BaseJoinIT {
 public void testLeftJoin() throws Exception {
 Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
 Connection conn = DriverManager.getConnection(getUrl(), props);
+String tableName1 = getTableName(conn, JOIN_ITEM_TABLE_FULL_NAME);
+String tableName2 = getTableName(conn, JOIN_SUPPLIER_TABLE_FULL_NAME);
 String query[] = new String[3];
-query[0] = "SELECT /*+ USE_SORT_MERGE_JOIN*/ item.\"item_id\", 
item.name, supp.\"supplier_id\", supp.name, next value for " + seqName + " FROM 
" + getTableName(conn, JOIN_ITEM_TABLE_FULL_NAME) + " item LEFT JOIN " + 
getTableName(conn, JOIN_SUPPLIER_TABLE_FULL_NAME) + " supp ON 
item.\"supplier_id\" = supp.\"supplier_id\" ORDER BY \"item_id\"";
-query[1] = "SELECT /*+ USE_SORT_MERGE_JOIN*/ " + getTableName(conn, 
JOIN_ITEM_TABLE_FULL_NAME) + ".\"item_id\", " + getTableName(conn, 
JOIN_ITEM_TABLE_FULL_NAME) + ".name, " + getTableName(conn, 
JOIN_SUPPLIER_TABLE_FULL_NAME) + ".\"supplier_id\", " + getTableName(conn, 
JOIN_SUPPLIER_T

Build failed in Jenkins: Phoenix Compile Level Compatibility with HBase #86

2016-10-19 Thread Apache Jenkins Server
See 

--
Started by timer
[EnvInject] - Loading node environment variables.
Building remotely on H10 (Ubuntu yahoo-not-h2 ubuntu docker) in workspace 

[Phoenix_Compile_Compat_wHBase] $ /bin/bash /tmp/hudson6664391134891571926.sh
core file size  (blocks, -c) 0
data seg size   (kbytes, -d) unlimited
scheduling priority (-e) 0
file size   (blocks, -f) unlimited
pending signals (-i) 386178
max locked memory   (kbytes, -l) 64
max memory size (kbytes, -m) unlimited
open files  (-n) 6
pipe size(512 bytes, -p) 8
POSIX message queues (bytes, -q) 819200
real-time priority  (-r) 0
stack size  (kbytes, -s) 8192
cpu time   (seconds, -t) unlimited
max user processes  (-u) 10240
virtual memory  (kbytes, -v) unlimited
file locks  (-x) unlimited
core id : 0
core id : 1
core id : 10
core id : 9
physical id : 0
physical id : 1
MemTotal:   49453340 kB
MemFree: 7437064 kB
Filesystem  Size  Used Avail Use% Mounted on
udev 24G   12K   24G   1% /dev
tmpfs   4.8G  944K  4.8G   1% /run
/dev/dm-0   3.6T  857G  2.6T  25% /
none4.0K 0  4.0K   0% /sys/fs/cgroup
none5.0M 0  5.0M   0% /run/lock
none 24G  284K   24G   1% /run/shm
none100M 0  100M   0% /run/user
/dev/sda2   237M  114M  111M  51% /boot
none3.6T  857G  2.6T  25% 
/var/lib/docker/aufs/mnt/270884c588a2c77b8d37c3b2ba4ebe1806dd55bffe4aebfe4285559e3fc1704f
apache-maven-2.2.1
apache-maven-3.0.4
apache-maven-3.0.5
apache-maven-3.2.1
apache-maven-3.2.5
apache-maven-3.3.3
apache-maven-3.3.9
latest
latest2
latest3


===
Verifying compile level compatibility with HBase 0.98 with Phoenix 
4.x-HBase-0.98
===

Cloning into 'hbase'...
Switched to a new branch '0.98'
Branch 0.98 set up to track remote branch 0.98 from origin.

main:
 [exec] 
~/jenkins-slave/workspace/Phoenix_Compile_Compat_wHBase/hbase/hbase-common 
~/jenkins-slave/workspace/Phoenix_Compile_Compat_wHBase/hbase/hbase-common
 [exec] 
~/jenkins-slave/workspace/Phoenix_Compile_Compat_wHBase/hbase/hbase-common

main:
[mkdir] Created dir: 

 [exec] tar: hadoop-snappy-nativelibs.tar: Cannot open: No such file or 
directory
 [exec] tar: Error is not recoverable: exiting now
 [exec] Result: 2

main:
[mkdir] Created dir: 

 [copy] Copying 20 files to 

[mkdir] Created dir: 

[mkdir] Created dir: 


main:
[mkdir] Created dir: 

 [copy] Copying 17 files to 

[mkdir] Created dir: 


main:
[mkdir] Created dir: 

 [copy] Copying 1 file to 

[mkdir] Created dir: 


HBase pom.xml:

Got HBase version as 0.98.23
Cloning into 'phoenix'...
Switched to a new branch '4.x-HBase-0.98'
Branch 4.x-HBase-0.98 set up to track remote branch 4.x-HBase-0.98 from origin.
ANTLR Parser Generator  Version 3.5.2
Output file 

 does not exist: must build 

PhoenixSQL.g


===
Verifying compil