Apache-Phoenix | 3.0 | Hadoop1 | Build Successful

2014-11-14 Thread Apache Jenkins Server
3.0 branch build status Successful
Source repository https://git-wip-us.apache.org/repos/asf/phoenix.git

Last Successful Compiled Artifacts https://builds.apache.org/job/Phoenix-3.0-hadoop1/lastSuccessfulBuild/artifact/

Last Complete Test Report https://builds.apache.org/job/Phoenix-3.0-hadoop1/lastCompletedBuild/testReport/

Changes
[maryannxue] PHOENIX-1799 Support many-to-many joins



[3/4] phoenix git commit: PHOENIX-1799 Support many-to-many joins

2014-11-14 Thread maryannxue
http://git-wip-us.apache.org/repos/asf/phoenix/blob/c647c6a2/phoenix-core/src/it/java/org/apache/phoenix/end2end/SortMergeJoinIT.java
--
diff --git 
a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SortMergeJoinIT.java 
b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SortMergeJoinIT.java
new file mode 100644
index 000..e676cdb
--- /dev/null
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SortMergeJoinIT.java
@@ -0,0 +1,2800 @@
+/*
+ * 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.JOIN_CUSTOMER_TABLE_FULL_NAME;
+import static org.apache.phoenix.util.TestUtil.JOIN_ITEM_TABLE_DISPLAY_NAME;
+import static org.apache.phoenix.util.TestUtil.JOIN_ITEM_TABLE_FULL_NAME;
+import static org.apache.phoenix.util.TestUtil.JOIN_ORDER_TABLE_DISPLAY_NAME;
+import static org.apache.phoenix.util.TestUtil.JOIN_ORDER_TABLE_FULL_NAME;
+import static org.apache.phoenix.util.TestUtil.JOIN_SCHEMA;
+import static 
org.apache.phoenix.util.TestUtil.JOIN_SUPPLIER_TABLE_DISPLAY_NAME;
+import static org.apache.phoenix.util.TestUtil.JOIN_SUPPLIER_TABLE_FULL_NAME;
+import static org.apache.phoenix.util.TestUtil.TEST_PROPERTIES;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import java.sql.Connection;
+import java.sql.Date;
+import java.sql.DriverManager;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.ResultSetMetaData;
+import java.sql.SQLException;
+import java.sql.Timestamp;
+import java.text.SimpleDateFormat;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+
+import org.apache.phoenix.exception.SQLExceptionCode;
+import org.apache.phoenix.query.QueryServices;
+import org.apache.phoenix.schema.TableAlreadyExistsException;
+import org.apache.phoenix.util.MetaDataUtil;
+import org.apache.phoenix.util.PropertiesUtil;
+import org.apache.phoenix.util.QueryUtil;
+import org.apache.phoenix.util.ReadOnlyProps;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameters;
+
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+@Category(HBaseManagedTimeTest.class)
+@RunWith(Parameterized.class)
+public class SortMergeJoinIT extends BaseHBaseManagedTimeIT {
+
+private SimpleDateFormat format = new SimpleDateFormat("-MM-dd 
HH:mm:ss");
+private String[] indexDDL;
+private String[] plans;
+
+public SortMergeJoinIT(String[] indexDDL, String[] plans) {
+this.indexDDL = indexDDL;
+this.plans = plans;
+}
+
+@BeforeClass
+@Shadower(classBeingShadowed = BaseHBaseManagedTimeIT.class)
+public static void doSetup() throws Exception {
+Map props = Maps.newHashMapWithExpectedSize(3);
+// Forces server cache to be used
+props.put(QueryServices.INDEX_MUTATE_BATCH_SIZE_THRESHOLD_ATTRIB, 
Integer.toString(2));
+// Must update config before starting server
+setUpTestDriver(new ReadOnlyProps(props.entrySet().iterator()));
+}
+
+@Before
+public void initTable() throws Exception {
+initTableValues();
+if (indexDDL != null && indexDDL.length > 0) {
+Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+Connection conn = DriverManager.getConnection(getUrl(), props);
+for (String ddl : indexDDL) {
+try {
+conn.createStatement().execute(ddl);
+} catch (TableAlreadyExistsException e) {
+}
+}
+conn.close();
+}
+}
+
+@Parameters
+public static Collection data() {
+List testCases = Lists.newArrayList();
+testCases

[2/4] phoenix git commit: PHOENIX-1799 Support many-to-many joins

2014-11-14 Thread maryannxue
http://git-wip-us.apache.org/repos/asf/phoenix/blob/c647c6a2/phoenix-core/src/main/java/org/apache/phoenix/compile/GroupByCompiler.java
--
diff --git 
a/phoenix-core/src/main/java/org/apache/phoenix/compile/GroupByCompiler.java 
b/phoenix-core/src/main/java/org/apache/phoenix/compile/GroupByCompiler.java
index a561a47..016cd52 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/GroupByCompiler.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/GroupByCompiler.java
@@ -136,7 +136,7 @@ public class GroupByCompiler {
  * @throws ColumnNotFoundException if column name could not be resolved
  * @throws AmbiguousColumnException if an unaliased column name is 
ambiguous across multiple tables
  */
