Build failed in Jenkins: Phoenix-4.x-HBase-1.1 #684

2018-03-13 Thread Apache Jenkins Server
See 


Changes:

[maryannxue] PHOENIX-4288 Indexes not used when ordering by primary key

[maryannxue] PHOENIX-3050 Handle DESC columns in child/parent join optimization

[maryannxue] PHOENIX-4437 Make QueryPlan.getEstimatedBytesToScan() independent 
of

[maryannxue] PHOENIX-1556 Base hash versus sort merge join decision on cost

[maryannxue] PHOENIX-4585 Prune local index regions used for join queries

[maryannxue] PHOENIX-4611 Not nullable column impact on join query plans

[maryannxue] Revert "PHOENIX-4611 Not nullable column impact on join query 
plans"

[maryannxue] PHOENIX-4585 Prune local index regions used for join queries 
(addendum)

[maryannxue] PHOENIX-4611 Not nullable column impact on join query plans

--
[...truncated 103.61 KB...]
[INFO] Tests run: 40, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 362.3 s 
- in org.apache.phoenix.end2end.index.GlobalImmutableTxIndexIT
[INFO] Running org.apache.phoenix.end2end.index.LocalImmutableNonTxIndexIT
[INFO] Tests run: 40, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 327.865 
s - in org.apache.phoenix.end2end.index.GlobalMutableNonTxIndexIT
[INFO] Tests run: 14, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 136.667 
s - in org.apache.phoenix.end2end.index.IndexMetadataIT
[INFO] Running org.apache.phoenix.end2end.index.LocalMutableNonTxIndexIT
[INFO] Running org.apache.phoenix.end2end.index.LocalImmutableTxIndexIT
[INFO] Tests run: 40, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 335.193 
s - in org.apache.phoenix.end2end.index.GlobalMutableTxIndexIT
[INFO] Running org.apache.phoenix.end2end.index.LocalMutableTxIndexIT
[INFO] Tests run: 18, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 241.985 
s - in org.apache.phoenix.end2end.index.IndexWithTableSchemaChangeIT
[INFO] Running org.apache.phoenix.end2end.index.MutableIndexIT
[INFO] Tests run: 37, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 363.258 
s - in org.apache.phoenix.end2end.index.IndexUsageIT
[INFO] Running org.apache.phoenix.end2end.index.SaltedIndexIT
[INFO] Tests run: 3, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 27.877 s 
- in org.apache.phoenix.end2end.index.SaltedIndexIT
[INFO] Running org.apache.phoenix.end2end.index.ViewIndexIT
[WARNING] Tests run: 12, Failures: 0, Errors: 0, Skipped: 2, Time elapsed: 
99.593 s - in org.apache.phoenix.end2end.index.ViewIndexIT
[INFO] Tests run: 40, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 446.134 
s - in org.apache.phoenix.end2end.index.LocalImmutableNonTxIndexIT
[INFO] Running org.apache.phoenix.end2end.index.txn.MutableRollbackIT
[INFO] Tests run: 40, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 442.682 
s - in org.apache.phoenix.end2end.index.LocalMutableNonTxIndexIT
[INFO] Running org.apache.phoenix.end2end.index.txn.RollbackIT
[INFO] Running org.apache.phoenix.end2end.join.HashJoinCacheIT
[INFO] Tests run: 1, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 13.036 s 
- in org.apache.phoenix.end2end.join.HashJoinCacheIT
[INFO] Running org.apache.phoenix.end2end.join.HashJoinGlobalIndexIT
[INFO] Tests run: 40, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 471.232 
s - in org.apache.phoenix.end2end.index.LocalImmutableTxIndexIT
[INFO] Tests run: 6, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 100.071 
s - in org.apache.phoenix.end2end.index.txn.MutableRollbackIT
[INFO] Running org.apache.phoenix.end2end.join.HashJoinMoreIT
[INFO] Tests run: 8, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 78.307 s 
- in org.apache.phoenix.end2end.index.txn.RollbackIT
[INFO] Running org.apache.phoenix.end2end.join.HashJoinNoIndexIT
[INFO] Running org.apache.phoenix.end2end.join.HashJoinLocalIndexIT
[INFO] Tests run: 40, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 466.603 
s - in org.apache.phoenix.end2end.index.LocalMutableTxIndexIT
[INFO] Running org.apache.phoenix.end2end.join.SortMergeJoinGlobalIndexIT
[ERROR] Tests run: 8, Failures: 1, Errors: 0, Skipped: 0, Time elapsed: 95.443 
s <<< FAILURE! - in org.apache.phoenix.end2end.join.HashJoinMoreIT
[ERROR] testBug2961(org.apache.phoenix.end2end.join.HashJoinMoreIT)  Time 
elapsed: 4.511 s  <<< FAILURE!
java.lang.AssertionError
at 
org.apache.phoenix.end2end.join.HashJoinMoreIT.testBug2961(HashJoinMoreIT.java:904)

[INFO] Running org.apache.phoenix.end2end.join.SortMergeJoinLocalIndexIT
[INFO] Tests run: 33, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 313.589 
s - in org.apache.phoenix.end2end.join.HashJoinNoIndexIT
[INFO] Running org.apache.phoenix.end2end.join.SortMergeJoinNoIndexIT
[INFO] Tests run: 72, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 791.928 
s - in org.apache.phoenix.end2end.index.MutableIndexIT
[INFO] Running org.apache.phoenix.end2end.join.SubqueryIT
[INFO] Tests run: 33, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 553.915 
s - in 

[2/9] phoenix git commit: PHOENIX-4322 DESC primary key column with variable length does not work in SkipScanFilter

2018-03-13 Thread maryannxue
PHOENIX-4322 DESC primary key column with variable length does not work in 
SkipScanFilter


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

Branch: refs/heads/4.x-HBase-0.98
Commit: 92b57c7893c91d90d78e30171e233043dbcb4583
Parents: 541d6ac
Author: maryannxue 
Authored: Tue Dec 5 10:59:41 2017 -0800
Committer: maryannxue 
Committed: Tue Mar 13 17:16:42 2018 -0700

--
 .../it/java/org/apache/phoenix/end2end/SortOrderIT.java  | 11 ++-
 .../expression/RowValueConstructorExpression.java|  4 ++--
 2 files changed, 12 insertions(+), 3 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/phoenix/blob/92b57c78/phoenix-core/src/it/java/org/apache/phoenix/end2end/SortOrderIT.java
--
diff --git 
a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SortOrderIT.java 
b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SortOrderIT.java
index 655dbb1..3f749c1 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SortOrderIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SortOrderIT.java
@@ -167,7 +167,16 @@ public class SortOrderIT extends ParallelStatsDisabledIT {
 runQueryTest(ddl, upsert("oid", "code"), insertedRows, new 
Object[][]{{"o2", 2}}, new WhereCondition("oid", "IN", "('o2')"),
 table);
 }