-public static GroupBy compile(StatementContext context, SelectStatement 
statement, TupleProjector tupleProjector) throws SQLException {
+public static GroupBy compile(StatementContext context, SelectStatement 
statement, TupleProjector tupleProjector, boolean isInRowKeyOrder) throws 
SQLException {
 List groupByNodes = statement.getGroupBy();
 /**
  * Distinct can use an aggregate plan if there's no group by.
@@ -179,7 +179,7 @@ public class GroupByCompiler {
 return GroupBy.EMPTY_GROUP_BY;
 }
 
-boolean isRowKeyOrderedGrouping = groupByVisitor.isOrderPreserving();
+boolean isRowKeyOrderedGrouping = isInRowKeyOrder && 
groupByVisitor.isOrderPreserving();
 List expressions = 
Lists.newArrayListWithCapacity(groupByEntries.size());
 List keyExpressions = expressions;
 String groupExprAttribName;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/c647c6a2/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 a609f60..f3d353b 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
@@ -20,7 +20,6 @@ package org.apache.phoenix.compile;
 import static org.apache.phoenix.schema.SaltingUtil.SALTING_COLUMN;
 
 import java.sql.SQLException;
-import java.sql.SQLFeatureNotSupportedException;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.Comparator;
@@ -293,6 +292,10 @@ public class JoinCompiler {
 return columnRefs;
 }
 
+public ParseNode getPostFiltersCombined() {
+return combine(postFilters);
+}
+
 public void addFilter(ParseNode filter) throws SQLException {
 if (joinSpecs.isEmpty()) {
 table.addFilter(filter);
@@ -320,7 +323,7 @@ public class JoinCompiler {
 for (JoinSpec joinSpec : joinSpecs) {
 JoinTable joinTable = joinSpec.getJoinTable();
 boolean hasSubJoin = !joinTable.getJoinSpecs().isEmpty();
-for (ComparisonParseNode node : joinSpec.getOnConditions()) {
+for (EqualParseNode node : joinSpec.getOnConditions()) {
 node.getLHS().accept(generalRefVisitor);
 if (hasSubJoin) {
 node.getRHS().accept(generalRefVisitor);
@@ -384,13 +387,12 @@ public class JoinCompiler {
 }
 
 public SelectStatement getAsSingleSubquery(SelectStatement query, 
boolean asSubquery) throws SQLException {
-if (!isFlat(query))
-throw new SQLFeatureNotSupportedException("Complex subqueries 
not supported as left join table.");
+assert (isFlat(query));
 
 if (asSubquery)
 return query;
 
-return NODE_FACTORY.select(query.getFrom(), select.getHint(), 
select.isDistinct(), select.getSelect(), query.getWhere(), select.getGroupBy(), 
select.getHaving(), select.getOrderBy(), select.getLimit(), 
select.getBindCount(), select.isAggregate(), select.hasSequence());
+return NODE_FACTORY.select(select, query.getFrom(), 
query.getWhere());
 }
 
 public boolean hasPostReference() {
@@ -427,7 +429,7 @@ public class JoinCompiler {
 
 public static class JoinSpec {
 private final JoinType type;
-private final List onConditions;
+private final List onConditions;
 private final JoinTable joinTable;
 private final boolean singleValueOnly;
 private Set dependencies;
@@ -436,7 +438,7 @@ public class JoinCompiler {
 private JoinSpec(JoinType type, ParseNode onNode, JoinTable joinTable, 
 boolean singleValueOnly, ColumnResolver resolver) throws 
SQLException {
 this.type =

[1/4] phoenix git commit: PHOENIX-1799 Support many-to-many joins

2014-11-14 Thread maryannxue
Repository: phoenix
Updated Branches:
  refs/heads/3.0 84fc0e872 -> c647c6a28


http://git-wip-us.apache.org/repos/asf/phoenix/blob/c647c6a2/phoenix-core/src/main/java/org/apache/phoenix/execute/TupleProjector.java
--
diff --git 
a/phoenix-core/src/main/java/org/apache/phoenix/execute/TupleProjector.java 
b/phoenix-core/src/main/java/org/apache/phoenix/execute/TupleProjector.java
index 14b488d..6a12f57 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/execute/TupleProjector.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/execute/TupleProjector.java
@@ -151,14 +151,21 @@ public class TupleProjector {
 public static class ProjectedValueTuple extends BaseTuple {
 private ImmutableBytesWritable keyPtr = new ImmutableBytesWritable();
 private long timestamp;
-private byte[] projectedValue;
+private ImmutableBytesWritable projectedValue = new 
ImmutableBytesWritable();
 private int bitSetLen;
 private KeyValue keyValue;
 
-private ProjectedValueTuple(byte[] keyBuffer, int keyOffset, int 
keyLength, long timestamp, byte[] projectedValue, int bitSetLen) {
+public ProjectedValueTuple(Tuple keyBase, long timestamp, byte[] 
projectedValue, int valueOffset, int valueLength, int bitSetLen) {
+keyBase.getKey(this.keyPtr);
+this.timestamp = timestamp;
+this.projectedValue.set(projectedValue, valueOffset, valueLength);
+this.bitSetLen = bitSetLen;
+}
+
+public ProjectedValueTuple(byte[] keyBuffer, int keyOffset, int 
keyLength, long timestamp, byte[] projectedValue, int valueOffset, int 
valueLength, int bitSetLen) {
 this.keyPtr.set(keyBuffer, keyOffset, keyLength);
 this.timestamp = timestamp;
-this.projectedValue = projectedValue;
+this.projectedValue.set(projectedValue, valueOffset, valueLength);
 this.bitSetLen = bitSetLen;
 }
 
@@ -170,7 +177,7 @@ public class TupleProjector {
 return timestamp;
 }
 
-public byte[] getProjectedValue() {
+public ImmutableBytesWritable getProjectedValue() {
 return projectedValue;
 }
 
@@ -195,7 +202,7 @@ public class TupleProjector {
 public KeyValue getValue(byte[] family, byte[] qualifier) {
 if (keyValue == null) {
 keyValue = KeyValueUtil.newKeyValue(keyPtr.get(), 
keyPtr.getOffset(), keyPtr.getLength(), 
-VALUE_COLUMN_FAMILY, VALUE_COLUMN_QUALIFIER, 
timestamp, projectedValue, 0, projectedValue.length);
+VALUE_COLUMN_FAMILY, VALUE_COLUMN_QUALIFIER, 
timestamp, projectedValue.get(), projectedValue.getOffset(), 
projectedValue.getLength());
 }
 return keyValue;
 }
@@ -203,7 +210,7 @@ public class TupleProjector {
 @Override
 public boolean getValue(byte[] family, byte[] qualifier,
 ImmutableBytesWritable ptr) {
-ptr.set(projectedValue);
+ptr.set(projectedValue.get(), projectedValue.getOffset(), 
projectedValue.getLength());
 return true;
 }
 
@@ -221,7 +228,7 @@ public class TupleProjector {
 public ProjectedValueTuple projectResults(Tuple tuple) {
byte[] bytesValue = schema.toBytes(tuple, getExpressions(), valueSet, 
ptr);
KeyValue base = tuple.getValue(0);
-return new ProjectedValueTuple(base.getBuffer(), base.getRowOffset(), 
base.getRowLength(), base.getTimestamp(), bytesValue, 
valueSet.getEstimatedLength());
+return new ProjectedValueTuple(base.getBuffer(), base.getRowOffset(), 
base.getRowLength(), base.getTimestamp(), bytesValue, 0, bytesValue.length, 
valueSet.getEstimatedLength());
 }
 
 public static void decodeProjectedValue(Tuple tuple, 
ImmutableBytesWritable ptr) throws IOException {
@@ -232,27 +239,33 @@ public class TupleProjector {
 
 public static ProjectedValueTuple mergeProjectedValue(ProjectedValueTuple 
dest, KeyValueSchema destSchema, ValueBitSet destBitSet,
Tuple src, KeyValueSchema srcSchema, ValueBitSet srcBitSet, int 
offset) throws IOException {
-   ImmutableBytesWritable destValue = new 
ImmutableBytesWritable(dest.getProjectedValue());
+   ImmutableBytesWritable destValue = dest.getProjectedValue();
+int origDestBitSetLen = dest.getBitSetLength();
destBitSet.clear();
-   destBitSet.or(destValue);
-   int origDestBitSetLen = dest.getBitSetLength();
-   ImmutableBytesWritable srcValue = new ImmutableBytesWritable();
-   decodeProjectedValue(src, srcValue);
-   srcBitSet.clear();
-   srcBitSet.or(srcValue);
-   int origSrcBitSetLen = srcBitSet.getEstimatedLength();
-   for (int i = 0; i < srcBitSet.getMaxSetBit(); i++) {
-   if (srcBitSet.get(i)) {
-   

[4/4] phoenix git commit: PHOENIX-1799 Support many-to-many joins

2014-11-14 Thread maryannxue
PHOENIX-1799 Support many-to-many joins


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

Branch: refs/heads/3.0
Commit: c647c6a28c7b050647f397d31df3c739aa9e9d6f
Parents: 84fc0e8
Author: maryannxue 
Authored: Fri Nov 14 16:58:26 2014 -0500
Committer: maryannxue 
Committed: Fri Nov 14 16:58:26 2014 -0500

--
 .../apache/phoenix/end2end/SortMergeJoinIT.java | 2800 ++
 .../apache/phoenix/compile/GroupByCompiler.java |4 +-
 .../apache/phoenix/compile/JoinCompiler.java|  147 +-
 .../apache/phoenix/compile/OrderByCompiler.java |   13 +-
 .../apache/phoenix/compile/QueryCompiler.java   |  191 +-
 .../phoenix/compile/SubselectRewriter.java  |   23 +-
 .../apache/phoenix/compile/UpsertCompiler.java  |5 +
 .../apache/phoenix/compile/WhereCompiler.java   |4 +-
 .../coprocessor/HashJoinRegionScanner.java  |   17 +-
 .../phoenix/execute/SortMergeJoinPlan.java  |  626 
 .../apache/phoenix/execute/TupleProjector.java  |   59 +-
 .../phoenix/iterate/FilterResultIterator.java   |3 +-
 .../phoenix/iterate/MappedByteBufferQueue.java  |  431 +++
 .../iterate/MappedByteBufferSortedQueue.java|  371 +--
 .../java/org/apache/phoenix/parse/HintNode.java |6 +-
 .../apache/phoenix/parse/ParseNodeFactory.java  |   11 +
 .../phoenix/parse/SelectStatementRewriter.java  |2 +-
 .../org/apache/phoenix/schema/ValueBitSet.java  |   11 +-
 18 files changed, 4265 insertions(+), 459 deletions(-)
--




Apache-Phoenix | Master | Build Successful

2014-11-14 Thread Apache Jenkins Server
Master branch build status Successful
Source repository https://git-wip-us.apache.org/repos/asf/phoenix.git

Last Successful Compiled Artifacts https://builds.apache.org/job/Phoenix-master/lastSuccessfulBuild/artifact/

Last Complete Test Report https://builds.apache.org/job/Phoenix-master/lastCompletedBuild/testReport/

Changes
[maryannxue] PHOENIX-1799 Support many-to-many joins



[1/4] phoenix git commit: PHOENIX-1799 Support many-to-many joins

2014-11-14 Thread maryannxue
Repository: phoenix
Updated Branches:
  refs/heads/4.0 4a675d84d -> ebc7ee42c


http://git-wip-us.apache.org/repos/asf/phoenix/blob/ebc7ee42/phoenix-core/src/main/java/org/apache/phoenix/execute/SortMergeJoinPlan.java
--
diff --git 
a/phoenix-core/src/main/java/org/apache/phoenix/execute/SortMergeJoinPlan.java 
b/phoenix-core/src/main/java/org/apache/phoenix/execute/SortMergeJoinPlan.java
new file mode 100644
index 000..03eda06
--- /dev/null
+++ 
b/phoenix-core/src/main/java/org/apache/phoenix/execute/SortMergeJoinPlan.java
@@ -0,0 +1,628 @@
+/*
+ * 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.execute;
+
+import java.io.IOException;
+import java.nio.MappedByteBuffer;
+import java.sql.ParameterMetaData;
+import java.sql.SQLException;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Queue;
+
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.KeyValueUtil;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.phoenix.compile.ExplainPlan;
+import org.apache.phoenix.compile.GroupByCompiler.GroupBy;
+import org.apache.phoenix.compile.OrderByCompiler.OrderBy;
+import org.apache.phoenix.compile.QueryPlan;
+import org.apache.phoenix.compile.RowProjector;
+import org.apache.phoenix.compile.StatementContext;
+import org.apache.phoenix.execute.TupleProjector.ProjectedValueTuple;
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.iterate.MappedByteBufferQueue;
+import org.apache.phoenix.iterate.ResultIterator;
+import org.apache.phoenix.jdbc.PhoenixParameterMetaData;
+import org.apache.phoenix.parse.FilterableStatement;
+import org.apache.phoenix.parse.JoinTableNode.JoinType;
+import org.apache.phoenix.query.KeyRange;
+import org.apache.phoenix.query.QueryServices;
+import org.apache.phoenix.query.QueryServicesOptions;
+import org.apache.phoenix.schema.KeyValueSchema;
+import org.apache.phoenix.schema.PColumn;
+import org.apache.phoenix.schema.PTable;
+import org.apache.phoenix.schema.TableRef;
+import org.apache.phoenix.schema.ValueBitSet;
+import org.apache.phoenix.schema.KeyValueSchema.KeyValueSchemaBuilder;
+import org.apache.phoenix.schema.tuple.ResultTuple;
+import org.apache.phoenix.schema.tuple.Tuple;
+import org.apache.phoenix.util.ResultUtil;
+import org.apache.phoenix.util.SchemaUtil;
+
+import com.google.common.collect.Lists;
+
+public class SortMergeJoinPlan implements QueryPlan {
+private static final byte[] EMPTY_PTR = new byte[0];
+
+private final StatementContext context;
+private final FilterableStatement statement;
+private final TableRef table;
+private final JoinType type;
+private final QueryPlan lhsPlan;
+private final QueryPlan rhsPlan;
+private final List lhsKeyExpressions;
+private final List rhsKeyExpressions;
+private final KeyValueSchema joinedSchema;
+private final KeyValueSchema lhsSchema;
+private final KeyValueSchema rhsSchema;
+private final int rhsFieldPosition;
+
+public SortMergeJoinPlan(StatementContext context, FilterableStatement 
statement, TableRef table, 
+JoinType type, QueryPlan lhsPlan, QueryPlan rhsPlan, 
List lhsKeyExpressions, List rhsKeyExpressions,
+PTable joinedTable, PTable lhsTable, PTable rhsTable, int 
rhsFieldPosition) {
+if (type == JoinType.Right) throw new 
IllegalArgumentException("JoinType should not be " + type);
+this.context = context;
+this.statement = statement;
+this.table = table;
+this.type = type;
+this.lhsPlan = lhsPlan;
+this.rhsPlan = rhsPlan;
+this.lhsKeyExpressions = lhsKeyExpressions;
+this.rhsKeyExpressions = rhsKeyExpressions;
+this.joinedSchema = buildSchema(joinedTable);
+this.lhsSchema = buildSchema(lhsTable);
+this.rhsSchema = buildSchema(rhsTable);
+this.rhsFieldPosition = rhsFieldPositio

[2/4] phoenix git commit: PHOENIX-1799 Support many-to-many joins

2014-11-14 Thread maryannxue
http://git-wip-us.apache.org/repos/asf/phoenix/blob/ebc7ee42/phoenix-core/src/main/java/org/apache/phoenix/compile/GroupByCompiler.java
--
diff --git 
a/phoenix-core/src/main/java/org/apache/phoenix/compile/GroupByCompiler.java 
b/phoenix-core/src/main/java/org/apache/phoenix/compile/GroupByCompiler.java
index a561a47..016cd52 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/GroupByCompiler.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/GroupByCompiler.java
@@ -136,7 +136,7 @@ public class GroupByCompiler {
  * @throws ColumnNotFoundException if column name could not be resolved
  * @throws AmbiguousColumnException if an unaliased column name is 
ambiguous across multiple tables
  */
-public static GroupBy compile(StatementContext context, SelectStatement 
statement, TupleProjector tupleProjector) throws SQLException {
+public static GroupBy compile(StatementContext context, SelectStatement 
statement, TupleProjector tupleProjector, boolean isInRowKeyOrder) throws 
SQLException {
 List groupByNodes = statement.getGroupBy();
 /**
  * Distinct can use an aggregate plan if there's no group by.
@@ -179,7 +179,7 @@ public class GroupByCompiler {
 return GroupBy.EMPTY_GROUP_BY;
 }
 
-boolean isRowKeyOrderedGrouping = groupByVisitor.isOrderPreserving();
+boolean isRowKeyOrderedGrouping = isInRowKeyOrder && 
groupByVisitor.isOrderPreserving();
 List expressions = 
Lists.newArrayListWithCapacity(groupByEntries.size());
 List keyExpressions = expressions;
 String groupExprAttribName;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/ebc7ee42/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 b519dc4..45b6603 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
@@ -20,7 +20,6 @@ package org.apache.phoenix.compile;
 import static org.apache.phoenix.schema.SaltingUtil.SALTING_COLUMN;
 
 import java.sql.SQLException;
-import java.sql.SQLFeatureNotSupportedException;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.Comparator;
@@ -293,6 +292,10 @@ public class JoinCompiler {
 return columnRefs;
 }
 
+public ParseNode getPostFiltersCombined() {
+return combine(postFilters);
+}
+
 public void addFilter(ParseNode filter) throws SQLException {
 if (joinSpecs.isEmpty()) {
 table.addFilter(filter);
@@ -320,7 +323,7 @@ public class JoinCompiler {
 for (JoinSpec joinSpec : joinSpecs) {
 JoinTable joinTable = joinSpec.getJoinTable();
 boolean hasSubJoin = !joinTable.getJoinSpecs().isEmpty();
-for (ComparisonParseNode node : joinSpec.getOnConditions()) {
+for (EqualParseNode node : joinSpec.getOnConditions()) {
 node.getLHS().accept(generalRefVisitor);
 if (hasSubJoin) {
 node.getRHS().accept(generalRefVisitor);
@@ -384,13 +387,12 @@ public class JoinCompiler {
 }
 
 public SelectStatement getAsSingleSubquery(SelectStatement query, 
boolean asSubquery) throws SQLException {
-if (!isFlat(query))
-throw new SQLFeatureNotSupportedException("Complex subqueries 
not supported as left join table.");
+assert (isFlat(query));
 
 if (asSubquery)
 return query;
 
-return NODE_FACTORY.select(query.getFrom(), select.getHint(), 
select.isDistinct(), select.getSelect(), query.getWhere(), select.getGroupBy(), 
select.getHaving(), select.getOrderBy(), select.getLimit(), 
select.getBindCount(), select.isAggregate(), select.hasSequence());
+return NODE_FACTORY.select(select, query.getFrom(), 
query.getWhere());
 }
 
 public boolean hasPostReference() {
@@ -427,7 +429,7 @@ public class JoinCompiler {
 
 public static class JoinSpec {
 private final JoinType type;
-private final List onConditions;
+private final List onConditions;
 private final JoinTable joinTable;
 private final boolean singleValueOnly;
 private Set dependencies;
@@ -436,7 +438,7 @@ public class JoinCompiler {
 private JoinSpec(JoinType type, ParseNode onNode, JoinTable joinTable, 
 boolean singleValueOnly, ColumnResolver resolver) throws 
SQLException {
 this.type =

[4/4] phoenix git commit: PHOENIX-1799 Support many-to-many joins

2014-11-14 Thread maryannxue
PHOENIX-1799 Support many-to-many joins


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

Branch: refs/heads/4.0
Commit: ebc7ee42cdb2b05a293f54dc687ca975db9acbc3
Parents: 4a675d8
Author: maryannxue 
Authored: Fri Nov 14 16:07:27 2014 -0500
Committer: maryannxue 
Committed: Fri Nov 14 16:07:27 2014 -0500

--
 .../apache/phoenix/end2end/SortMergeJoinIT.java | 2822 ++
 .../apache/phoenix/compile/GroupByCompiler.java |4 +-
 .../apache/phoenix/compile/JoinCompiler.java|  147 +-
 .../apache/phoenix/compile/OrderByCompiler.java |   11 +-
 .../apache/phoenix/compile/QueryCompiler.java   |  191 +-
 .../phoenix/compile/QueryCompiler.java.orig |  444 ---
 .../phoenix/compile/SubselectRewriter.java  |   23 +-
 .../apache/phoenix/compile/UpsertCompiler.java  |5 +
 .../apache/phoenix/compile/WhereCompiler.java   |4 +-
 .../coprocessor/HashJoinRegionScanner.java  |   17 +-
 .../phoenix/execute/SortMergeJoinPlan.java  |  628 
 .../apache/phoenix/execute/TupleProjector.java  |   59 +-
 .../phoenix/iterate/FilterResultIterator.java   |3 +-
 .../phoenix/iterate/MappedByteBufferQueue.java  |  431 +++
 .../iterate/MappedByteBufferSortedQueue.java|  372 +--
 .../java/org/apache/phoenix/parse/HintNode.java |6 +-
 .../apache/phoenix/parse/ParseNodeFactory.java  |   11 +
 .../phoenix/parse/SelectStatementRewriter.java  |2 +-
 .../org/apache/phoenix/schema/ValueBitSet.java  |   11 +-
 19 files changed, 4288 insertions(+), 903 deletions(-)
--




[3/4] phoenix git commit: PHOENIX-1799 Support many-to-many joins

2014-11-14 Thread maryannxue
http://git-wip-us.apache.org/repos/asf/phoenix/blob/ebc7ee42/phoenix-core/src/it/java/org/apache/phoenix/end2end/SortMergeJoinIT.java
--
diff --git 
a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SortMergeJoinIT.java 
b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SortMergeJoinIT.java
new file mode 100644
index 000..469388b
--- /dev/null
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SortMergeJoinIT.java
@@ -0,0 +1,2822 @@
+/*
+ * 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.JOIN_CUSTOMER_TABLE_FULL_NAME;
+import static org.apache.phoenix.util.TestUtil.JOIN_ITEM_TABLE_DISPLAY_NAME;
+import static org.apache.phoenix.util.TestUtil.JOIN_ITEM_TABLE_FULL_NAME;
+import static org.apache.phoenix.util.TestUtil.JOIN_ORDER_TABLE_DISPLAY_NAME;
+import static org.apache.phoenix.util.TestUtil.JOIN_ORDER_TABLE_FULL_NAME;
+import static org.apache.phoenix.util.TestUtil.JOIN_SCHEMA;
+import static 
org.apache.phoenix.util.TestUtil.JOIN_SUPPLIER_TABLE_DISPLAY_NAME;
+import static org.apache.phoenix.util.TestUtil.JOIN_SUPPLIER_TABLE_FULL_NAME;
+import static org.apache.phoenix.util.TestUtil.TEST_PROPERTIES;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import java.sql.Connection;
+import java.sql.Date;
+import java.sql.DriverManager;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.ResultSetMetaData;
+import java.sql.SQLException;
+import java.sql.Timestamp;
+import java.text.SimpleDateFormat;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+
+import org.apache.phoenix.exception.SQLExceptionCode;
+import org.apache.phoenix.query.QueryServices;
+import org.apache.phoenix.schema.TableAlreadyExistsException;
+import org.apache.phoenix.util.MetaDataUtil;
+import org.apache.phoenix.util.PropertiesUtil;
+import org.apache.phoenix.util.QueryUtil;
+import org.apache.phoenix.util.ReadOnlyProps;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameters;
+
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+@Category(HBaseManagedTimeTest.class)
+@RunWith(Parameterized.class)
+public class SortMergeJoinIT extends BaseHBaseManagedTimeIT {
+
+private SimpleDateFormat format = new SimpleDateFormat("-MM-dd 
HH:mm:ss");
+private String[] indexDDL;
+private String[] plans;
+
+public SortMergeJoinIT(String[] indexDDL, String[] plans) {
+this.indexDDL = indexDDL;
+this.plans = plans;
+}
+
+@BeforeClass
+@Shadower(classBeingShadowed = BaseHBaseManagedTimeIT.class)
+public static void doSetup() throws Exception {
+Map props = Maps.newHashMapWithExpectedSize(3);
+// Forces server cache to be used
+props.put(QueryServices.INDEX_MUTATE_BATCH_SIZE_THRESHOLD_ATTRIB, 
Integer.toString(2));
+// Must update config before starting server
+setUpTestDriver(new ReadOnlyProps(props.entrySet().iterator()));
+}
+
+@Before
+public void initTable() throws Exception {
+initTableValues();
+if (indexDDL != null && indexDDL.length > 0) {
+Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+Connection conn = DriverManager.getConnection(getUrl(), props);
+for (String ddl : indexDDL) {
+try {
+conn.createStatement().execute(ddl);
+} catch (TableAlreadyExistsException e) {
+}
+}
+conn.close();
+}
+}
+
+@Parameters
+public static Collection data() {
+List testCases = Lists.newArrayList();
+testCases

Jenkins build is back to normal : Phoenix | Master #487

2014-11-14 Thread Apache Jenkins Server
See 



phoenix git commit: PHOENIX-1799 Support many-to-many joins

2014-11-14 Thread maryannxue
Repository: phoenix
Updated Branches:
  refs/heads/master eddc846d8 -> bc89c9a51


PHOENIX-1799 Support many-to-many joins


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

Branch: refs/heads/master
Commit: bc89c9a513323001622ee699b6e5dde6a843c795
Parents: eddc846
Author: maryannxue 
Authored: Fri Nov 14 15:54:25 2014 -0500
Committer: maryannxue 
Committed: Fri Nov 14 15:54:25 2014 -0500

--
 .../org/apache/phoenix/iterate/MappedByteBufferSortedQueue.java | 1 +
 1 file changed, 1 insertion(+)
--


http://git-wip-us.apache.org/repos/asf/phoenix/blob/bc89c9a5/phoenix-core/src/main/java/org/apache/phoenix/iterate/MappedByteBufferSortedQueue.java
--
diff --git 
a/phoenix-core/src/main/java/org/apache/phoenix/iterate/MappedByteBufferSortedQueue.java
 
b/phoenix-core/src/main/java/org/apache/phoenix/iterate/MappedByteBufferSortedQueue.java
index 03a360e..ae2f452 100644
--- 
a/phoenix-core/src/main/java/org/apache/phoenix/iterate/MappedByteBufferSortedQueue.java
+++ 
b/phoenix-core/src/main/java/org/apache/phoenix/iterate/MappedByteBufferSortedQueue.java
@@ -82,6 +82,7 @@ public class MappedByteBufferSortedQueue extends 
MappedByteBufferQueue

[2/5] phoenix git commit: PHOENIX-1799 Support many-to-many joins

2014-11-14 Thread maryannxue
http://git-wip-us.apache.org/repos/asf/phoenix/blob/a715a796/phoenix-core/src/it/java/org/apache/phoenix/end2end/SortMergeJoinIT.java
--
diff --git 
a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SortMergeJoinIT.java 
b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SortMergeJoinIT.java
new file mode 100644
index 000..469388b
--- /dev/null
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SortMergeJoinIT.java
@@ -0,0 +1,2822 @@
+/*
+ * 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.JOIN_CUSTOMER_TABLE_FULL_NAME;
+import static org.apache.phoenix.util.TestUtil.JOIN_ITEM_TABLE_DISPLAY_NAME;
+import static org.apache.phoenix.util.TestUtil.JOIN_ITEM_TABLE_FULL_NAME;
+import static org.apache.phoenix.util.TestUtil.JOIN_ORDER_TABLE_DISPLAY_NAME;
+import static org.apache.phoenix.util.TestUtil.JOIN_ORDER_TABLE_FULL_NAME;
+import static org.apache.phoenix.util.TestUtil.JOIN_SCHEMA;
+import static 
org.apache.phoenix.util.TestUtil.JOIN_SUPPLIER_TABLE_DISPLAY_NAME;
+import static org.apache.phoenix.util.TestUtil.JOIN_SUPPLIER_TABLE_FULL_NAME;
+import static org.apache.phoenix.util.TestUtil.TEST_PROPERTIES;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import java.sql.Connection;
+import java.sql.Date;
+import java.sql.DriverManager;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.ResultSetMetaData;
+import java.sql.SQLException;
+import java.sql.Timestamp;
+import java.text.SimpleDateFormat;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+
+import org.apache.phoenix.exception.SQLExceptionCode;
+import org.apache.phoenix.query.QueryServices;
+import org.apache.phoenix.schema.TableAlreadyExistsException;
+import org.apache.phoenix.util.MetaDataUtil;
+import org.apache.phoenix.util.PropertiesUtil;
+import org.apache.phoenix.util.QueryUtil;
+import org.apache.phoenix.util.ReadOnlyProps;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameters;
+
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+@Category(HBaseManagedTimeTest.class)
+@RunWith(Parameterized.class)
+public class SortMergeJoinIT extends BaseHBaseManagedTimeIT {
+
+private SimpleDateFormat format = new SimpleDateFormat("-MM-dd 
HH:mm:ss");
+private String[] indexDDL;
+private String[] plans;
+
+public SortMergeJoinIT(String[] indexDDL, String[] plans) {
+this.indexDDL = indexDDL;
+this.plans = plans;
+}
+
+@BeforeClass
+@Shadower(classBeingShadowed = BaseHBaseManagedTimeIT.class)
+public static void doSetup() throws Exception {
+Map props = Maps.newHashMapWithExpectedSize(3);
+// Forces server cache to be used
+props.put(QueryServices.INDEX_MUTATE_BATCH_SIZE_THRESHOLD_ATTRIB, 
Integer.toString(2));
+// Must update config before starting server
+setUpTestDriver(new ReadOnlyProps(props.entrySet().iterator()));
+}
+
+@Before
+public void initTable() throws Exception {
+initTableValues();
+if (indexDDL != null && indexDDL.length > 0) {
+Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+Connection conn = DriverManager.getConnection(getUrl(), props);
+for (String ddl : indexDDL) {
+try {
+conn.createStatement().execute(ddl);
+} catch (TableAlreadyExistsException e) {
+}
+}
+conn.close();
+}
+}
+
+@Parameters
+public static Collection data() {
+List testCases = Lists.newArrayList();
+testCases

[1/5] phoenix git commit: PHOENIX-1799 Support many-to-many joins

2014-11-14 Thread maryannxue
Repository: phoenix
Updated Branches:
  refs/heads/master 9b285dae0 -> eddc846d8


http://git-wip-us.apache.org/repos/asf/phoenix/blob/a715a796/phoenix-core/src/main/java/org/apache/phoenix/execute/SortMergeJoinPlan.java
--
diff --git 
a/phoenix-core/src/main/java/org/apache/phoenix/execute/SortMergeJoinPlan.java 
b/phoenix-core/src/main/java/org/apache/phoenix/execute/SortMergeJoinPlan.java
new file mode 100644
index 000..03eda06
--- /dev/null
+++ 
b/phoenix-core/src/main/java/org/apache/phoenix/execute/SortMergeJoinPlan.java
@@ -0,0 +1,628 @@
+/*
+ * 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.execute;
+
+import java.io.IOException;
+import java.nio.MappedByteBuffer;
+import java.sql.ParameterMetaData;
+import java.sql.SQLException;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Queue;
+
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.KeyValueUtil;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.phoenix.compile.ExplainPlan;
+import org.apache.phoenix.compile.GroupByCompiler.GroupBy;
+import org.apache.phoenix.compile.OrderByCompiler.OrderBy;
+import org.apache.phoenix.compile.QueryPlan;
+import org.apache.phoenix.compile.RowProjector;
+import org.apache.phoenix.compile.StatementContext;
+import org.apache.phoenix.execute.TupleProjector.ProjectedValueTuple;
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.iterate.MappedByteBufferQueue;
+import org.apache.phoenix.iterate.ResultIterator;
+import org.apache.phoenix.jdbc.PhoenixParameterMetaData;
+import org.apache.phoenix.parse.FilterableStatement;
+import org.apache.phoenix.parse.JoinTableNode.JoinType;
+import org.apache.phoenix.query.KeyRange;
+import org.apache.phoenix.query.QueryServices;
+import org.apache.phoenix.query.QueryServicesOptions;
+import org.apache.phoenix.schema.KeyValueSchema;
+import org.apache.phoenix.schema.PColumn;
+import org.apache.phoenix.schema.PTable;
+import org.apache.phoenix.schema.TableRef;
+import org.apache.phoenix.schema.ValueBitSet;
+import org.apache.phoenix.schema.KeyValueSchema.KeyValueSchemaBuilder;
+import org.apache.phoenix.schema.tuple.ResultTuple;
+import org.apache.phoenix.schema.tuple.Tuple;
+import org.apache.phoenix.util.ResultUtil;
+import org.apache.phoenix.util.SchemaUtil;
+
+import com.google.common.collect.Lists;
+
+public class SortMergeJoinPlan implements QueryPlan {
+private static final byte[] EMPTY_PTR = new byte[0];
+
+private final StatementContext context;
+private final FilterableStatement statement;
+private final TableRef table;
+private final JoinType type;
+private final QueryPlan lhsPlan;
+private final QueryPlan rhsPlan;
+private final List lhsKeyExpressions;
+private final List rhsKeyExpressions;
+private final KeyValueSchema joinedSchema;
+private final KeyValueSchema lhsSchema;
+private final KeyValueSchema rhsSchema;
+private final int rhsFieldPosition;
+
+public SortMergeJoinPlan(StatementContext context, FilterableStatement 
statement, TableRef table, 
+JoinType type, QueryPlan lhsPlan, QueryPlan rhsPlan, 
List lhsKeyExpressions, List rhsKeyExpressions,
+PTable joinedTable, PTable lhsTable, PTable rhsTable, int 
rhsFieldPosition) {
+if (type == JoinType.Right) throw new 
IllegalArgumentException("JoinType should not be " + type);
+this.context = context;
+this.statement = statement;
+this.table = table;
+this.type = type;
+this.lhsPlan = lhsPlan;
+this.rhsPlan = rhsPlan;
+this.lhsKeyExpressions = lhsKeyExpressions;
+this.rhsKeyExpressions = rhsKeyExpressions;
+this.joinedSchema = buildSchema(joinedTable);
+this.lhsSchema = buildSchema(lhsTable);
+this.rhsSchema = buildSchema(rhsTable);
+this.rhsFieldPosition = rhsFieldPosi

[4/5] phoenix git commit: PHOENIX-1799 Support many-to-many joins

2014-11-14 Thread maryannxue
http://git-wip-us.apache.org/repos/asf/phoenix/blob/eddc846d/phoenix-core/src/main/java/org/apache/phoenix/parse/SelectStatementRewriter.java
--
diff --git 
a/phoenix-core/src/main/java/org/apache/phoenix/parse/SelectStatementRewriter.java
 
b/phoenix-core/src/main/java/org/apache/phoenix/parse/SelectStatementRewriter.java
index ab06d46..3152abe 100644
--- 
a/phoenix-core/src/main/java/org/apache/phoenix/parse/SelectStatementRewriter.java
+++ 
b/phoenix-core/src/main/java/org/apache/phoenix/parse/SelectStatementRewriter.java
@@ -52,7 +52,7 @@ public class SelectStatementRewriter extends 
ParseNodeRewriter {
 SelectStatementRewriter rewriter = new 
SelectStatementRewriter(removeNodes);
 where = where.accept(rewriter);
 // Return new SELECT statement with updated WHERE clause
-return NODE_FACTORY.select(statement, where, statement.getHaving());
+return NODE_FACTORY.select(statement, where);
 }
 
 /**

http://git-wip-us.apache.org/repos/asf/phoenix/blob/eddc846d/phoenix-core/src/main/java/org/apache/phoenix/schema/ValueBitSet.java
--
diff --git 
a/phoenix-core/src/main/java/org/apache/phoenix/schema/ValueBitSet.java 
b/phoenix-core/src/main/java/org/apache/phoenix/schema/ValueBitSet.java
index 47acdac..7931659 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/ValueBitSet.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/ValueBitSet.java
@@ -145,10 +145,14 @@ public class ValueBitSet {
 }
 
 public void or(ImmutableBytesWritable ptr) {
-if (schema == null) {
+or(ptr, isVarLength() ? Bytes.SIZEOF_SHORT + 1 : Bytes.SIZEOF_SHORT);
+}
+
+public void or(ImmutableBytesWritable ptr, int length) {
+if (schema == null || length == 0) {
 return;
 }
-if (isVarLength()) {
+if (length > Bytes.SIZEOF_SHORT) {
 int offset = ptr.getOffset() + ptr.getLength() - 
Bytes.SIZEOF_SHORT;
 short nLongs = Bytes.toShort(ptr.get(), offset);
 offset -= nLongs * Bytes.SIZEOF_LONG;
@@ -160,7 +164,7 @@ public class ValueBitSet {
 } else {
 long l = Bytes.toShort(ptr.get(), ptr.getOffset() + 
ptr.getLength() - Bytes.SIZEOF_SHORT);
 bits[0] |= l;
-maxSetBit = Math.max(maxSetBit, BITS_PER_SHORT - 1);
+maxSetBit = Math.max(maxSetBit, (bits[0] == 0 ? 0 : 
BITS_PER_SHORT) - 1);
 }
 
 }
@@ -196,3 +200,4 @@ public class ValueBitSet {
 maxSetBit = Math.max(maxSetBit, isSet.maxSetBit);
 }
 }
+



[3/5] phoenix git commit: PHOENIX-1799 Support many-to-many joins

2014-11-14 Thread maryannxue
PHOENIX-1799 Support many-to-many joins


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

Branch: refs/heads/master
Commit: a715a796e4962a452ff321ea2252643b06d9ce01
Parents: 9b285da
Author: maryannxue 
Authored: Fri Nov 14 15:27:40 2014 -0500
Committer: maryannxue 
Committed: Fri Nov 14 15:27:40 2014 -0500

--
 .../apache/phoenix/end2end/SortMergeJoinIT.java | 2822 ++
 .../phoenix/execute/SortMergeJoinPlan.java  |  628 
 .../phoenix/iterate/MappedByteBufferQueue.java  |  431 +++
 3 files changed, 3881 insertions(+)
--




[5/5] phoenix git commit: PHOENIX-1799 Support many-to-many joins

2014-11-14 Thread maryannxue
PHOENIX-1799 Support many-to-many joins


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

Branch: refs/heads/master
Commit: eddc846d85f97f3cc2206aba27a8d544c4db3384
Parents: a715a79
Author: maryannxue 
Authored: Fri Nov 14 15:29:43 2014 -0500
Committer: maryannxue 
Committed: Fri Nov 14 15:29:43 2014 -0500

--
 .../apache/phoenix/compile/GroupByCompiler.java |   4 +-
 .../apache/phoenix/compile/JoinCompiler.java| 147 
 .../apache/phoenix/compile/OrderByCompiler.java |  11 +-
 .../apache/phoenix/compile/QueryCompiler.java   | 191 +++---
 .../phoenix/compile/SubselectRewriter.java  |  23 +-
 .../apache/phoenix/compile/UpsertCompiler.java  |   5 +
 .../apache/phoenix/compile/WhereCompiler.java   |   4 +-
 .../coprocessor/HashJoinRegionScanner.java  |  17 +-
 .../apache/phoenix/execute/TupleProjector.java  |  59 +--
 .../phoenix/iterate/FilterResultIterator.java   |   3 +-
 .../iterate/MappedByteBufferSortedQueue.java| 370 ---
 .../java/org/apache/phoenix/parse/HintNode.java |   6 +-
 .../apache/phoenix/parse/ParseNodeFactory.java  |  11 +
 .../phoenix/parse/SelectStatementRewriter.java  |   2 +-
 .../org/apache/phoenix/schema/ValueBitSet.java  |  11 +-
 15 files changed, 406 insertions(+), 458 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/phoenix/blob/eddc846d/phoenix-core/src/main/java/org/apache/phoenix/compile/GroupByCompiler.java
--
diff --git 
a/phoenix-core/src/main/java/org/apache/phoenix/compile/GroupByCompiler.java 
b/phoenix-core/src/main/java/org/apache/phoenix/compile/GroupByCompiler.java
index a561a47..016cd52 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/GroupByCompiler.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/GroupByCompiler.java
@@ -136,7 +136,7 @@ public class GroupByCompiler {
  * @throws ColumnNotFoundException if column name could not be resolved
  * @throws AmbiguousColumnException if an unaliased column name is 
ambiguous across multiple tables
  */
-public static GroupBy compile(StatementContext context, SelectStatement 
statement, TupleProjector tupleProjector) throws SQLException {
+public static GroupBy compile(StatementContext context, SelectStatement 
statement, TupleProjector tupleProjector, boolean isInRowKeyOrder) throws 
SQLException {
 List groupByNodes = statement.getGroupBy();
 /**
  * Distinct can use an aggregate plan if there's no group by.
@@ -179,7 +179,7 @@ public class GroupByCompiler {
 return GroupBy.EMPTY_GROUP_BY;
 }
 
-boolean isRowKeyOrderedGrouping = groupByVisitor.isOrderPreserving();
+boolean isRowKeyOrderedGrouping = isInRowKeyOrder && 
groupByVisitor.isOrderPreserving();
 List expressions = 
Lists.newArrayListWithCapacity(groupByEntries.size());
 List keyExpressions = expressions;
 String groupExprAttribName;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/eddc846d/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 b519dc4..45b6603 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
@@ -20,7 +20,6 @@ package org.apache.phoenix.compile;
 import static org.apache.phoenix.schema.SaltingUtil.SALTING_COLUMN;
 
 import java.sql.SQLException;
-import java.sql.SQLFeatureNotSupportedException;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.Comparator;
@@ -293,6 +292,10 @@ public class JoinCompiler {
 return columnRefs;
 }
 
+public ParseNode getPostFiltersCombined() {
+return combine(postFilters);
+}
+
 public void addFilter(ParseNode filter) throws SQLException {
 if (joinSpecs.isEmpty()) {
 table.addFilter(filter);
@@ -320,7 +323,7 @@ public class JoinCompiler {
 for (JoinSpec joinSpec : joinSpecs) {
 JoinTable joinTable = joinSpec.getJoinTable();
 boolean hasSubJoin = !joinTable.getJoinSpecs().isEmpty();
-for (ComparisonParseNode node : joinSpec.getOnConditions()) {
+for (EqualParseNode node : joinSpec.getOnConditions()) {
 node.getLHS().accept(gener