-
+
+@Test
+public void inDescCompositePK3() throws Exception {
+String table = generateUniqueName();
+String ddl = "CREATE table " + table + " (oid VARCHAR NOT NULL, code 
VARCHAR NOT NULL constraint pk primary key (oid DESC, code DESC))";
+Object[][] insertedRows = new Object[][]{{"o1", "1"}, {"o2", "2"}, 
{"o3", "3"}};
+runQueryTest(ddl, upsert("oid", "code"), insertedRows, new 
Object[][]{{"o2", "2"}, {"o1", "1"}}, new WhereCondition("(oid, code)", "IN", 
"(('o2', '2'), ('o1', '1'))"),
+table);
+}
+
 @Test
 public void likeDescCompositePK1() throws Exception {
 String table = generateUniqueName();

http://git-wip-us.apache.org/repos/asf/phoenix/blob/92b57c78/phoenix-core/src/main/java/org/apache/phoenix/expression/RowValueConstructorExpression.java
--
diff --git 
a/phoenix-core/src/main/java/org/apache/phoenix/expression/RowValueConstructorExpression.java
 
b/phoenix-core/src/main/java/org/apache/phoenix/expression/RowValueConstructorExpression.java
index 15f6e3e..9bb7234 100644
--- 
a/phoenix-core/src/main/java/org/apache/phoenix/expression/RowValueConstructorExpression.java
+++ 
b/phoenix-core/src/main/java/org/apache/phoenix/expression/RowValueConstructorExpression.java
@@ -199,8 +199,8 @@ public class RowValueConstructorExpression extends 
BaseCompoundExpression {
 // as otherwise we need it to ensure sort order is correct
 for (int k = expressionCount -1 ; 
 k >=0 &&  getChildren().get(k).getDataType() != 
null 
-  && 
!getChildren().get(k).getDataType().isFixedWidth() 
-  && outputBytes[outputSize-1] == 
QueryConstants.SEPARATOR_BYTE ; k--) {
+  && 
!getChildren().get(k).getDataType().isFixedWidth()
+  && outputBytes[outputSize-1] == 
SchemaUtil.getSeparatorByte(true, false, getChildren().get(k)) ; k--) {
 outputSize--;
 }
 ptr.set(outputBytes, 0, outputSize);



[8/9] phoenix git commit: PHOENIX-4585 Prune local index regions used for join queries

2018-03-13 Thread maryannxue
PHOENIX-4585 Prune local index regions used for join queries


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

Branch: refs/heads/4.x-HBase-0.98
Commit: babda3258921fdf4de595ba734d972860d58a0a4
Parents: 6914d54
Author: maryannxue 
Authored: Fri Feb 16 11:29:25 2018 -0800
Committer: maryannxue 
Committed: Tue Mar 13 21:31:00 2018 -0700

--
 .../apache/phoenix/compile/JoinCompiler.java|  37 ++--
 .../apache/phoenix/compile/QueryCompiler.java   |  60 +++---
 .../phoenix/compile/QueryCompilerTest.java  | 186 ++-
 3 files changed, 238 insertions(+), 45 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/phoenix/blob/babda325/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 f5a7e39..4020cf9 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
@@ -1199,7 +1199,8 @@ public class JoinCompiler {
 return AndExpression.create(expressions);
 }
 
-public static SelectStatement optimize(PhoenixStatement statement, 
SelectStatement select, final ColumnResolver resolver) throws SQLException {
+public static Pair> optimize(
+PhoenixStatement statement, SelectStatement select, final 
ColumnResolver resolver) throws SQLException {
 TableRef groupByTableRef = null;
 TableRef orderByTableRef = null;
 if (select.getGroupBy() != null && !select.getGroupBy().isEmpty()) {
@@ -1226,7 +1227,7 @@ public class JoinCompiler {
 QueryCompiler compiler = new QueryCompiler(statement, select, 
resolver, false, null);
 List binds = statement.getParameters();
 StatementContext ctx = new StatementContext(statement, resolver, 
new Scan(), new SequenceManager(statement));
-QueryPlan plan = compiler.compileJoinQuery(ctx, binds, join, 
false, false, null);
+QueryPlan plan = compiler.compileJoinQuery(ctx, binds, join, 
false, false, null, Collections.emptyMap());
 TableRef table = plan.getTableRef();
 if (groupByTableRef != null && !groupByTableRef.equals(table)) {
 groupByTableRef = null;
@@ -1236,7 +1237,8 @@ public class JoinCompiler {
 }
 }
 
-final Map replacement = new HashMap();
+Map replacementMap = null;
+Map dataPlanMap = null;
 
 for (Table table : join.getTables()) {
 if (table.isSubselect())
@@ -1245,19 +1247,30 @@ public class JoinCompiler {
 List groupBy = tableRef.equals(groupByTableRef) ? 
select.getGroupBy() : null;
 List orderBy = tableRef.equals(orderByTableRef) ? 
select.getOrderBy() : null;
 SelectStatement stmt = 
getSubqueryForOptimizedPlan(select.getHint(), table.getDynamicColumns(), 
table.getTableSamplingRate(), tableRef, join.getColumnRefs(), 
table.getPreFiltersCombined(), groupBy, orderBy, table.isWildCardSelect(), 
select.hasSequence(), select.getUdfParseNodes());
-// TODO: As port of PHOENIX-4585, we need to make sure this plan 
has a pointer to the data plan
-// when an index is used instead of the data table, and that this 
method returns that
-// state for downstream processing.
 // TODO: It seems inefficient to be recompiling the statement 
again and again inside of this optimize call
-QueryPlan plan = 
statement.getConnection().getQueryServices().getOptimizer().optimize(statement, 
stmt);
-if (!plan.getTableRef().equals(tableRef)) {
-replacement.put(tableRef, plan.getTableRef());
+QueryPlan dataPlan =
+new QueryCompiler(
+statement, stmt,
+FromCompiler.getResolverForQuery(stmt, 
statement.getConnection()),
+false, null)
+.compile();
+QueryPlan plan = 
statement.getConnection().getQueryServices().getOptimizer().optimize(statement, 
dataPlan);
+TableRef newTableRef = plan.getTableRef();
+if (!newTableRef.equals(tableRef)) {
+if (replacementMap == null) {
+  

[3/9] phoenix git commit: PHOENIX-3050 Handle DESC columns in child/parent join optimization

2018-03-13 Thread maryannxue
PHOENIX-3050 Handle DESC columns in child/parent join optimization


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

Branch: refs/heads/4.x-HBase-0.98
Commit: 977699afe0d66f1434b8bc1c5a751767e563d6ce
Parents: 92b57c7
Author: maryannxue 
Authored: Wed Dec 6 12:07:16 2017 -0800
Committer: maryannxue 
Committed: Tue Mar 13 17:17:16 2018 -0700

--
 .../phoenix/end2end/join/HashJoinMoreIT.java |  5 +
 .../org/apache/phoenix/compile/JoinCompiler.java | 19 +--
 .../apache/phoenix/compile/QueryCompiler.java|  6 +++---
 .../apache/phoenix/compile/WhereOptimizer.java   |  5 -
 4 files changed, 21 insertions(+), 14 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/phoenix/blob/977699af/phoenix-core/src/it/java/org/apache/phoenix/end2end/join/HashJoinMoreIT.java
--
diff --git 
a/phoenix-core/src/it/java/org/apache/phoenix/end2end/join/HashJoinMoreIT.java 
b/phoenix-core/src/it/java/org/apache/phoenix/end2end/join/HashJoinMoreIT.java
index 37ffd02..f09f1d3 100644
--- 
a/phoenix-core/src/it/java/org/apache/phoenix/end2end/join/HashJoinMoreIT.java
+++ 
b/phoenix-core/src/it/java/org/apache/phoenix/end2end/join/HashJoinMoreIT.java
@@ -895,6 +895,11 @@ public class HashJoinMoreIT extends 
ParallelStatsDisabledIT {
 + "FROM ( SELECT ACCOUNT_ID, BUCKET_ID, OBJECT_ID, 
MAX(OBJECT_VERSION) AS MAXVER "
 + "   FROM test2961 GROUP BY ACCOUNT_ID, BUCKET_ID, 
OBJECT_ID) AS X "
 + "   INNER JOIN test2961 AS OBJ ON X.ACCOUNT_ID = 
OBJ.ACCOUNT_ID AND X.BUCKET_ID = OBJ.BUCKET_ID AND X.OBJECT_ID = OBJ.OBJECT_ID 
AND  X.MAXVER = OBJ.OBJECT_VERSION";
+rs = conn.createStatement().executeQuery("explain " + q);
+String plan = QueryUtil.getExplainPlan(rs);
+String dynamicFilter = "DYNAMIC SERVER FILTER BY (OBJ.ACCOUNT_ID, 
OBJ.BUCKET_ID, OBJ.OBJECT_ID, OBJ.OBJECT_VERSION) IN ((X.ACCOUNT_ID, 
X.BUCKET_ID, X.OBJECT_ID, X.MAXVER))";
+assertTrue("Expected '" + dynamicFilter + "' to be used for the 
query, but got:\n" + plan,
+plan.contains(dynamicFilter));
 rs = conn.createStatement().executeQuery(q);
 assertTrue(rs.next());
 assertEquals("", rs.getString(4));

http://git-wip-us.apache.org/repos/asf/phoenix/blob/977699af/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 f9d8711..f3c4c24 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
@@ -108,6 +108,12 @@ import com.google.common.collect.Sets;
 
 public class JoinCompiler {
 
+public enum Strategy {
+HASH_BUILD_LEFT,
+HASH_BUILD_RIGHT,
+SORT_MERGE,
+}
+
 public enum ColumnRefType {
 JOINLOCAL,
 GENERAL,
@@ -487,7 +493,7 @@ public class JoinCompiler {
 return dependencies;
 }
 
-public Pair 
compileJoinConditions(StatementContext lhsCtx, StatementContext rhsCtx, boolean 
sortExpressions) throws SQLException {
+public Pair 
compileJoinConditions(StatementContext lhsCtx, StatementContext rhsCtx, 
Strategy strategy) throws SQLException {
 if (onConditions.isEmpty()) {
 return new Pair(
 Collections. 
singletonList(LiteralExpression.newConstant(1)),
@@ -503,15 +509,16 @@ public class JoinCompiler {
 rhsCompiler.reset();
 Expression right = condition.getRHS().accept(rhsCompiler);
 PDataType toType = getCommonType(left.getDataType(), 
right.getDataType());
-if (left.getDataType() != toType || left.getSortOrder() == 
SortOrder.DESC) {
-left = CoerceExpression.create(left, toType, 
SortOrder.ASC, left.getMaxLength());
+SortOrder toSortOrder = strategy == Strategy.SORT_MERGE ? 
SortOrder.ASC : (strategy == Strategy.HASH_BUILD_LEFT ? right.getSortOrder() : 
left.getSortOrder());
+if (left.getDataType() != toType || left.getSortOrder() != 
toSortOrder) {
+left = CoerceExpression.create(left, toType, toSortOrder, 
left.getMaxLength());
 }
-  

[6/9] phoenix git commit: PHOENIX-1556 Base hash versus sort merge join decision on cost

2018-03-13 Thread maryannxue
http://git-wip-us.apache.org/repos/asf/phoenix/blob/6914d54d/phoenix-core/src/main/java/org/apache/phoenix/execute/ClientAggregatePlan.java
--
diff --git 
a/phoenix-core/src/main/java/org/apache/phoenix/execute/ClientAggregatePlan.java
 
b/phoenix-core/src/main/java/org/apache/phoenix/execute/ClientAggregatePlan.java
index a15ab35..21cbc2d 100644
--- 
a/phoenix-core/src/main/java/org/apache/phoenix/execute/ClientAggregatePlan.java
+++ 
b/phoenix-core/src/main/java/org/apache/phoenix/execute/ClientAggregatePlan.java
@@ -35,6 +35,10 @@ import org.apache.phoenix.compile.QueryPlan;
 import org.apache.phoenix.compile.RowProjector;
 import org.apache.phoenix.compile.StatementContext;
 import org.apache.phoenix.coprocessor.BaseScannerRegionObserver;
+import org.apache.phoenix.execute.visitor.AvgRowWidthVisitor;
+import org.apache.phoenix.execute.visitor.ByteCountVisitor;
+import org.apache.phoenix.execute.visitor.QueryPlanVisitor;
+import org.apache.phoenix.execute.visitor.RowCountVisitor;
 import org.apache.phoenix.expression.Expression;
 import org.apache.phoenix.expression.OrderByExpression;
 import org.apache.phoenix.expression.aggregator.Aggregators;
@@ -90,25 +94,30 @@ public class ClientAggregatePlan extends 
ClientProcessingPlan {
 
 @Override
 public Cost getCost() {
-Long byteCount = null;
-try {
-byteCount = getEstimatedBytesToScan();
-} catch (SQLException e) {
-// ignored.
-}
-
-if (byteCount == null) {
+Double outputBytes = this.accept(new ByteCountVisitor());
+Double inputRows = this.getDelegate().accept(new RowCountVisitor());
+Double rowWidth = this.accept(new AvgRowWidthVisitor());
+if (inputRows == null || outputBytes == null || rowWidth == null) {
 return Cost.UNKNOWN;
 }
+double inputBytes = inputRows * rowWidth;
+double rowsBeforeHaving = RowCountVisitor.aggregate(
+RowCountVisitor.filter(
+inputRows.doubleValue(),
+RowCountVisitor.stripSkipScanFilter(
+context.getScan().getFilter())),
+groupBy);
+double rowsAfterHaving = RowCountVisitor.filter(rowsBeforeHaving, 
having);
+double bytesBeforeHaving = rowWidth * rowsBeforeHaving;
+double bytesAfterHaving = rowWidth * rowsAfterHaving;
 
 int parallelLevel = CostUtil.estimateParallelLevel(
 false, context.getConnection().getQueryServices());
-Cost cost = CostUtil.estimateAggregateCost(byteCount,
-groupBy, clientAggregators.getEstimatedByteSize(), 
parallelLevel);
+Cost cost = CostUtil.estimateAggregateCost(
+inputBytes, bytesBeforeHaving, groupBy, parallelLevel);
 if (!orderBy.getOrderByExpressions().isEmpty()) {
-double outputBytes = CostUtil.estimateAggregateOutputBytes(
-byteCount, groupBy, 
clientAggregators.getEstimatedByteSize());
-Cost orderByCost = CostUtil.estimateOrderByCost(outputBytes, 
parallelLevel);
+Cost orderByCost = CostUtil.estimateOrderByCost(
+bytesAfterHaving, outputBytes, parallelLevel);
 cost = cost.plus(orderByCost);
 }
 return super.getCost().plus(cost);
@@ -210,7 +219,16 @@ public class ClientAggregatePlan extends 
ClientProcessingPlan {
 public GroupBy getGroupBy() {
 return groupBy;
 }
-
+
+@Override
+public  T accept(QueryPlanVisitor visitor) {
+return visitor.visit(this);
+}
+
+public Expression getHaving() {
+return having;
+}
+
 private static class ClientGroupedAggregatingResultIterator extends 
BaseGroupedAggregatingResultIterator {
 private final List groupByExpressions;
 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/6914d54d/phoenix-core/src/main/java/org/apache/phoenix/execute/ClientProcessingPlan.java
--
diff --git 
a/phoenix-core/src/main/java/org/apache/phoenix/execute/ClientProcessingPlan.java
 
b/phoenix-core/src/main/java/org/apache/phoenix/execute/ClientProcessingPlan.java
index ac43919..75ba8f2 100644
--- 
a/phoenix-core/src/main/java/org/apache/phoenix/execute/ClientProcessingPlan.java
+++ 
b/phoenix-core/src/main/java/org/apache/phoenix/execute/ClientProcessingPlan.java
@@ -85,4 +85,8 @@ public abstract class ClientProcessingPlan extends 
DelegateQueryPlan {
 public FilterableStatement getStatement() {
 return statement;
 }
+
+public Expression getWhere() {
+return where;
+}
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/6914d54d/phoenix-core/src/main/java/org/apache/phoenix/execute/ClientScanPlan.java
--
diff --git 

[7/9] phoenix git commit: PHOENIX-1556 Base hash versus sort merge join decision on cost

2018-03-13 Thread maryannxue
PHOENIX-1556 Base hash versus sort merge join decision on cost


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

Branch: refs/heads/4.x-HBase-0.98
Commit: 6914d54d99b4fafae44d1a3397c44ba6e5d10368
Parents: 2c75823
Author: maryannxue 
Authored: Mon Feb 12 14:07:30 2018 -0800
Committer: maryannxue 
Committed: Tue Mar 13 21:19:01 2018 -0700

--
 .../phoenix/end2end/CostBasedDecisionIT.java| 420 -
 .../apache/phoenix/compile/JoinCompiler.java|  43 ++
 .../phoenix/compile/ListJarsQueryPlan.java  |   8 +-
 .../apache/phoenix/compile/QueryCompiler.java   | 455 ++-
 .../org/apache/phoenix/compile/QueryPlan.java   |   2 +
 .../apache/phoenix/compile/TraceQueryPlan.java  |   6 +
 .../apache/phoenix/execute/AggregatePlan.java   |  41 +-
 .../phoenix/execute/ClientAggregatePlan.java|  46 +-
 .../phoenix/execute/ClientProcessingPlan.java   |   4 +
 .../apache/phoenix/execute/ClientScanPlan.java  |  22 +-
 .../apache/phoenix/execute/CorrelatePlan.java   |  26 +-
 .../apache/phoenix/execute/CursorFetchPlan.java |   6 +
 .../apache/phoenix/execute/HashJoinPlan.java| 128 --
 .../execute/LiteralResultIterationPlan.java |   6 +
 .../org/apache/phoenix/execute/ScanPlan.java|  14 +-
 .../phoenix/execute/SortMergeJoinPlan.java  |  20 +-
 .../phoenix/execute/TupleProjectionPlan.java|   6 +
 .../org/apache/phoenix/execute/UnionPlan.java   |  12 +-
 .../apache/phoenix/execute/UnnestArrayPlan.java |   6 +
 .../execute/visitor/AvgRowWidthVisitor.java | 205 +
 .../execute/visitor/ByteCountVisitor.java   | 125 +
 .../execute/visitor/QueryPlanVisitor.java   |  46 ++
 .../execute/visitor/RowCountVisitor.java| 335 ++
 .../apache/phoenix/jdbc/PhoenixStatement.java   |   6 +
 .../java/org/apache/phoenix/util/CostUtil.java  |  61 +--
 .../query/ParallelIteratorsSplitTest.java   |   6 +
 26 files changed, 1615 insertions(+), 440 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/phoenix/blob/6914d54d/phoenix-core/src/it/java/org/apache/phoenix/end2end/CostBasedDecisionIT.java
--
diff --git 
a/phoenix-core/src/it/java/org/apache/phoenix/end2end/CostBasedDecisionIT.java 
b/phoenix-core/src/it/java/org/apache/phoenix/end2end/CostBasedDecisionIT.java
index a3584ce..493855a 100644
--- 
a/phoenix-core/src/it/java/org/apache/phoenix/end2end/CostBasedDecisionIT.java
+++ 
b/phoenix-core/src/it/java/org/apache/phoenix/end2end/CostBasedDecisionIT.java
@@ -32,12 +32,16 @@ import org.apache.phoenix.query.QueryServices;
 import org.apache.phoenix.util.PropertiesUtil;
 import org.apache.phoenix.util.QueryUtil;
 import org.apache.phoenix.util.ReadOnlyProps;
+
 import org.junit.BeforeClass;
 import org.junit.Test;
 
 import com.google.common.collect.Maps;
 
 public class CostBasedDecisionIT extends BaseUniqueNamesOwnClusterIT {
+private final String testTable500;
+private final String testTable990;
+private final String testTable1000;
 
 @BeforeClass
 public static void doSetup() throws Exception {
@@ -46,9 +50,16 @@ public class CostBasedDecisionIT extends 
BaseUniqueNamesOwnClusterIT {
 props.put(QueryServices.STATS_UPDATE_FREQ_MS_ATTRIB, Long.toString(5));
 props.put(QueryServices.USE_STATS_FOR_PARALLELIZATION, 
Boolean.toString(true));
 props.put(QueryServices.COST_BASED_OPTIMIZER_ENABLED, 
Boolean.toString(true));
+props.put(QueryServices.MAX_SERVER_CACHE_SIZE_ATTRIB, 
Long.toString(15));
 setUpTestDriver(new ReadOnlyProps(props.entrySet().iterator()));
 }
 
+public CostBasedDecisionIT() throws Exception {
+testTable500 = initTestTableValues(500);
+testTable990 = initTestTableValues(990);
+testTable1000 = initTestTableValues(1000);
+}
+
 @Test
 public void testCostOverridesStaticPlanOrdering1() throws Exception {
 Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
@@ -64,10 +75,7 @@ public class CostBasedDecisionIT extends 
BaseUniqueNamesOwnClusterIT {
 
 String query = "SELECT rowkey, c1, c2 FROM " + tableName + " where 
c1 LIKE 'X0%' ORDER BY rowkey";
 // Use the data table plan that opts out order-by when stats are 
not available.
-ResultSet rs = conn.createStatement().executeQuery("explain " + 
query);
-String plan = QueryUtil.getExplainPlan(rs);
-assertTrue("Expected 'FULL SCAN' in the plan:\n" + plan + ".",
-plan.contains("FULL SCAN"));
+

[9/9] phoenix git commit: PHOENIX-4611 Not nullable column impact on join query plans

2018-03-13 Thread maryannxue
PHOENIX-4611 Not nullable column impact on join query plans


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

Branch: refs/heads/4.x-HBase-0.98
Commit: 9bb7811f001d00cea42da6185c3645d7d14e4a16
Parents: babda32
Author: maryannxue 
Authored: Fri Feb 16 21:03:46 2018 -0800
Committer: maryannxue 
Committed: Tue Mar 13 21:31:58 2018 -0700

--
 .../java/org/apache/phoenix/end2end/SortMergeJoinMoreIT.java | 8 
 .../java/org/apache/phoenix/end2end/join/HashJoinMoreIT.java | 4 ++--
 .../main/java/org/apache/phoenix/compile/JoinCompiler.java   | 4 
 3 files changed, 10 insertions(+), 6 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/phoenix/blob/9bb7811f/phoenix-core/src/it/java/org/apache/phoenix/end2end/SortMergeJoinMoreIT.java
--
diff --git 
a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SortMergeJoinMoreIT.java 
b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SortMergeJoinMoreIT.java
index a132728..dea349a 100644
--- 
a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SortMergeJoinMoreIT.java
+++ 
b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SortMergeJoinMoreIT.java
@@ -440,8 +440,8 @@ public class SortMergeJoinMoreIT extends 
ParallelStatsDisabledIT {
 "SERVER AGGREGATE INTO ORDERED DISTINCT ROWS 
BY [BUCKET, \"TIMESTAMP\", SRC_LOCATION, DST_LOCATION]\n" +
 "CLIENT MERGE SORT\n" +
 "CLIENT SORTED BY [BUCKET, \"TIMESTAMP\"]\n" +
-"CLIENT SORTED BY [E.BUCKET, E.TIMESTAMP]\n" +
-"CLIENT AGGREGATE INTO DISTINCT ROWS BY [E.BUCKET, 
E.TIMESTAMP]"
+"CLIENT SORTED BY [E.TIMESTAMP, E.BUCKET]\n" +
+"CLIENT AGGREGATE INTO DISTINCT ROWS BY [E.TIMESTAMP, 
E.BUCKET]"
 :
 "SORT-MERGE-JOIN (INNER) TABLES\n" +
 "CLIENT PARALLEL 2-WAY SKIP SCAN ON 2 RANGES OVER 
" + eventCountTableName + " [0,'5SEC',~14629935200,'Tr/Bal'] - 
[1,'5SEC',~14629934200,'Tr/Bal']\n" +
@@ -456,8 +456,8 @@ public class SortMergeJoinMoreIT extends 
ParallelStatsDisabledIT {
 "SERVER DISTINCT PREFIX FILTER OVER [BUCKET, 
\"TIMESTAMP\", SRC_LOCATION, DST_LOCATION]\n" +
 "SERVER AGGREGATE INTO ORDERED DISTINCT ROWS 
BY [BUCKET, \"TIMESTAMP\", SRC_LOCATION, DST_LOCATION]\n" +
 "CLIENT MERGE SORT\n" +
-"CLIENT SORTED BY [E.BUCKET, E.TIMESTAMP]\n" +
-"CLIENT AGGREGATE INTO DISTINCT ROWS BY [E.BUCKET, 
E.TIMESTAMP]";
+"CLIENT SORTED BY [E.TIMESTAMP, E.BUCKET]\n" +
+"CLIENT AGGREGATE INTO DISTINCT ROWS BY [E.TIMESTAMP, 
E.BUCKET]";
 
 ResultSet rs = conn.createStatement().executeQuery("explain " 
+ q);
 assertEquals(p, QueryUtil.getExplainPlan(rs));

http://git-wip-us.apache.org/repos/asf/phoenix/blob/9bb7811f/phoenix-core/src/it/java/org/apache/phoenix/end2end/join/HashJoinMoreIT.java
--
diff --git 
a/phoenix-core/src/it/java/org/apache/phoenix/end2end/join/HashJoinMoreIT.java 
b/phoenix-core/src/it/java/org/apache/phoenix/end2end/join/HashJoinMoreIT.java
index f09f1d3..3a1b015 100644
--- 
a/phoenix-core/src/it/java/org/apache/phoenix/end2end/join/HashJoinMoreIT.java
+++ 
b/phoenix-core/src/it/java/org/apache/phoenix/end2end/join/HashJoinMoreIT.java
@@ -786,7 +786,7 @@ public class HashJoinMoreIT extends ParallelStatsDisabledIT 
{
 String p = i == 0 ?
 "CLIENT PARALLEL 2-WAY SKIP SCAN ON 2 RANGES OVER 
EVENT_COUNT [0,'5SEC',~14629935200,'Tr/Bal'] - 
[1,'5SEC',~14629934200,'Tr/Bal']\n" +
 "SERVER FILTER BY FIRST KEY ONLY\n" +
-"SERVER AGGREGATE INTO DISTINCT ROWS BY [E.BUCKET, 
\"E.TIMESTAMP\"]\n" +
+"SERVER AGGREGATE INTO DISTINCT ROWS BY 
[\"E.TIMESTAMP\", E.BUCKET]\n" +
 "CLIENT MERGE SORT\n" +
 "PARALLEL INNER-JOIN TABLE 0 (SKIP MERGE)\n" +
 "CLIENT PARALLEL 2-WAY SKIP SCAN ON 2 RANGES 
OVER " + t[i] + " [0,'5SEC',~14629935200,'Tr/Bal'] - 
[1,'5SEC',~14629934200,'Tr/Bal']\n" +
@@ -795,7 +795,7 @@ public class HashJoinMoreIT extends 

[1/9] phoenix git commit: PHOENIX-4288 Indexes not used when ordering by primary key

2018-03-13 Thread maryannxue
Repository: phoenix
Updated Branches:
  refs/heads/4.x-HBase-0.98 1e83415f1 -> 9bb7811f0


PHOENIX-4288 Indexes not used when ordering by primary key


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

Branch: refs/heads/4.x-HBase-0.98
Commit: 541d6ac22866fe7571365e063a23108c6ca1ea63
Parents: 1e83415
Author: maryannxue 
Authored: Tue Dec 5 10:52:46 2017 -0800
Committer: maryannxue 
Committed: Tue Mar 13 17:16:04 2018 -0700

--
 .../phoenix/end2end/CostBasedDecisionIT.java| 466 +++
 .../phoenix/compile/ListJarsQueryPlan.java  |   6 +
 .../org/apache/phoenix/compile/QueryPlan.java   |   5 +-
 .../apache/phoenix/compile/TraceQueryPlan.java  |   6 +
 .../apache/phoenix/execute/AggregatePlan.java   |  30 +-
 .../apache/phoenix/execute/BaseQueryPlan.java   |  19 +-
 .../phoenix/execute/ClientAggregatePlan.java|  28 ++
 .../apache/phoenix/execute/ClientScanPlan.java  |  25 +
 .../apache/phoenix/execute/CorrelatePlan.java   |  25 +
 .../phoenix/execute/DelegateQueryPlan.java  |   6 +
 .../apache/phoenix/execute/HashJoinPlan.java|  29 ++
 .../execute/LiteralResultIterationPlan.java |   6 +
 .../org/apache/phoenix/execute/ScanPlan.java|  25 +
 .../phoenix/execute/SortMergeJoinPlan.java  |  18 +
 .../org/apache/phoenix/execute/UnionPlan.java   |  10 +
 .../apache/phoenix/jdbc/PhoenixStatement.java   |   6 +
 .../java/org/apache/phoenix/optimize/Cost.java  | 123 +
 .../apache/phoenix/optimize/QueryOptimizer.java |  28 +-
 .../org/apache/phoenix/query/QueryServices.java |   2 +
 .../phoenix/query/QueryServicesOptions.java |   6 +-
 .../java/org/apache/phoenix/util/CostUtil.java  |  90 
 .../query/ParallelIteratorsSplitTest.java   |   6 +
 22 files changed, 951 insertions(+), 14 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/phoenix/blob/541d6ac2/phoenix-core/src/it/java/org/apache/phoenix/end2end/CostBasedDecisionIT.java
--
diff --git 
a/phoenix-core/src/it/java/org/apache/phoenix/end2end/CostBasedDecisionIT.java 
b/phoenix-core/src/it/java/org/apache/phoenix/end2end/CostBasedDecisionIT.java
new file mode 100644
index 000..a3584ce
--- /dev/null
+++ 
b/phoenix-core/src/it/java/org/apache/phoenix/end2end/CostBasedDecisionIT.java
@@ -0,0 +1,466 @@
+/*
+ * 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.assertTrue;
+
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.util.Map;
+import java.util.Properties;
+
+import org.apache.phoenix.query.BaseTest;
+import org.apache.phoenix.query.QueryServices;
+import org.apache.phoenix.util.PropertiesUtil;
+import org.apache.phoenix.util.QueryUtil;
+import org.apache.phoenix.util.ReadOnlyProps;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import com.google.common.collect.Maps;
+
+public class CostBasedDecisionIT extends BaseUniqueNamesOwnClusterIT {
+
+@BeforeClass
+public static void doSetup() throws Exception {
+Map props = Maps.newHashMapWithExpectedSize(1);
+props.put(QueryServices.STATS_GUIDEPOST_WIDTH_BYTES_ATTRIB, 
Long.toString(20));
+props.put(QueryServices.STATS_UPDATE_FREQ_MS_ATTRIB, Long.toString(5));
+props.put(QueryServices.USE_STATS_FOR_PARALLELIZATION, 
Boolean.toString(true));
+props.put(QueryServices.COST_BASED_OPTIMIZER_ENABLED, 
Boolean.toString(true));
+setUpTestDriver(new ReadOnlyProps(props.entrySet().iterator()));
+}
+
+@Test
+public void testCostOverridesStaticPlanOrdering1() throws Exception {
+Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);

[4/9] phoenix git commit: PHOENIX-4437 Make QueryPlan.getEstimatedBytesToScan() independent of getExplainPlan() and pull optimize() out of getExplainPlan()

2018-03-13 Thread maryannxue
PHOENIX-4437 Make QueryPlan.getEstimatedBytesToScan() independent of 
getExplainPlan() and pull optimize() out of getExplainPlan()


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

Branch: refs/heads/4.x-HBase-0.98
Commit: 7ef96fe1bed43f3ac3dae900a3e6a83791faf697
Parents: 977699a
Author: maryannxue 
Authored: Thu Dec 21 10:31:04 2017 -0800
Committer: maryannxue 
Committed: Tue Mar 13 20:08:24 2018 -0700

--
 .../end2end/ExplainPlanWithStatsEnabledIT.java  |  4 +-
 .../apache/phoenix/execute/BaseQueryPlan.java   | 45 ++
 .../apache/phoenix/execute/HashJoinPlan.java| 59 +-
 .../phoenix/execute/SortMergeJoinPlan.java  | 63 ++--
 .../org/apache/phoenix/execute/UnionPlan.java   | 53 
 .../apache/phoenix/jdbc/PhoenixStatement.java   |  9 ++-
 6 files changed, 120 insertions(+), 113 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/phoenix/blob/7ef96fe1/phoenix-core/src/it/java/org/apache/phoenix/end2end/ExplainPlanWithStatsEnabledIT.java
--
diff --git 
a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ExplainPlanWithStatsEnabledIT.java
 
b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ExplainPlanWithStatsEnabledIT.java
index 76ed7ba..a835e84 100644
--- 
a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ExplainPlanWithStatsEnabledIT.java
+++ 
b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ExplainPlanWithStatsEnabledIT.java
@@ -303,8 +303,8 @@ public class ExplainPlanWithStatsEnabledIT extends 
ParallelStatsEnabledIT {
 try (Connection conn = DriverManager.getConnection(getUrl())) {
 conn.setAutoCommit(false);
 Estimate info = getByteRowEstimates(conn, sql, binds);
-assertEquals((Long) 200L, info.estimatedBytes);
-assertEquals((Long) 2L, info.estimatedRows);
+assertEquals((Long) 176l, info.estimatedBytes);
+assertEquals((Long) 2l, info.estimatedRows);
 assertTrue(info.estimateInfoTs > 0);
 }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/7ef96fe1/phoenix-core/src/main/java/org/apache/phoenix/execute/BaseQueryPlan.java
--
diff --git 
a/phoenix-core/src/main/java/org/apache/phoenix/execute/BaseQueryPlan.java 
b/phoenix-core/src/main/java/org/apache/phoenix/execute/BaseQueryPlan.java
index 38ed926..c4edf31 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/execute/BaseQueryPlan.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/execute/BaseQueryPlan.java
@@ -118,7 +118,7 @@ public abstract class BaseQueryPlan implements QueryPlan {
 protected Long estimatedRows;
 protected Long estimatedSize;
 protected Long estimateInfoTimestamp;
-private boolean explainPlanCalled;
+private boolean getEstimatesCalled;
 
 
 protected BaseQueryPlan(
@@ -500,32 +500,17 @@ public abstract class BaseQueryPlan implements QueryPlan {
 
 @Override
 public ExplainPlan getExplainPlan() throws SQLException {
-explainPlanCalled = true;
 if (context.getScanRanges() == ScanRanges.NOTHING) {
 return new ExplainPlan(Collections.singletonList("DEGENERATE SCAN 
OVER " + getTableRef().getTable().getName().getString()));
 }
 
-// If cost-based optimizer is enabled, we need to initialize a dummy 
iterator to
-// get the stats for computing costs.
-boolean costBased =
-
context.getConnection().getQueryServices().getConfiguration().getBoolean(
-QueryServices.COST_BASED_OPTIMIZER_ENABLED, 
QueryServicesOptions.DEFAULT_COST_BASED_OPTIMIZER_ENABLED);
-if (costBased) {
-ResultIterator iterator = iterator();
-iterator.close();
-}
-// Optimize here when getting explain plan, as queries don't get 
optimized until after compilation
-QueryPlan plan = 
context.getConnection().getQueryServices().getOptimizer().optimize(context.getStatement(),
 this);
-ExplainPlan exp = plan instanceof BaseQueryPlan ? new 
ExplainPlan(getPlanSteps(plan.iterator())) : plan.getExplainPlan();
-if (!costBased) { // do not override estimates if they are used for 
cost calculation.
-this.estimatedRows = plan.getEstimatedRowsToScan();
-this.estimatedSize = plan.getEstimatedBytesToScan();
-this.estimateInfoTimestamp = plan.getEstimateInfoTimestamp();
-}
-return exp;
+ResultIterator 

[5/9] phoenix git commit: PHOENIX-3505 Avoid NPE on close() in OrderedResultIterator

2018-03-13 Thread maryannxue
PHOENIX-3505 Avoid NPE on close() in OrderedResultIterator


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

Branch: refs/heads/4.x-HBase-0.98
Commit: 2c758234186e5a4d70cdc6501df19f9b0d9ec601
Parents: 7ef96fe
Author: Josh Elser 
Authored: Wed Nov 23 11:16:35 2016 -0500
Committer: maryannxue 
Committed: Tue Mar 13 20:08:46 2018 -0700

--
 .../phoenix/iterate/OrderedResultIterator.java  |  5 ++-
 .../iterate/OrderedResultIteratorTest.java  | 41 
 2 files changed, 45 insertions(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/phoenix/blob/2c758234/phoenix-core/src/main/java/org/apache/phoenix/iterate/OrderedResultIterator.java
--
diff --git 
a/phoenix-core/src/main/java/org/apache/phoenix/iterate/OrderedResultIterator.java
 
b/phoenix-core/src/main/java/org/apache/phoenix/iterate/OrderedResultIterator.java
index 36ca00b..36b274a 100644
--- 
a/phoenix-core/src/main/java/org/apache/phoenix/iterate/OrderedResultIterator.java
+++ 
b/phoenix-core/src/main/java/org/apache/phoenix/iterate/OrderedResultIterator.java
@@ -280,7 +280,10 @@ public class OrderedResultIterator implements 
PeekingResultIterator {
 
 @Override
 public void close() throws SQLException {
-resultIterator.close();
+// Guard against resultIterator being null
+if (null != resultIterator) {
+resultIterator.close();
+}
 resultIterator = PeekingResultIterator.EMPTY_ITERATOR;
 }
 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/2c758234/phoenix-core/src/test/java/org/apache/phoenix/iterate/OrderedResultIteratorTest.java
--
diff --git 
a/phoenix-core/src/test/java/org/apache/phoenix/iterate/OrderedResultIteratorTest.java
 
b/phoenix-core/src/test/java/org/apache/phoenix/iterate/OrderedResultIteratorTest.java
new file mode 100644
index 000..50ed8e9
--- /dev/null
+++ 
b/phoenix-core/src/test/java/org/apache/phoenix/iterate/OrderedResultIteratorTest.java
@@ -0,0 +1,41 @@
+/*
+ * 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.iterate;
+
+import java.sql.SQLException;
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.phoenix.expression.OrderByExpression;
+import org.junit.Test;
+
+/**
+ * Test class for {@link OrderedResultIterator}.
+ */
+public class OrderedResultIteratorTest {
+
+  @Test
+  public void testNullIteratorOnClose() throws SQLException {
+  ResultIterator delegate =  ResultIterator.EMPTY_ITERATOR;
+  List orderByExpressions = 
Collections.singletonList(null);
+  int thresholdBytes = Integer.MAX_VALUE;
+  OrderedResultIterator iterator = new OrderedResultIterator(delegate, 
orderByExpressions, thresholdBytes);
+  // Should not throw an exception
+  iterator.close();
+  }
+
+}



Apache-Phoenix | 4.x-HBase-1.1 | Build Successful

2018-03-13 Thread Apache Jenkins Server
4.x-HBase-1.1 branch build status Successful

Source repository https://git-wip-us.apache.org/repos/asf?p=phoenix.git;a=shortlog;h=refs/heads/4.x-HBase-1.1

Compiled Artifacts https://builds.apache.org/job/Phoenix-4.x-HBase-1.1/lastSuccessfulBuild/artifact/

Test Report https://builds.apache.org/job/Phoenix-4.x-HBase-1.1/lastCompletedBuild/testReport/

Changes
[tdsilva] PHOENIX-4634 Looking up a parent index table of a tenant child view



Build times for last couple of runsLatest build time is the right most | Legend blue: normal, red: test failure, gray: timeout


[8/9] phoenix git commit: PHOENIX-4585 Prune local index regions used for join queries (addendum)

2018-03-13 Thread maryannxue
PHOENIX-4585 Prune local index regions used for join queries (addendum)


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

Branch: refs/heads/4.x-HBase-1.1
Commit: eafcdd1ac7875aca48f869adc29f6f0d88942559
Parents: 4b2659b
Author: maryannxue 
Authored: Tue Mar 13 17:02:01 2018 -0700
Committer: maryannxue 
Committed: Tue Mar 13 17:02:01 2018 -0700

--
 .../src/test/java/org/apache/phoenix/compile/QueryCompilerTest.java | 1 -
 1 file changed, 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/phoenix/blob/eafcdd1a/phoenix-core/src/test/java/org/apache/phoenix/compile/QueryCompilerTest.java
--
diff --git 
a/phoenix-core/src/test/java/org/apache/phoenix/compile/QueryCompilerTest.java 
b/phoenix-core/src/test/java/org/apache/phoenix/compile/QueryCompilerTest.java
index 0d5a0ff..1e615c1 100644
--- 
a/phoenix-core/src/test/java/org/apache/phoenix/compile/QueryCompilerTest.java
+++ 
b/phoenix-core/src/test/java/org/apache/phoenix/compile/QueryCompilerTest.java
@@ -4728,5 +4728,4 @@ public class QueryCompilerTest extends 
BaseConnectionlessQueryTest {
 return Collections.emptyList();
 }
 }
->>> 11308c8aa... PHOENIX-4585 Prune local index regions used for join 
queries
 }



[5/9] phoenix git commit: PHOENIX-4585 Prune local index regions used for join queries

2018-03-13 Thread maryannxue
PHOENIX-4585 Prune local index regions used for join queries


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

Branch: refs/heads/4.x-HBase-1.1
Commit: adbce12f90ad3341aa7991bb0445fe1d11cda597
Parents: 6b40a36
Author: maryannxue 
Authored: Fri Feb 16 11:29:25 2018 -0800
Committer: maryannxue 
Committed: Tue Mar 13 16:54:27 2018 -0700

--
 .../apache/phoenix/compile/JoinCompiler.java|  37 ++--
 .../apache/phoenix/compile/QueryCompiler.java   |  56 +++---
 .../phoenix/compile/QueryCompilerTest.java  | 187 ++-
 3 files changed, 237 insertions(+), 43 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/phoenix/blob/adbce12f/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 f5a7e39..4020cf9 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
@@ -1199,7 +1199,8 @@ public class JoinCompiler {
 return AndExpression.create(expressions);
 }
 
-public static SelectStatement optimize(PhoenixStatement statement, 
SelectStatement select, final ColumnResolver resolver) throws SQLException {
+public static Pair> optimize(
+PhoenixStatement statement, SelectStatement select, final 
ColumnResolver resolver) throws SQLException {
 TableRef groupByTableRef = null;
 TableRef orderByTableRef = null;
 if (select.getGroupBy() != null && !select.getGroupBy().isEmpty()) {
@@ -1226,7 +1227,7 @@ public class JoinCompiler {
 QueryCompiler compiler = new QueryCompiler(statement, select, 
resolver, false, null);
 List binds = statement.getParameters();
 StatementContext ctx = new StatementContext(statement, resolver, 
new Scan(), new SequenceManager(statement));
-QueryPlan plan = compiler.compileJoinQuery(ctx, binds, join, 
false, false, null);
+QueryPlan plan = compiler.compileJoinQuery(ctx, binds, join, 
false, false, null, Collections.emptyMap());
 TableRef table = plan.getTableRef();
 if (groupByTableRef != null && !groupByTableRef.equals(table)) {
 groupByTableRef = null;
@@ -1236,7 +1237,8 @@ public class JoinCompiler {
 }
 }
 
-final Map replacement = new HashMap();
+Map replacementMap = null;
+Map dataPlanMap = null;
 
 for (Table table : join.getTables()) {
 if (table.isSubselect())
@@ -1245,19 +1247,30 @@ public class JoinCompiler {
 List groupBy = tableRef.equals(groupByTableRef) ? 
select.getGroupBy() : null;
 List orderBy = tableRef.equals(orderByTableRef) ? 
select.getOrderBy() : null;
 SelectStatement stmt = 
getSubqueryForOptimizedPlan(select.getHint(), table.getDynamicColumns(), 
table.getTableSamplingRate(), tableRef, join.getColumnRefs(), 
table.getPreFiltersCombined(), groupBy, orderBy, table.isWildCardSelect(), 
select.hasSequence(), select.getUdfParseNodes());
-// TODO: As port of PHOENIX-4585, we need to make sure this plan 
has a pointer to the data plan
-// when an index is used instead of the data table, and that this 
method returns that
-// state for downstream processing.
 // TODO: It seems inefficient to be recompiling the statement 
again and again inside of this optimize call
-QueryPlan plan = 
statement.getConnection().getQueryServices().getOptimizer().optimize(statement, 
stmt);
-if (!plan.getTableRef().equals(tableRef)) {
-replacement.put(tableRef, plan.getTableRef());
+QueryPlan dataPlan =
+new QueryCompiler(
+statement, stmt,
+FromCompiler.getResolverForQuery(stmt, 
statement.getConnection()),
+false, null)
+.compile();
+QueryPlan plan = 
statement.getConnection().getQueryServices().getOptimizer().optimize(statement, 
dataPlan);
+TableRef newTableRef = plan.getTableRef();
+if (!newTableRef.equals(tableRef)) {
+if (replacementMap == null) {
+   

[1/9] phoenix git commit: PHOENIX-3050 Handle DESC columns in child/parent join optimization

2018-03-13 Thread maryannxue
Repository: phoenix
Updated Branches:
  refs/heads/4.x-HBase-1.1 49f19ba68 -> c65c81a47


PHOENIX-3050 Handle DESC columns in child/parent join optimization


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

Branch: refs/heads/4.x-HBase-1.1
Commit: 33ca96e656f3eac2802bcd5d3a5e1e4780455cf5
Parents: 49f19ba
Author: maryannxue 
Authored: Wed Dec 6 12:07:16 2017 -0800
Committer: maryannxue 
Committed: Tue Mar 13 16:44:33 2018 -0700

--
 .../phoenix/end2end/join/HashJoinMoreIT.java |  5 +
 .../org/apache/phoenix/compile/JoinCompiler.java | 19 +--
 .../apache/phoenix/compile/QueryCompiler.java|  6 +++---
 .../apache/phoenix/compile/WhereOptimizer.java   |  5 -
 4 files changed, 21 insertions(+), 14 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/phoenix/blob/33ca96e6/phoenix-core/src/it/java/org/apache/phoenix/end2end/join/HashJoinMoreIT.java
--
diff --git 
a/phoenix-core/src/it/java/org/apache/phoenix/end2end/join/HashJoinMoreIT.java 
b/phoenix-core/src/it/java/org/apache/phoenix/end2end/join/HashJoinMoreIT.java
index 37ffd02..f09f1d3 100644
--- 
a/phoenix-core/src/it/java/org/apache/phoenix/end2end/join/HashJoinMoreIT.java
+++ 
b/phoenix-core/src/it/java/org/apache/phoenix/end2end/join/HashJoinMoreIT.java
@@ -895,6 +895,11 @@ public class HashJoinMoreIT extends 
ParallelStatsDisabledIT {
 + "FROM ( SELECT ACCOUNT_ID, BUCKET_ID, OBJECT_ID, 
MAX(OBJECT_VERSION) AS MAXVER "
 + "   FROM test2961 GROUP BY ACCOUNT_ID, BUCKET_ID, 
OBJECT_ID) AS X "
 + "   INNER JOIN test2961 AS OBJ ON X.ACCOUNT_ID = 
OBJ.ACCOUNT_ID AND X.BUCKET_ID = OBJ.BUCKET_ID AND X.OBJECT_ID = OBJ.OBJECT_ID 
AND  X.MAXVER = OBJ.OBJECT_VERSION";
+rs = conn.createStatement().executeQuery("explain " + q);
+String plan = QueryUtil.getExplainPlan(rs);
+String dynamicFilter = "DYNAMIC SERVER FILTER BY (OBJ.ACCOUNT_ID, 
OBJ.BUCKET_ID, OBJ.OBJECT_ID, OBJ.OBJECT_VERSION) IN ((X.ACCOUNT_ID, 
X.BUCKET_ID, X.OBJECT_ID, X.MAXVER))";
+assertTrue("Expected '" + dynamicFilter + "' to be used for the 
query, but got:\n" + plan,
+plan.contains(dynamicFilter));
 rs = conn.createStatement().executeQuery(q);
 assertTrue(rs.next());
 assertEquals("", rs.getString(4));

http://git-wip-us.apache.org/repos/asf/phoenix/blob/33ca96e6/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 f9d8711..f3c4c24 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
@@ -108,6 +108,12 @@ import com.google.common.collect.Sets;
 
 public class JoinCompiler {
 
+public enum Strategy {
+HASH_BUILD_LEFT,
+HASH_BUILD_RIGHT,
+SORT_MERGE,
+}
+
 public enum ColumnRefType {
 JOINLOCAL,
 GENERAL,
@@ -487,7 +493,7 @@ public class JoinCompiler {
 return dependencies;
 }
 
-public Pair 
compileJoinConditions(StatementContext lhsCtx, StatementContext rhsCtx, boolean 
sortExpressions) throws SQLException {
+public Pair 
compileJoinConditions(StatementContext lhsCtx, StatementContext rhsCtx, 
Strategy strategy) throws SQLException {
 if (onConditions.isEmpty()) {
 return new Pair(
 Collections. 
singletonList(LiteralExpression.newConstant(1)),
@@ -503,15 +509,16 @@ public class JoinCompiler {
 rhsCompiler.reset();
 Expression right = condition.getRHS().accept(rhsCompiler);
 PDataType toType = getCommonType(left.getDataType(), 
right.getDataType());
-if (left.getDataType() != toType || left.getSortOrder() == 
SortOrder.DESC) {
-left = CoerceExpression.create(left, toType, 
SortOrder.ASC, left.getMaxLength());
+SortOrder toSortOrder = strategy == Strategy.SORT_MERGE ? 
SortOrder.ASC : (strategy == Strategy.HASH_BUILD_LEFT ? right.getSortOrder() : 
left.getSortOrder());
+if (left.getDataType() != toType || left.getSortOrder() != 
toSortOrder) {
+left = 

[9/9] phoenix git commit: PHOENIX-4611 Not nullable column impact on join query plans

2018-03-13 Thread maryannxue
PHOENIX-4611 Not nullable column impact on join query plans


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

Branch: refs/heads/4.x-HBase-1.1
Commit: c65c81a47a65127a60886a984378c268598fd2e0
Parents: eafcdd1
Author: maryannxue 
Authored: Fri Feb 16 21:03:46 2018 -0800
Committer: maryannxue 
Committed: Tue Mar 13 17:02:13 2018 -0700

--
 .../java/org/apache/phoenix/end2end/SortMergeJoinMoreIT.java | 8 
 .../java/org/apache/phoenix/end2end/join/HashJoinMoreIT.java | 4 ++--
 .../main/java/org/apache/phoenix/compile/JoinCompiler.java   | 4 
 .../org/apache/phoenix/compile/TupleProjectionCompiler.java  | 2 +-
 4 files changed, 11 insertions(+), 7 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/phoenix/blob/c65c81a4/phoenix-core/src/it/java/org/apache/phoenix/end2end/SortMergeJoinMoreIT.java
--
diff --git 
a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SortMergeJoinMoreIT.java 
b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SortMergeJoinMoreIT.java
index a132728..dea349a 100644
--- 
a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SortMergeJoinMoreIT.java
+++ 
b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SortMergeJoinMoreIT.java
@@ -440,8 +440,8 @@ public class SortMergeJoinMoreIT extends 
ParallelStatsDisabledIT {
 "SERVER AGGREGATE INTO ORDERED DISTINCT ROWS 
BY [BUCKET, \"TIMESTAMP\", SRC_LOCATION, DST_LOCATION]\n" +
 "CLIENT MERGE SORT\n" +
 "CLIENT SORTED BY [BUCKET, \"TIMESTAMP\"]\n" +
-"CLIENT SORTED BY [E.BUCKET, E.TIMESTAMP]\n" +
-"CLIENT AGGREGATE INTO DISTINCT ROWS BY [E.BUCKET, 
E.TIMESTAMP]"
+"CLIENT SORTED BY [E.TIMESTAMP, E.BUCKET]\n" +
+"CLIENT AGGREGATE INTO DISTINCT ROWS BY [E.TIMESTAMP, 
E.BUCKET]"
 :
 "SORT-MERGE-JOIN (INNER) TABLES\n" +
 "CLIENT PARALLEL 2-WAY SKIP SCAN ON 2 RANGES OVER 
" + eventCountTableName + " [0,'5SEC',~14629935200,'Tr/Bal'] - 
[1,'5SEC',~14629934200,'Tr/Bal']\n" +
@@ -456,8 +456,8 @@ public class SortMergeJoinMoreIT extends 
ParallelStatsDisabledIT {
 "SERVER DISTINCT PREFIX FILTER OVER [BUCKET, 
\"TIMESTAMP\", SRC_LOCATION, DST_LOCATION]\n" +
 "SERVER AGGREGATE INTO ORDERED DISTINCT ROWS 
BY [BUCKET, \"TIMESTAMP\", SRC_LOCATION, DST_LOCATION]\n" +
 "CLIENT MERGE SORT\n" +
-"CLIENT SORTED BY [E.BUCKET, E.TIMESTAMP]\n" +
-"CLIENT AGGREGATE INTO DISTINCT ROWS BY [E.BUCKET, 
E.TIMESTAMP]";
+"CLIENT SORTED BY [E.TIMESTAMP, E.BUCKET]\n" +
+"CLIENT AGGREGATE INTO DISTINCT ROWS BY [E.TIMESTAMP, 
E.BUCKET]";
 
 ResultSet rs = conn.createStatement().executeQuery("explain " 
+ q);
 assertEquals(p, QueryUtil.getExplainPlan(rs));

http://git-wip-us.apache.org/repos/asf/phoenix/blob/c65c81a4/phoenix-core/src/it/java/org/apache/phoenix/end2end/join/HashJoinMoreIT.java
--
diff --git 
a/phoenix-core/src/it/java/org/apache/phoenix/end2end/join/HashJoinMoreIT.java 
b/phoenix-core/src/it/java/org/apache/phoenix/end2end/join/HashJoinMoreIT.java
index f09f1d3..3a1b015 100644
--- 
a/phoenix-core/src/it/java/org/apache/phoenix/end2end/join/HashJoinMoreIT.java
+++ 
b/phoenix-core/src/it/java/org/apache/phoenix/end2end/join/HashJoinMoreIT.java
@@ -786,7 +786,7 @@ public class HashJoinMoreIT extends ParallelStatsDisabledIT 
{
 String p = i == 0 ?
 "CLIENT PARALLEL 2-WAY SKIP SCAN ON 2 RANGES OVER 
EVENT_COUNT [0,'5SEC',~14629935200,'Tr/Bal'] - 
[1,'5SEC',~14629934200,'Tr/Bal']\n" +
 "SERVER FILTER BY FIRST KEY ONLY\n" +
-"SERVER AGGREGATE INTO DISTINCT ROWS BY [E.BUCKET, 
\"E.TIMESTAMP\"]\n" +
+"SERVER AGGREGATE INTO DISTINCT ROWS BY 
[\"E.TIMESTAMP\", E.BUCKET]\n" +
 "CLIENT MERGE SORT\n" +
 "PARALLEL INNER-JOIN TABLE 0 (SKIP MERGE)\n" +
 "CLIENT PARALLEL 2-WAY SKIP SCAN ON 2 RANGES 
OVER " + t[i] + " [0,'5SEC',~14629935200,'Tr/Bal'] - 
[1,'5SEC',~14629934200,'Tr/Bal']\n" +
@@ 

[7/9] phoenix git commit: Revert "PHOENIX-4611 Not nullable column impact on join query plans"

2018-03-13 Thread maryannxue
Revert "PHOENIX-4611 Not nullable column impact on join query plans"


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

Branch: refs/heads/4.x-HBase-1.1
Commit: 4b2659b85ed3a68722a81d4ff7a6b8fdda9b79ed
Parents: 8162813
Author: maryannxue 
Authored: Tue Mar 13 16:59:05 2018 -0700
Committer: maryannxue 
Committed: Tue Mar 13 16:59:05 2018 -0700

--
 .../java/org/apache/phoenix/end2end/SortMergeJoinMoreIT.java | 8 
 .../java/org/apache/phoenix/end2end/join/HashJoinMoreIT.java | 4 ++--
 .../main/java/org/apache/phoenix/compile/JoinCompiler.java   | 4 
 .../org/apache/phoenix/compile/TupleProjectionCompiler.java  | 2 +-
 4 files changed, 7 insertions(+), 11 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/phoenix/blob/4b2659b8/phoenix-core/src/it/java/org/apache/phoenix/end2end/SortMergeJoinMoreIT.java
--
diff --git 
a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SortMergeJoinMoreIT.java 
b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SortMergeJoinMoreIT.java
index dea349a..a132728 100644
--- 
a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SortMergeJoinMoreIT.java
+++ 
b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SortMergeJoinMoreIT.java
@@ -440,8 +440,8 @@ public class SortMergeJoinMoreIT extends 
ParallelStatsDisabledIT {
 "SERVER AGGREGATE INTO ORDERED DISTINCT ROWS 
BY [BUCKET, \"TIMESTAMP\", SRC_LOCATION, DST_LOCATION]\n" +
 "CLIENT MERGE SORT\n" +
 "CLIENT SORTED BY [BUCKET, \"TIMESTAMP\"]\n" +
-"CLIENT SORTED BY [E.TIMESTAMP, E.BUCKET]\n" +
-"CLIENT AGGREGATE INTO DISTINCT ROWS BY [E.TIMESTAMP, 
E.BUCKET]"
+"CLIENT SORTED BY [E.BUCKET, E.TIMESTAMP]\n" +
+"CLIENT AGGREGATE INTO DISTINCT ROWS BY [E.BUCKET, 
E.TIMESTAMP]"
 :
 "SORT-MERGE-JOIN (INNER) TABLES\n" +
 "CLIENT PARALLEL 2-WAY SKIP SCAN ON 2 RANGES OVER 
" + eventCountTableName + " [0,'5SEC',~14629935200,'Tr/Bal'] - 
[1,'5SEC',~14629934200,'Tr/Bal']\n" +
@@ -456,8 +456,8 @@ public class SortMergeJoinMoreIT extends 
ParallelStatsDisabledIT {
 "SERVER DISTINCT PREFIX FILTER OVER [BUCKET, 
\"TIMESTAMP\", SRC_LOCATION, DST_LOCATION]\n" +
 "SERVER AGGREGATE INTO ORDERED DISTINCT ROWS 
BY [BUCKET, \"TIMESTAMP\", SRC_LOCATION, DST_LOCATION]\n" +
 "CLIENT MERGE SORT\n" +
-"CLIENT SORTED BY [E.TIMESTAMP, E.BUCKET]\n" +
-"CLIENT AGGREGATE INTO DISTINCT ROWS BY [E.TIMESTAMP, 
E.BUCKET]";
+"CLIENT SORTED BY [E.BUCKET, E.TIMESTAMP]\n" +
+"CLIENT AGGREGATE INTO DISTINCT ROWS BY [E.BUCKET, 
E.TIMESTAMP]";
 
 ResultSet rs = conn.createStatement().executeQuery("explain " 
+ q);
 assertEquals(p, QueryUtil.getExplainPlan(rs));

http://git-wip-us.apache.org/repos/asf/phoenix/blob/4b2659b8/phoenix-core/src/it/java/org/apache/phoenix/end2end/join/HashJoinMoreIT.java
--
diff --git 
a/phoenix-core/src/it/java/org/apache/phoenix/end2end/join/HashJoinMoreIT.java 
b/phoenix-core/src/it/java/org/apache/phoenix/end2end/join/HashJoinMoreIT.java
index 3a1b015..f09f1d3 100644
--- 
a/phoenix-core/src/it/java/org/apache/phoenix/end2end/join/HashJoinMoreIT.java
+++ 
b/phoenix-core/src/it/java/org/apache/phoenix/end2end/join/HashJoinMoreIT.java
@@ -786,7 +786,7 @@ public class HashJoinMoreIT extends ParallelStatsDisabledIT 
{
 String p = i == 0 ?
 "CLIENT PARALLEL 2-WAY SKIP SCAN ON 2 RANGES OVER 
EVENT_COUNT [0,'5SEC',~14629935200,'Tr/Bal'] - 
[1,'5SEC',~14629934200,'Tr/Bal']\n" +
 "SERVER FILTER BY FIRST KEY ONLY\n" +
-"SERVER AGGREGATE INTO DISTINCT ROWS BY 
[\"E.TIMESTAMP\", E.BUCKET]\n" +
+"SERVER AGGREGATE INTO DISTINCT ROWS BY [E.BUCKET, 
\"E.TIMESTAMP\"]\n" +
 "CLIENT MERGE SORT\n" +
 "PARALLEL INNER-JOIN TABLE 0 (SKIP MERGE)\n" +
 "CLIENT PARALLEL 2-WAY SKIP SCAN ON 2 RANGES 
OVER " + t[i] + " [0,'5SEC',~14629935200,'Tr/Bal'] - 

[3/9] phoenix git commit: PHOENIX-1556 Base hash versus sort merge join decision on cost

2018-03-13 Thread maryannxue
http://git-wip-us.apache.org/repos/asf/phoenix/blob/6b40a36b/phoenix-core/src/main/java/org/apache/phoenix/execute/ClientAggregatePlan.java
--
diff --git 
a/phoenix-core/src/main/java/org/apache/phoenix/execute/ClientAggregatePlan.java
 
b/phoenix-core/src/main/java/org/apache/phoenix/execute/ClientAggregatePlan.java
index a15ab35..21cbc2d 100644
--- 
a/phoenix-core/src/main/java/org/apache/phoenix/execute/ClientAggregatePlan.java
+++ 
b/phoenix-core/src/main/java/org/apache/phoenix/execute/ClientAggregatePlan.java
@@ -35,6 +35,10 @@ import org.apache.phoenix.compile.QueryPlan;
 import org.apache.phoenix.compile.RowProjector;
 import org.apache.phoenix.compile.StatementContext;
 import org.apache.phoenix.coprocessor.BaseScannerRegionObserver;
+import org.apache.phoenix.execute.visitor.AvgRowWidthVisitor;
+import org.apache.phoenix.execute.visitor.ByteCountVisitor;
+import org.apache.phoenix.execute.visitor.QueryPlanVisitor;
+import org.apache.phoenix.execute.visitor.RowCountVisitor;
 import org.apache.phoenix.expression.Expression;
 import org.apache.phoenix.expression.OrderByExpression;
 import org.apache.phoenix.expression.aggregator.Aggregators;
@@ -90,25 +94,30 @@ public class ClientAggregatePlan extends 
ClientProcessingPlan {
 
 @Override
 public Cost getCost() {
-Long byteCount = null;
-try {
-byteCount = getEstimatedBytesToScan();
-} catch (SQLException e) {
-// ignored.
-}
-
-if (byteCount == null) {
+Double outputBytes = this.accept(new ByteCountVisitor());
+Double inputRows = this.getDelegate().accept(new RowCountVisitor());
+Double rowWidth = this.accept(new AvgRowWidthVisitor());
+if (inputRows == null || outputBytes == null || rowWidth == null) {
 return Cost.UNKNOWN;
 }
+double inputBytes = inputRows * rowWidth;
+double rowsBeforeHaving = RowCountVisitor.aggregate(
+RowCountVisitor.filter(
+inputRows.doubleValue(),
+RowCountVisitor.stripSkipScanFilter(
+context.getScan().getFilter())),
+groupBy);
+double rowsAfterHaving = RowCountVisitor.filter(rowsBeforeHaving, 
having);
+double bytesBeforeHaving = rowWidth * rowsBeforeHaving;
+double bytesAfterHaving = rowWidth * rowsAfterHaving;
 
 int parallelLevel = CostUtil.estimateParallelLevel(
 false, context.getConnection().getQueryServices());
-Cost cost = CostUtil.estimateAggregateCost(byteCount,
-groupBy, clientAggregators.getEstimatedByteSize(), 
parallelLevel);
+Cost cost = CostUtil.estimateAggregateCost(
+inputBytes, bytesBeforeHaving, groupBy, parallelLevel);
 if (!orderBy.getOrderByExpressions().isEmpty()) {
-double outputBytes = CostUtil.estimateAggregateOutputBytes(
-byteCount, groupBy, 
clientAggregators.getEstimatedByteSize());
-Cost orderByCost = CostUtil.estimateOrderByCost(outputBytes, 
parallelLevel);
+Cost orderByCost = CostUtil.estimateOrderByCost(
+bytesAfterHaving, outputBytes, parallelLevel);
 cost = cost.plus(orderByCost);
 }
 return super.getCost().plus(cost);
@@ -210,7 +219,16 @@ public class ClientAggregatePlan extends 
ClientProcessingPlan {
 public GroupBy getGroupBy() {
 return groupBy;
 }
-
+
+@Override
+public  T accept(QueryPlanVisitor visitor) {
+return visitor.visit(this);
+}
+
+public Expression getHaving() {
+return having;
+}
+
 private static class ClientGroupedAggregatingResultIterator extends 
BaseGroupedAggregatingResultIterator {
 private final List groupByExpressions;
 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/6b40a36b/phoenix-core/src/main/java/org/apache/phoenix/execute/ClientProcessingPlan.java
--
diff --git 
a/phoenix-core/src/main/java/org/apache/phoenix/execute/ClientProcessingPlan.java
 
b/phoenix-core/src/main/java/org/apache/phoenix/execute/ClientProcessingPlan.java
index ac43919..75ba8f2 100644
--- 
a/phoenix-core/src/main/java/org/apache/phoenix/execute/ClientProcessingPlan.java
+++ 
b/phoenix-core/src/main/java/org/apache/phoenix/execute/ClientProcessingPlan.java
@@ -85,4 +85,8 @@ public abstract class ClientProcessingPlan extends 
DelegateQueryPlan {
 public FilterableStatement getStatement() {
 return statement;
 }
+
+public Expression getWhere() {
+return where;
+}
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/6b40a36b/phoenix-core/src/main/java/org/apache/phoenix/execute/ClientScanPlan.java
--
diff --git 

[2/9] phoenix git commit: PHOENIX-4437 Make QueryPlan.getEstimatedBytesToScan() independent of getExplainPlan() and pull optimize() out of getExplainPlan()

2018-03-13 Thread maryannxue
PHOENIX-4437 Make QueryPlan.getEstimatedBytesToScan() independent of 
getExplainPlan() and pull optimize() out of getExplainPlan()


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

Branch: refs/heads/4.x-HBase-1.1
Commit: 56cdcb9043a8e10933428b2145b0d57b58d3e284
Parents: 33ca96e
Author: maryannxue 
Authored: Thu Dec 21 10:31:04 2017 -0800
Committer: maryannxue 
Committed: Tue Mar 13 16:47:25 2018 -0700

--
 .../end2end/ExplainPlanWithStatsEnabledIT.java  |  4 +-
 .../apache/phoenix/execute/BaseQueryPlan.java   | 45 ++
 .../apache/phoenix/execute/HashJoinPlan.java| 59 +-
 .../phoenix/execute/SortMergeJoinPlan.java  | 63 ++--
 .../org/apache/phoenix/execute/UnionPlan.java   | 53 
 .../apache/phoenix/jdbc/PhoenixStatement.java   |  9 ++-
 6 files changed, 120 insertions(+), 113 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/phoenix/blob/56cdcb90/phoenix-core/src/it/java/org/apache/phoenix/end2end/ExplainPlanWithStatsEnabledIT.java
--
diff --git 
a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ExplainPlanWithStatsEnabledIT.java
 
b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ExplainPlanWithStatsEnabledIT.java
index 969bbfb..39bdb67 100644
--- 
a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ExplainPlanWithStatsEnabledIT.java
+++ 
b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ExplainPlanWithStatsEnabledIT.java
@@ -303,8 +303,8 @@ public class ExplainPlanWithStatsEnabledIT extends 
ParallelStatsEnabledIT {
 try (Connection conn = DriverManager.getConnection(getUrl())) {
 conn.setAutoCommit(false);
 Estimate info = getByteRowEstimates(conn, sql, binds);
-assertEquals((Long) 200L, info.estimatedBytes);
-assertEquals((Long) 2L, info.estimatedRows);
+assertEquals((Long) 176l, info.estimatedBytes);
+assertEquals((Long) 2l, info.estimatedRows);
 assertTrue(info.estimateInfoTs > 0);
 }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/56cdcb90/phoenix-core/src/main/java/org/apache/phoenix/execute/BaseQueryPlan.java
--
diff --git 
a/phoenix-core/src/main/java/org/apache/phoenix/execute/BaseQueryPlan.java 
b/phoenix-core/src/main/java/org/apache/phoenix/execute/BaseQueryPlan.java
index 968b259..b152030 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/execute/BaseQueryPlan.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/execute/BaseQueryPlan.java
@@ -118,7 +118,7 @@ public abstract class BaseQueryPlan implements QueryPlan {
 protected Long estimatedRows;
 protected Long estimatedSize;
 protected Long estimateInfoTimestamp;
-private boolean explainPlanCalled;
+private boolean getEstimatesCalled;
 
 
 protected BaseQueryPlan(
@@ -506,32 +506,17 @@ public abstract class BaseQueryPlan implements QueryPlan {
 
 @Override
 public ExplainPlan getExplainPlan() throws SQLException {
-explainPlanCalled = true;
 if (context.getScanRanges() == ScanRanges.NOTHING) {
 return new ExplainPlan(Collections.singletonList("DEGENERATE SCAN 
OVER " + getTableRef().getTable().getName().getString()));
 }
 
-// If cost-based optimizer is enabled, we need to initialize a dummy 
iterator to
-// get the stats for computing costs.
-boolean costBased =
-
context.getConnection().getQueryServices().getConfiguration().getBoolean(
-QueryServices.COST_BASED_OPTIMIZER_ENABLED, 
QueryServicesOptions.DEFAULT_COST_BASED_OPTIMIZER_ENABLED);
-if (costBased) {
-ResultIterator iterator = iterator();
-iterator.close();
-}
-// Optimize here when getting explain plan, as queries don't get 
optimized until after compilation
-QueryPlan plan = 
context.getConnection().getQueryServices().getOptimizer().optimize(context.getStatement(),
 this);
-ExplainPlan exp = plan instanceof BaseQueryPlan ? new 
ExplainPlan(getPlanSteps(plan.iterator())) : plan.getExplainPlan();
-if (!costBased) { // do not override estimates if they are used for 
cost calculation.
-this.estimatedRows = plan.getEstimatedRowsToScan();
-this.estimatedSize = plan.getEstimatedBytesToScan();
-this.estimateInfoTimestamp = plan.getEstimateInfoTimestamp();
-}
-return exp;
+ResultIterator 

[4/9] phoenix git commit: PHOENIX-1556 Base hash versus sort merge join decision on cost

2018-03-13 Thread maryannxue
PHOENIX-1556 Base hash versus sort merge join decision on cost


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

Branch: refs/heads/4.x-HBase-1.1
Commit: 6b40a36b51eb92361dc2cfdca291f4391c7fcc01
Parents: 56cdcb9
Author: maryannxue 
Authored: Mon Feb 12 14:07:30 2018 -0800
Committer: maryannxue 
Committed: Tue Mar 13 16:48:01 2018 -0700

--
 .../phoenix/end2end/CostBasedDecisionIT.java| 420 -
 .../apache/phoenix/compile/JoinCompiler.java|  43 ++
 .../phoenix/compile/ListJarsQueryPlan.java  |   8 +-
 .../apache/phoenix/compile/QueryCompiler.java   | 449 ++-
 .../org/apache/phoenix/compile/QueryPlan.java   |   2 +
 .../apache/phoenix/compile/TraceQueryPlan.java  |   6 +
 .../apache/phoenix/execute/AggregatePlan.java   |  41 +-
 .../phoenix/execute/ClientAggregatePlan.java|  46 +-
 .../phoenix/execute/ClientProcessingPlan.java   |   4 +
 .../apache/phoenix/execute/ClientScanPlan.java  |  22 +-
 .../apache/phoenix/execute/CorrelatePlan.java   |  26 +-
 .../apache/phoenix/execute/CursorFetchPlan.java |   6 +
 .../apache/phoenix/execute/HashJoinPlan.java| 128 --
 .../execute/LiteralResultIterationPlan.java |   6 +
 .../org/apache/phoenix/execute/ScanPlan.java|  14 +-
 .../phoenix/execute/SortMergeJoinPlan.java  |  20 +-
 .../phoenix/execute/TupleProjectionPlan.java|   6 +
 .../org/apache/phoenix/execute/UnionPlan.java   |  12 +-
 .../apache/phoenix/execute/UnnestArrayPlan.java |   6 +
 .../execute/visitor/AvgRowWidthVisitor.java | 205 +
 .../execute/visitor/ByteCountVisitor.java   | 125 ++
 .../execute/visitor/QueryPlanVisitor.java   |  46 ++
 .../execute/visitor/RowCountVisitor.java| 335 ++
 .../apache/phoenix/jdbc/PhoenixStatement.java   |   6 +
 .../java/org/apache/phoenix/util/CostUtil.java  |  61 +--
 .../query/ParallelIteratorsSplitTest.java   |   6 +
 26 files changed, 1612 insertions(+), 437 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/phoenix/blob/6b40a36b/phoenix-core/src/it/java/org/apache/phoenix/end2end/CostBasedDecisionIT.java
--
diff --git 
a/phoenix-core/src/it/java/org/apache/phoenix/end2end/CostBasedDecisionIT.java 
b/phoenix-core/src/it/java/org/apache/phoenix/end2end/CostBasedDecisionIT.java
index a3584ce..493855a 100644
--- 
a/phoenix-core/src/it/java/org/apache/phoenix/end2end/CostBasedDecisionIT.java
+++ 
b/phoenix-core/src/it/java/org/apache/phoenix/end2end/CostBasedDecisionIT.java
@@ -32,12 +32,16 @@ import org.apache.phoenix.query.QueryServices;
 import org.apache.phoenix.util.PropertiesUtil;
 import org.apache.phoenix.util.QueryUtil;
 import org.apache.phoenix.util.ReadOnlyProps;
+
 import org.junit.BeforeClass;
 import org.junit.Test;
 
 import com.google.common.collect.Maps;
 
 public class CostBasedDecisionIT extends BaseUniqueNamesOwnClusterIT {
+private final String testTable500;
+private final String testTable990;
+private final String testTable1000;
 
 @BeforeClass
 public static void doSetup() throws Exception {
@@ -46,9 +50,16 @@ public class CostBasedDecisionIT extends 
BaseUniqueNamesOwnClusterIT {
 props.put(QueryServices.STATS_UPDATE_FREQ_MS_ATTRIB, Long.toString(5));
 props.put(QueryServices.USE_STATS_FOR_PARALLELIZATION, 
Boolean.toString(true));
 props.put(QueryServices.COST_BASED_OPTIMIZER_ENABLED, 
Boolean.toString(true));
+props.put(QueryServices.MAX_SERVER_CACHE_SIZE_ATTRIB, 
Long.toString(15));
 setUpTestDriver(new ReadOnlyProps(props.entrySet().iterator()));
 }
 
+public CostBasedDecisionIT() throws Exception {
+testTable500 = initTestTableValues(500);
+testTable990 = initTestTableValues(990);
+testTable1000 = initTestTableValues(1000);
+}
+
 @Test
 public void testCostOverridesStaticPlanOrdering1() throws Exception {
 Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
@@ -64,10 +75,7 @@ public class CostBasedDecisionIT extends 
BaseUniqueNamesOwnClusterIT {
 
 String query = "SELECT rowkey, c1, c2 FROM " + tableName + " where 
c1 LIKE 'X0%' ORDER BY rowkey";
 // Use the data table plan that opts out order-by when stats are 
not available.
-ResultSet rs = conn.createStatement().executeQuery("explain " + 
query);
-String plan = QueryUtil.getExplainPlan(rs);
-assertTrue("Expected 'FULL SCAN' in the plan:\n" + plan + ".",
-plan.contains("FULL SCAN"));
+

[6/9] phoenix git commit: PHOENIX-4611 Not nullable column impact on join query plans

2018-03-13 Thread maryannxue
PHOENIX-4611 Not nullable column impact on join query plans


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

Branch: refs/heads/4.x-HBase-1.1
Commit: 8162813ffac3003dc70dca5c40a3386ea7a1cbbc
Parents: adbce12
Author: maryannxue 
Authored: Fri Feb 16 21:03:46 2018 -0800
Committer: maryannxue 
Committed: Tue Mar 13 16:55:16 2018 -0700

--
 .../java/org/apache/phoenix/end2end/SortMergeJoinMoreIT.java | 8 
 .../java/org/apache/phoenix/end2end/join/HashJoinMoreIT.java | 4 ++--
 .../main/java/org/apache/phoenix/compile/JoinCompiler.java   | 4 
 .../org/apache/phoenix/compile/TupleProjectionCompiler.java  | 2 +-
 4 files changed, 11 insertions(+), 7 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/phoenix/blob/8162813f/phoenix-core/src/it/java/org/apache/phoenix/end2end/SortMergeJoinMoreIT.java
--
diff --git 
a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SortMergeJoinMoreIT.java 
b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SortMergeJoinMoreIT.java
index a132728..dea349a 100644
--- 
a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SortMergeJoinMoreIT.java
+++ 
b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SortMergeJoinMoreIT.java
@@ -440,8 +440,8 @@ public class SortMergeJoinMoreIT extends 
ParallelStatsDisabledIT {
 "SERVER AGGREGATE INTO ORDERED DISTINCT ROWS 
BY [BUCKET, \"TIMESTAMP\", SRC_LOCATION, DST_LOCATION]\n" +
 "CLIENT MERGE SORT\n" +
 "CLIENT SORTED BY [BUCKET, \"TIMESTAMP\"]\n" +
-"CLIENT SORTED BY [E.BUCKET, E.TIMESTAMP]\n" +
-"CLIENT AGGREGATE INTO DISTINCT ROWS BY [E.BUCKET, 
E.TIMESTAMP]"
+"CLIENT SORTED BY [E.TIMESTAMP, E.BUCKET]\n" +
+"CLIENT AGGREGATE INTO DISTINCT ROWS BY [E.TIMESTAMP, 
E.BUCKET]"
 :
 "SORT-MERGE-JOIN (INNER) TABLES\n" +
 "CLIENT PARALLEL 2-WAY SKIP SCAN ON 2 RANGES OVER 
" + eventCountTableName + " [0,'5SEC',~14629935200,'Tr/Bal'] - 
[1,'5SEC',~14629934200,'Tr/Bal']\n" +
@@ -456,8 +456,8 @@ public class SortMergeJoinMoreIT extends 
ParallelStatsDisabledIT {
 "SERVER DISTINCT PREFIX FILTER OVER [BUCKET, 
\"TIMESTAMP\", SRC_LOCATION, DST_LOCATION]\n" +
 "SERVER AGGREGATE INTO ORDERED DISTINCT ROWS 
BY [BUCKET, \"TIMESTAMP\", SRC_LOCATION, DST_LOCATION]\n" +
 "CLIENT MERGE SORT\n" +
-"CLIENT SORTED BY [E.BUCKET, E.TIMESTAMP]\n" +
-"CLIENT AGGREGATE INTO DISTINCT ROWS BY [E.BUCKET, 
E.TIMESTAMP]";
+"CLIENT SORTED BY [E.TIMESTAMP, E.BUCKET]\n" +
+"CLIENT AGGREGATE INTO DISTINCT ROWS BY [E.TIMESTAMP, 
E.BUCKET]";
 
 ResultSet rs = conn.createStatement().executeQuery("explain " 
+ q);
 assertEquals(p, QueryUtil.getExplainPlan(rs));

http://git-wip-us.apache.org/repos/asf/phoenix/blob/8162813f/phoenix-core/src/it/java/org/apache/phoenix/end2end/join/HashJoinMoreIT.java
--
diff --git 
a/phoenix-core/src/it/java/org/apache/phoenix/end2end/join/HashJoinMoreIT.java 
b/phoenix-core/src/it/java/org/apache/phoenix/end2end/join/HashJoinMoreIT.java
index f09f1d3..3a1b015 100644
--- 
a/phoenix-core/src/it/java/org/apache/phoenix/end2end/join/HashJoinMoreIT.java
+++ 
b/phoenix-core/src/it/java/org/apache/phoenix/end2end/join/HashJoinMoreIT.java
@@ -786,7 +786,7 @@ public class HashJoinMoreIT extends ParallelStatsDisabledIT 
{
 String p = i == 0 ?
 "CLIENT PARALLEL 2-WAY SKIP SCAN ON 2 RANGES OVER 
EVENT_COUNT [0,'5SEC',~14629935200,'Tr/Bal'] - 
[1,'5SEC',~14629934200,'Tr/Bal']\n" +
 "SERVER FILTER BY FIRST KEY ONLY\n" +
-"SERVER AGGREGATE INTO DISTINCT ROWS BY [E.BUCKET, 
\"E.TIMESTAMP\"]\n" +
+"SERVER AGGREGATE INTO DISTINCT ROWS BY 
[\"E.TIMESTAMP\", E.BUCKET]\n" +
 "CLIENT MERGE SORT\n" +
 "PARALLEL INNER-JOIN TABLE 0 (SKIP MERGE)\n" +
 "CLIENT PARALLEL 2-WAY SKIP SCAN ON 2 RANGES 
OVER " + t[i] + " [0,'5SEC',~14629935200,'Tr/Bal'] - 
[1,'5SEC',~14629934200,'Tr/Bal']\n" +
@@ 

phoenix git commit: PHOENIX-4288 Indexes not used when ordering by primary key

2018-03-13 Thread maryannxue
Repository: phoenix
Updated Branches:
  refs/heads/4.x-HBase-1.1 20f4cf99f -> 49f19ba68


PHOENIX-4288 Indexes not used when ordering by primary key


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

Branch: refs/heads/4.x-HBase-1.1
Commit: 49f19ba684457a6208985d52900da9081fdce7d6
Parents: 20f4cf9
Author: maryannxue 
Authored: Tue Dec 5 10:52:46 2017 -0800
Committer: maryannxue 
Committed: Tue Mar 13 16:43:01 2018 -0700

--
 .../phoenix/end2end/CostBasedDecisionIT.java| 466 +++
 .../phoenix/compile/ListJarsQueryPlan.java  |   6 +
 .../org/apache/phoenix/compile/QueryPlan.java   |   5 +-
 .../apache/phoenix/compile/TraceQueryPlan.java  |   6 +
 .../apache/phoenix/execute/AggregatePlan.java   |  30 +-
 .../apache/phoenix/execute/BaseQueryPlan.java   |  19 +-
 .../phoenix/execute/ClientAggregatePlan.java|  28 ++
 .../apache/phoenix/execute/ClientScanPlan.java  |  25 +
 .../apache/phoenix/execute/CorrelatePlan.java   |  25 +
 .../phoenix/execute/DelegateQueryPlan.java  |   6 +
 .../apache/phoenix/execute/HashJoinPlan.java|  29 ++
 .../execute/LiteralResultIterationPlan.java |   6 +
 .../org/apache/phoenix/execute/ScanPlan.java|  25 +
 .../phoenix/execute/SortMergeJoinPlan.java  |  18 +
 .../org/apache/phoenix/execute/UnionPlan.java   |  10 +
 .../apache/phoenix/jdbc/PhoenixStatement.java   |   6 +
 .../java/org/apache/phoenix/optimize/Cost.java  | 123 +
 .../apache/phoenix/optimize/QueryOptimizer.java |  28 +-
 .../org/apache/phoenix/query/QueryServices.java |   2 +
 .../phoenix/query/QueryServicesOptions.java |   6 +-
 .../java/org/apache/phoenix/util/CostUtil.java  |  90 
 .../query/ParallelIteratorsSplitTest.java   |   6 +
 22 files changed, 951 insertions(+), 14 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/phoenix/blob/49f19ba6/phoenix-core/src/it/java/org/apache/phoenix/end2end/CostBasedDecisionIT.java
--
diff --git 
a/phoenix-core/src/it/java/org/apache/phoenix/end2end/CostBasedDecisionIT.java 
b/phoenix-core/src/it/java/org/apache/phoenix/end2end/CostBasedDecisionIT.java
new file mode 100644
index 000..a3584ce
--- /dev/null
+++ 
b/phoenix-core/src/it/java/org/apache/phoenix/end2end/CostBasedDecisionIT.java
@@ -0,0 +1,466 @@
+/*
+ * 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.assertTrue;
+
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.util.Map;
+import java.util.Properties;
+
+import org.apache.phoenix.query.BaseTest;
+import org.apache.phoenix.query.QueryServices;
+import org.apache.phoenix.util.PropertiesUtil;
+import org.apache.phoenix.util.QueryUtil;
+import org.apache.phoenix.util.ReadOnlyProps;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import com.google.common.collect.Maps;
+
+public class CostBasedDecisionIT extends BaseUniqueNamesOwnClusterIT {
+
+@BeforeClass
+public static void doSetup() throws Exception {
+Map props = Maps.newHashMapWithExpectedSize(1);
+props.put(QueryServices.STATS_GUIDEPOST_WIDTH_BYTES_ATTRIB, 
Long.toString(20));
+props.put(QueryServices.STATS_UPDATE_FREQ_MS_ATTRIB, Long.toString(5));
+props.put(QueryServices.USE_STATS_FOR_PARALLELIZATION, 
Boolean.toString(true));
+props.put(QueryServices.COST_BASED_OPTIMIZER_ENABLED, 
Boolean.toString(true));
+setUpTestDriver(new ReadOnlyProps(props.entrySet().iterator()));
+}
+
+@Test
+public void testCostOverridesStaticPlanOrdering1() throws Exception {
+Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+ 

Jenkins build is back to normal : Phoenix-4.x-HBase-1.3 #61

2018-03-13 Thread Apache Jenkins Server
See 




Apache-Phoenix | origin/4.11-HBase-1.3 | Build Still Failing

2018-03-13 Thread Apache Jenkins Server
<<< text/html; charset=UTF-8: Unrecognized >>>


Build failed in Jenkins: Phoenix | Master #1958

2018-03-13 Thread Apache Jenkins Server
See 


Changes:

[tdsilva] PHOENIX-4634 Looking up a parent index table of a tenant child view

--
[...truncated 1.34 MB...]
[INFO] Running org.apache.phoenix.end2end.join.SortMergeJoinGlobalIndexIT
[INFO] Running org.apache.phoenix.end2end.join.SortMergeJoinLocalIndexIT
[INFO] Running org.apache.phoenix.end2end.join.SortMergeJoinNoIndexIT
[INFO] Tests run: 33, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 360.919 
s - in org.apache.phoenix.end2end.join.HashJoinNoIndexIT
[INFO] Tests run: 24, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 401.68 
s - in org.apache.phoenix.end2end.join.SubqueryIT
[INFO] Running org.apache.phoenix.end2end.join.SubqueryUsingSortMergeJoinIT
[INFO] Tests run: 34, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 374.905 
s - in org.apache.phoenix.end2end.join.SortMergeJoinNoIndexIT
[INFO] Tests run: 72, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 834.309 
s - in org.apache.phoenix.end2end.index.MutableIndexIT
[INFO] Tests run: 33, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 626.921 
s - in org.apache.phoenix.end2end.join.HashJoinGlobalIndexIT
[INFO] Running org.apache.phoenix.end2end.salted.SaltedTableIT
[INFO] Running org.apache.phoenix.end2end.salted.SaltedTableUpsertSelectIT
[INFO] Tests run: 8, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 27.137 s 
- in org.apache.phoenix.end2end.salted.SaltedTableIT
[INFO] Running org.apache.phoenix.iterate.RoundRobinResultIteratorIT
[INFO] Running org.apache.phoenix.end2end.salted.SaltedTableVarLengthRowKeyIT
[INFO] Tests run: 1, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 4.397 s 
- in org.apache.phoenix.end2end.salted.SaltedTableVarLengthRowKeyIT
[INFO] Running org.apache.phoenix.iterate.PhoenixQueryTimeoutIT
[INFO] Running org.apache.phoenix.replication.SystemCatalogWALEntryFilterIT
[INFO] Tests run: 2, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 0.209 s 
- in org.apache.phoenix.replication.SystemCatalogWALEntryFilterIT
[INFO] Running org.apache.phoenix.rpc.UpdateCacheIT
[INFO] Tests run: 2, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 9.412 s 
- in org.apache.phoenix.iterate.PhoenixQueryTimeoutIT
[INFO] Running org.apache.phoenix.trace.PhoenixTableMetricsWriterIT
[INFO] Tests run: 6, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 50.934 s 
- in org.apache.phoenix.end2end.salted.SaltedTableUpsertSelectIT
[INFO] Running org.apache.phoenix.trace.PhoenixTracingEndToEndIT
[INFO] Tests run: 2, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 14.404 s 
- in org.apache.phoenix.trace.PhoenixTableMetricsWriterIT
[INFO] Running org.apache.phoenix.tx.FlappingTransactionIT
[INFO] Tests run: 4, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 19.103 s 
- in org.apache.phoenix.tx.FlappingTransactionIT
[INFO] Running org.apache.phoenix.tx.ParameterizedTransactionIT
[INFO] Tests run: 7, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 41.259 s 
- in org.apache.phoenix.rpc.UpdateCacheIT
[INFO] Running org.apache.phoenix.tx.TransactionIT
[INFO] Tests run: 9, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 76.308 s 
- in org.apache.phoenix.iterate.RoundRobinResultIteratorIT
[INFO] Running org.apache.phoenix.tx.TxCheckpointIT
[INFO] Tests run: 8, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 104.282 
s - in org.apache.phoenix.trace.PhoenixTracingEndToEndIT
[INFO] Tests run: 8, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 97.08 s 
- in org.apache.phoenix.tx.TransactionIT
[INFO] Running org.apache.phoenix.util.IndexScrutinyIT
[INFO] Tests run: 34, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 667.127 
s - in org.apache.phoenix.end2end.join.SortMergeJoinGlobalIndexIT
[INFO] Tests run: 15, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 280.707 
s - in org.apache.phoenix.end2end.join.SubqueryUsingSortMergeJoinIT
[INFO] Tests run: 3, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 29.726 s 
- in org.apache.phoenix.util.IndexScrutinyIT
[INFO] Tests run: 34, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 825.296 
s - in org.apache.phoenix.end2end.join.HashJoinLocalIndexIT
[INFO] Tests run: 34, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 815.431 
s - in org.apache.phoenix.end2end.join.SortMergeJoinLocalIndexIT
[WARNING] Tests run: 52, Failures: 0, Errors: 0, Skipped: 4, Time elapsed: 
357.121 s - in org.apache.phoenix.tx.ParameterizedTransactionIT
[INFO] Tests run: 40, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 440.3 s 
- in org.apache.phoenix.tx.TxCheckpointIT
[INFO] 
[INFO] Results:
[INFO] 
[ERROR] Failures: 
[ERROR]   ConcurrentMutationsIT.testConcurrentDeletesAndUpsertValues:214 
Expected to find PK in data table: (0,0)
[ERROR]   DefaultColumnValueIT.testDefaultIndexed:978
[ERROR]   RowValueConstructorIT.testRVCLastPkIsTable1stPkIndex:1584
[ERROR]   
IndexMetadataIT.testMutableTableOnlyHasPrimaryKeyIndex:623->helpTestTableOnlyHasPrimaryKeyIndex:662
[ERROR] 

Apache-Phoenix | 4.x-HBase-1.2 | Build Successful

2018-03-13 Thread Apache Jenkins Server
4.x-HBase-1.2 branch build status Successful

Source repository https://git-wip-us.apache.org/repos/asf?p=phoenix.git;a=shortlog;h=refs/heads/4.x-HBase-1.2

Compiled Artifacts https://builds.apache.org/job/Phoenix-4.x-HBase-1.2/lastSuccessfulBuild/artifact/

Test Report https://builds.apache.org/job/Phoenix-4.x-HBase-1.2/lastCompletedBuild/testReport/

Changes
[tdsilva] PHOENIX-4634 Looking up a parent index table of a tenant child view



Build times for last couple of runsLatest build time is the right most | Legend blue: normal, red: test failure, gray: timeout


Apache-Phoenix | 4.x-HBase-1.1 | Build Successful

2018-03-13 Thread Apache Jenkins Server
4.x-HBase-1.1 branch build status Successful

Source repository https://git-wip-us.apache.org/repos/asf?p=phoenix.git;a=shortlog;h=refs/heads/4.x-HBase-1.1

Compiled Artifacts https://builds.apache.org/job/Phoenix-4.x-HBase-1.1/lastSuccessfulBuild/artifact/

Test Report https://builds.apache.org/job/Phoenix-4.x-HBase-1.1/lastCompletedBuild/testReport/

Changes
[jtaylor] PHOENIX-4571 Adds dependency on servlet-api for PQS

[jtaylor] PHOENIX-4650 Upgrade i18n-util dependency to version 1.0.4 (Shehzaad



Build times for last couple of runsLatest build time is the right most | Legend blue: normal, red: test failure, gray: timeout


Apache Phoenix - Timeout crawler - Build https://builds.apache.org/job/Phoenix-4.x-HBase-0.98/1832/

2018-03-13 Thread Apache Jenkins Server
[...truncated 61 lines...]

Build failed in Jenkins: Phoenix | 4.x-HBase-0.98 #1832

2018-03-13 Thread Apache Jenkins Server
See 


Changes:

[jtaylor] PHOENIX-4650 Upgrade i18n-util dependency to version 1.0.4 (Shehzaad

[tdsilva] PHOENIX-4634 Looking up a parent index table of a tenant child view

--
[...truncated 141.78 KB...]
at 
org.apache.phoenix.index.PhoenixTransactionalIndexer.postBatchMutateIndispensably(PhoenixTransactionalIndexer.java:240)
at 
org.apache.hadoop.hbase.regionserver.RegionCoprocessorHost$37.call(RegionCoprocessorHost.java:1040)
at 
org.apache.hadoop.hbase.regionserver.RegionCoprocessorHost$RegionOperation.call(RegionCoprocessorHost.java:1656)
at 
org.apache.hadoop.hbase.regionserver.RegionCoprocessorHost.execOperation(RegionCoprocessorHost.java:1733)
at 
org.apache.hadoop.hbase.regionserver.RegionCoprocessorHost.execOperation(RegionCoprocessorHost.java:1688)
at 
org.apache.hadoop.hbase.regionserver.RegionCoprocessorHost.postBatchMutateIndispensably(RegionCoprocessorHost.java:1036)
at 
org.apache.hadoop.hbase.regionserver.HRegion.doMiniBatchMutation(HRegion.java:2767)
at 
org.apache.hadoop.hbase.regionserver.HRegion.batchMutate(HRegion.java:2359)
at 
org.apache.hadoop.hbase.regionserver.HRegion.batchMutate(HRegion.java:2314)
at 
org.apache.hadoop.hbase.regionserver.HRegion.batchMutate(HRegion.java:2318)
at 
org.apache.hadoop.hbase.regionserver.HRegionServer.doBatchOp(HRegionServer.java:4678)
at 
org.apache.hadoop.hbase.regionserver.HRegionServer.doNonAtomicRegionMutation(HRegionServer.java:3835)
at 
org.apache.hadoop.hbase.regionserver.HRegionServer.multi(HRegionServer.java:3680)
at 
org.apache.hadoop.hbase.protobuf.generated.ClientProtos$ClientService$2.callBlockingMethod(ClientProtos.java:32500)
at org.apache.hadoop.hbase.ipc.RpcServer.call(RpcServer.java:2195)
at org.apache.hadoop.hbase.ipc.CallRunner.run(CallRunner.java:104)
at 
org.apache.hadoop.hbase.ipc.RpcExecutor.consumerLoop(RpcExecutor.java:133)
at org.apache.hadoop.hbase.ipc.RpcExecutor$1.run(RpcExecutor.java:108)
at java.lang.Thread.run(Thread.java:745)
Caused by: org.apache.hadoop.hbase.client.RetriesExhaustedWithDetailsException: 
Failed 1 action: RegionOpeningException: 1 time, servers with issues: 
asf926.gq1.ygridcore.net,37188,1520968856542, 
at 
org.apache.hadoop.hbase.client.AsyncProcess$BatchErrors.makeException(AsyncProcess.java:211)
at 
org.apache.hadoop.hbase.client.AsyncProcess$BatchErrors.access$500(AsyncProcess.java:195)
at 
org.apache.hadoop.hbase.client.AsyncProcess.getErrors(AsyncProcess.java:1082)
at 
org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplementation.processBatchCallback(HConnectionManager.java:2479)
at org.apache.hadoop.hbase.client.HTable.batchCallback(HTable.java:898)
at org.apache.hadoop.hbase.client.HTable.batchCallback(HTable.java:913)
at org.apache.hadoop.hbase.client.HTable.batch(HTable.java:888)
at 
org.apache.phoenix.hbase.index.write.ParallelWriterIndexCommitter$1.call(ParallelWriterIndexCommitter.java:170)
at 
org.apache.phoenix.hbase.index.write.ParallelWriterIndexCommitter$1.call(ParallelWriterIndexCommitter.java:133)
at java.util.concurrent.FutureTask.run(FutureTask.java:262)
at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
... 1 more
: 1 time, servers with issues: asf926.gq1.ygridcore.net,37188,1520968856542, 
at 
org.apache.phoenix.tx.ParameterizedTransactionIT.testNonTxToTxTable(ParameterizedTransactionIT.java:288)
Caused by: org.apache.hadoop.hbase.client.RetriesExhaustedWithDetailsException: 
Failed 1 action: org.apache.hadoop.hbase.DoNotRetryIOException: Failed 1 
action: RegionOpeningException: 1 time, servers with issues: 
asf926.gq1.ygridcore.net,37188,1520968856542, 
at 
org.apache.phoenix.util.ServerUtil.createIOException(ServerUtil.java:80)
at 
org.apache.phoenix.util.ServerUtil.throwIOException(ServerUtil.java:62)
at 
org.apache.phoenix.index.PhoenixTransactionalIndexer.postBatchMutateIndispensably(PhoenixTransactionalIndexer.java:240)
at 
org.apache.hadoop.hbase.regionserver.RegionCoprocessorHost$37.call(RegionCoprocessorHost.java:1040)
at 
org.apache.hadoop.hbase.regionserver.RegionCoprocessorHost$RegionOperation.call(RegionCoprocessorHost.java:1656)
at 
org.apache.hadoop.hbase.regionserver.RegionCoprocessorHost.execOperation(RegionCoprocessorHost.java:1733)
at 
org.apache.hadoop.hbase.regionserver.RegionCoprocessorHost.execOperation(RegionCoprocessorHost.java:1688)
at 
org.apache.hadoop.hbase.regionserver.RegionCoprocessorHost.postBatchMutateIndispensably(RegionCoprocessorHost.java:1036)
at 

Apache Phoenix - Timeout crawler - Build https://builds.apache.org/job/Phoenix-master/1957/

2018-03-13 Thread Apache Jenkins Server
[...truncated 50 lines...]

Build failed in Jenkins: Phoenix | Master #1957

2018-03-13 Thread Apache Jenkins Server
See 


Changes:

[jtaylor] PHOENIX-4650 Upgrade i18n-util dependency to version 1.0.4 (Shehzaad

--
[...truncated 222.60 KB...]
[INFO] Running org.apache.phoenix.end2end.join.SortMergeJoinLocalIndexIT
[INFO] Running org.apache.phoenix.end2end.join.SortMergeJoinNoIndexIT
[INFO] Running org.apache.phoenix.end2end.join.SubqueryIT
[INFO] Tests run: 33, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 357.969 
s - in org.apache.phoenix.end2end.join.HashJoinNoIndexIT
[INFO] Running org.apache.phoenix.end2end.join.SubqueryUsingSortMergeJoinIT
[INFO] Tests run: 34, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 363.742 
s - in org.apache.phoenix.end2end.join.SortMergeJoinNoIndexIT
[INFO] Tests run: 72, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 799.035 
s - in org.apache.phoenix.end2end.index.MutableIndexIT
[INFO] Tests run: 33, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 616.279 
s - in org.apache.phoenix.end2end.join.HashJoinGlobalIndexIT
[INFO] Tests run: 24, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 393.69 
s - in org.apache.phoenix.end2end.join.SubqueryIT
[INFO] Running org.apache.phoenix.end2end.salted.SaltedTableIT
[INFO] Running org.apache.phoenix.end2end.salted.SaltedTableUpsertSelectIT
[INFO] Tests run: 8, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 26.718 s 
- in org.apache.phoenix.end2end.salted.SaltedTableIT
[INFO] Running org.apache.phoenix.iterate.RoundRobinResultIteratorIT
[INFO] Running org.apache.phoenix.end2end.salted.SaltedTableVarLengthRowKeyIT
[INFO] Tests run: 1, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 4.439 s 
- in org.apache.phoenix.end2end.salted.SaltedTableVarLengthRowKeyIT
[INFO] Running org.apache.phoenix.replication.SystemCatalogWALEntryFilterIT
[INFO] Tests run: 2, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 0.299 s 
- in org.apache.phoenix.replication.SystemCatalogWALEntryFilterIT
[INFO] Running org.apache.phoenix.rpc.UpdateCacheIT
[INFO] Running org.apache.phoenix.iterate.PhoenixQueryTimeoutIT
[INFO] Tests run: 2, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 9.74 s - 
in org.apache.phoenix.iterate.PhoenixQueryTimeoutIT
[INFO] Running org.apache.phoenix.trace.PhoenixTableMetricsWriterIT
[INFO] Tests run: 6, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 50.865 s 
- in org.apache.phoenix.end2end.salted.SaltedTableUpsertSelectIT
[INFO] Running org.apache.phoenix.trace.PhoenixTracingEndToEndIT
[INFO] Tests run: 2, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 14.448 s 
- in org.apache.phoenix.trace.PhoenixTableMetricsWriterIT
[INFO] Running org.apache.phoenix.tx.FlappingTransactionIT
[INFO] Tests run: 7, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 40.42 s 
- in org.apache.phoenix.rpc.UpdateCacheIT
[INFO] Running org.apache.phoenix.tx.ParameterizedTransactionIT
[INFO] Tests run: 4, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 19.234 s 
- in org.apache.phoenix.tx.FlappingTransactionIT
[INFO] Running org.apache.phoenix.tx.TransactionIT
[INFO] Tests run: 9, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 74.077 s 
- in org.apache.phoenix.iterate.RoundRobinResultIteratorIT
[INFO] Running org.apache.phoenix.tx.TxCheckpointIT
[INFO] Tests run: 34, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 636.961 
s - in org.apache.phoenix.end2end.join.SortMergeJoinGlobalIndexIT
[INFO] Tests run: 8, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 104.48 s 
- in org.apache.phoenix.trace.PhoenixTracingEndToEndIT
[INFO] Tests run: 8, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 91.455 s 
- in org.apache.phoenix.tx.TransactionIT
[INFO] Tests run: 15, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 277.392 
s - in org.apache.phoenix.end2end.join.SubqueryUsingSortMergeJoinIT
[INFO] Running org.apache.phoenix.util.IndexScrutinyIT
[INFO] Tests run: 34, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 818.928 
s - in org.apache.phoenix.end2end.join.HashJoinLocalIndexIT
[INFO] Tests run: 3, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 26.757 s 
- in org.apache.phoenix.util.IndexScrutinyIT
[INFO] Tests run: 34, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 800.302 
s - in org.apache.phoenix.end2end.join.SortMergeJoinLocalIndexIT
[WARNING] Tests run: 52, Failures: 0, Errors: 0, Skipped: 4, Time elapsed: 
337.021 s - in org.apache.phoenix.tx.ParameterizedTransactionIT
[INFO] Tests run: 40, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 425.663 
s - in org.apache.phoenix.tx.TxCheckpointIT
[INFO] 
[INFO] Results:
[INFO] 
[ERROR] Failures: 
[ERROR]   ConcurrentMutationsIT.testConcurrentDeletesAndUpsertValues:214 
Expected to find PK in data table: (0,0)
[ERROR]   DefaultColumnValueIT.testDefaultIndexed:978
[ERROR]   RowValueConstructorIT.testRVCLastPkIsTable1stPkIndex:1584
[ERROR]   
IndexMetadataIT.testMutableTableOnlyHasPrimaryKeyIndex:623->helpTestTableOnlyHasPrimaryKeyIndex:662
[ERROR] Errors: 

Build failed in Jenkins: Phoenix-4.x-HBase-1.3 #60

2018-03-13 Thread Apache Jenkins Server
See 


Changes:

[jtaylor] PHOENIX-4650 Upgrade i18n-util dependency to version 1.0.4 (Shehzaad

--
[...truncated 662.13 KB...]
[INFO] Running org.apache.phoenix.end2end.index.MutableIndexIT
[INFO] Running org.apache.phoenix.end2end.index.SaltedIndexIT
[INFO] Tests run: 3, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 27.164 s 
- in org.apache.phoenix.end2end.index.SaltedIndexIT
[INFO] Running org.apache.phoenix.end2end.index.ViewIndexIT
[INFO] Tests run: 18, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 251.003 
s - in org.apache.phoenix.end2end.index.IndexWithTableSchemaChangeIT
[INFO] Running org.apache.phoenix.end2end.index.txn.MutableRollbackIT
[WARNING] Tests run: 12, Failures: 0, Errors: 0, Skipped: 2, Time elapsed: 
111.914 s - in org.apache.phoenix.end2end.index.ViewIndexIT
[INFO] Tests run: 37, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 379.84 
s - in org.apache.phoenix.end2end.index.IndexUsageIT
[INFO] Running org.apache.phoenix.end2end.index.txn.RollbackIT
[INFO] Running org.apache.phoenix.end2end.join.HashJoinCacheIT
[INFO] Tests run: 1, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 13.609 s 
- in org.apache.phoenix.end2end.join.HashJoinCacheIT
[INFO] Running org.apache.phoenix.end2end.join.HashJoinGlobalIndexIT
[INFO] Tests run: 6, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 111.239 
s - in org.apache.phoenix.end2end.index.txn.MutableRollbackIT
[INFO] Running org.apache.phoenix.end2end.join.HashJoinLocalIndexIT
[INFO] Tests run: 40, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 457.438 
s - in org.apache.phoenix.end2end.index.LocalImmutableNonTxIndexIT
[INFO] Tests run: 8, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 83.954 s 
- in org.apache.phoenix.end2end.index.txn.RollbackIT
[INFO] Running org.apache.phoenix.end2end.join.HashJoinNoIndexIT
[INFO] Running org.apache.phoenix.end2end.join.HashJoinMoreIT
[INFO] Tests run: 40, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 486.177 
s - in org.apache.phoenix.end2end.index.LocalImmutableTxIndexIT
[INFO] Tests run: 40, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 489.991 
s - in org.apache.phoenix.end2end.index.LocalMutableTxIndexIT
[INFO] Tests run: 40, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 457.692 
s - in org.apache.phoenix.end2end.index.LocalMutableNonTxIndexIT
[INFO] Running org.apache.phoenix.end2end.join.SortMergeJoinGlobalIndexIT
[INFO] Tests run: 8, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 101.792 
s - in org.apache.phoenix.end2end.join.HashJoinMoreIT
[INFO] Running org.apache.phoenix.end2end.join.SubqueryIT
[INFO] Running org.apache.phoenix.end2end.join.SortMergeJoinLocalIndexIT
[INFO] Running org.apache.phoenix.end2end.join.SortMergeJoinNoIndexIT
[INFO] Tests run: 33, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 352.366 
s - in org.apache.phoenix.end2end.join.HashJoinNoIndexIT
[INFO] Running org.apache.phoenix.end2end.join.SubqueryUsingSortMergeJoinIT
[INFO] Tests run: 24, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 396.912 
s - in org.apache.phoenix.end2end.join.SubqueryIT
[INFO] Tests run: 34, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 373.961 
s - in org.apache.phoenix.end2end.join.SortMergeJoinNoIndexIT
[INFO] Tests run: 72, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 819.894 
s - in org.apache.phoenix.end2end.index.MutableIndexIT
[INFO] Tests run: 33, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 617.872 
s - in org.apache.phoenix.end2end.join.HashJoinGlobalIndexIT
[INFO] Running org.apache.phoenix.end2end.salted.SaltedTableIT
[INFO] Running org.apache.phoenix.end2end.salted.SaltedTableUpsertSelectIT
[INFO] Running org.apache.phoenix.end2end.salted.SaltedTableVarLengthRowKeyIT
[INFO] Tests run: 1, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 4.345 s 
- in org.apache.phoenix.end2end.salted.SaltedTableVarLengthRowKeyIT
[INFO] Running org.apache.phoenix.iterate.RoundRobinResultIteratorIT
[INFO] Running org.apache.phoenix.iterate.PhoenixQueryTimeoutIT
[INFO] Tests run: 8, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 26.613 s 
- in org.apache.phoenix.end2end.salted.SaltedTableIT
[INFO] Running org.apache.phoenix.replication.SystemCatalogWALEntryFilterIT
[INFO] Tests run: 2, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 0.366 s 
- in org.apache.phoenix.replication.SystemCatalogWALEntryFilterIT
[INFO] Running org.apache.phoenix.rpc.UpdateCacheIT
[INFO] Tests run: 2, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 9.192 s 
- in org.apache.phoenix.iterate.PhoenixQueryTimeoutIT
[INFO] Running org.apache.phoenix.trace.PhoenixTableMetricsWriterIT
[INFO] Tests run: 2, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 14.455 s 
- in org.apache.phoenix.trace.PhoenixTableMetricsWriterIT
[INFO] Running org.apache.phoenix.trace.PhoenixTracingEndToEndIT
[INFO] Tests run: 6, Failures: 0, Errors: 0, Skipped: 0, Time 

Apache-Phoenix | 4.x-HBase-1.2 | Build Successful

2018-03-13 Thread Apache Jenkins Server
4.x-HBase-1.2 branch build status Successful

Source repository https://git-wip-us.apache.org/repos/asf?p=phoenix.git;a=shortlog;h=refs/heads/4.x-HBase-1.2

Compiled Artifacts https://builds.apache.org/job/Phoenix-4.x-HBase-1.2/lastSuccessfulBuild/artifact/

Test Report https://builds.apache.org/job/Phoenix-4.x-HBase-1.2/lastCompletedBuild/testReport/

Changes
[jtaylor] PHOENIX-4650 Upgrade i18n-util dependency to version 1.0.4 (Shehzaad



Build times for last couple of runsLatest build time is the right most | Legend blue: normal, red: test failure, gray: timeout


phoenix git commit: PHOENIX-4634 Looking up a parent index table of a tenant child view fails in BaseColumnResolver createTableRef()

2018-03-13 Thread tdsilva
Repository: phoenix
Updated Branches:
  refs/heads/4.x-HBase-1.3 31dad52c0 -> f81cce37e


PHOENIX-4634 Looking up a parent index table of a tenant child view fails in 
BaseColumnResolver createTableRef()


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

Branch: refs/heads/4.x-HBase-1.3
Commit: f81cce37eb431be16f83dd356941d85a23cd02de
Parents: 31dad52
Author: Thomas D'Silva 
Authored: Tue Mar 6 11:14:47 2018 -0800
Committer: Thomas D'Silva 
Committed: Tue Mar 13 10:39:10 2018 -0700

--
 .../apache/phoenix/end2end/PhoenixDriverIT.java |   2 +-
 .../index/ChildViewsUseParentViewIndexIT.java   |  10 +-
 .../apache/phoenix/schema/MetaDataClient.java   | 159 ---
 .../org/apache/phoenix/util/PhoenixRuntime.java |  24 +--
 4 files changed, 115 insertions(+), 80 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/phoenix/blob/f81cce37/phoenix-core/src/it/java/org/apache/phoenix/end2end/PhoenixDriverIT.java
--
diff --git 
a/phoenix-core/src/it/java/org/apache/phoenix/end2end/PhoenixDriverIT.java 
b/phoenix-core/src/it/java/org/apache/phoenix/end2end/PhoenixDriverIT.java
index 407e9cf..c93d2aa 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/PhoenixDriverIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/PhoenixDriverIT.java
@@ -159,7 +159,7 @@ public class PhoenixDriverIT extends 
BaseUniqueNamesOwnClusterIT {
 stmt.executeQuery(sql);
 PTable indexTable = stmt.getQueryPlan().getTableRef().getTable();
 String tableName = indexTable.getName().getString();
-String expectedTableName = baseTableIndexName + 
QueryConstants.CHILD_VIEW_INDEX_NAME_SEPARATOR + viewName;
+String expectedTableName = viewName + 
QueryConstants.CHILD_VIEW_INDEX_NAME_SEPARATOR + baseTableIndexName;
 assertEquals("Parent Index table is not used ", expectedTableName, 
tableName);
 
 // verify that we can look up the index using PhoenixRuntime from 
a different client

http://git-wip-us.apache.org/repos/asf/phoenix/blob/f81cce37/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/ChildViewsUseParentViewIndexIT.java
--
diff --git 
a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/ChildViewsUseParentViewIndexIT.java
 
b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/ChildViewsUseParentViewIndexIT.java
index 1e60bf1..6c8f9e8 100644
--- 
a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/ChildViewsUseParentViewIndexIT.java
+++ 
b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/ChildViewsUseParentViewIndexIT.java
@@ -37,11 +37,11 @@ public class ChildViewsUseParentViewIndexIT extends 
ParallelStatsDisabledIT {
 
 @Test
 public void testIndexOnParentViewWithTenantSpecificConnection() throws 
Exception {
-final String baseTableName = generateUniqueName();
-final String globalViewName = generateUniqueName();
-final String globalViewIdxName = generateUniqueName();
-final String tenantViewName1 = generateUniqueName();
-final String tenantViewName2 = generateUniqueName();
+final String baseTableName = "BT_" + generateUniqueName();
+final String globalViewName = "GV_" + generateUniqueName();
+final String globalViewIdxName = "GVI_" + generateUniqueName();
+final String tenantViewName1 = "TV1_" + generateUniqueName();
+final String tenantViewName2 = "TV2_" + generateUniqueName();
 
 // Set up props with TenantId
 Properties props  = new Properties();

http://git-wip-us.apache.org/repos/asf/phoenix/blob/f81cce37/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 e7f7795..d252879 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
@@ -596,66 +596,111 @@ public class MetaDataClient {
 int tryCount = 0;
 MetaDataMutationResult result;
 
-do {
-final byte[] schemaBytes = PVarchar.INSTANCE.toBytes(schemaName);
-final byte[] tableBytes = PVarchar.INSTANCE.toBytes(tableName);
-ConnectionQueryServices queryServices = 

phoenix git commit: PHOENIX-4634 Looking up a parent index table of a tenant child view fails in BaseColumnResolver createTableRef()

2018-03-13 Thread tdsilva
Repository: phoenix
Updated Branches:
  refs/heads/4.x-HBase-1.1 80df1c3c9 -> 20f4cf99f


PHOENIX-4634 Looking up a parent index table of a tenant child view fails in 
BaseColumnResolver createTableRef()


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

Branch: refs/heads/4.x-HBase-1.1
Commit: 20f4cf99fb6154c251b68e8c02016dec8417d622
Parents: 80df1c3
Author: Thomas D'Silva 
Authored: Tue Mar 6 11:14:47 2018 -0800
Committer: Thomas D'Silva 
Committed: Tue Mar 13 10:38:54 2018 -0700

--
 .../apache/phoenix/end2end/PhoenixDriverIT.java |   2 +-
 .../index/ChildViewsUseParentViewIndexIT.java   |  10 +-
 .../apache/phoenix/schema/MetaDataClient.java   | 159 ---
 .../org/apache/phoenix/util/PhoenixRuntime.java |  24 +--
 4 files changed, 115 insertions(+), 80 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/phoenix/blob/20f4cf99/phoenix-core/src/it/java/org/apache/phoenix/end2end/PhoenixDriverIT.java
--
diff --git 
a/phoenix-core/src/it/java/org/apache/phoenix/end2end/PhoenixDriverIT.java 
b/phoenix-core/src/it/java/org/apache/phoenix/end2end/PhoenixDriverIT.java
index 407e9cf..c93d2aa 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/PhoenixDriverIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/PhoenixDriverIT.java
@@ -159,7 +159,7 @@ public class PhoenixDriverIT extends 
BaseUniqueNamesOwnClusterIT {
 stmt.executeQuery(sql);
 PTable indexTable = stmt.getQueryPlan().getTableRef().getTable();
 String tableName = indexTable.getName().getString();
-String expectedTableName = baseTableIndexName + 
QueryConstants.CHILD_VIEW_INDEX_NAME_SEPARATOR + viewName;
+String expectedTableName = viewName + 
QueryConstants.CHILD_VIEW_INDEX_NAME_SEPARATOR + baseTableIndexName;
 assertEquals("Parent Index table is not used ", expectedTableName, 
tableName);
 
 // verify that we can look up the index using PhoenixRuntime from 
a different client

http://git-wip-us.apache.org/repos/asf/phoenix/blob/20f4cf99/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/ChildViewsUseParentViewIndexIT.java
--
diff --git 
a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/ChildViewsUseParentViewIndexIT.java
 
b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/ChildViewsUseParentViewIndexIT.java
index 1e60bf1..6c8f9e8 100644
--- 
a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/ChildViewsUseParentViewIndexIT.java
+++ 
b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/ChildViewsUseParentViewIndexIT.java
@@ -37,11 +37,11 @@ public class ChildViewsUseParentViewIndexIT extends 
ParallelStatsDisabledIT {
 
 @Test
 public void testIndexOnParentViewWithTenantSpecificConnection() throws 
Exception {
-final String baseTableName = generateUniqueName();
-final String globalViewName = generateUniqueName();
-final String globalViewIdxName = generateUniqueName();
-final String tenantViewName1 = generateUniqueName();
-final String tenantViewName2 = generateUniqueName();
+final String baseTableName = "BT_" + generateUniqueName();
+final String globalViewName = "GV_" + generateUniqueName();
+final String globalViewIdxName = "GVI_" + generateUniqueName();
+final String tenantViewName1 = "TV1_" + generateUniqueName();
+final String tenantViewName2 = "TV2_" + generateUniqueName();
 
 // Set up props with TenantId
 Properties props  = new Properties();

http://git-wip-us.apache.org/repos/asf/phoenix/blob/20f4cf99/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 24eaef8..5f305a5 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
@@ -590,66 +590,111 @@ public class MetaDataClient {
 int tryCount = 0;
 MetaDataMutationResult result;
 
-do {
-final byte[] schemaBytes = PVarchar.INSTANCE.toBytes(schemaName);
-final byte[] tableBytes = PVarchar.INSTANCE.toBytes(tableName);
-ConnectionQueryServices queryServices = 

phoenix git commit: PHOENIX-4634 Looking up a parent index table of a tenant child view fails in BaseColumnResolver createTableRef()

2018-03-13 Thread tdsilva
Repository: phoenix
Updated Branches:
  refs/heads/4.x-HBase-0.98 6d0cbbb6f -> 1e83415f1


PHOENIX-4634 Looking up a parent index table of a tenant child view fails in 
BaseColumnResolver createTableRef()


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

Branch: refs/heads/4.x-HBase-0.98
Commit: 1e83415f1cfb8dc1e96b53861b8e3097e4c8b4f7
Parents: 6d0cbbb
Author: Thomas D'Silva 
Authored: Tue Mar 6 11:14:47 2018 -0800
Committer: Thomas D'Silva 
Committed: Tue Mar 13 10:38:45 2018 -0700

--
 .../apache/phoenix/end2end/PhoenixDriverIT.java |   2 +-
 .../index/ChildViewsUseParentViewIndexIT.java   |  10 +-
 .../apache/phoenix/schema/MetaDataClient.java   | 159 ---
 .../org/apache/phoenix/util/PhoenixRuntime.java |  24 +--
 4 files changed, 115 insertions(+), 80 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/phoenix/blob/1e83415f/phoenix-core/src/it/java/org/apache/phoenix/end2end/PhoenixDriverIT.java
--
diff --git 
a/phoenix-core/src/it/java/org/apache/phoenix/end2end/PhoenixDriverIT.java 
b/phoenix-core/src/it/java/org/apache/phoenix/end2end/PhoenixDriverIT.java
index 407e9cf..c93d2aa 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/PhoenixDriverIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/PhoenixDriverIT.java
@@ -159,7 +159,7 @@ public class PhoenixDriverIT extends 
BaseUniqueNamesOwnClusterIT {
 stmt.executeQuery(sql);
 PTable indexTable = stmt.getQueryPlan().getTableRef().getTable();
 String tableName = indexTable.getName().getString();
-String expectedTableName = baseTableIndexName + 
QueryConstants.CHILD_VIEW_INDEX_NAME_SEPARATOR + viewName;
+String expectedTableName = viewName + 
QueryConstants.CHILD_VIEW_INDEX_NAME_SEPARATOR + baseTableIndexName;
 assertEquals("Parent Index table is not used ", expectedTableName, 
tableName);
 
 // verify that we can look up the index using PhoenixRuntime from 
a different client

http://git-wip-us.apache.org/repos/asf/phoenix/blob/1e83415f/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/ChildViewsUseParentViewIndexIT.java
--
diff --git 
a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/ChildViewsUseParentViewIndexIT.java
 
b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/ChildViewsUseParentViewIndexIT.java
index 1e60bf1..6c8f9e8 100644
--- 
a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/ChildViewsUseParentViewIndexIT.java
+++ 
b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/ChildViewsUseParentViewIndexIT.java
@@ -37,11 +37,11 @@ public class ChildViewsUseParentViewIndexIT extends 
ParallelStatsDisabledIT {
 
 @Test
 public void testIndexOnParentViewWithTenantSpecificConnection() throws 
Exception {
-final String baseTableName = generateUniqueName();
-final String globalViewName = generateUniqueName();
-final String globalViewIdxName = generateUniqueName();
-final String tenantViewName1 = generateUniqueName();
-final String tenantViewName2 = generateUniqueName();
+final String baseTableName = "BT_" + generateUniqueName();
+final String globalViewName = "GV_" + generateUniqueName();
+final String globalViewIdxName = "GVI_" + generateUniqueName();
+final String tenantViewName1 = "TV1_" + generateUniqueName();
+final String tenantViewName2 = "TV2_" + generateUniqueName();
 
 // Set up props with TenantId
 Properties props  = new Properties();

http://git-wip-us.apache.org/repos/asf/phoenix/blob/1e83415f/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 3b38ede..6687db6 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
@@ -591,66 +591,111 @@ public class MetaDataClient {
 int tryCount = 0;
 MetaDataMutationResult result;
 
-do {
-final byte[] schemaBytes = PVarchar.INSTANCE.toBytes(schemaName);
-final byte[] tableBytes = PVarchar.INSTANCE.toBytes(tableName);
-ConnectionQueryServices queryServices = 

phoenix git commit: PHOENIX-4634 Looking up a parent index table of a tenant child view fails in BaseColumnResolver createTableRef()

2018-03-13 Thread tdsilva
Repository: phoenix
Updated Branches:
  refs/heads/master 7091820ea -> 4e677818e


PHOENIX-4634 Looking up a parent index table of a tenant child view fails in 
BaseColumnResolver createTableRef()


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

Branch: refs/heads/master
Commit: 4e677818e2a2453e3e078506e3e096301df4564f
Parents: 7091820
Author: Thomas D'Silva 
Authored: Tue Mar 6 11:14:47 2018 -0800
Committer: Thomas D'Silva 
Committed: Tue Mar 13 11:04:00 2018 -0700

--
 .../apache/phoenix/end2end/PhoenixDriverIT.java |   2 +-
 .../index/ChildViewsUseParentViewIndexIT.java   |  10 +-
 .../apache/phoenix/schema/MetaDataClient.java   | 159 ---
 .../org/apache/phoenix/util/PhoenixRuntime.java |  24 +--
 4 files changed, 115 insertions(+), 80 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/phoenix/blob/4e677818/phoenix-core/src/it/java/org/apache/phoenix/end2end/PhoenixDriverIT.java
--
diff --git 
a/phoenix-core/src/it/java/org/apache/phoenix/end2end/PhoenixDriverIT.java 
b/phoenix-core/src/it/java/org/apache/phoenix/end2end/PhoenixDriverIT.java
index 407e9cf..c93d2aa 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/PhoenixDriverIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/PhoenixDriverIT.java
@@ -159,7 +159,7 @@ public class PhoenixDriverIT extends 
BaseUniqueNamesOwnClusterIT {
 stmt.executeQuery(sql);
 PTable indexTable = stmt.getQueryPlan().getTableRef().getTable();
 String tableName = indexTable.getName().getString();
-String expectedTableName = baseTableIndexName + 
QueryConstants.CHILD_VIEW_INDEX_NAME_SEPARATOR + viewName;
+String expectedTableName = viewName + 
QueryConstants.CHILD_VIEW_INDEX_NAME_SEPARATOR + baseTableIndexName;
 assertEquals("Parent Index table is not used ", expectedTableName, 
tableName);
 
 // verify that we can look up the index using PhoenixRuntime from 
a different client

http://git-wip-us.apache.org/repos/asf/phoenix/blob/4e677818/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/ChildViewsUseParentViewIndexIT.java
--
diff --git 
a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/ChildViewsUseParentViewIndexIT.java
 
b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/ChildViewsUseParentViewIndexIT.java
index 1e60bf1..6c8f9e8 100644
--- 
a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/ChildViewsUseParentViewIndexIT.java
+++ 
b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/ChildViewsUseParentViewIndexIT.java
@@ -37,11 +37,11 @@ public class ChildViewsUseParentViewIndexIT extends 
ParallelStatsDisabledIT {
 
 @Test
 public void testIndexOnParentViewWithTenantSpecificConnection() throws 
Exception {
-final String baseTableName = generateUniqueName();
-final String globalViewName = generateUniqueName();
-final String globalViewIdxName = generateUniqueName();
-final String tenantViewName1 = generateUniqueName();
-final String tenantViewName2 = generateUniqueName();
+final String baseTableName = "BT_" + generateUniqueName();
+final String globalViewName = "GV_" + generateUniqueName();
+final String globalViewIdxName = "GVI_" + generateUniqueName();
+final String tenantViewName1 = "TV1_" + generateUniqueName();
+final String tenantViewName2 = "TV2_" + generateUniqueName();
 
 // Set up props with TenantId
 Properties props  = new Properties();

http://git-wip-us.apache.org/repos/asf/phoenix/blob/4e677818/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 e7f7795..d252879 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
@@ -596,66 +596,111 @@ public class MetaDataClient {
 int tryCount = 0;
 MetaDataMutationResult result;
 
-do {
-final byte[] schemaBytes = PVarchar.INSTANCE.toBytes(schemaName);
-final byte[] tableBytes = PVarchar.INSTANCE.toBytes(tableName);
-ConnectionQueryServices queryServices = 

phoenix git commit: PHOENIX-4650 Upgrade i18n-util dependency to version 1.0.4 (Shehzaad Nakhoda)

2018-03-13 Thread jamestaylor
Repository: phoenix
Updated Branches:
  refs/heads/5.x-HBase-2.0 3c0dbcc2d -> 33d6db781


PHOENIX-4650 Upgrade i18n-util dependency to version 1.0.4 (Shehzaad Nakhoda)


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

Branch: refs/heads/5.x-HBase-2.0
Commit: 33d6db7810f0eaacc4b3b3f719a6713b3d53ae67
Parents: 3c0dbcc
Author: James Taylor 
Authored: Tue Mar 13 10:10:55 2018 -0700
Committer: James Taylor 
Committed: Tue Mar 13 10:33:57 2018 -0700

--
 .../org/apache/phoenix/end2end/CollationKeyFunctionIT.java| 6 +++---
 .../phoenix/expression/function/CollationKeyFunctionTest.java | 7 ---
 pom.xml   | 2 +-
 3 files changed, 8 insertions(+), 7 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/phoenix/blob/33d6db78/phoenix-core/src/it/java/org/apache/phoenix/end2end/CollationKeyFunctionIT.java
--
diff --git 
a/phoenix-core/src/it/java/org/apache/phoenix/end2end/CollationKeyFunctionIT.java
 
b/phoenix-core/src/it/java/org/apache/phoenix/end2end/CollationKeyFunctionIT.java
index 9f9e4d1..e954874 100644
--- 
a/phoenix-core/src/it/java/org/apache/phoenix/end2end/CollationKeyFunctionIT.java
+++ 
b/phoenix-core/src/it/java/org/apache/phoenix/end2end/CollationKeyFunctionIT.java
@@ -72,12 +72,12 @@ public class CollationKeyFunctionIT extends 
ParallelStatsDisabledIT {
 
@Test
public void testZhSort() throws Exception {
-   queryWithCollKeyDefaultArgsWithExpectedOrder("zh", false, 0, 6, 
new Integer[] { 3, 0, 1, 6, 5, 4, 2 });
+   queryWithCollKeyDefaultArgsWithExpectedOrder("zh", false, 0, 6, 
new Integer[] { 4, 3, 1, 5, 2, 0, 6 });
}
 
@Test
public void testZhTwSort() throws Exception {
-   queryWithCollKeyDefaultArgsWithExpectedOrder("zh_TW", false, 0, 
6, new Integer[] { 0, 3, 4, 1, 5, 2, 6 });
+   queryWithCollKeyDefaultArgsWithExpectedOrder("zh_TW", false, 0, 
6, new Integer[] { 4, 3, 1, 5, 2, 0, 6 });
}
 
@Test
@@ -87,7 +87,7 @@ public class CollationKeyFunctionIT extends 
ParallelStatsDisabledIT {
 
@Test
public void testZhStrokeSort() throws Exception {
-   queryWithCollKeyDefaultArgsWithExpectedOrder("zh__STROKE", 
false, 0, 6, new Integer[] { 0, 1, 3, 4, 6, 2, 5 });
+   queryWithCollKeyDefaultArgsWithExpectedOrder("zh__STROKE", 
false, 0, 6, new Integer[] { 4, 2, 0, 3, 1, 6, 5 });
}
 
@Test

http://git-wip-us.apache.org/repos/asf/phoenix/blob/33d6db78/phoenix-core/src/test/java/org/apache/phoenix/expression/function/CollationKeyFunctionTest.java
--
diff --git 
a/phoenix-core/src/test/java/org/apache/phoenix/expression/function/CollationKeyFunctionTest.java
 
b/phoenix-core/src/test/java/org/apache/phoenix/expression/function/CollationKeyFunctionTest.java
index 950875b..93154e3 100644
--- 
a/phoenix-core/src/test/java/org/apache/phoenix/expression/function/CollationKeyFunctionTest.java
+++ 
b/phoenix-core/src/test/java/org/apache/phoenix/expression/function/CollationKeyFunctionTest.java
@@ -90,12 +90,13 @@ public class CollationKeyFunctionTest {
 
@Test
public void testZhSort() throws Exception {
-   testSortOrderNoEquals(chineseChars, "zh", Boolean.FALSE, null, 
null, new Integer[] { 3, 0, 1, 6, 5, 4, 2 });
+   testSortOrderNoEquals(chineseChars, "zh", Boolean.FALSE, null, 
null, new Integer[] { 4, 3, 1, 5, 2, 0, 6 });
+
}
 
@Test
public void testZhTwSort() throws Exception {
-   testSortOrderNoEquals(chineseChars, "zh_TW", Boolean.FALSE, 
null, null, new Integer[] { 0, 3, 4, 1, 5, 2, 6 });
+   testSortOrderNoEquals(chineseChars, "zh_TW", Boolean.FALSE, 
null, null, new Integer[] { 4, 3, 1, 5, 2, 0, 6 });
}
 
@Test
@@ -107,7 +108,7 @@ public class CollationKeyFunctionTest {
@Test
public void testZhStrokeSort() throws Exception {
testSortOrderNoEquals(chineseChars, "zh__STROKE", 
Boolean.FALSE, null, null,
-   new Integer[] { 0, 1, 3, 4, 6, 2, 5 });
+   new Integer[] { 4, 2, 0, 3, 1, 6, 5 });
}
 
@Test

http://git-wip-us.apache.org/repos/asf/phoenix/blob/33d6db78/pom.xml
--
diff --git a/pom.xml b/pom.xml
index 7408c7c..6abc4b4 100644
--- a/pom.xml
+++ b/pom.xml
@@ -104,7 +104,7 @@
 

[1/2] phoenix git commit: PHOENIX-4571 Adds dependency on servlet-api for PQS

2018-03-13 Thread jamestaylor
Repository: phoenix
Updated Branches:
  refs/heads/4.x-HBase-1.1 8b24575e1 -> 80df1c3c9


PHOENIX-4571 Adds dependency on servlet-api for PQS


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

Branch: refs/heads/4.x-HBase-1.1
Commit: 8da55890eb86c39127f7095244646fe6df47bb70
Parents: 8b24575
Author: Josh Elser 
Authored: Tue Jan 30 16:06:08 2018 -0500
Committer: James Taylor 
Committed: Tue Mar 13 10:21:45 2018 -0700

--
 phoenix-queryserver/pom.xml | 4 
 pom.xml | 6 ++
 2 files changed, 10 insertions(+)
--


http://git-wip-us.apache.org/repos/asf/phoenix/blob/8da55890/phoenix-queryserver/pom.xml
--
diff --git a/phoenix-queryserver/pom.xml b/phoenix-queryserver/pom.xml
index 4fa7c38..6664996 100644
--- a/phoenix-queryserver/pom.xml
+++ b/phoenix-queryserver/pom.xml
@@ -147,6 +147,10 @@
   commons-logging
   commons-logging
 
+
+  javax.servlet
+  javax.servlet-api
+
 
 
   org.mockito

http://git-wip-us.apache.org/repos/asf/phoenix/blob/8da55890/pom.xml
--
diff --git a/pom.xml b/pom.xml
index feee424..e260d24 100644
--- a/pom.xml
+++ b/pom.xml
@@ -106,6 +106,7 @@
 2.11
 2.9.5
 1.0.1
+3.1.0
 
 1.8.5
 4.12
@@ -933,6 +934,11 @@
 i18n-util
 ${i18n-util.version}
   
+  
+javax.servlet
+javax.servlet-api
+${servlet.api.version}
+  
 
   
 



[2/2] phoenix git commit: PHOENIX-4650 Upgrade i18n-util dependency to version 1.0.4 (Shehzaad Nakhoda)

2018-03-13 Thread jamestaylor
PHOENIX-4650 Upgrade i18n-util dependency to version 1.0.4 (Shehzaad Nakhoda)


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

Branch: refs/heads/4.x-HBase-1.1
Commit: 80df1c3c9a7b46e0a7156e2320bd937e902101d0
Parents: 8da5589
Author: James Taylor 
Authored: Tue Mar 13 10:10:55 2018 -0700
Committer: James Taylor 
Committed: Tue Mar 13 10:22:06 2018 -0700

--
 .../org/apache/phoenix/end2end/CollationKeyFunctionIT.java| 6 +++---
 .../phoenix/expression/function/CollationKeyFunctionTest.java | 7 ---
 pom.xml   | 2 +-
 3 files changed, 8 insertions(+), 7 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/phoenix/blob/80df1c3c/phoenix-core/src/it/java/org/apache/phoenix/end2end/CollationKeyFunctionIT.java
--
diff --git 
a/phoenix-core/src/it/java/org/apache/phoenix/end2end/CollationKeyFunctionIT.java
 
b/phoenix-core/src/it/java/org/apache/phoenix/end2end/CollationKeyFunctionIT.java
index 9f9e4d1..e954874 100644
--- 
a/phoenix-core/src/it/java/org/apache/phoenix/end2end/CollationKeyFunctionIT.java
+++ 
b/phoenix-core/src/it/java/org/apache/phoenix/end2end/CollationKeyFunctionIT.java
@@ -72,12 +72,12 @@ public class CollationKeyFunctionIT extends 
ParallelStatsDisabledIT {
 
@Test
public void testZhSort() throws Exception {
-   queryWithCollKeyDefaultArgsWithExpectedOrder("zh", false, 0, 6, 
new Integer[] { 3, 0, 1, 6, 5, 4, 2 });
+   queryWithCollKeyDefaultArgsWithExpectedOrder("zh", false, 0, 6, 
new Integer[] { 4, 3, 1, 5, 2, 0, 6 });
}
 
@Test
public void testZhTwSort() throws Exception {
-   queryWithCollKeyDefaultArgsWithExpectedOrder("zh_TW", false, 0, 
6, new Integer[] { 0, 3, 4, 1, 5, 2, 6 });
+   queryWithCollKeyDefaultArgsWithExpectedOrder("zh_TW", false, 0, 
6, new Integer[] { 4, 3, 1, 5, 2, 0, 6 });
}
 
@Test
@@ -87,7 +87,7 @@ public class CollationKeyFunctionIT extends 
ParallelStatsDisabledIT {
 
@Test
public void testZhStrokeSort() throws Exception {
-   queryWithCollKeyDefaultArgsWithExpectedOrder("zh__STROKE", 
false, 0, 6, new Integer[] { 0, 1, 3, 4, 6, 2, 5 });
+   queryWithCollKeyDefaultArgsWithExpectedOrder("zh__STROKE", 
false, 0, 6, new Integer[] { 4, 2, 0, 3, 1, 6, 5 });
}
 
@Test

http://git-wip-us.apache.org/repos/asf/phoenix/blob/80df1c3c/phoenix-core/src/test/java/org/apache/phoenix/expression/function/CollationKeyFunctionTest.java
--
diff --git 
a/phoenix-core/src/test/java/org/apache/phoenix/expression/function/CollationKeyFunctionTest.java
 
b/phoenix-core/src/test/java/org/apache/phoenix/expression/function/CollationKeyFunctionTest.java
index 950875b..93154e3 100644
--- 
a/phoenix-core/src/test/java/org/apache/phoenix/expression/function/CollationKeyFunctionTest.java
+++ 
b/phoenix-core/src/test/java/org/apache/phoenix/expression/function/CollationKeyFunctionTest.java
@@ -90,12 +90,13 @@ public class CollationKeyFunctionTest {
 
@Test
public void testZhSort() throws Exception {
-   testSortOrderNoEquals(chineseChars, "zh", Boolean.FALSE, null, 
null, new Integer[] { 3, 0, 1, 6, 5, 4, 2 });
+   testSortOrderNoEquals(chineseChars, "zh", Boolean.FALSE, null, 
null, new Integer[] { 4, 3, 1, 5, 2, 0, 6 });
+
}
 
@Test
public void testZhTwSort() throws Exception {
-   testSortOrderNoEquals(chineseChars, "zh_TW", Boolean.FALSE, 
null, null, new Integer[] { 0, 3, 4, 1, 5, 2, 6 });
+   testSortOrderNoEquals(chineseChars, "zh_TW", Boolean.FALSE, 
null, null, new Integer[] { 4, 3, 1, 5, 2, 0, 6 });
}
 
@Test
@@ -107,7 +108,7 @@ public class CollationKeyFunctionTest {
@Test
public void testZhStrokeSort() throws Exception {
testSortOrderNoEquals(chineseChars, "zh__STROKE", 
Boolean.FALSE, null, null,
-   new Integer[] { 0, 1, 3, 4, 6, 2, 5 });
+   new Integer[] { 4, 2, 0, 3, 1, 6, 5 });
}
 
@Test

http://git-wip-us.apache.org/repos/asf/phoenix/blob/80df1c3c/pom.xml
--
diff --git a/pom.xml b/pom.xml
index e260d24..53ea215 100644
--- a/pom.xml
+++ b/pom.xml
@@ -105,7 +105,7 @@
 2.11.8
 2.11
 2.9.5
-1.0.1
+1.0.4
 3.1.0
 
 1.8.5



phoenix git commit: PHOENIX-4650 Upgrade i18n-util dependency to version 1.0.4 (Shehzaad Nakhoda)

2018-03-13 Thread jamestaylor
Repository: phoenix
Updated Branches:
  refs/heads/4.x-cdh5.11.2 7f2a04dd8 -> a41c7f4f8


PHOENIX-4650 Upgrade i18n-util dependency to version 1.0.4 (Shehzaad Nakhoda)


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

Branch: refs/heads/4.x-cdh5.11.2
Commit: a41c7f4f890a107efa3625a8fe0fbcf6eef978eb
Parents: 7f2a04d
Author: James Taylor 
Authored: Tue Mar 13 10:10:55 2018 -0700
Committer: James Taylor 
Committed: Tue Mar 13 10:16:54 2018 -0700

--
 .../org/apache/phoenix/end2end/CollationKeyFunctionIT.java| 6 +++---
 .../phoenix/expression/function/CollationKeyFunctionTest.java | 7 ---
 pom.xml   | 2 +-
 3 files changed, 8 insertions(+), 7 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/phoenix/blob/a41c7f4f/phoenix-core/src/it/java/org/apache/phoenix/end2end/CollationKeyFunctionIT.java
--
diff --git 
a/phoenix-core/src/it/java/org/apache/phoenix/end2end/CollationKeyFunctionIT.java
 
b/phoenix-core/src/it/java/org/apache/phoenix/end2end/CollationKeyFunctionIT.java
index 9f9e4d1..e954874 100644
--- 
a/phoenix-core/src/it/java/org/apache/phoenix/end2end/CollationKeyFunctionIT.java
+++ 
b/phoenix-core/src/it/java/org/apache/phoenix/end2end/CollationKeyFunctionIT.java
@@ -72,12 +72,12 @@ public class CollationKeyFunctionIT extends 
ParallelStatsDisabledIT {
 
@Test
public void testZhSort() throws Exception {
-   queryWithCollKeyDefaultArgsWithExpectedOrder("zh", false, 0, 6, 
new Integer[] { 3, 0, 1, 6, 5, 4, 2 });
+   queryWithCollKeyDefaultArgsWithExpectedOrder("zh", false, 0, 6, 
new Integer[] { 4, 3, 1, 5, 2, 0, 6 });
}
 
@Test
public void testZhTwSort() throws Exception {
-   queryWithCollKeyDefaultArgsWithExpectedOrder("zh_TW", false, 0, 
6, new Integer[] { 0, 3, 4, 1, 5, 2, 6 });
+   queryWithCollKeyDefaultArgsWithExpectedOrder("zh_TW", false, 0, 
6, new Integer[] { 4, 3, 1, 5, 2, 0, 6 });
}
 
@Test
@@ -87,7 +87,7 @@ public class CollationKeyFunctionIT extends 
ParallelStatsDisabledIT {
 
@Test
public void testZhStrokeSort() throws Exception {
-   queryWithCollKeyDefaultArgsWithExpectedOrder("zh__STROKE", 
false, 0, 6, new Integer[] { 0, 1, 3, 4, 6, 2, 5 });
+   queryWithCollKeyDefaultArgsWithExpectedOrder("zh__STROKE", 
false, 0, 6, new Integer[] { 4, 2, 0, 3, 1, 6, 5 });
}
 
@Test

http://git-wip-us.apache.org/repos/asf/phoenix/blob/a41c7f4f/phoenix-core/src/test/java/org/apache/phoenix/expression/function/CollationKeyFunctionTest.java
--
diff --git 
a/phoenix-core/src/test/java/org/apache/phoenix/expression/function/CollationKeyFunctionTest.java
 
b/phoenix-core/src/test/java/org/apache/phoenix/expression/function/CollationKeyFunctionTest.java
index 950875b..93154e3 100644
--- 
a/phoenix-core/src/test/java/org/apache/phoenix/expression/function/CollationKeyFunctionTest.java
+++ 
b/phoenix-core/src/test/java/org/apache/phoenix/expression/function/CollationKeyFunctionTest.java
@@ -90,12 +90,13 @@ public class CollationKeyFunctionTest {
 
@Test
public void testZhSort() throws Exception {
-   testSortOrderNoEquals(chineseChars, "zh", Boolean.FALSE, null, 
null, new Integer[] { 3, 0, 1, 6, 5, 4, 2 });
+   testSortOrderNoEquals(chineseChars, "zh", Boolean.FALSE, null, 
null, new Integer[] { 4, 3, 1, 5, 2, 0, 6 });
+
}
 
@Test
public void testZhTwSort() throws Exception {
-   testSortOrderNoEquals(chineseChars, "zh_TW", Boolean.FALSE, 
null, null, new Integer[] { 0, 3, 4, 1, 5, 2, 6 });
+   testSortOrderNoEquals(chineseChars, "zh_TW", Boolean.FALSE, 
null, null, new Integer[] { 4, 3, 1, 5, 2, 0, 6 });
}
 
@Test
@@ -107,7 +108,7 @@ public class CollationKeyFunctionTest {
@Test
public void testZhStrokeSort() throws Exception {
testSortOrderNoEquals(chineseChars, "zh__STROKE", 
Boolean.FALSE, null, null,
-   new Integer[] { 0, 1, 3, 4, 6, 2, 5 });
+   new Integer[] { 4, 2, 0, 3, 1, 6, 5 });
}
 
@Test

http://git-wip-us.apache.org/repos/asf/phoenix/blob/a41c7f4f/pom.xml
--
diff --git a/pom.xml b/pom.xml
index 574f8ba..0bb9ed5 100644
--- a/pom.xml
+++ b/pom.xml
@@ -148,7 +148,7 @@
 

phoenix git commit: PHOENIX-4650 Upgrade i18n-util dependency to version 1.0.4 (Shehzaad Nakhoda)

2018-03-13 Thread jamestaylor
Repository: phoenix
Updated Branches:
  refs/heads/4.x-HBase-1.2 4756e0d28 -> c62829df0


PHOENIX-4650 Upgrade i18n-util dependency to version 1.0.4 (Shehzaad Nakhoda)


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

Branch: refs/heads/4.x-HBase-1.2
Commit: c62829df000ab7bdee4056710273fc2a20631b20
Parents: 4756e0d
Author: James Taylor 
Authored: Tue Mar 13 10:10:55 2018 -0700
Committer: James Taylor 
Committed: Tue Mar 13 10:15:09 2018 -0700

--
 .../org/apache/phoenix/end2end/CollationKeyFunctionIT.java| 6 +++---
 .../phoenix/expression/function/CollationKeyFunctionTest.java | 7 ---
 pom.xml   | 2 +-
 3 files changed, 8 insertions(+), 7 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/phoenix/blob/c62829df/phoenix-core/src/it/java/org/apache/phoenix/end2end/CollationKeyFunctionIT.java
--
diff --git 
a/phoenix-core/src/it/java/org/apache/phoenix/end2end/CollationKeyFunctionIT.java
 
b/phoenix-core/src/it/java/org/apache/phoenix/end2end/CollationKeyFunctionIT.java
index 9f9e4d1..e954874 100644
--- 
a/phoenix-core/src/it/java/org/apache/phoenix/end2end/CollationKeyFunctionIT.java
+++ 
b/phoenix-core/src/it/java/org/apache/phoenix/end2end/CollationKeyFunctionIT.java
@@ -72,12 +72,12 @@ public class CollationKeyFunctionIT extends 
ParallelStatsDisabledIT {
 
@Test
public void testZhSort() throws Exception {
-   queryWithCollKeyDefaultArgsWithExpectedOrder("zh", false, 0, 6, 
new Integer[] { 3, 0, 1, 6, 5, 4, 2 });
+   queryWithCollKeyDefaultArgsWithExpectedOrder("zh", false, 0, 6, 
new Integer[] { 4, 3, 1, 5, 2, 0, 6 });
}
 
@Test
public void testZhTwSort() throws Exception {
-   queryWithCollKeyDefaultArgsWithExpectedOrder("zh_TW", false, 0, 
6, new Integer[] { 0, 3, 4, 1, 5, 2, 6 });
+   queryWithCollKeyDefaultArgsWithExpectedOrder("zh_TW", false, 0, 
6, new Integer[] { 4, 3, 1, 5, 2, 0, 6 });
}
 
@Test
@@ -87,7 +87,7 @@ public class CollationKeyFunctionIT extends 
ParallelStatsDisabledIT {
 
@Test
public void testZhStrokeSort() throws Exception {
-   queryWithCollKeyDefaultArgsWithExpectedOrder("zh__STROKE", 
false, 0, 6, new Integer[] { 0, 1, 3, 4, 6, 2, 5 });
+   queryWithCollKeyDefaultArgsWithExpectedOrder("zh__STROKE", 
false, 0, 6, new Integer[] { 4, 2, 0, 3, 1, 6, 5 });
}
 
@Test

http://git-wip-us.apache.org/repos/asf/phoenix/blob/c62829df/phoenix-core/src/test/java/org/apache/phoenix/expression/function/CollationKeyFunctionTest.java
--
diff --git 
a/phoenix-core/src/test/java/org/apache/phoenix/expression/function/CollationKeyFunctionTest.java
 
b/phoenix-core/src/test/java/org/apache/phoenix/expression/function/CollationKeyFunctionTest.java
index 950875b..93154e3 100644
--- 
a/phoenix-core/src/test/java/org/apache/phoenix/expression/function/CollationKeyFunctionTest.java
+++ 
b/phoenix-core/src/test/java/org/apache/phoenix/expression/function/CollationKeyFunctionTest.java
@@ -90,12 +90,13 @@ public class CollationKeyFunctionTest {
 
@Test
public void testZhSort() throws Exception {
-   testSortOrderNoEquals(chineseChars, "zh", Boolean.FALSE, null, 
null, new Integer[] { 3, 0, 1, 6, 5, 4, 2 });
+   testSortOrderNoEquals(chineseChars, "zh", Boolean.FALSE, null, 
null, new Integer[] { 4, 3, 1, 5, 2, 0, 6 });
+
}
 
@Test
public void testZhTwSort() throws Exception {
-   testSortOrderNoEquals(chineseChars, "zh_TW", Boolean.FALSE, 
null, null, new Integer[] { 0, 3, 4, 1, 5, 2, 6 });
+   testSortOrderNoEquals(chineseChars, "zh_TW", Boolean.FALSE, 
null, null, new Integer[] { 4, 3, 1, 5, 2, 0, 6 });
}
 
@Test
@@ -107,7 +108,7 @@ public class CollationKeyFunctionTest {
@Test
public void testZhStrokeSort() throws Exception {
testSortOrderNoEquals(chineseChars, "zh__STROKE", 
Boolean.FALSE, null, null,
-   new Integer[] { 0, 1, 3, 4, 6, 2, 5 });
+   new Integer[] { 4, 2, 0, 3, 1, 6, 5 });
}
 
@Test

http://git-wip-us.apache.org/repos/asf/phoenix/blob/c62829df/pom.xml
--
diff --git a/pom.xml b/pom.xml
index 90523c1..0724fd5 100644
--- a/pom.xml
+++ b/pom.xml
@@ -105,7 +105,7 @@
 

phoenix git commit: PHOENIX-4650 Upgrade i18n-util dependency to version 1.0.4 (Shehzaad Nakhoda)

2018-03-13 Thread jamestaylor
Repository: phoenix
Updated Branches:
  refs/heads/master cc494a1ba -> 7091820ea


PHOENIX-4650 Upgrade i18n-util dependency to version 1.0.4 (Shehzaad Nakhoda)


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

Branch: refs/heads/master
Commit: 7091820ea63072a42154fe74134126b8b48d386c
Parents: cc494a1
Author: James Taylor 
Authored: Tue Mar 13 10:10:55 2018 -0700
Committer: James Taylor 
Committed: Tue Mar 13 10:13:26 2018 -0700

--
 .../org/apache/phoenix/end2end/CollationKeyFunctionIT.java| 6 +++---
 .../phoenix/expression/function/CollationKeyFunctionTest.java | 7 ---
 pom.xml   | 2 +-
 3 files changed, 8 insertions(+), 7 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/phoenix/blob/7091820e/phoenix-core/src/it/java/org/apache/phoenix/end2end/CollationKeyFunctionIT.java
--
diff --git 
a/phoenix-core/src/it/java/org/apache/phoenix/end2end/CollationKeyFunctionIT.java
 
b/phoenix-core/src/it/java/org/apache/phoenix/end2end/CollationKeyFunctionIT.java
index 9f9e4d1..e954874 100644
--- 
a/phoenix-core/src/it/java/org/apache/phoenix/end2end/CollationKeyFunctionIT.java
+++ 
b/phoenix-core/src/it/java/org/apache/phoenix/end2end/CollationKeyFunctionIT.java
@@ -72,12 +72,12 @@ public class CollationKeyFunctionIT extends 
ParallelStatsDisabledIT {
 
@Test
public void testZhSort() throws Exception {
-   queryWithCollKeyDefaultArgsWithExpectedOrder("zh", false, 0, 6, 
new Integer[] { 3, 0, 1, 6, 5, 4, 2 });
+   queryWithCollKeyDefaultArgsWithExpectedOrder("zh", false, 0, 6, 
new Integer[] { 4, 3, 1, 5, 2, 0, 6 });
}
 
@Test
public void testZhTwSort() throws Exception {
-   queryWithCollKeyDefaultArgsWithExpectedOrder("zh_TW", false, 0, 
6, new Integer[] { 0, 3, 4, 1, 5, 2, 6 });
+   queryWithCollKeyDefaultArgsWithExpectedOrder("zh_TW", false, 0, 
6, new Integer[] { 4, 3, 1, 5, 2, 0, 6 });
}
 
@Test
@@ -87,7 +87,7 @@ public class CollationKeyFunctionIT extends 
ParallelStatsDisabledIT {
 
@Test
public void testZhStrokeSort() throws Exception {
-   queryWithCollKeyDefaultArgsWithExpectedOrder("zh__STROKE", 
false, 0, 6, new Integer[] { 0, 1, 3, 4, 6, 2, 5 });
+   queryWithCollKeyDefaultArgsWithExpectedOrder("zh__STROKE", 
false, 0, 6, new Integer[] { 4, 2, 0, 3, 1, 6, 5 });
}
 
@Test

http://git-wip-us.apache.org/repos/asf/phoenix/blob/7091820e/phoenix-core/src/test/java/org/apache/phoenix/expression/function/CollationKeyFunctionTest.java
--
diff --git 
a/phoenix-core/src/test/java/org/apache/phoenix/expression/function/CollationKeyFunctionTest.java
 
b/phoenix-core/src/test/java/org/apache/phoenix/expression/function/CollationKeyFunctionTest.java
index 950875b..93154e3 100644
--- 
a/phoenix-core/src/test/java/org/apache/phoenix/expression/function/CollationKeyFunctionTest.java
+++ 
b/phoenix-core/src/test/java/org/apache/phoenix/expression/function/CollationKeyFunctionTest.java
@@ -90,12 +90,13 @@ public class CollationKeyFunctionTest {
 
@Test
public void testZhSort() throws Exception {
-   testSortOrderNoEquals(chineseChars, "zh", Boolean.FALSE, null, 
null, new Integer[] { 3, 0, 1, 6, 5, 4, 2 });
+   testSortOrderNoEquals(chineseChars, "zh", Boolean.FALSE, null, 
null, new Integer[] { 4, 3, 1, 5, 2, 0, 6 });
+
}
 
@Test
public void testZhTwSort() throws Exception {
-   testSortOrderNoEquals(chineseChars, "zh_TW", Boolean.FALSE, 
null, null, new Integer[] { 0, 3, 4, 1, 5, 2, 6 });
+   testSortOrderNoEquals(chineseChars, "zh_TW", Boolean.FALSE, 
null, null, new Integer[] { 4, 3, 1, 5, 2, 0, 6 });
}
 
@Test
@@ -107,7 +108,7 @@ public class CollationKeyFunctionTest {
@Test
public void testZhStrokeSort() throws Exception {
testSortOrderNoEquals(chineseChars, "zh__STROKE", 
Boolean.FALSE, null, null,
-   new Integer[] { 0, 1, 3, 4, 6, 2, 5 });
+   new Integer[] { 4, 2, 0, 3, 1, 6, 5 });
}
 
@Test

http://git-wip-us.apache.org/repos/asf/phoenix/blob/7091820e/pom.xml
--
diff --git a/pom.xml b/pom.xml
index c8d3271..4af01d8 100644
--- a/pom.xml
+++ b/pom.xml
@@ -105,7 +105,7 @@
 2.11.8
 2.11

phoenix git commit: PHOENIX-4650 Upgrade i18n-util dependency to version 1.0.4 (Shehzaad Nakhoda)

2018-03-13 Thread jamestaylor
Repository: phoenix
Updated Branches:
  refs/heads/4.x-HBase-1.3 a73fb08ca -> 31dad52c0


PHOENIX-4650 Upgrade i18n-util dependency to version 1.0.4 (Shehzaad Nakhoda)


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

Branch: refs/heads/4.x-HBase-1.3
Commit: 31dad52c0df8ed2b6db47ea7ba8cf1c1303085bf
Parents: a73fb08
Author: James Taylor 
Authored: Tue Mar 13 10:10:55 2018 -0700
Committer: James Taylor 
Committed: Tue Mar 13 10:12:35 2018 -0700

--
 .../org/apache/phoenix/end2end/CollationKeyFunctionIT.java| 6 +++---
 .../phoenix/expression/function/CollationKeyFunctionTest.java | 7 ---
 pom.xml   | 2 +-
 3 files changed, 8 insertions(+), 7 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/phoenix/blob/31dad52c/phoenix-core/src/it/java/org/apache/phoenix/end2end/CollationKeyFunctionIT.java
--
diff --git 
a/phoenix-core/src/it/java/org/apache/phoenix/end2end/CollationKeyFunctionIT.java
 
b/phoenix-core/src/it/java/org/apache/phoenix/end2end/CollationKeyFunctionIT.java
index 9f9e4d1..e954874 100644
--- 
a/phoenix-core/src/it/java/org/apache/phoenix/end2end/CollationKeyFunctionIT.java
+++ 
b/phoenix-core/src/it/java/org/apache/phoenix/end2end/CollationKeyFunctionIT.java
@@ -72,12 +72,12 @@ public class CollationKeyFunctionIT extends 
ParallelStatsDisabledIT {
 
@Test
public void testZhSort() throws Exception {
-   queryWithCollKeyDefaultArgsWithExpectedOrder("zh", false, 0, 6, 
new Integer[] { 3, 0, 1, 6, 5, 4, 2 });
+   queryWithCollKeyDefaultArgsWithExpectedOrder("zh", false, 0, 6, 
new Integer[] { 4, 3, 1, 5, 2, 0, 6 });
}
 
@Test
public void testZhTwSort() throws Exception {
-   queryWithCollKeyDefaultArgsWithExpectedOrder("zh_TW", false, 0, 
6, new Integer[] { 0, 3, 4, 1, 5, 2, 6 });
+   queryWithCollKeyDefaultArgsWithExpectedOrder("zh_TW", false, 0, 
6, new Integer[] { 4, 3, 1, 5, 2, 0, 6 });
}
 
@Test
@@ -87,7 +87,7 @@ public class CollationKeyFunctionIT extends 
ParallelStatsDisabledIT {
 
@Test
public void testZhStrokeSort() throws Exception {
-   queryWithCollKeyDefaultArgsWithExpectedOrder("zh__STROKE", 
false, 0, 6, new Integer[] { 0, 1, 3, 4, 6, 2, 5 });
+   queryWithCollKeyDefaultArgsWithExpectedOrder("zh__STROKE", 
false, 0, 6, new Integer[] { 4, 2, 0, 3, 1, 6, 5 });
}
 
@Test

http://git-wip-us.apache.org/repos/asf/phoenix/blob/31dad52c/phoenix-core/src/test/java/org/apache/phoenix/expression/function/CollationKeyFunctionTest.java
--
diff --git 
a/phoenix-core/src/test/java/org/apache/phoenix/expression/function/CollationKeyFunctionTest.java
 
b/phoenix-core/src/test/java/org/apache/phoenix/expression/function/CollationKeyFunctionTest.java
index 950875b..93154e3 100644
--- 
a/phoenix-core/src/test/java/org/apache/phoenix/expression/function/CollationKeyFunctionTest.java
+++ 
b/phoenix-core/src/test/java/org/apache/phoenix/expression/function/CollationKeyFunctionTest.java
@@ -90,12 +90,13 @@ public class CollationKeyFunctionTest {
 
@Test
public void testZhSort() throws Exception {
-   testSortOrderNoEquals(chineseChars, "zh", Boolean.FALSE, null, 
null, new Integer[] { 3, 0, 1, 6, 5, 4, 2 });
+   testSortOrderNoEquals(chineseChars, "zh", Boolean.FALSE, null, 
null, new Integer[] { 4, 3, 1, 5, 2, 0, 6 });
+
}
 
@Test
public void testZhTwSort() throws Exception {
-   testSortOrderNoEquals(chineseChars, "zh_TW", Boolean.FALSE, 
null, null, new Integer[] { 0, 3, 4, 1, 5, 2, 6 });
+   testSortOrderNoEquals(chineseChars, "zh_TW", Boolean.FALSE, 
null, null, new Integer[] { 4, 3, 1, 5, 2, 0, 6 });
}
 
@Test
@@ -107,7 +108,7 @@ public class CollationKeyFunctionTest {
@Test
public void testZhStrokeSort() throws Exception {
testSortOrderNoEquals(chineseChars, "zh__STROKE", 
Boolean.FALSE, null, null,
-   new Integer[] { 0, 1, 3, 4, 6, 2, 5 });
+   new Integer[] { 4, 2, 0, 3, 1, 6, 5 });
}
 
@Test

http://git-wip-us.apache.org/repos/asf/phoenix/blob/31dad52c/pom.xml
--
diff --git a/pom.xml b/pom.xml
index b860099..deaf848 100644
--- a/pom.xml
+++ b/pom.xml
@@ -105,7 +105,7 @@
 

Build failed in Jenkins: Phoenix Compile Compatibility with HBase #575

2018-03-13 Thread Apache Jenkins Server
See 


--
[...truncated 39.68 KB...]
[ERROR] 
:[364,5]
 method does not override or implement a method from a supertype
[ERROR] 
:[370,5]
 method does not override or implement a method from a supertype
[ERROR] 
:[376,5]
 method does not override or implement a method from a supertype
[ERROR] 
:[382,5]
 method does not override or implement a method from a supertype
[ERROR] Failed to execute goal 
org.apache.maven.plugins:maven-compiler-plugin:3.0:compile (default-compile) on 
project phoenix-core: Compilation failure: Compilation failure: 
[ERROR] 
:[34,39]
 cannot find symbol
[ERROR]   symbol:   class MetricRegistry
[ERROR]   location: package org.apache.hadoop.hbase.metrics
[ERROR] 
:[144,16]
 cannot find symbol
[ERROR]   symbol:   class MetricRegistry
[ERROR]   location: class 
org.apache.phoenix.coprocessor.PhoenixMetaDataCoprocessorHost.PhoenixMetaDataControllerEnvironment
[ERROR] 
:[24,35]
 cannot find symbol
[ERROR]   symbol:   class DelegatingHBaseRpcController
[ERROR]   location: package org.apache.hadoop.hbase.ipc
[ERROR] 
:[25,35]
 cannot find symbol
[ERROR]   symbol:   class HBaseRpcController
[ERROR]   location: package org.apache.hadoop.hbase.ipc
[ERROR] 
:[37,37]
 cannot find symbol
[ERROR]   symbol: class DelegatingHBaseRpcController
[ERROR] 
:[56,38]
 cannot find symbol
[ERROR]   symbol:   class HBaseRpcController
[ERROR]   location: class 
org.apache.hadoop.hbase.ipc.controller.MetadataRpcController
[ERROR] 
:[26,35]
 cannot find symbol
[ERROR]   symbol:   class HBaseRpcController
[ERROR]   location: package org.apache.hadoop.hbase.ipc
[ERROR] 
:[40,12]
 cannot find symbol
[ERROR]   symbol:   class HBaseRpcController
[ERROR]   location: class 
org.apache.hadoop.hbase.ipc.controller.InterRegionServerMetadataRpcControllerFactory
[ERROR] 
:[46,12]
 cannot find symbol
[ERROR]   symbol:   class HBaseRpcController
[ERROR]   location: class 
org.apache.hadoop.hbase.ipc.controller.InterRegionServerMetadataRpcControllerFactory
[ERROR] 
:[52,12]
 cannot find symbol
[ERROR]   symbol:   class HBaseRpcController
[ERROR]   location: class 
org.apache.hadoop.hbase.ipc.controller.InterRegionServerMetadataRpcControllerFactory
[ERROR] 
:[57,46]
 cannot