From 610f2e6636419c610966a942cdd57131c84cd256 Mon Sep 17 00:00:00 2001 From: "navis.ryu" Date: Thu, 9 Apr 2015 16:58:01 +0900 Subject: [PATCH 1/2] TAJO-1542 Refactoring of HashJoinExecs --- .../java/org/apache/tajo/catalog/Schema.java | 20 + .../org/apache/tajo/storage/EmptyTuple.java | 140 +----- .../org/apache/tajo/storage/NullTuple.java | 172 +++++++ .../java/org/apache/tajo/storage/VTuple.java | 20 +- .../engine/planner/PhysicalPlannerImpl.java | 22 +- .../BasicPhysicalExecutorVisitor.java | 8 - .../planner/physical/CommonHashJoinExec.java | 186 +++++++ .../planner/physical/CommonJoinExec.java | 132 ++++- .../physical/HashFullOuterJoinExec.java | 247 +++------ .../engine/planner/physical/HashJoinExec.java | 212 +------- .../physical/HashLeftAntiJoinExec.java | 59 +-- .../physical/HashLeftOuterJoinExec.java | 292 +---------- .../physical/HashLeftSemiJoinExec.java | 48 +- .../planner/physical/NLLeftOuterJoinExec.java | 101 ---- .../engine/planner/physical/PhysicalExec.java | 27 + .../physical/PhysicalExecutorVisitor.java | 3 - .../physical/RightOuterMergeJoinExec.java | 40 +- .../apache/tajo/engine/utils/CacheHolder.java | 3 +- .../physical/TestHashSemiJoinExec.java | 9 +- .../physical/TestLeftOuterHashJoinExec.java | 104 ++-- .../physical/TestLeftOuterNLJoinExec.java | 474 ------------------ .../testJoinFilterOfRowPreservedTable1.sql | 2 +- .../testJoinFilterOfRowPreservedTable1.result | 2 +- .../expr/AggregationFunctionCallEval.java | 4 +- .../apache/tajo/plan/expr/AlgebraicUtil.java | 5 + .../org/apache/tajo/plan/expr/EvalNode.java | 39 +- .../org/apache/tajo/plan/expr/InEval.java | 2 +- .../plan/expr/PatternMatchPredicateEval.java | 2 +- .../tajo/plan/expr/WindowFunctionEval.java | 2 +- .../org/apache/tajo/storage/FrameTuple.java | 14 +- 30 files changed, 823 insertions(+), 1568 deletions(-) create mode 100644 tajo-common/src/main/java/org/apache/tajo/storage/NullTuple.java create mode 100644 tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/CommonHashJoinExec.java delete mode 100644 tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/NLLeftOuterJoinExec.java delete mode 100644 tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestLeftOuterNLJoinExec.java diff --git a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/Schema.java b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/Schema.java index fcbd177392..5ccc38f66f 100644 --- a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/Schema.java +++ b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/Schema.java @@ -311,6 +311,15 @@ public boolean containsAll(Collection columns) { return fields.containsAll(columns); } + public boolean containsAny(Collection columns) { + for (Column column : columns) { + if (fields.contains(column)) { + return true; + } + } + return false; + } + public synchronized Schema addColumn(String name, TypeDesc typeDesc) { String normalized = name; if(fieldsByQualifiedName.containsKey(normalized)) { @@ -383,6 +392,17 @@ public SchemaProto getProto() { return builder.build(); } + public Set getAliases() { + Set aliases = new HashSet(); + for (Column column : fields) { + if (column.hasQualifier()) { + String qualifier = column.getQualifier(); + aliases.add(qualifier.startsWith("default.") ? qualifier.substring(8) : qualifier); + } + } + return aliases; + } + private static class SchemaProtoBuilder implements ColumnVisitor { private SchemaProto.Builder builder; public SchemaProtoBuilder(SchemaProto.Builder builder) { diff --git a/tajo-common/src/main/java/org/apache/tajo/storage/EmptyTuple.java b/tajo-common/src/main/java/org/apache/tajo/storage/EmptyTuple.java index 89e72ed092..cdcebd7aa2 100644 --- a/tajo-common/src/main/java/org/apache/tajo/storage/EmptyTuple.java +++ b/tajo-common/src/main/java/org/apache/tajo/storage/EmptyTuple.java @@ -18,17 +18,12 @@ package org.apache.tajo.storage; -import org.apache.tajo.datum.Datum; -import org.apache.tajo.datum.NullDatum; -import org.apache.tajo.datum.ProtobufDatum; - /* This class doesn’t have content datum. if selected column is zero, this is useful * e.g. select count(*) from table * */ -public class EmptyTuple implements Tuple, Cloneable { +public class EmptyTuple extends NullTuple { private static EmptyTuple tuple; - private static Datum[] EMPTY_VALUES = new Datum[0]; static { tuple = new EmptyTuple(); @@ -39,138 +34,11 @@ public static EmptyTuple get() { } private EmptyTuple() { + super(0); } @Override - public int size() { - return 0; - } - - public boolean contains(int fieldId) { - return false; - } - - @Override - public boolean isNull(int fieldid) { - return true; - } - - @Override - public boolean isNotNull(int fieldid) { - return false; - } - - @Override - public void clear() { - } - - @Override - public void put(int fieldId, Datum value) { - throw new UnsupportedOperationException(); - } - - @Override - public void put(int fieldId, Datum[] values) { - throw new UnsupportedOperationException(); - } - - @Override - public void put(int fieldId, Tuple tuple) { - throw new UnsupportedOperationException(); - } - - @Override - public void put(Datum[] values) { - throw new UnsupportedOperationException(); - } - - @Override - public Datum get(int fieldId) { - return NullDatum.get(); - } - - @Override - public void setOffset(long offset) { - - } - - @Override - public long getOffset() { - return -1; - } - - @Override - public boolean getBool(int fieldId) { - return NullDatum.get().asBool(); - } - - @Override - public byte getByte(int fieldId) { - return NullDatum.get().asByte(); - } - - @Override - public char getChar(int fieldId) { - return NullDatum.get().asChar(); - } - - @Override - public byte[] getBytes(int fieldId) { - return NullDatum.get().asByteArray(); - } - - @Override - public short getInt2(int fieldId) { - return NullDatum.get().asInt2(); - } - - @Override - public int getInt4(int fieldId) { - return NullDatum.get().asInt4(); - } - - @Override - public long getInt8(int fieldId) { - return NullDatum.get().asInt8(); - } - - @Override - public float getFloat4(int fieldId) { - return NullDatum.get().asFloat4(); - } - - @Override - public double getFloat8(int fieldId) { - return NullDatum.get().asFloat8(); - } - - @Override - public String getText(int fieldId) { - return NullDatum.get().asChars(); - } - - @Override - public ProtobufDatum getProtobufDatum(int fieldId) { - throw new UnsupportedOperationException(); - } - - @Override - public Datum getInterval(int fieldId) { - return NullDatum.get(); - } - - @Override - public char[] getUnicodeChars(int fieldId) { - return NullDatum.get().asUnicodeChars(); - } - - @Override - public Tuple clone() throws CloneNotSupportedException { - throw new CloneNotSupportedException(); - } - - @Override - public Datum[] getValues() { - return EMPTY_VALUES; + public Tuple clone() { + return this; } } diff --git a/tajo-common/src/main/java/org/apache/tajo/storage/NullTuple.java b/tajo-common/src/main/java/org/apache/tajo/storage/NullTuple.java new file mode 100644 index 0000000000..8b990a8bbc --- /dev/null +++ b/tajo-common/src/main/java/org/apache/tajo/storage/NullTuple.java @@ -0,0 +1,172 @@ +/** + * 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.tajo.storage; + +import org.apache.tajo.datum.Datum; +import org.apache.tajo.datum.NullDatum; +import org.apache.tajo.datum.ProtobufDatum; + +import java.util.Arrays; + +public class NullTuple implements Tuple, Cloneable { + + public static Tuple create(int size) { + return new NullTuple(size); + } + + private final int size; + + NullTuple(int size) { + this.size = size; + } + + @Override + public int size() { + return size; + } + + public boolean contains(int fieldId) { + return fieldId < size; + } + + @Override + public boolean isNull(int fieldid) { + return true; + } + + @Override + public boolean isNotNull(int fieldid) { + return false; + } + + @Override + public void clear() { + } + + @Override + public void put(int fieldId, Datum value) { + throw new UnsupportedOperationException(); + } + + @Override + public void put(int fieldId, Datum[] values) { + throw new UnsupportedOperationException(); + } + + @Override + public void put(int fieldId, Tuple tuple) { + throw new UnsupportedOperationException(); + } + + @Override + public void put(Datum[] values) { + throw new UnsupportedOperationException(); + } + + @Override + public Datum get(int fieldId) { + return NullDatum.get(); + } + + @Override + public void setOffset(long offset) { + } + + @Override + public long getOffset() { + return 0; + } + + @Override + public boolean getBool(int fieldId) { + return NullDatum.get().asBool(); + } + + @Override + public byte getByte(int fieldId) { + return NullDatum.get().asByte(); + } + + @Override + public char getChar(int fieldId) { + return NullDatum.get().asChar(); + } + + @Override + public byte[] getBytes(int fieldId) { + return NullDatum.get().asByteArray(); + } + + @Override + public short getInt2(int fieldId) { + return NullDatum.get().asInt2(); + } + + @Override + public int getInt4(int fieldId) { + return NullDatum.get().asInt4(); + } + + @Override + public long getInt8(int fieldId) { + return NullDatum.get().asInt8(); + } + + @Override + public float getFloat4(int fieldId) { + return NullDatum.get().asFloat4(); + } + + @Override + public double getFloat8(int fieldId) { + return NullDatum.get().asFloat8(); + } + + @Override + public String getText(int fieldId) { + return NullDatum.get().asChars(); + } + + @Override + public ProtobufDatum getProtobufDatum(int fieldId) { + throw new UnsupportedOperationException(); + } + + @Override + public Datum getInterval(int fieldId) { + return NullDatum.get(); + } + + @Override + public char[] getUnicodeChars(int fieldId) { + return NullDatum.get().asUnicodeChars(); + } + + @Override + public Tuple clone() throws CloneNotSupportedException { + return new NullTuple(size); + } + + @Override + public Datum[] getValues() { + Datum[] datum = new Datum[size]; + Arrays.fill(datum, NullDatum.get()); + return datum; + } +} diff --git a/tajo-common/src/main/java/org/apache/tajo/storage/VTuple.java b/tajo-common/src/main/java/org/apache/tajo/storage/VTuple.java index 5e839b7e58..da69eb09eb 100644 --- a/tajo-common/src/main/java/org/apache/tajo/storage/VTuple.java +++ b/tajo-common/src/main/java/org/apache/tajo/storage/VTuple.java @@ -201,6 +201,7 @@ public Tuple clone() throws CloneNotSupportedException { return tuple; } + @Override public String toString() { return toDisplayString(getValues()); } @@ -225,22 +226,15 @@ public boolean equals(Object obj) { } public static String toDisplayString(Datum [] values) { - boolean first = true; StringBuilder str = new StringBuilder(); - str.append("("); - for(int i=0; i < values.length; i++) { - if(values[i] != null) { - if(first) { - first = false; - } else { - str.append(", "); - } - str.append(i) - .append("=>") - .append(values[i]); + str.append('('); + for (Datum datum : values) { + if (str.length() > 1) { + str.append(','); } + str.append(datum); } - str.append(")"); + str.append(')'); return str.toString(); } } diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/PhysicalPlannerImpl.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/PhysicalPlannerImpl.java index f1327932e6..9fa4295dd0 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/PhysicalPlannerImpl.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/PhysicalPlannerImpl.java @@ -465,14 +465,14 @@ private PhysicalExec createLeftOuterJoinPlan(TaskAttemptContext context, JoinNod case IN_MEMORY_HASH_JOIN: LOG.info("Left Outer Join (" + plan.getPID() +") chooses [Hash Join]."); return new HashLeftOuterJoinExec(context, plan, leftExec, rightExec); - case NESTED_LOOP_JOIN: - //the right operand is too large, so we opt for NL implementation of left outer join - LOG.info("Left Outer Join (" + plan.getPID() +") chooses [Nested Loop Join]."); - return new NLLeftOuterJoinExec(context, plan, leftExec, rightExec); + case MERGE_JOIN: + //the right operand is too large, so we opt for merge join implementation + LOG.info("Left Outer Join (" + plan.getPID() +") chooses [Merge Join]."); + return createRightOuterMergeJoinPlan(context, plan, rightExec, leftExec); default: LOG.error("Invalid Left Outer Join Algorithm Enforcer: " + algorithm.name()); - LOG.error("Choose a fallback inner join algorithm: " + JoinAlgorithm.IN_MEMORY_HASH_JOIN.name()); - return new HashLeftOuterJoinExec(context, plan, leftExec, rightExec); + LOG.error("Choose a fallback to join algorithm: " + JoinAlgorithm.MERGE_JOIN); + return createRightOuterMergeJoinPlan(context, plan, rightExec, leftExec); } } else { return createBestLeftOuterJoinPlan(context, plan, leftExec, rightExec); @@ -499,9 +499,9 @@ private PhysicalExec createBestLeftOuterJoinPlan(TaskAttemptContext context, Joi return new HashLeftOuterJoinExec(context, plan, leftExec, rightExec); } else { - //the right operand is too large, so we opt for NL implementation of left outer join - LOG.info("Left Outer Join (" + plan.getPID() +") chooses [Nested Loop Join]."); - return new NLLeftOuterJoinExec(context, plan, leftExec, rightExec); + //the right operand is too large, so we opt for merge join implementation + LOG.info("Left Outer Join (" + plan.getPID() +") chooses [Merge Join]."); + return createRightOuterMergeJoinPlan(context, plan, rightExec, leftExec); } } @@ -565,7 +565,7 @@ private PhysicalExec createRightOuterJoinPlan(TaskAttemptContext context, JoinNo return createRightOuterMergeJoinPlan(context, plan, leftExec, rightExec); default: LOG.error("Invalid Right Outer Join Algorithm Enforcer: " + algorithm.name()); - LOG.error("Choose a fallback merge join algorithm: " + JoinAlgorithm.MERGE_JOIN.name()); + LOG.error("Choose a fallback to join algorithm: " + JoinAlgorithm.MERGE_JOIN); return createRightOuterMergeJoinPlan(context, plan, leftExec, rightExec); } } else { @@ -588,7 +588,7 @@ private PhysicalExec createFullOuterJoinPlan(TaskAttemptContext context, JoinNod default: LOG.error("Invalid Full Outer Join Algorithm Enforcer: " + algorithm.name()); - LOG.error("Choose a fallback merge join algorithm: " + JoinAlgorithm.MERGE_JOIN.name()); + LOG.error("Choose a fallback to join algorithm: " + JoinAlgorithm.MERGE_JOIN); return createFullOuterMergeJoinPlan(context, plan, leftExec, rightExec); } } else { diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/BasicPhysicalExecutorVisitor.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/BasicPhysicalExecutorVisitor.java index 42611b05bd..c2d93bb16f 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/BasicPhysicalExecutorVisitor.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/BasicPhysicalExecutorVisitor.java @@ -65,8 +65,6 @@ public RESULT visit(PhysicalExec exec, Stack stack, CONTEXT contex return visitMergeJoin(context, (MergeJoinExec) exec, stack); } else if (exec instanceof NLJoinExec) { return visitNLJoin(context, (NLJoinExec) exec, stack); - } else if (exec instanceof NLLeftOuterJoinExec) { - return visitNLLeftOuterJoin(context, (NLLeftOuterJoinExec) exec, stack); } else if (exec instanceof ProjectionExec) { return visitProjection(context, (ProjectionExec) exec, stack); } else if (exec instanceof RangeShuffleFileWriteExec) { @@ -213,12 +211,6 @@ public RESULT visitNLJoin(CONTEXT context, NLJoinExec exec, Stack return visitBinaryExecutor(context, exec, stack); } - @Override - public RESULT visitNLLeftOuterJoin(CONTEXT context, NLLeftOuterJoinExec exec, Stack stack) - throws PhysicalPlanningException { - return visitBinaryExecutor(context, exec, stack); - } - @Override public RESULT visitProjection(CONTEXT context, ProjectionExec exec, Stack stack) throws PhysicalPlanningException { diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/CommonHashJoinExec.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/CommonHashJoinExec.java new file mode 100644 index 0000000000..df96fb5841 --- /dev/null +++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/CommonHashJoinExec.java @@ -0,0 +1,186 @@ +/** + * 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.tajo.engine.planner.physical; + +import org.apache.tajo.catalog.Column; +import org.apache.tajo.catalog.statistics.TableStats; +import org.apache.tajo.engine.utils.CacheHolder; +import org.apache.tajo.engine.utils.TableCacheKey; +import org.apache.tajo.plan.logical.JoinNode; +import org.apache.tajo.plan.util.PlannerUtil; +import org.apache.tajo.storage.Tuple; +import org.apache.tajo.storage.VTuple; +import org.apache.tajo.worker.ExecutionBlockSharedResource; +import org.apache.tajo.worker.TaskAttemptContext; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; + +public abstract class CommonHashJoinExec extends CommonJoinExec { + + protected final List joinKeyPairs; + + // temporal tuples and states for nested loop join + protected boolean first = true; + protected Map tupleSlots; + + protected Iterator iterator; + + protected final Tuple keyTuple; + + protected final int rightNumCols; + protected final int leftNumCols; + + protected final int[] leftKeyList; + protected final int[] rightKeyList; + + protected boolean finished; + + public CommonHashJoinExec(TaskAttemptContext context, JoinNode plan, PhysicalExec outer, PhysicalExec inner) { + super(context, plan, outer, inner); + + // HashJoin only can manage equi join key pairs. + this.joinKeyPairs = PlannerUtil.getJoinKeyPairs(joinQual, outer.getSchema(), + inner.getSchema(), false); + + leftKeyList = new int[joinKeyPairs.size()]; + rightKeyList = new int[joinKeyPairs.size()]; + + for (int i = 0; i < joinKeyPairs.size(); i++) { + leftKeyList[i] = outer.getSchema().getColumnId(joinKeyPairs.get(i)[0].getQualifiedName()); + } + + for (int i = 0; i < joinKeyPairs.size(); i++) { + rightKeyList[i] = inner.getSchema().getColumnId(joinKeyPairs.get(i)[1].getQualifiedName()); + } + + leftNumCols = outer.getSchema().size(); + rightNumCols = inner.getSchema().size(); + + keyTuple = new VTuple(leftKeyList.length); + } + + protected void loadRightToHashTable() throws IOException { + ScanExec scanExec = PhysicalPlanUtil.findExecutor(rightChild, ScanExec.class); + if (scanExec.canBroadcast()) { + /* If this table can broadcast, all tasks in a node will share the same cache */ + TableCacheKey key = CacheHolder.BroadcastCacheHolder.getCacheKey( + context, scanExec.getCanonicalName(), scanExec.getFragments()); + loadRightFromCache(key); + } else { + this.tupleSlots = convert(buildRightToHashTable(), false); + } + + first = false; + } + + protected void loadRightFromCache(TableCacheKey key) throws IOException { + ExecutionBlockSharedResource sharedResource = context.getSharedResource(); + + CacheHolder>> holder; + synchronized (sharedResource.getLock()) { + if (sharedResource.hasBroadcastCache(key)) { + holder = sharedResource.getBroadcastCache(key); + } else { + Map> built = buildRightToHashTable(); + holder = new CacheHolder.BroadcastCacheHolder(built, rightChild.getInputStats(), null); + sharedResource.addBroadcastCache(key, holder); + } + } + this.tupleSlots = convert(holder.getData(), true); + } + + protected Map> buildRightToHashTable() throws IOException { + Tuple tuple; + Map> map = new HashMap>(100000); + + while (!context.isStopped() && (tuple = rightChild.next()) != null) { + Tuple keyTuple = new VTuple(joinKeyPairs.size()); + for (int i = 0; i < rightKeyList.length; i++) { + keyTuple.put(i, tuple.get(rightKeyList[i])); + } + + List newValue = map.get(keyTuple); + if (newValue == null) { + map.put(keyTuple, newValue = new ArrayList()); + } + // if source is scan or groupby, it needs not to be cloned + newValue.add(new VTuple(tuple)); + } + return map; + } + + // todo: convert loaded data to cache condition + protected abstract Map convert(Map> hashed, boolean fromCache) + throws IOException; + + protected Tuple toKey(final Tuple outerTuple) { + for (int i = 0; i < leftKeyList.length; i++) { + keyTuple.put(i, outerTuple.get(leftKeyList[i])); + } + return keyTuple; + } + + @Override + public void rescan() throws IOException { + super.rescan(); + finished = false; + iterator = null; + } + + @Override + public void close() throws IOException { + super.close(); + iterator = null; + if (tupleSlots != null) { + tupleSlots.clear(); + tupleSlots = null; + } + } + + @Override + public TableStats getInputStats() { + if (leftChild == null) { + return inputStats; + } + TableStats leftInputStats = leftChild.getInputStats(); + inputStats.setNumBytes(0); + inputStats.setReadBytes(0); + inputStats.setNumRows(0); + + if (leftInputStats != null) { + inputStats.setNumBytes(leftInputStats.getNumBytes()); + inputStats.setReadBytes(leftInputStats.getReadBytes()); + inputStats.setNumRows(leftInputStats.getNumRows()); + } + + TableStats rightInputStats = rightChild.getInputStats(); + if (rightInputStats != null) { + inputStats.setNumBytes(inputStats.getNumBytes() + rightInputStats.getNumBytes()); + inputStats.setReadBytes(inputStats.getReadBytes() + rightInputStats.getReadBytes()); + inputStats.setNumRows(inputStats.getNumRows() + rightInputStats.getNumRows()); + } + + return inputStats; + } +} diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/CommonJoinExec.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/CommonJoinExec.java index 2535edffdc..a743407b32 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/CommonJoinExec.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/CommonJoinExec.java @@ -18,36 +18,137 @@ package org.apache.tajo.engine.planner.physical; +import com.google.common.base.Predicate; +import com.google.common.collect.Iterators; +import com.google.common.collect.Lists; +import org.apache.tajo.catalog.Column; +import org.apache.tajo.catalog.Schema; import org.apache.tajo.catalog.SchemaUtil; import org.apache.tajo.engine.planner.Projector; +import org.apache.tajo.plan.expr.AlgebraicUtil; +import org.apache.tajo.plan.expr.BinaryEval; import org.apache.tajo.plan.expr.EvalNode; +import org.apache.tajo.plan.expr.EvalTreeUtil; import org.apache.tajo.plan.logical.JoinNode; +import org.apache.tajo.storage.FrameTuple; +import org.apache.tajo.storage.NullTuple; +import org.apache.tajo.storage.Tuple; +import org.apache.tajo.storage.VTuple; import org.apache.tajo.worker.TaskAttemptContext; import java.io.IOException; +import java.util.Arrays; +import java.util.Iterator; +import java.util.LinkedHashSet; +import java.util.List; -// common join exec except HashLeftOuterJoinExec +// common exec for all join execs public abstract class CommonJoinExec extends BinaryPhysicalExec { // from logical plan protected JoinNode plan; protected final boolean hasJoinQual; - protected EvalNode joinQual; + protected EvalNode joinQual; // ex) a.id = b.id + protected EvalNode leftJoinFilter; // ex) a > 10 + protected EvalNode rightJoinFilter; // ex) b > 5 + + protected final Schema leftSchema; + protected final Schema rightSchema; + + protected final FrameTuple frameTuple; + protected final Tuple outTuple; // projection protected Projector projector; - public CommonJoinExec(TaskAttemptContext context, JoinNode plan, PhysicalExec outer, - PhysicalExec inner) { + public CommonJoinExec(TaskAttemptContext context, JoinNode plan, PhysicalExec outer, PhysicalExec inner) { super(context, SchemaUtil.merge(outer.getSchema(), inner.getSchema()), plan.getOutSchema(), outer, inner); this.plan = plan; - this.joinQual = plan.getJoinQual(); - this.hasJoinQual = plan.hasJoinQual(); + this.leftSchema = outer.getSchema(); + this.rightSchema = inner.getSchema(); + if (plan.hasJoinQual()) { + EvalNode[] extracted = extractJoinConditions(plan.getJoinQual(), leftSchema, rightSchema); + joinQual = extracted[0]; + leftJoinFilter = extracted[1]; + rightJoinFilter = extracted[2]; + } + this.hasJoinQual = joinQual != null; // for projection this.projector = new Projector(context, inSchema, outSchema, plan.getTargets()); + + // for join + this.frameTuple = new FrameTuple(); + this.outTuple = new VTuple(outSchema.size()); + } + + private EvalNode[] extractJoinConditions(EvalNode joinQual, Schema leftSchema, Schema rightSchema) { + List joinQuals = Lists.newArrayList(); + List leftFilters = Lists.newArrayList(); + List rightFilters = Lists.newArrayList(); + for (EvalNode eachQual : AlgebraicUtil.toConjunctiveNormalFormArray(joinQual)) { + if (EvalTreeUtil.isJoinQual(null, leftSchema, rightSchema, eachQual, true)) { + joinQuals.add(eachQual); + continue; + } + if (!(eachQual instanceof BinaryEval)) { + continue; + } + BinaryEval binaryEval = (BinaryEval)eachQual; + LinkedHashSet leftColumns = EvalTreeUtil.findUniqueColumns(binaryEval.getLeftExpr()); + LinkedHashSet rightColumns = EvalTreeUtil.findUniqueColumns(binaryEval.getRightExpr()); + boolean leftInLeft = leftSchema.containsAny(leftColumns); + boolean rightInLeft = leftSchema.containsAny(rightColumns); + boolean leftInRight = rightSchema.containsAny(leftColumns); + boolean rightInRight = rightSchema.containsAny(rightColumns); + if ((leftInLeft || leftInRight) && (rightInLeft || rightInRight)) { +// throw new IllegalStateException("Invalid filter " + binaryEval.toString()); + continue; // todo this happens sometimes. why? + } + if (leftInLeft || rightInLeft) { + leftFilters.add(eachQual); + } else if (leftInRight || rightInRight) { + rightFilters.add(eachQual); + } + } + return new EvalNode[] { + joinQuals.isEmpty() ? null : AlgebraicUtil.createSingletonExprFromCNF(joinQuals), + leftFilters.isEmpty() ? null : AlgebraicUtil.createSingletonExprFromCNF(leftFilters), + rightFilters.isEmpty() ? null : AlgebraicUtil.createSingletonExprFromCNF(rightFilters) + }; + } + + public JoinNode getPlan() { + return plan; + } + + protected boolean leftFiltered(Tuple left) { + return leftJoinFilter != null && !leftJoinFilter.eval(left).asBool(); + } + + protected boolean rightFiltered(Tuple right) { + return rightJoinFilter != null && !rightJoinFilter.eval(right).asBool(); + } + + protected Iterator rightFiltered(Iterable rightTuples) { + if (rightTuples == null) { + return Iterators.emptyIterator(); + } + if (rightJoinFilter == null) { + return rightTuples.iterator(); + } + return Iterators.filter(rightTuples.iterator(), new Predicate() { + @Override + public boolean apply(Tuple input) { + return rightJoinFilter.eval(input).asBool(); + } + }); + } + + protected Iterator nullIterator(int length) { + return Arrays.asList(NullTuple.create(length)).iterator(); } @Override @@ -56,6 +157,12 @@ public void init() throws IOException { if (hasJoinQual) { joinQual.bind(context.getEvalContext(), inSchema); } + if (leftJoinFilter != null) { + leftJoinFilter.bind(leftSchema); + } + if (rightJoinFilter != null) { + rightJoinFilter.bind(rightSchema); + } } @Override @@ -63,10 +170,7 @@ protected void compile() { if (hasJoinQual) { joinQual = context.getPrecompiledEval(inSchema, joinQual); } - } - - public JoinNode getPlan() { - return plan; + // compile filters? } @Override @@ -74,6 +178,14 @@ public void close() throws IOException { super.close(); plan = null; joinQual = null; + leftJoinFilter = null; + rightJoinFilter = null; projector = null; } + + @Override + public String toString() { + return getClass().getSimpleName() + + " [" + leftSchema.getAliases() + " : " + rightSchema.getAliases() + "]"; + } } diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/HashFullOuterJoinExec.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/HashFullOuterJoinExec.java index 6e28ae0f55..e2b0a866cd 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/HashFullOuterJoinExec.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/HashFullOuterJoinExec.java @@ -18,101 +18,59 @@ package org.apache.tajo.engine.planner.physical; -import org.apache.tajo.catalog.Column; -import org.apache.tajo.engine.utils.TupleUtil; -import org.apache.tajo.plan.util.PlannerUtil; import org.apache.tajo.plan.logical.JoinNode; -import org.apache.tajo.storage.FrameTuple; +import org.apache.tajo.storage.NullTuple; import org.apache.tajo.storage.Tuple; -import org.apache.tajo.storage.VTuple; +import org.apache.tajo.util.Pair; import org.apache.tajo.worker.TaskAttemptContext; import java.io.IOException; import java.util.*; +public class HashFullOuterJoinExec extends CommonHashJoinExec>> { -public class HashFullOuterJoinExec extends CommonJoinExec { - - protected List joinKeyPairs; - - // temporal tuples and states for nested loop join - protected boolean first = true; - protected FrameTuple frameTuple; - protected Tuple outTuple = null; - protected Map> tupleSlots; - protected Iterator iterator = null; - protected Tuple leftTuple; - protected Tuple leftKeyTuple; - - protected int [] leftKeyList; - protected int [] rightKeyList; - - protected boolean finished = false; - protected boolean shouldGetLeftTuple = true; - - private int rightNumCols; - private int leftNumCols; - private Map matched; + private boolean finalLoop; // final loop for right unmatched public HashFullOuterJoinExec(TaskAttemptContext context, JoinNode plan, PhysicalExec outer, PhysicalExec inner) { super(context, plan, outer, inner); - this.tupleSlots = new HashMap>(10000); - - // this hashmap mirrors the evolution of the tupleSlots, with the same keys. For each join key, - // we have a boolean flag, initially false (whether this join key had at least one match on the left operand) - this.matched = new HashMap(10000); - - // HashJoin only can manage equi join key pairs. - this.joinKeyPairs = PlannerUtil.getJoinKeyPairs(joinQual, outer.getSchema(), inner.getSchema(), - false); - - leftKeyList = new int[joinKeyPairs.size()]; - rightKeyList = new int[joinKeyPairs.size()]; - - for (int i = 0; i < joinKeyPairs.size(); i++) { - leftKeyList[i] = outer.getSchema().getColumnId(joinKeyPairs.get(i)[0].getQualifiedName()); - } - - for (int i = 0; i < joinKeyPairs.size(); i++) { - rightKeyList[i] = inner.getSchema().getColumnId(joinKeyPairs.get(i)[1].getQualifiedName()); - } - - // for join - frameTuple = new FrameTuple(); - outTuple = new VTuple(outSchema.size()); - leftKeyTuple = new VTuple(leftKeyList.length); - - leftNumCols = outer.getSchema().size(); - rightNumCols = inner.getSchema().size(); } - protected void getKeyLeftTuple(final Tuple outerTuple, Tuple keyTuple) { - for (int i = 0; i < leftKeyList.length; i++) { - keyTuple.put(i, outerTuple.get(leftKeyList[i])); - } - } + public Iterator getUnmatchedRight() { - public Tuple getNextUnmatchedRight() { + return new Iterator() { - List newValue; - Tuple returnedTuple; - // get a keyTUple from the matched hashmap with a boolean false value - for(Tuple aKeyTuple : matched.keySet()) { - if(matched.get(aKeyTuple) == false) { - newValue = tupleSlots.get(aKeyTuple); - returnedTuple = newValue.remove(0); - tupleSlots.put(aKeyTuple, newValue); + private Iterator>> iterator1 = tupleSlots.values().iterator(); + private Iterator iterator2; - // after taking the last element from the list in tupleSlots, set flag true in matched as well - if(newValue.isEmpty()){ - matched.put(aKeyTuple, true); + @Override + public boolean hasNext() { + if (hasMore()) { + return true; } + for (iterator2 = null; !hasMore() && iterator1.hasNext();) { + Pair> next = iterator1.next(); + if (!next.getFirst()) { + iterator2 = next.getSecond().iterator(); + } + } + return hasMore(); + } - return returnedTuple; + private boolean hasMore() { + return iterator2 != null && iterator2.hasNext(); } - } - return null; + + @Override + public Tuple next() { + return iterator2.next(); + } + + @Override + public void remove() { + throw new UnsupportedOperationException("remove"); + } + }; } public Tuple next() throws IOException { @@ -120,112 +78,67 @@ public Tuple next() throws IOException { loadRightToHashTable(); } - Tuple rightTuple; - boolean found = false; - - while(!context.isStopped() && !finished) { - if (shouldGetLeftTuple) { // initially, it is true. - // getting new outer - leftTuple = leftChild.next(); // it comes from a disk - if (leftTuple == null) { // if no more tuples in left tuples on disk, a join is completed. - // in this stage we can begin outputing tuples from the right operand (which were before in tupleSlots) null padded on the left side - Tuple unmatchedRightTuple = getNextUnmatchedRight(); - if( unmatchedRightTuple == null) { - finished = true; - outTuple = null; - return null; - } else { - Tuple nullPaddedTuple = TupleUtil.createNullPaddedTuple(leftNumCols); - frameTuple.set(nullPaddedTuple, unmatchedRightTuple); - projector.eval(frameTuple, outTuple); - - return outTuple; - } - } - - // getting corresponding right - getKeyLeftTuple(leftTuple, leftKeyTuple); // get a left key tuple - List rightTuples = tupleSlots.get(leftKeyTuple); - if (rightTuples != null) { // found right tuples on in-memory hash table. - iterator = rightTuples.iterator(); - shouldGetLeftTuple = false; - } else { - //this left tuple doesn't have a match on the right.But full outer join => we should keep it anyway - //output a tuple with the nulls padded rightTuple - Tuple nullPaddedTuple = TupleUtil.createNullPaddedTuple(rightNumCols); - frameTuple.set(leftTuple, nullPaddedTuple); - projector.eval(frameTuple, outTuple); - // we simulate we found a match, which is exactly the null padded one - shouldGetLeftTuple = true; - return outTuple; - } - } - - // getting a next right tuple on in-memory hash table. - rightTuple = iterator.next(); - frameTuple.set(leftTuple, rightTuple); // evaluate a join condition on both tuples - - if (joinQual.eval(frameTuple).isTrue()) { // if both tuples are joinable + while (!context.isStopped() && !finished) { + if (iterator != null && iterator.hasNext()) { + frameTuple.setRight(iterator.next()); projector.eval(frameTuple, outTuple); - found = true; - getKeyLeftTuple(leftTuple, leftKeyTuple); - matched.put(leftKeyTuple, true); + return outTuple; } - - if (!iterator.hasNext()) { // no more right tuples for this hash key - shouldGetLeftTuple = true; + if (finalLoop) { + finished = true; + return null; } - - if (found) { - break; + Tuple leftTuple = leftChild.next(); + if (leftTuple == null) { + // if no more tuples in left tuples, a join is completed. + // in this stage we can begin outputing tuples from the right operand (which were before in tupleSlots) null padded on the left side + frameTuple.setLeft(NullTuple.create(leftNumCols)); + iterator = getUnmatchedRight(); + finalLoop = true; + continue; } - } - return outTuple; - } - - protected void loadRightToHashTable() throws IOException { - Tuple tuple; - Tuple keyTuple; + frameTuple.setLeft(leftTuple); - while (!context.isStopped() && (tuple = rightChild.next()) != null) { - keyTuple = new VTuple(joinKeyPairs.size()); - for (int i = 0; i < rightKeyList.length; i++) { - keyTuple.put(i, tuple.get(rightKeyList[i])); + if (leftFiltered(leftTuple)) { + iterator = nullIterator(rightNumCols); + continue; } - - List newValue = tupleSlots.get(keyTuple); - if (newValue != null) { - newValue.add(tuple); - } else { - newValue = new ArrayList(); - newValue.add(tuple); - tupleSlots.put(keyTuple, newValue); - matched.put(keyTuple,false); + // getting corresponding right + Pair> hashed = tupleSlots.get(toKey(leftTuple)); + if (hashed == null) { + iterator = nullIterator(rightNumCols); + continue; + } + Iterator rightTuples = rightFiltered(hashed.getSecond()); + if (!rightTuples.hasNext()) { + iterator = nullIterator(rightNumCols); + continue; } + iterator = rightTuples; + hashed.setFirst(true); // match found } - first = false; + + return null; } @Override - public void rescan() throws IOException { - super.rescan(); - - tupleSlots.clear(); - first = true; - - finished = false; - iterator = null; - shouldGetLeftTuple = true; + protected Map>> convert(Map> hashed, boolean fromCache) + throws IOException { + Map>> tuples = new HashMap>>(hashed.size()); + for (Map.Entry> entry : hashed.entrySet()) { + // flag: initially false (whether this join key had at least one match on the counter part) + tuples.put(entry.getKey(), new Pair>(false, entry.getValue())); + } + return tuples; } @Override - public void close() throws IOException { - super.close(); - tupleSlots.clear(); - matched.clear(); - tupleSlots = null; - matched = null; - iterator = null; + public void rescan() throws IOException { + super.rescan(); + for (Pair> value : tupleSlots.values()) { + value.setFirst(false); + } + finalLoop = false; } } diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/HashJoinExec.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/HashJoinExec.java index 48f3682610..a4215fafb4 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/HashJoinExec.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/HashJoinExec.java @@ -18,225 +18,59 @@ package org.apache.tajo.engine.planner.physical; -import org.apache.tajo.catalog.Column; -import org.apache.tajo.catalog.statistics.TableStats; -import org.apache.tajo.engine.utils.CacheHolder; -import org.apache.tajo.engine.utils.TableCacheKey; import org.apache.tajo.plan.logical.JoinNode; -import org.apache.tajo.plan.util.PlannerUtil; -import org.apache.tajo.storage.FrameTuple; import org.apache.tajo.storage.Tuple; -import org.apache.tajo.storage.VTuple; -import org.apache.tajo.worker.ExecutionBlockSharedResource; import org.apache.tajo.worker.TaskAttemptContext; import java.io.IOException; import java.util.*; -public class HashJoinExec extends CommonJoinExec { - - protected List joinKeyPairs; - - // temporal tuples and states for nested loop join - protected boolean first = true; - protected FrameTuple frameTuple; - protected Tuple outTuple = null; - protected Map> tupleSlots; - protected Iterator iterator = null; - protected Tuple leftTuple; - protected Tuple leftKeyTuple; - - protected int [] leftKeyList; - protected int [] rightKeyList; - - protected boolean finished = false; - protected boolean shouldGetLeftTuple = true; - - private TableStats cachedRightTableStats; +public class HashJoinExec extends CommonHashJoinExec> { public HashJoinExec(TaskAttemptContext context, JoinNode plan, PhysicalExec leftExec, PhysicalExec rightExec) { super(context, plan, leftExec, rightExec); - - // HashJoin only can manage equi join key pairs. - this.joinKeyPairs = PlannerUtil.getJoinKeyPairs(joinQual, leftExec.getSchema(), - rightExec.getSchema(), false); - - leftKeyList = new int[joinKeyPairs.size()]; - rightKeyList = new int[joinKeyPairs.size()]; - - for (int i = 0; i < joinKeyPairs.size(); i++) { - leftKeyList[i] = leftExec.getSchema().getColumnId(joinKeyPairs.get(i)[0].getQualifiedName()); - } - - for (int i = 0; i < joinKeyPairs.size(); i++) { - rightKeyList[i] = rightExec.getSchema().getColumnId(joinKeyPairs.get(i)[1].getQualifiedName()); - } - - // for join - frameTuple = new FrameTuple(); - outTuple = new VTuple(outSchema.size()); - leftKeyTuple = new VTuple(leftKeyList.length); } - protected void getKeyLeftTuple(final Tuple outerTuple, Tuple keyTuple) { - for (int i = 0; i < leftKeyList.length; i++) { - keyTuple.put(i, outerTuple.get(leftKeyList[i])); - } + @Override + protected Map> convert(Map> hashed, boolean fromCache) + throws IOException { + return fromCache ? new HashMap>(hashed) : hashed; } + @Override public Tuple next() throws IOException { if (first) { loadRightToHashTable(); } - Tuple rightTuple; - boolean found = false; - - while(!context.isStopped() && !finished) { - if (shouldGetLeftTuple) { // initially, it is true. - // getting new outer - leftTuple = leftChild.next(); // it comes from a disk - if (leftTuple == null) { // if no more tuples in left tuples on disk, a join is completed. - finished = true; - return null; - } - - // getting corresponding right - getKeyLeftTuple(leftTuple, leftKeyTuple); // get a left key tuple - List rightTuples = tupleSlots.get(leftKeyTuple); - if (rightTuples != null) { // found right tuples on in-memory hash table. - iterator = rightTuples.iterator(); - shouldGetLeftTuple = false; - } else { - shouldGetLeftTuple = true; - continue; - } - } - - // getting a next right tuple on in-memory hash table. - rightTuple = iterator.next(); - frameTuple.set(leftTuple, rightTuple); // evaluate a join condition on both tuples - if (joinQual.eval(frameTuple).isTrue()) { // if both tuples are joinable + while (!context.isStopped() && !finished) { + if (iterator != null && iterator.hasNext()) { + frameTuple.setRight(iterator.next()); projector.eval(frameTuple, outTuple); - found = true; - } - - if (!iterator.hasNext()) { // no more right tuples for this hash key - shouldGetLeftTuple = true; - } - - if (found) { - break; - } - } - - return new VTuple(outTuple); - } - - protected void loadRightToHashTable() throws IOException { - ScanExec scanExec = PhysicalPlanUtil.findExecutor(rightChild, ScanExec.class); - if (scanExec.canBroadcast()) { - /* If this table can broadcast, all tasks in a node will share the same cache */ - TableCacheKey key = CacheHolder.BroadcastCacheHolder.getCacheKey( - context, scanExec.getCanonicalName(), scanExec.getFragments()); - loadRightFromCache(key); - } else { - this.tupleSlots = buildRightToHashTable(); - } - - first = false; - } - - protected void loadRightFromCache(TableCacheKey key) throws IOException { - ExecutionBlockSharedResource sharedResource = context.getSharedResource(); - synchronized (sharedResource.getLock()) { - if (sharedResource.hasBroadcastCache(key)) { - CacheHolder>> data = sharedResource.getBroadcastCache(key); - this.tupleSlots = data.getData(); - this.cachedRightTableStats = data.getTableStats(); - } else { - CacheHolder.BroadcastCacheHolder holder = - new CacheHolder.BroadcastCacheHolder(buildRightToHashTable(), rightChild.getInputStats(), null); - sharedResource.addBroadcastCache(key, holder); - CacheHolder>> data = sharedResource.getBroadcastCache(key); - this.tupleSlots = data.getData(); - this.cachedRightTableStats = data.getTableStats(); + return outTuple; } - } - } - - private Map> buildRightToHashTable() throws IOException { - Tuple tuple; - Tuple keyTuple; - Map> map = new HashMap>(100000); - - while (!context.isStopped() && (tuple = rightChild.next()) != null) { - keyTuple = new VTuple(joinKeyPairs.size()); - for (int i = 0; i < rightKeyList.length; i++) { - keyTuple.put(i, tuple.get(rightKeyList[i])); + Tuple leftTuple = leftChild.next(); // it comes from a disk + if (leftTuple == null || leftFiltered(leftTuple)) { // if no more tuples in left tuples on disk, a join is completed. + finished = leftTuple == null; + continue; } - List newValue = map.get(keyTuple); + frameTuple.setLeft(leftTuple); - if (newValue != null) { - newValue.add(tuple); - } else { - newValue = new ArrayList(); - newValue.add(tuple); - map.put(keyTuple, newValue); + // getting corresponding right + Iterable hashed = getRights(toKey(leftTuple)); + Iterator rightTuples = rightFiltered(hashed); + if (rightTuples.hasNext()) { + iterator = rightTuples; } } - return map; - } - - @Override - public void rescan() throws IOException { - super.rescan(); - - tupleSlots.clear(); - first = true; - - finished = false; - iterator = null; - shouldGetLeftTuple = true; + return null; } - @Override - public void close() throws IOException { - super.close(); - if (tupleSlots != null) { - tupleSlots.clear(); - tupleSlots = null; - } - - iterator = null; + private Iterable getRights(Tuple key) { + return tupleSlots.get(key); } - @Override - public TableStats getInputStats() { - if (leftChild == null) { - return inputStats; - } - TableStats leftInputStats = leftChild.getInputStats(); - inputStats.setNumBytes(0); - inputStats.setReadBytes(0); - inputStats.setNumRows(0); - - if (leftInputStats != null) { - inputStats.setNumBytes(leftInputStats.getNumBytes()); - inputStats.setReadBytes(leftInputStats.getReadBytes()); - inputStats.setNumRows(leftInputStats.getNumRows()); - } - - TableStats rightInputStats = cachedRightTableStats == null ? rightChild.getInputStats() : cachedRightTableStats; - if (rightInputStats != null) { - inputStats.setNumBytes(inputStats.getNumBytes() + rightInputStats.getNumBytes()); - inputStats.setReadBytes(inputStats.getReadBytes() + rightInputStats.getReadBytes()); - inputStats.setNumRows(inputStats.getNumRows() + rightInputStats.getNumRows()); - } - - return inputStats; - } } diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/HashLeftAntiJoinExec.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/HashLeftAntiJoinExec.java index 881bf84605..8239270580 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/HashLeftAntiJoinExec.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/HashLeftAntiJoinExec.java @@ -19,10 +19,8 @@ package org.apache.tajo.engine.planner.physical; import org.apache.tajo.worker.TaskAttemptContext; -import org.apache.tajo.datum.NullDatum; import org.apache.tajo.plan.logical.JoinNode; import org.apache.tajo.storage.Tuple; -import org.apache.tajo.storage.VTuple; import java.io.IOException; import java.util.List; @@ -33,16 +31,10 @@ * If not found, it returns the tuple of the FROM side table with null padding. */ public class HashLeftAntiJoinExec extends HashJoinExec { - private Tuple rightNullTuple; public HashLeftAntiJoinExec(TaskAttemptContext context, JoinNode plan, PhysicalExec fromSideChild, PhysicalExec notInSideChild) { super(context, plan, fromSideChild, notInSideChild); - // NUll Tuple - rightNullTuple = new VTuple(leftChild.outColumnNum); - for (int i = 0; i < leftChild.outColumnNum; i++) { - rightNullTuple.put(i, NullDatum.get()); - } } /** @@ -56,54 +48,33 @@ public HashLeftAntiJoinExec(TaskAttemptContext context, JoinNode plan, PhysicalE * @return The tuple which is unmatched to a given join condition. * @throws IOException */ + @Override public Tuple next() throws IOException { if (first) { loadRightToHashTable(); } - Tuple rightTuple; - boolean notFound; - while(!context.isStopped() && !finished) { - - // getting new outer - leftTuple = leftChild.next(); // it comes from a disk - if (leftTuple == null) { // if no more tuples in left tuples on disk, a join is completed. - finished = true; - return null; - } - - // Try to find a hash bucket in in-memory hash table - getKeyLeftTuple(leftTuple, leftKeyTuple); - List rightTuples = tupleSlots.get(leftKeyTuple); - if (rightTuples != null) { - // if found, it gets a hash bucket from the hash table. - iterator = rightTuples.iterator(); - } else { - // if not found, it returns a tuple. - frameTuple.set(leftTuple, rightNullTuple); + if (iterator != null && iterator.hasNext()) { + frameTuple.setRight(iterator.next()); projector.eval(frameTuple, outTuple); return outTuple; } - - // Reach here only when a hash bucket is found. Then, it checks all tuples in the found bucket. - // If it finds a matched tuple, it escapes the loop for all tuples in the hash bucket. - notFound = true; - while (!context.isStopped() && notFound && iterator.hasNext()) { - rightTuple = iterator.next(); - frameTuple.set(leftTuple, rightTuple); - if (joinQual.eval(frameTuple).isTrue()) { // if the matched one is found - notFound = false; - } + // getting new outer + Tuple leftTuple = leftChild.next(); // it comes from a disk + if (leftTuple == null || leftFiltered(leftTuple)) { // if no more tuples in left tuples on disk, a join is completed. + finished = leftTuple == null; + continue; } - if (notFound) { // if there is no matched tuple - frameTuple.set(leftTuple, rightNullTuple); - projector.eval(frameTuple, outTuple); - break; + frameTuple.setLeft(leftTuple); + + // Try to find a hash bucket in in-memory hash table + List hashed = tupleSlots.get(toKey(leftTuple)); + if (hashed == null || !rightFiltered(hashed).hasNext()) { + iterator = nullIterator(0); } } - - return outTuple; + return null; } } diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/HashLeftOuterJoinExec.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/HashLeftOuterJoinExec.java index 6f573d0e06..8613eacb7b 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/HashLeftOuterJoinExec.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/HashLeftOuterJoinExec.java @@ -18,307 +18,61 @@ package org.apache.tajo.engine.planner.physical; -import com.google.common.collect.Lists; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; -import org.apache.tajo.catalog.Column; -import org.apache.tajo.catalog.statistics.TableStats; -import org.apache.tajo.engine.planner.Projector; -import org.apache.tajo.engine.utils.CacheHolder; -import org.apache.tajo.engine.utils.TableCacheKey; -import org.apache.tajo.engine.utils.TupleUtil; -import org.apache.tajo.plan.util.PlannerUtil; -import org.apache.tajo.catalog.SchemaUtil; -import org.apache.tajo.plan.expr.AlgebraicUtil; -import org.apache.tajo.plan.expr.EvalNode; -import org.apache.tajo.plan.expr.EvalTreeUtil; import org.apache.tajo.plan.logical.JoinNode; -import org.apache.tajo.storage.FrameTuple; import org.apache.tajo.storage.Tuple; -import org.apache.tajo.storage.VTuple; -import org.apache.tajo.worker.ExecutionBlockSharedResource; import org.apache.tajo.worker.TaskAttemptContext; import java.io.IOException; import java.util.*; +public class HashLeftOuterJoinExec extends HashJoinExec { -public class HashLeftOuterJoinExec extends BinaryPhysicalExec { - // from logical plan - protected JoinNode plan; - protected EvalNode joinQual; // ex) a.id = b.id - protected EvalNode joinFilter; // ex) a > 10 - - protected List joinKeyPairs; - - // temporal tuples and states for nested loop join - protected boolean first = true; - protected FrameTuple frameTuple; - protected Tuple outTuple = null; - protected Map> tupleSlots; - protected Iterator iterator = null; - protected Tuple leftTuple; - protected Tuple leftKeyTuple; - - protected int [] leftKeyList; - protected int [] rightKeyList; - - protected boolean finished = false; - protected boolean shouldGetLeftTuple = true; - - // projection - protected Projector projector; - - private int rightNumCols; - private TableStats cachedRightTableStats; private static final Log LOG = LogFactory.getLog(HashLeftOuterJoinExec.class); public HashLeftOuterJoinExec(TaskAttemptContext context, JoinNode plan, PhysicalExec leftChild, PhysicalExec rightChild) { - super(context, SchemaUtil.merge(leftChild.getSchema(), rightChild.getSchema()), - plan.getOutSchema(), leftChild, rightChild); - this.plan = plan; - - List joinQuals = Lists.newArrayList(); - List joinFilters = Lists.newArrayList(); - for (EvalNode eachQual : AlgebraicUtil.toConjunctiveNormalFormArray(plan.getJoinQual())) { - if (EvalTreeUtil.isJoinQual(eachQual, true)) { - joinQuals.add(eachQual); - } else { - joinFilters.add(eachQual); - } - } - - this.joinQual = AlgebraicUtil.createSingletonExprFromCNF(joinQuals.toArray(new EvalNode[joinQuals.size()])); - if (joinFilters.size() > 0) { - this.joinFilter = AlgebraicUtil.createSingletonExprFromCNF(joinFilters.toArray(new EvalNode[joinFilters.size()])); - } else { - this.joinFilter = null; - } - - // HashJoin only can manage equi join key pairs. - this.joinKeyPairs = PlannerUtil.getJoinKeyPairs(joinQual, leftChild.getSchema(), - rightChild.getSchema(), false); - - leftKeyList = new int[joinKeyPairs.size()]; - rightKeyList = new int[joinKeyPairs.size()]; - - for (int i = 0; i < joinKeyPairs.size(); i++) { - leftKeyList[i] = leftChild.getSchema().getColumnId(joinKeyPairs.get(i)[0].getQualifiedName()); - } - - for (int i = 0; i < joinKeyPairs.size(); i++) { - rightKeyList[i] = rightChild.getSchema().getColumnId(joinKeyPairs.get(i)[1].getQualifiedName()); - } - - // for projection - this.projector = new Projector(context, inSchema, outSchema, plan.getTargets()); - - // for join - frameTuple = new FrameTuple(); - outTuple = new VTuple(outSchema.size()); - leftKeyTuple = new VTuple(leftKeyList.length); - - rightNumCols = rightChild.getSchema().size(); - - joinQual.bind(context.getEvalContext(), inSchema); - if (joinFilter != null) { - joinFilter.bind(context.getEvalContext(), inSchema); - } + super(context, plan, leftChild, rightChild); } @Override - protected void compile() { - joinQual = context.getPrecompiledEval(inSchema, joinQual); - } - - protected void getKeyLeftTuple(final Tuple outerTuple, Tuple keyTuple) { - for (int i = 0; i < leftKeyList.length; i++) { - keyTuple.put(i, outerTuple.get(leftKeyList[i])); - } - } - public Tuple next() throws IOException { if (first) { loadRightToHashTable(); } - Tuple rightTuple; - boolean found = false; - - while(!context.isStopped() && !finished) { - - if (shouldGetLeftTuple) { // initially, it is true. - // getting new outer - leftTuple = leftChild.next(); // it comes from a disk - if (leftTuple == null) { // if no more tuples in left tuples on disk, a join is completed. - finished = true; - return null; - } - - // getting corresponding right - getKeyLeftTuple(leftTuple, leftKeyTuple); // get a left key tuple - List rightTuples = tupleSlots.get(leftKeyTuple); - if (rightTuples != null) { // found right tuples on in-memory hash table. - iterator = rightTuples.iterator(); - shouldGetLeftTuple = false; - } else { - // this left tuple doesn't have a match on the right, and output a tuple with the nulls padded rightTuple - Tuple nullPaddedTuple = TupleUtil.createNullPaddedTuple(rightNumCols); - frameTuple.set(leftTuple, nullPaddedTuple); - projector.eval(frameTuple, outTuple); - // we simulate we found a match, which is exactly the null padded one - shouldGetLeftTuple = true; - return outTuple; - } - } - - // getting a next right tuple on in-memory hash table. - rightTuple = iterator.next(); - if (!iterator.hasNext()) { // no more right tuples for this hash key - shouldGetLeftTuple = true; - } - - frameTuple.set(leftTuple, rightTuple); // evaluate a join condition on both tuples - - // if there is no join filter, it is always true. - boolean satisfiedWithFilter = joinFilter == null || joinFilter.eval(frameTuple).isTrue(); - boolean satisfiedWithJoinCondition = joinQual.eval(frameTuple).isTrue(); - - // if a composited tuple satisfies with both join filter and join condition - if (satisfiedWithJoinCondition && satisfiedWithFilter) { - projector.eval(frameTuple, outTuple); - return outTuple; - } else { - - // if join filter is satisfied, the left outer join (LOJ) operator should return the null padded tuple - // only once. Then, LOJ operator should take the next left tuple. - if (!satisfiedWithFilter) { - shouldGetLeftTuple = true; - } - - // null padding - Tuple nullPaddedTuple = TupleUtil.createNullPaddedTuple(rightNumCols); - frameTuple.set(leftTuple, nullPaddedTuple); - + while (!context.isStopped() && !finished) { + if (iterator != null && iterator.hasNext()) { + frameTuple.setRight(iterator.next()); projector.eval(frameTuple, outTuple); return outTuple; } - } - - return outTuple; - } - - protected void loadRightToHashTable() throws IOException { - ScanExec scanExec = PhysicalPlanUtil.findExecutor(rightChild, ScanExec.class); - if (scanExec.canBroadcast()) { - /* If this table can broadcast, all tasks in a node will share the same cache */ - TableCacheKey key = CacheHolder.BroadcastCacheHolder.getCacheKey( - context, scanExec.getCanonicalName(), scanExec.getFragments()); - loadRightFromCache(key); - } else { - this.tupleSlots = buildRightToHashTable(); - } - - first = false; - } - - protected void loadRightFromCache(TableCacheKey key) throws IOException { - ExecutionBlockSharedResource sharedResource = context.getSharedResource(); - synchronized (sharedResource.getLock()) { - if (sharedResource.hasBroadcastCache(key)) { - CacheHolder>> data = sharedResource.getBroadcastCache(key); - this.tupleSlots = data.getData(); - this.cachedRightTableStats = data.getTableStats(); - } else { - CacheHolder.BroadcastCacheHolder holder = - new CacheHolder.BroadcastCacheHolder(buildRightToHashTable(), rightChild.getInputStats(), null); - sharedResource.addBroadcastCache(key, holder); - CacheHolder>> data = sharedResource.getBroadcastCache(key); - this.tupleSlots = data.getData(); - this.cachedRightTableStats = data.getTableStats(); + Tuple leftTuple = leftChild.next(); // it comes from a disk + if (leftTuple == null) { // if no more tuples in left tuples on disk, a join is completed. + finished = true; + return null; } - } - } + frameTuple.setLeft(leftTuple); - private Map> buildRightToHashTable() throws IOException { - Tuple tuple; - Tuple keyTuple; - Map> map = new HashMap>(100000); - - while (!context.isStopped() && (tuple = rightChild.next()) != null) { - keyTuple = new VTuple(joinKeyPairs.size()); - for (int i = 0; i < rightKeyList.length; i++) { - keyTuple.put(i, tuple.get(rightKeyList[i])); + if (leftFiltered(leftTuple)) { + iterator = nullIterator(rightNumCols); + continue; } - List newValue = map.get(keyTuple); - - if (newValue != null) { - newValue.add(tuple); - } else { - newValue = new ArrayList(); - newValue.add(tuple); - map.put(keyTuple, newValue); + // getting corresponding right + List hashed = tupleSlots.get(toKey(leftTuple)); + Iterator rightTuples = rightFiltered(hashed); + if (!rightTuples.hasNext()) { + //this left tuple doesn't have a match on the right.But full outer join => we should keep it anyway + //output a tuple with the nulls padded rightTuple + iterator = nullIterator(rightNumCols); + continue; } + iterator = rightTuples; } - return map; - } - - @Override - public void rescan() throws IOException { - super.rescan(); - - tupleSlots.clear(); - first = true; - - finished = false; - iterator = null; - shouldGetLeftTuple = true; - } - - - @Override - public void close() throws IOException { - super.close(); - tupleSlots.clear(); - tupleSlots = null; - iterator = null; - plan = null; - joinQual = null; - joinFilter = null; - projector = null; - } - - public JoinNode getPlan() { - return this.plan; - } - - @Override - public TableStats getInputStats() { - if (leftChild == null) { - return inputStats; - } - TableStats leftInputStats = leftChild.getInputStats(); - inputStats.setNumBytes(0); - inputStats.setReadBytes(0); - inputStats.setNumRows(0); - - if (leftInputStats != null) { - inputStats.setNumBytes(leftInputStats.getNumBytes()); - inputStats.setReadBytes(leftInputStats.getReadBytes()); - inputStats.setNumRows(leftInputStats.getNumRows()); - } - - TableStats rightInputStats = cachedRightTableStats == null ? rightChild.getInputStats() : cachedRightTableStats; - if (rightInputStats != null) { - inputStats.setNumBytes(inputStats.getNumBytes() + rightInputStats.getNumBytes()); - inputStats.setReadBytes(inputStats.getReadBytes() + rightInputStats.getReadBytes()); - inputStats.setNumRows(inputStats.getNumRows() + rightInputStats.getNumRows()); - } - - return inputStats; + return null; } } diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/HashLeftSemiJoinExec.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/HashLeftSemiJoinExec.java index 32e6d081eb..41e842a533 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/HashLeftSemiJoinExec.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/HashLeftSemiJoinExec.java @@ -50,50 +50,34 @@ public HashLeftSemiJoinExec(TaskAttemptContext context, JoinNode plan, PhysicalE * @return The tuple which is firstly matched to a given join condition. * @throws java.io.IOException */ + @Override public Tuple next() throws IOException { if (first) { loadRightToHashTable(); } - Tuple rightTuple; - boolean notFound; - while(!context.isStopped() && !finished) { - - // getting new outer - leftTuple = leftChild.next(); // it comes from a disk - if (leftTuple == null) { // if no more tuples in left tuples on disk, a join is completed. - finished = true; - return null; + if (iterator != null && iterator.hasNext()) { + frameTuple.setRight(iterator.next()); + projector.eval(frameTuple, outTuple); + return outTuple; } - - // Try to find a hash bucket in in-memory hash table - getKeyLeftTuple(leftTuple, leftKeyTuple); - List rightTuples = tupleSlots.get(leftKeyTuple); - if (rightTuples != null) { - // if found, it gets a hash bucket from the hash table. - iterator = rightTuples.iterator(); - } else { + // getting new outer + Tuple leftTuple = leftChild.next(); // it comes from a disk + if (leftTuple == null || leftFiltered(leftTuple)) { // if no more tuples in left tuples on disk, a join is completed. + finished = leftTuple == null; continue; } - // Reach here only when a hash bucket is found. Then, it checks all tuples in the found bucket. - // If it finds any matched tuple, it returns the tuple immediately. - notFound = true; - while (notFound && iterator.hasNext()) { - rightTuple = iterator.next(); - frameTuple.set(leftTuple, rightTuple); - if (joinQual.eval(frameTuple).isTrue()) { // if the matched one is found - notFound = false; - projector.eval(frameTuple, outTuple); - } - } + frameTuple.setLeft(leftTuple); - if (!notFound) { // if there is no matched tuple - break; + // Try to find a hash bucket in in-memory hash table + List hashed = tupleSlots.get(toKey(leftTuple)); + if (hashed != null && rightFiltered(hashed).hasNext()) { + // if found, it gets a hash bucket from the hash table. + iterator = nullIterator(0); } } - - return outTuple; + return null; } } diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/NLLeftOuterJoinExec.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/NLLeftOuterJoinExec.java deleted file mode 100644 index 735623dbdf..0000000000 --- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/NLLeftOuterJoinExec.java +++ /dev/null @@ -1,101 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.tajo.engine.planner.physical; - -import org.apache.tajo.engine.utils.TupleUtil; -import org.apache.tajo.plan.logical.JoinNode; -import org.apache.tajo.storage.FrameTuple; -import org.apache.tajo.storage.Tuple; -import org.apache.tajo.storage.VTuple; -import org.apache.tajo.worker.TaskAttemptContext; - -import java.io.IOException; - -public class NLLeftOuterJoinExec extends CommonJoinExec { - // temporal tuples and states for nested loop join - private boolean needNextRightTuple; - private FrameTuple frameTuple; - private Tuple leftTuple = null; - private Tuple rightTuple = null; - private Tuple outTuple = null; - - private boolean foundAtLeastOneMatch; - private int rightNumCols; - - public NLLeftOuterJoinExec(TaskAttemptContext context, JoinNode plan, PhysicalExec leftChild, - PhysicalExec rightChild) { - super(context, plan, leftChild, rightChild); - // for join - needNextRightTuple = true; - frameTuple = new FrameTuple(); - outTuple = new VTuple(outSchema.size()); - - foundAtLeastOneMatch = false; - rightNumCols = rightChild.getSchema().size(); - } - - public Tuple next() throws IOException { - while (!context.isStopped()) { - if (needNextRightTuple) { - leftTuple = leftChild.next(); - if (leftTuple == null) { - return null; - } - needNextRightTuple = false; - // a new tuple from the left child has initially no matches on the right operand - foundAtLeastOneMatch = false; - } - rightTuple = rightChild.next(); - - if (rightTuple == null) { - // the scan of the right operand is finished with no matches found - if(foundAtLeastOneMatch == false){ - //output a tuple with the nulls padded rightTuple - Tuple nullPaddedTuple = TupleUtil.createNullPaddedTuple(rightNumCols); - frameTuple.set(leftTuple, nullPaddedTuple); - projector.eval(frameTuple, outTuple); - // we simulate we found a match, which is exactly the null padded one - foundAtLeastOneMatch = true; - needNextRightTuple = true; - rightChild.rescan(); - return outTuple; - } else { - needNextRightTuple = true; - rightChild.rescan(); - continue; - } - } - - frameTuple.set(leftTuple, rightTuple); - ; - if (joinQual.eval(frameTuple).isTrue()) { - projector.eval(frameTuple, outTuple); - foundAtLeastOneMatch = true; - return outTuple; - } - } - return null; - } - - @Override - public void rescan() throws IOException { - super.rescan(); - needNextRightTuple = true; - } -} diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/PhysicalExec.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/PhysicalExec.java index de14c9a9d7..ec831b7dc0 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/PhysicalExec.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/PhysicalExec.java @@ -29,6 +29,7 @@ import org.apache.tajo.worker.TaskAttemptContext; import java.io.IOException; +import java.util.Iterator; import java.util.UUID; public abstract class PhysicalExec implements SchemaObject { @@ -86,4 +87,30 @@ protected Path getExecutorTmpDir() { public TableStats getInputStats() { return null; } + + public Iterator newIterator(final PhysicalExec source) { + return new Iterator() { + + private Tuple tuple; + + @Override + public boolean hasNext() { + try { + return (tuple = source.next()) == null; + } catch (IOException e) { + throw new RuntimeException(e); + } + } + + @Override + public Tuple next() { + return tuple; + } + + @Override + public void remove() { + throw new UnsupportedOperationException("remove"); + } + }; + } } diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/PhysicalExecutorVisitor.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/PhysicalExecutorVisitor.java index 505b599ad0..c4d90a564a 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/PhysicalExecutorVisitor.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/PhysicalExecutorVisitor.java @@ -79,9 +79,6 @@ RESULT visitMergeJoin(CONTEXT context, MergeJoinExec exec, Stack s RESULT visitNLJoin(CONTEXT context, NLJoinExec exec, Stack stack) throws PhysicalPlanningException; - RESULT visitNLLeftOuterJoin(CONTEXT context, NLLeftOuterJoinExec exec, Stack stack) - throws PhysicalPlanningException; - RESULT visitProjection(CONTEXT context, ProjectionExec exec, Stack stack) throws PhysicalPlanningException; diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/RightOuterMergeJoinExec.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/RightOuterMergeJoinExec.java index 7abfbe655d..fd825b1f45 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/RightOuterMergeJoinExec.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/RightOuterMergeJoinExec.java @@ -102,7 +102,6 @@ private Tuple createNullPaddedTuple(int columnNum){ * @throws IOException */ public Tuple next() throws IOException { - Tuple previous; while (!context.isStopped()) { boolean newRound = false; @@ -121,7 +120,7 @@ public Tuple next() throws IOException { // The finalizing stage, where remaining tuples on the only right are transformed into left-padded results if (end) { - if (initRightDone == false) { + if (!initRightDone) { // maybe the left operand was empty => the right one didn't have the chance to initialize rightTuple = rightChild.next(); initRightDone = true; @@ -160,18 +159,24 @@ public Tuple next() throws IOException { } } - if(rightTuple == null){ + if(rightTuple == null) { rightTuple = rightChild.next(); - - if(rightTuple != null){ - initRightDone = true; - } - else { + if (rightTuple == null) { initRightDone = true; end = true; continue; } } + if (rightFiltered(rightTuple)) { + Tuple nullPaddedTuple = createNullPaddedTuple(leftNumCols); + frameTuple.set(nullPaddedTuple, rightTuple); + projector.eval(frameTuple, outTuple); + + rightTuple = null; + return outTuple; + } + initRightDone = true; + ////////////////////////////////////////////////////////////////////// // END INITIALIZATION STAGE ////////////////////////////////////////////////////////////////////// @@ -203,10 +208,7 @@ public Tuple next() throws IOException { // we simulate we found a match, which is exactly the null padded one // BEFORE RETURN, MOVE FORWARD - rightTuple = rightChild.next(); - if(rightTuple == null) { - end = true; - } + rightTuple = null; return outTuple; } else if (cmp < 0) { @@ -223,6 +225,7 @@ public Tuple next() throws IOException { // END MOVE FORWARDING STAGE ////////////////////////////////////////////////////////////////////// + Tuple previous = null; // once a match is found, retain all tuples with this key in tuple slots on each side if(!end) { endInPopulationStage = false; @@ -257,6 +260,19 @@ public Tuple next() throws IOException { endInPopulationStage = true; } } // if end false + if (previous != null && rightFiltered(previous)) { + Tuple nullPaddedTuple = createNullPaddedTuple(leftNumCols); + frameTuple.set(nullPaddedTuple, previous); + projector.eval(frameTuple, outTuple); + + // reset tuple slots for a new round + leftTupleSlots.clear(); + innerTupleSlots.clear(); + posRightTupleSlots = -1; + posLeftTupleSlots = -1; + + return outTuple; + } } // if newRound diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/utils/CacheHolder.java b/tajo-core/src/main/java/org/apache/tajo/engine/utils/CacheHolder.java index 6a5c0bf929..addca4914e 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/utils/CacheHolder.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/utils/CacheHolder.java @@ -18,7 +18,6 @@ package org.apache.tajo.engine.utils; -import com.google.common.collect.Maps; import org.apache.tajo.catalog.proto.CatalogProtos; import org.apache.tajo.catalog.statistics.TableStats; import org.apache.tajo.storage.Tuple; @@ -66,7 +65,7 @@ public BroadcastCacheHolder(Map> data, TableStats tableStats, @Override public Map> getData() { - return Maps.newHashMap(data); + return data; } @Override diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestHashSemiJoinExec.java b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestHashSemiJoinExec.java index 42a018bdac..ab69234309 100644 --- a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestHashSemiJoinExec.java +++ b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestHashSemiJoinExec.java @@ -205,11 +205,12 @@ public final void testHashSemiJoin() throws IOException, PlanningException { exec.init(); // expect result without duplicated tuples. while ((tuple = exec.next()) != null) { + System.out.println("[TestHashSemiJoinExec/testHashSemiJoin] " + tuple); count++; - assertTrue(i == tuple.get(0).asInt4()); - assertTrue(i == tuple.get(1).asInt4()); - assertTrue(("dept_" + i).equals(tuple.get(2).asChars())); - assertTrue(10 + i == tuple.get(3).asInt4()); + assertEquals(i, tuple.get(0).asInt4()); + assertEquals(i, tuple.get(1).asInt4()); + assertEquals("dept_" + i, tuple.get(2).asChars()); + assertEquals(10 + i, tuple.get(3).asInt4()); i += 2; } diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestLeftOuterHashJoinExec.java b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestLeftOuterHashJoinExec.java index 5747e58aed..e403947b66 100644 --- a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestLeftOuterHashJoinExec.java +++ b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestLeftOuterHashJoinExec.java @@ -319,24 +319,17 @@ public final void testLeftOuter_HashJoinExec1() throws IOException, PlanningExce PhysicalExec exec = phyPlanner.createPlan(ctx, plan); ProjectionExec proj = (ProjectionExec) exec; - if (proj.getChild() instanceof NLLeftOuterJoinExec) { - //for this small data set this is not likely to happen - - assertEquals(1, 0); - } - else{ - Tuple tuple; - int count = 0; - int i = 1; - exec.init(); - - while ((tuple = exec.next()) != null) { - //TODO check contents - count = count + 1; - } - exec.close(); - assertEquals(5, count); + Tuple tuple; + int count = 0; + int i = 1; + exec.init(); + + while ((tuple = exec.next()) != null) { + //TODO check contents + count = count + 1; } + exec.close(); + assertEquals(5, count); } @Test @@ -361,24 +354,17 @@ public final void testLeftOuter_HashJoinExec2() throws IOException, PlanningExce PhysicalExec exec = phyPlanner.createPlan(ctx, plan); ProjectionExec proj = (ProjectionExec) exec; - if (proj.getChild() instanceof NLLeftOuterJoinExec) { - //for this small data set this is not likely to happen - - assertEquals(1, 0); - } - else{ - Tuple tuple; - int count = 0; - int i = 1; - exec.init(); - - while ((tuple = exec.next()) != null) { - //TODO check contents - count = count + 1; - } - exec.close(); - assertEquals(7, count); + Tuple tuple; + int count = 0; + int i = 1; + exec.init(); + + while ((tuple = exec.next()) != null) { + //TODO check contents + count = count + 1; } + exec.close(); + assertEquals(7, count); } @@ -403,24 +389,17 @@ public final void testLeftOuter_HashJoinExec3() throws IOException, PlanningExce PhysicalExec exec = phyPlanner.createPlan(ctx, plan); ProjectionExec proj = (ProjectionExec) exec; - if (proj.getChild() instanceof NLLeftOuterJoinExec) { - //for this small data set this is not likely to happen - - assertEquals(1, 0); - } - else{ - Tuple tuple; - int count = 0; - int i = 1; - exec.init(); - - while ((tuple = exec.next()) != null) { - //TODO check contents - count = count + 1; - } - exec.close(); - assertEquals(7, count); + Tuple tuple; + int count = 0; + int i = 1; + exec.init(); + + while ((tuple = exec.next()) != null) { + //TODO check contents + count = count + 1; } + exec.close(); + assertEquals(7, count); } @@ -445,22 +424,15 @@ public final void testLeftOuter_HashJoinExec4() throws IOException, PlanningExce PhysicalExec exec = phyPlanner.createPlan(ctx, plan); ProjectionExec proj = (ProjectionExec) exec; - if (proj.getChild() instanceof NLLeftOuterJoinExec) { - //for this small data set this is not likely to happen - - assertEquals(1, 0); - } - else{ - int count = 0; - exec.init(); - - while (exec.next() != null) { - //TODO check contents - count = count + 1; - } - exec.close(); - assertEquals(0, count); + int count = 0; + exec.init(); + + while (exec.next() != null) { + //TODO check contents + count = count + 1; } + exec.close(); + assertEquals(0, count); } diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestLeftOuterNLJoinExec.java b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestLeftOuterNLJoinExec.java deleted file mode 100644 index ab4b88161f..0000000000 --- a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestLeftOuterNLJoinExec.java +++ /dev/null @@ -1,474 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.tajo.engine.planner.physical; - -import org.apache.hadoop.fs.Path; -import org.apache.tajo.LocalTajoTestingUtility; -import org.apache.tajo.TajoTestingCluster; -import org.apache.tajo.algebra.Expr; -import org.apache.tajo.catalog.*; -import org.apache.tajo.catalog.proto.CatalogProtos.StoreType; -import org.apache.tajo.common.TajoDataTypes.Type; -import org.apache.tajo.conf.TajoConf; -import org.apache.tajo.datum.Datum; -import org.apache.tajo.datum.DatumFactory; -import org.apache.tajo.engine.parser.SQLAnalyzer; -import org.apache.tajo.plan.LogicalPlanner; -import org.apache.tajo.engine.planner.PhysicalPlanner; -import org.apache.tajo.engine.planner.PhysicalPlannerImpl; -import org.apache.tajo.plan.PlanningException; -import org.apache.tajo.engine.planner.enforce.Enforcer; -import org.apache.tajo.plan.logical.LogicalNode; -import org.apache.tajo.engine.query.QueryContext; -import org.apache.tajo.storage.*; -import org.apache.tajo.storage.fragment.FileFragment; -import org.apache.tajo.util.CommonTestingUtil; -import org.apache.tajo.util.TUtil; -import org.apache.tajo.worker.TaskAttemptContext; -import org.junit.After; -import org.junit.Before; -import org.junit.Test; - -import java.io.IOException; - -import static org.apache.tajo.TajoConstants.DEFAULT_DATABASE_NAME; -import static org.apache.tajo.TajoConstants.DEFAULT_TABLESPACE_NAME; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNull; - -public class TestLeftOuterNLJoinExec { - private TajoConf conf; - private final String TEST_PATH = TajoTestingCluster.DEFAULT_TEST_DIRECTORY + "/TestLeftOuterNLJoinExec"; - private TajoTestingCluster util; - private CatalogService catalog; - private SQLAnalyzer analyzer; - private LogicalPlanner planner; - private QueryContext defaultContext; - private Path testDir; - - private TableDesc dep3; - private TableDesc job3; - private TableDesc emp3; - private TableDesc phone3; - - private final String DEP3_NAME = CatalogUtil.buildFQName(DEFAULT_DATABASE_NAME, "dep3"); - private final String JOB3_NAME = CatalogUtil.buildFQName(DEFAULT_DATABASE_NAME, "job3"); - private final String EMP3_NAME = CatalogUtil.buildFQName(DEFAULT_DATABASE_NAME, "emp3"); - private final String PHONE3_NAME = CatalogUtil.buildFQName(DEFAULT_DATABASE_NAME, "phone3"); - - @Before - public void setUp() throws Exception { - util = new TajoTestingCluster(); - catalog = util.startCatalogCluster().getCatalog(); - testDir = CommonTestingUtil.getTestDir(TEST_PATH); - catalog.createTablespace(DEFAULT_TABLESPACE_NAME, testDir.toUri().toString()); - catalog.createDatabase(DEFAULT_DATABASE_NAME, DEFAULT_TABLESPACE_NAME); - conf = util.getConfiguration(); - - //----------------- dep3 ------------------------------ - // dep_id | dep_name | loc_id - //-------------------------------- - // 0 | dep_0 | 1000 - // 1 | dep_1 | 1001 - // 2 | dep_2 | 1002 - // 3 | dep_3 | 1003 - // 4 | dep_4 | 1004 - // 5 | dep_5 | 1005 - // 6 | dep_6 | 1006 - // 7 | dep_7 | 1007 - // 8 | dep_8 | 1008 - // 9 | dep_9 | 1009 - Schema dep3Schema = new Schema(); - dep3Schema.addColumn("dep_id", Type.INT4); - dep3Schema.addColumn("dep_name", Type.TEXT); - dep3Schema.addColumn("loc_id", Type.INT4); - - - TableMeta dep3Meta = CatalogUtil.newTableMeta(StoreType.CSV); - Path dep3Path = new Path(testDir, "dep3.csv"); - Appender appender1 = ((FileStorageManager)StorageManager.getFileStorageManager(conf)) - .getAppender(dep3Meta, dep3Schema, dep3Path); - appender1.init(); - Tuple tuple = new VTuple(dep3Schema.size()); - for (int i = 0; i < 10; i++) { - tuple.put(new Datum[] { DatumFactory.createInt4(i), - DatumFactory.createText("dept_" + i), - DatumFactory.createInt4(1000 + i) }); - appender1.addTuple(tuple); - } - - appender1.flush(); - appender1.close(); - dep3 = CatalogUtil.newTableDesc(DEP3_NAME, dep3Schema, dep3Meta, dep3Path); - catalog.createTable(dep3); - - //----------------- job3 ------------------------------ - // job_id | job_title - // ---------------------- - // 101 | job_101 - // 102 | job_102 - // 103 | job_103 - - Schema job3Schema = new Schema(); - job3Schema.addColumn("job_id", Type.INT4); - job3Schema.addColumn("job_title", Type.TEXT); - - - TableMeta job3Meta = CatalogUtil.newTableMeta(StoreType.CSV); - Path job3Path = new Path(testDir, "job3.csv"); - Appender appender2 = ((FileStorageManager)StorageManager.getFileStorageManager(conf)) - .getAppender(job3Meta, job3Schema, job3Path); - appender2.init(); - Tuple tuple2 = new VTuple(job3Schema.size()); - for (int i = 1; i < 4; i++) { - int x = 100 + i; - tuple2.put(new Datum[] { DatumFactory.createInt4(100 + i), - DatumFactory.createText("job_" + x) }); - appender2.addTuple(tuple2); - } - - appender2.flush(); - appender2.close(); - job3 = CatalogUtil.newTableDesc(JOB3_NAME, job3Schema, job3Meta, job3Path); - catalog.createTable(job3); - - - - //---------------------emp3 -------------------- - // emp_id | first_name | last_name | dep_id | salary | job_id - // ------------------------------------------------------------ - // 11 | fn_11 | ln_11 | 1 | 123 | 101 - // 13 | fn_13 | ln_13 | 3 | 369 | 103 - // 15 | fn_15 | ln_15 | 5 | 615 | null - // 17 | fn_17 | ln_17 | 7 | 861 | null - // 19 | fn_19 | ln_19 | 9 | 1107 | null - // 21 | fn_21 | ln_21 | 1 | 123 | 101 - // 23 | fn_23 | ln_23 | 3 | 369 | 103 - - Schema emp3Schema = new Schema(); - emp3Schema.addColumn("emp_id", Type.INT4); - emp3Schema.addColumn("first_name", Type.TEXT); - emp3Schema.addColumn("last_name", Type.TEXT); - emp3Schema.addColumn("dep_id", Type.INT4); - emp3Schema.addColumn("salary", Type.FLOAT4); - emp3Schema.addColumn("job_id", Type.INT4); - - - TableMeta emp3Meta = CatalogUtil.newTableMeta(StoreType.CSV); - Path emp3Path = new Path(testDir, "emp3.csv"); - Appender appender3 = ((FileStorageManager)StorageManager.getFileStorageManager(conf)) - .getAppender(emp3Meta, emp3Schema, emp3Path); - appender3.init(); - Tuple tuple3 = new VTuple(emp3Schema.size()); - - for (int i = 1; i < 4; i += 2) { - int x = 10 + i; - tuple3.put(new Datum[] { DatumFactory.createInt4(10 + i), - DatumFactory.createText("firstname_" + x), - DatumFactory.createText("lastname_" + x), - DatumFactory.createInt4(i), - DatumFactory.createFloat4(123 * i), - DatumFactory.createInt4(100 + i) }); - appender3.addTuple(tuple3); - - int y = 20 + i; - tuple3.put(new Datum[] { DatumFactory.createInt4(20 + i), - DatumFactory.createText("firstname_" + y), - DatumFactory.createText("lastname_" + y), - DatumFactory.createInt4(i), - DatumFactory.createFloat4(123 * i), - DatumFactory.createInt4(100 + i) }); - appender3.addTuple(tuple3); - } - - for (int i = 5; i < 10; i += 2) { - int x = 10 + i; - tuple3.put(new Datum[] { DatumFactory.createInt4(10 + i), - DatumFactory.createText("firstname_" + x), - DatumFactory.createText("lastname_" + x), - DatumFactory.createInt4(i), - DatumFactory.createFloat4(123 * i), - DatumFactory.createNullDatum() }); - appender3.addTuple(tuple3); - } - - appender3.flush(); - appender3.close(); - emp3 = CatalogUtil.newTableDesc(EMP3_NAME, emp3Schema, emp3Meta, emp3Path); - catalog.createTable(emp3); - - // ---------------------phone3 -------------------- - // emp_id | phone_number - // ----------------------------------------------- - // this table is empty, no rows - - Schema phone3Schema = new Schema(); - phone3Schema.addColumn("emp_id", Type.INT4); - phone3Schema.addColumn("phone_number", Type.TEXT); - - - TableMeta phone3Meta = CatalogUtil.newTableMeta(StoreType.CSV); - Path phone3Path = new Path(testDir, "phone3.csv"); - Appender appender5 = ((FileStorageManager)StorageManager.getFileStorageManager(conf)) - .getAppender(phone3Meta, phone3Schema, phone3Path); - appender5.init(); - - appender5.flush(); - appender5.close(); - phone3 = CatalogUtil.newTableDesc(PHONE3_NAME, phone3Schema, phone3Meta, phone3Path); - catalog.createTable(phone3); - - analyzer = new SQLAnalyzer(); - planner = new LogicalPlanner(catalog); - - defaultContext = LocalTajoTestingUtility.createDummyContext(conf); - } - - @After - public void tearDown() throws Exception { - util.shutdownCatalogCluster(); - } - - String[] QUERIES = { - "select dep3.dep_id, dep_name, emp_id, salary from dep3 left outer join emp3 on dep3.dep_id = emp3.dep_id", //0 no nulls - "select job3.job_id, job_title, emp_id, salary from job3 left outer join emp3 on job3.job_id=emp3.job_id", //1 nulls on the right operand - "select job3.job_id, job_title, emp_id, salary from emp3 left outer join job3 on job3.job_id=emp3.job_id", //2 nulls on the left side - "select emp3.emp_id, first_name, phone_number from emp3 left outer join phone3 on emp3.emp_id = phone3.emp_id", //3 one operand is empty - "select phone_number, emp3.emp_id, first_name from phone3 left outer join emp3 on emp3.emp_id = phone3.emp_id" //4 one operand is empty - }; - - @Test - public final void testLeftOuterNLJoinExec0() throws IOException, PlanningException { - FileFragment[] dep3Frags = FileStorageManager.splitNG(conf, DEP3_NAME, dep3.getMeta(), new Path(dep3.getPath()), - Integer.MAX_VALUE); - FileFragment[] emp3Frags = FileStorageManager.splitNG(conf, EMP3_NAME, emp3.getMeta(), new Path(emp3.getPath()), - Integer.MAX_VALUE); - - FileFragment[] merged = TUtil.concat(dep3Frags, emp3Frags); - - Path workDir = CommonTestingUtil.getTestDir(TajoTestingCluster.DEFAULT_TEST_DIRECTORY + "/TestLeftOuterNLJoinExec0"); - TaskAttemptContext ctx = new TaskAttemptContext(new QueryContext(conf), - LocalTajoTestingUtility.newTaskAttemptId(), merged, workDir); - ctx.setEnforcer(new Enforcer()); - Expr context = analyzer.parse(QUERIES[0]); - LogicalNode plan = planner.createPlan(defaultContext, context).getRootBlock().getRoot(); - - - PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf); - PhysicalExec exec = phyPlanner.createPlan(ctx, plan); - - //maybe plan results with hash join exec algorithm usage. Must convert from HashLeftOuterJoinExec into NLLeftOuterJoinExec - ProjectionExec proj = (ProjectionExec) exec; - if (proj.getChild() instanceof HashLeftOuterJoinExec) { - HashLeftOuterJoinExec join = proj.getChild(); - NLLeftOuterJoinExec aJoin = new NLLeftOuterJoinExec(ctx, join.getPlan(), join.getLeftChild(), join.getRightChild()); - proj.setChild(aJoin); - exec = proj; - } - - int count = 0; - exec.init(); - while (exec.next() != null) { - //TODO check contents - count = count + 1; - } - assertNull(exec.next()); - exec.close(); - assertEquals(12, count); - } - - - @Test - public final void testLeftOuterNLJoinExec1() throws IOException, PlanningException { - FileFragment[] job3Frags = FileStorageManager.splitNG(conf, JOB3_NAME, job3.getMeta(), new Path(job3.getPath()), - Integer.MAX_VALUE); - FileFragment[] emp3Frags = FileStorageManager.splitNG(conf, EMP3_NAME, emp3.getMeta(), new Path(emp3.getPath()), - Integer.MAX_VALUE); - - FileFragment[] merged = TUtil.concat(job3Frags, emp3Frags); - - - Path workDir = CommonTestingUtil.getTestDir(TajoTestingCluster.DEFAULT_TEST_DIRECTORY + "/TestLeftOuter_NLJoinExec1"); - TaskAttemptContext ctx = new TaskAttemptContext(new QueryContext(conf), - LocalTajoTestingUtility.newTaskAttemptId(), merged, workDir); - ctx.setEnforcer(new Enforcer()); - Expr context = analyzer.parse(QUERIES[1]); - LogicalNode plan = planner.createPlan(defaultContext, context).getRootBlock().getRoot(); - - - PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf); - PhysicalExec exec = phyPlanner.createPlan(ctx, plan); - - //maybe plan results with hash join exec algorithm usage. Must convert from HashLeftOuterJoinExec into NLLeftOuterJoinExec - ProjectionExec proj = (ProjectionExec) exec; - if (proj.getChild() instanceof HashLeftOuterJoinExec) { - HashLeftOuterJoinExec join = proj.getChild(); - NLLeftOuterJoinExec aJoin = new NLLeftOuterJoinExec(ctx, join.getPlan(), join.getLeftChild(), join.getRightChild()); - proj.setChild(aJoin); - exec = proj; - - } - - - Tuple tuple; - int i = 1; - int count = 0; - exec.init(); - while ((tuple = exec.next()) != null) { - //TODO check contents - count = count + 1; - - } - exec.close(); - assertEquals(5, count); - } - - @Test - public final void testLeftOuter_NLJoinExec2() throws IOException, PlanningException { - FileFragment[] emp3Frags = FileStorageManager.splitNG(conf, EMP3_NAME, emp3.getMeta(), new Path(emp3.getPath()), - Integer.MAX_VALUE); - FileFragment[] job3Frags = FileStorageManager.splitNG(conf, JOB3_NAME, job3.getMeta(), new Path(job3.getPath()), - Integer.MAX_VALUE); - - FileFragment[] merged = TUtil.concat(emp3Frags, job3Frags); - - Path workDir = CommonTestingUtil.getTestDir(TajoTestingCluster.DEFAULT_TEST_DIRECTORY + "/TestLeftOuter_NLJoinExec2"); - TaskAttemptContext ctx = new TaskAttemptContext(new QueryContext(conf), - LocalTajoTestingUtility.newTaskAttemptId(), merged, workDir); - ctx.setEnforcer(new Enforcer()); - Expr context = analyzer.parse(QUERIES[2]); - LogicalNode plan = planner.createPlan(defaultContext, context).getRootBlock().getRoot(); - - - PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf); - PhysicalExec exec = phyPlanner.createPlan(ctx, plan); - - //maybe plan results with hash join exec algorithm usage. Must convert from HashLeftOuterJoinExec into NLLeftOuterJoinExec - ProjectionExec proj = (ProjectionExec) exec; - if (proj.getChild() instanceof HashLeftOuterJoinExec) { - HashLeftOuterJoinExec join = proj.getChild(); - NLLeftOuterJoinExec aJoin = new NLLeftOuterJoinExec(ctx, join.getPlan(), join.getLeftChild(), join.getRightChild()); - proj.setChild(aJoin); - exec = proj; - - } - - - Tuple tuple; - int i = 1; - int count = 0; - exec.init(); - while ((tuple = exec.next()) != null) { - //TODO check contents - count = count + 1; - - } - exec.close(); - assertEquals(7, count); - } - - - @Test - public final void testLeftOuter_NLJoinExec3() throws IOException, PlanningException { - FileFragment[] emp3Frags = FileStorageManager.splitNG(conf, EMP3_NAME, emp3.getMeta(), new Path(emp3.getPath()), - Integer.MAX_VALUE); - FileFragment[] phone3Frags = FileStorageManager.splitNG(conf, PHONE3_NAME, phone3.getMeta(), new Path(phone3.getPath()), - Integer.MAX_VALUE); - - FileFragment[] merged = TUtil.concat(emp3Frags, phone3Frags); - - Path workDir = CommonTestingUtil.getTestDir(TajoTestingCluster.DEFAULT_TEST_DIRECTORY + "/TestLeftOuter_NLJoinExec3"); - TaskAttemptContext ctx = new TaskAttemptContext(new QueryContext(conf), - LocalTajoTestingUtility.newTaskAttemptId(), merged, workDir); - ctx.setEnforcer(new Enforcer()); - Expr context = analyzer.parse(QUERIES[3]); - LogicalNode plan = planner.createPlan(defaultContext, context).getRootBlock().getRoot(); - - - PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf); - PhysicalExec exec = phyPlanner.createPlan(ctx, plan); - - //maybe plan results with hash join exec algorithm usage. Must convert from HashLeftOuterJoinExec into NLLeftOuterJoinExec - ProjectionExec proj = (ProjectionExec) exec; - if (proj.getChild() instanceof HashLeftOuterJoinExec) { - HashLeftOuterJoinExec join = proj.getChild(); - NLLeftOuterJoinExec aJoin = new NLLeftOuterJoinExec(ctx, join.getPlan(), join.getLeftChild(), join.getRightChild()); - proj.setChild(aJoin); - exec = proj; - - } - - - Tuple tuple; - int i = 1; - int count = 0; - exec.init(); - while ((tuple = exec.next()) != null) { - //TODO check contents - count = count + 1; - - } - exec.close(); - assertEquals(7, count); - } - - @Test - public final void testLeftOuter_NLJoinExec4() throws IOException, PlanningException { - FileFragment[] emp3Frags = FileStorageManager.splitNG(conf, EMP3_NAME, emp3.getMeta(), new Path(emp3.getPath()), - Integer.MAX_VALUE); - FileFragment[] phone3Frags = FileStorageManager.splitNG(conf, PHONE3_NAME, phone3.getMeta(), new Path(phone3.getPath()), - Integer.MAX_VALUE); - - FileFragment[] merged = TUtil.concat(phone3Frags, emp3Frags); - - Path workDir = CommonTestingUtil.getTestDir(TajoTestingCluster.DEFAULT_TEST_DIRECTORY + "/TestLeftOuter_NLJoinExec4"); - TaskAttemptContext ctx = new TaskAttemptContext(new QueryContext(conf), - LocalTajoTestingUtility.newTaskAttemptId(), merged, workDir); - ctx.setEnforcer(new Enforcer()); - Expr context = analyzer.parse(QUERIES[4]); - LogicalNode plan = planner.createPlan(defaultContext, context).getRootBlock().getRoot(); - - - PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf); - PhysicalExec exec = phyPlanner.createPlan(ctx, plan); - - //maybe plan results with hash join exec algorithm usage. Must convert from HashLeftOuterJoinExec into NLLeftOuterJoinExec - ProjectionExec proj = (ProjectionExec) exec; - if (proj.getChild() instanceof HashLeftOuterJoinExec) { - HashLeftOuterJoinExec join = proj.getChild(); - NLLeftOuterJoinExec aJoin = new NLLeftOuterJoinExec(ctx, join.getPlan(), join.getLeftChild(), join.getRightChild()); - proj.setChild(aJoin); - exec = proj; - - } - - - Tuple tuple; - int i = 1; - int count = 0; - exec.init(); - while ((tuple = exec.next()) != null) { - //TODO check contents - count = count + 1; - - } - exec.close(); - assertEquals(0, count); - } -} diff --git a/tajo-core/src/test/resources/queries/TestJoinQuery/testJoinFilterOfRowPreservedTable1.sql b/tajo-core/src/test/resources/queries/TestJoinQuery/testJoinFilterOfRowPreservedTable1.sql index 66274d7d69..50ea371780 100644 --- a/tajo-core/src/test/resources/queries/TestJoinQuery/testJoinFilterOfRowPreservedTable1.sql +++ b/tajo-core/src/test/resources/queries/TestJoinQuery/testJoinFilterOfRowPreservedTable1.sql @@ -5,4 +5,4 @@ select n_regionkey from region left outer join nation on n_regionkey = r_regionkey and r_name in ('AMERICA', 'ASIA') -order by r_name; \ No newline at end of file +order by r_name,n_name; \ No newline at end of file diff --git a/tajo-core/src/test/resources/results/TestJoinQuery/testJoinFilterOfRowPreservedTable1.result b/tajo-core/src/test/resources/results/TestJoinQuery/testJoinFilterOfRowPreservedTable1.result index 82d5562138..d489e3e90b 100644 --- a/tajo-core/src/test/resources/results/TestJoinQuery/testJoinFilterOfRowPreservedTable1.result +++ b/tajo-core/src/test/resources/results/TestJoinQuery/testJoinFilterOfRowPreservedTable1.result @@ -6,10 +6,10 @@ AMERICA,1,BRAZIL,1 AMERICA,1,CANADA,1 AMERICA,1,PERU,1 AMERICA,1,UNITED STATES,1 +ASIA,2,CHINA,2 ASIA,2,INDIA,2 ASIA,2,INDONESIA,2 ASIA,2,JAPAN,2 -ASIA,2,CHINA,2 ASIA,2,VIETNAM,2 EUROPE,3,null,null MIDDLE EAST,4,null,null \ No newline at end of file diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/AggregationFunctionCallEval.java b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/AggregationFunctionCallEval.java index cfcc829717..33d656556a 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/AggregationFunctionCallEval.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/AggregationFunctionCallEval.java @@ -78,7 +78,7 @@ public EvalNode bind(EvalContext evalContext, Schema schema) { } public void merge(FunctionContext context, Tuple tuple) { - if (!isBinded) { + if (!isBound) { throw new IllegalStateException("bind() must be called before merge()"); } mergeParam(context, evalParams(tuple)); @@ -99,7 +99,7 @@ public T eval(Tuple tuple) { } public Datum terminate(FunctionContext context) { - if (!isBinded) { + if (!isBound) { throw new IllegalStateException("bind() must be called before terminate()"); } if (!finalPhase) { diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/AlgebraicUtil.java b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/AlgebraicUtil.java index 6cf72723e9..c6b7354e04 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/AlgebraicUtil.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/AlgebraicUtil.java @@ -21,6 +21,7 @@ import org.apache.tajo.catalog.Column; import java.util.ArrayList; +import java.util.Collection; import java.util.List; import java.util.Map; import java.util.Stack; @@ -327,6 +328,10 @@ public static boolean isIndexableOperator(EvalNode expr) { (expr.getType() == EvalType.LIKE && !((LikePredicateEval)expr).isLeadingWildCard()); } + public static EvalNode createSingletonExprFromCNF(Collection cnfExprs) { + return createSingletonExprFromCNF(cnfExprs.toArray(new EvalNode[cnfExprs.size()])); + } + /** * Convert a list of conjunctive normal forms into a singleton expression. * diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/EvalNode.java b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/EvalNode.java index 9abb0bc72d..b154532f12 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/EvalNode.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/EvalNode.java @@ -35,43 +35,44 @@ * It is also used for evaluation. */ public abstract class EvalNode implements Cloneable, GsonObject, ProtoObject { - @Expose protected EvalType type; - protected boolean isBinded = false; + @Expose + protected EvalType type; + protected transient boolean isBound; public EvalNode() { } - public EvalNode(EvalType type) { - this.type = type; - } - - public EvalType getType() { - return this.type; - } - - public abstract DataType getValueType(); + public EvalNode(EvalType type) { + this.type = type; + } + + public EvalType getType() { + return this.type; + } + + public abstract DataType getValueType(); public abstract int childNum(); public abstract EvalNode getChild(int idx); - - public abstract String getName(); + + public abstract String getName(); @Override - public String toJson() { + public String toJson() { return PlanGsonHelper.toJson(this, EvalNode.class); - } + } public EvalNode bind(@Nullable EvalContext evalContext, Schema schema) { for (int i = 0; i < childNum(); i++) { getChild(i).bind(evalContext, schema); } - isBinded = true; + isBound = true; return this; } - public T eval(Tuple tuple) { - if (!isBinded) { + public T eval(Tuple tuple) { + if (!isBound) { throw new IllegalStateException("bind() must be called before eval()"); } return null; @@ -87,7 +88,7 @@ public T eval(Tuple tuple) { public Object clone() throws CloneNotSupportedException { EvalNode evalNode = (EvalNode) super.clone(); evalNode.type = type; - evalNode.isBinded = isBinded; + evalNode.isBound = isBound; return evalNode; } diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/InEval.java b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/InEval.java index c968bda393..7052663152 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/InEval.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/InEval.java @@ -58,7 +58,7 @@ public String getName() { @Override public Datum eval(Tuple tuple) { - if (!isBinded) { + if (!isBound) { throw new IllegalStateException("bind() must be called before eval()"); } if (values == null) { diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/PatternMatchPredicateEval.java b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/PatternMatchPredicateEval.java index cdd8dfb575..ec143f7ed2 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/PatternMatchPredicateEval.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/PatternMatchPredicateEval.java @@ -82,7 +82,7 @@ public EvalNode bind(EvalContext evalContext, Schema schema) { @Override public Datum eval(Tuple tuple) { - if (!isBinded) { + if (!isBound) { throw new IllegalStateException("bind() must be called before eval()"); } Datum predicand = leftExpr.eval(tuple); diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/WindowFunctionEval.java b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/WindowFunctionEval.java index 0b60d14234..d86bc989e7 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/WindowFunctionEval.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/WindowFunctionEval.java @@ -63,7 +63,7 @@ protected void mergeParam(FunctionContext context, Tuple params) { @Override public Datum terminate(FunctionContext context) { - if (!isBinded) { + if (!isBound) { throw new IllegalStateException("bind() must be called before terminate()"); } return functionInvoke.terminate(context); diff --git a/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/FrameTuple.java b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/FrameTuple.java index 8b7e2e06a8..a5561ed7cf 100644 --- a/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/FrameTuple.java +++ b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/FrameTuple.java @@ -30,7 +30,7 @@ /** * An instance of FrameTuple is an immutable tuple. * It contains two tuples and pretends to be one instance of Tuple for - * join qual evaluatations. + * join qual evaluations. */ public class FrameTuple implements Tuple, Cloneable { private int size; @@ -52,6 +52,18 @@ public void set(Tuple left, Tuple right) { this.right = right; } + public FrameTuple setLeft(Tuple left) { + this.left = left; + this.leftSize = left.size(); + return this; + } + + public FrameTuple setRight(Tuple right) { + this.right = right; + this.size = leftSize + right.size(); + return this; + } + @Override public int size() { return size; From f16223339e09cb36c27ac561f25b8a759c43597f Mon Sep 17 00:00:00 2001 From: "navis.ryu" Date: Mon, 27 Apr 2015 17:03:17 +0900 Subject: [PATCH 2/2] TAJO-1554 Implement hash-theta join --- .../java/org/apache/tajo/catalog/Schema.java | 2 +- .../java/org/apache/tajo/storage/VTuple.java | 19 +- .../engine/planner/PhysicalPlannerImpl.java | 51 +++-- .../engine/planner/physical/BNLJoinExec.java | 208 +++++++----------- .../planner/physical/CommonHashJoinExec.java | 48 +++- .../planner/physical/CommonJoinExec.java | 193 +++++++++++----- .../physical/HashFullOuterJoinExec.java | 43 ++-- .../engine/planner/physical/HashJoinExec.java | 20 +- .../physical/HashLeftAntiJoinExec.java | 5 +- .../physical/HashLeftOuterJoinExec.java | 11 +- .../physical/HashLeftSemiJoinExec.java | 5 +- .../physical/MergeFullOuterJoinExec.java | 22 +- .../planner/physical/MergeJoinExec.java | 10 +- .../engine/planner/physical/NLJoinExec.java | 22 +- .../physical/RightOuterMergeJoinExec.java | 4 +- .../planner/physical/SelectionExec.java | 11 +- .../planner/physical/ThetaOperation.java | 83 +++++++ .../org/apache/tajo/QueryTestCaseBase.java | 10 +- .../org/apache/tajo/TestHashThetaJoin.java | 97 ++++++++ .../engine/planner/TestLogicalPlanner.java | 2 +- ...CrossJoinWithThetaJoinConditionInWhere.sql | 2 +- .../testThetaJoinFullOuter.1.result | 44 ++++ .../testThetaJoinFullOuter.2.result | 36 +++ .../testThetaJoinFullOuter.3.result | 40 ++++ .../testThetaJoinFullOuter.4.result | 32 +++ .../testThetaJoinFullOuter.5.result | 49 +++++ .../testThetaJoinInner.1.result | 43 ++++ .../testThetaJoinInner.2.result | 33 +++ .../testThetaJoinInner.3.result | 38 ++++ .../testThetaJoinInner.4.result | 28 +++ .../testThetaJoinInner.5.result | 48 ++++ .../testThetaJoinLeftOuter.1.result | 43 ++++ .../testThetaJoinLeftOuter.2.result | 35 +++ .../testThetaJoinLeftOuter.3.result | 39 ++++ .../testThetaJoinLeftOuter.4.result | 31 +++ .../testThetaJoinLeftOuter.5.result | 48 ++++ .../testThetaJoinRightOuter.1.result | 44 ++++ .../testThetaJoinRightOuter.2.result | 34 +++ .../testThetaJoinRightOuter.3.result | 39 ++++ .../testThetaJoinRightOuter.4.result | 29 +++ .../testThetaJoinRightOuter.5.result | 49 +++++ .../testExplainSelectPhysical.1.result | 2 +- .../testExplainSelectPhysical.2.result | 2 +- .../testExplainSelectPhysical.3.result | 2 +- .../org/apache/tajo/plan/LogicalPlanner.java | 18 +- .../apache/tajo/plan/expr/AlgebraicUtil.java | 1 + .../apache/tajo/plan/expr/EvalTreeUtil.java | 1 + .../rewrite/rules/FilterPushDownRule.java | 5 + .../org/apache/tajo/storage/FrameTuple.java | 8 +- 49 files changed, 1391 insertions(+), 298 deletions(-) create mode 100644 tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/ThetaOperation.java create mode 100644 tajo-core/src/test/java/org/apache/tajo/TestHashThetaJoin.java create mode 100644 tajo-core/src/test/resources/results/TestHashThetaJoin/testThetaJoinFullOuter.1.result create mode 100644 tajo-core/src/test/resources/results/TestHashThetaJoin/testThetaJoinFullOuter.2.result create mode 100644 tajo-core/src/test/resources/results/TestHashThetaJoin/testThetaJoinFullOuter.3.result create mode 100644 tajo-core/src/test/resources/results/TestHashThetaJoin/testThetaJoinFullOuter.4.result create mode 100644 tajo-core/src/test/resources/results/TestHashThetaJoin/testThetaJoinFullOuter.5.result create mode 100644 tajo-core/src/test/resources/results/TestHashThetaJoin/testThetaJoinInner.1.result create mode 100644 tajo-core/src/test/resources/results/TestHashThetaJoin/testThetaJoinInner.2.result create mode 100644 tajo-core/src/test/resources/results/TestHashThetaJoin/testThetaJoinInner.3.result create mode 100644 tajo-core/src/test/resources/results/TestHashThetaJoin/testThetaJoinInner.4.result create mode 100644 tajo-core/src/test/resources/results/TestHashThetaJoin/testThetaJoinInner.5.result create mode 100644 tajo-core/src/test/resources/results/TestHashThetaJoin/testThetaJoinLeftOuter.1.result create mode 100644 tajo-core/src/test/resources/results/TestHashThetaJoin/testThetaJoinLeftOuter.2.result create mode 100644 tajo-core/src/test/resources/results/TestHashThetaJoin/testThetaJoinLeftOuter.3.result create mode 100644 tajo-core/src/test/resources/results/TestHashThetaJoin/testThetaJoinLeftOuter.4.result create mode 100644 tajo-core/src/test/resources/results/TestHashThetaJoin/testThetaJoinLeftOuter.5.result create mode 100644 tajo-core/src/test/resources/results/TestHashThetaJoin/testThetaJoinRightOuter.1.result create mode 100644 tajo-core/src/test/resources/results/TestHashThetaJoin/testThetaJoinRightOuter.2.result create mode 100644 tajo-core/src/test/resources/results/TestHashThetaJoin/testThetaJoinRightOuter.3.result create mode 100644 tajo-core/src/test/resources/results/TestHashThetaJoin/testThetaJoinRightOuter.4.result create mode 100644 tajo-core/src/test/resources/results/TestHashThetaJoin/testThetaJoinRightOuter.5.result diff --git a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/Schema.java b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/Schema.java index 5ccc38f66f..34fc42f4ce 100644 --- a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/Schema.java +++ b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/Schema.java @@ -313,7 +313,7 @@ public boolean containsAll(Collection columns) { public boolean containsAny(Collection columns) { for (Column column : columns) { - if (fields.contains(column)) { + if (contains(column)) { return true; } } diff --git a/tajo-common/src/main/java/org/apache/tajo/storage/VTuple.java b/tajo-common/src/main/java/org/apache/tajo/storage/VTuple.java index da69eb09eb..c2231f0d45 100644 --- a/tajo-common/src/main/java/org/apache/tajo/storage/VTuple.java +++ b/tajo-common/src/main/java/org/apache/tajo/storage/VTuple.java @@ -28,7 +28,7 @@ import java.net.InetAddress; import java.util.Arrays; -public class VTuple implements Tuple, Cloneable { +public class VTuple implements Tuple, Cloneable, Comparable { @Expose public Datum [] values; @Expose private long offset; @@ -237,4 +237,21 @@ public static String toDisplayString(Datum [] values) { str.append(')'); return str.toString(); } + + @Override + public int compareTo(Tuple other) { + for (int i = 0; i < size(); i++) { + if (isNull(i) && other.isNull(i)) { + continue; + } + if (isNull(i) || other.isNull(i)) { + return isNull(i) ? 1 : -1; + } + int compare = get(i).compareTo(other.get(i)); + if (compare != 0) { + return compare; + } + } + return 0; + } } diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/PhysicalPlannerImpl.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/PhysicalPlannerImpl.java index 9fa4295dd0..83448229b3 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/PhysicalPlannerImpl.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/PhysicalPlannerImpl.java @@ -281,9 +281,24 @@ public boolean checkIfInMemoryInnerJoinIsPossible(TaskAttemptContext context, Lo return inMemoryInnerJoinFlag; } - public PhysicalExec createJoinPlan(TaskAttemptContext context, JoinNode joinNode, PhysicalExec leftExec, - PhysicalExec rightExec) throws IOException { + public CommonJoinExec createJoinPlan(TaskAttemptContext context, JoinNode joinNode, PhysicalExec leftExec, + PhysicalExec rightExec) throws IOException { + CommonJoinExec joinExec; + try { + joinExec = createJoinExec(context, joinNode, leftExec, rightExec); + } catch (Exception e) { + LOG.warn("Failed to make join exec, failing back to CROSS join", e); + return createCrossJoinPlan(context, joinNode, leftExec, rightExec); + } + if (joinExec.isThetaJoin() && !joinExec.isHashJoin()) { + LOG.warn("Generic theta join is not supported, yet"); + return createCrossJoinPlan(context, joinNode, leftExec, rightExec); + } + return joinExec; + } + private CommonJoinExec createJoinExec(TaskAttemptContext context, JoinNode joinNode, PhysicalExec leftExec, + PhysicalExec rightExec) throws IOException { switch (joinNode.getJoinType()) { case CROSS: return createCrossJoinPlan(context, joinNode, leftExec, rightExec); @@ -317,7 +332,7 @@ public PhysicalExec createJoinPlan(TaskAttemptContext context, JoinNode joinNode } } - private PhysicalExec createCrossJoinPlan(TaskAttemptContext context, JoinNode plan, + private CommonJoinExec createCrossJoinPlan(TaskAttemptContext context, JoinNode plan, PhysicalExec leftExec, PhysicalExec rightExec) throws IOException { Enforcer enforcer = context.getEnforcer(); EnforceProperty property = getAlgorithmEnforceProperty(enforcer, plan); @@ -343,7 +358,7 @@ private PhysicalExec createCrossJoinPlan(TaskAttemptContext context, JoinNode pl } } - private PhysicalExec createInnerJoinPlan(TaskAttemptContext context, JoinNode plan, + private CommonJoinExec createInnerJoinPlan(TaskAttemptContext context, JoinNode plan, PhysicalExec leftExec, PhysicalExec rightExec) throws IOException { Enforcer enforcer = context.getEnforcer(); EnforceProperty property = getAlgorithmEnforceProperty(enforcer, plan); @@ -416,7 +431,7 @@ private PhysicalExec createInnerJoinPlan(TaskAttemptContext context, JoinNode pl return new PhysicalExec [] {smaller, larger}; } - private PhysicalExec createBestInnerJoinPlan(TaskAttemptContext context, JoinNode plan, + private CommonJoinExec createBestInnerJoinPlan(TaskAttemptContext context, JoinNode plan, PhysicalExec leftExec, PhysicalExec rightExec) throws IOException { boolean inMemoryHashJoin = false; if (checkIfInMemoryInnerJoinIsPossible(context, plan.getLeftChild(), true) @@ -434,7 +449,7 @@ private PhysicalExec createBestInnerJoinPlan(TaskAttemptContext context, JoinNod } } - private MergeJoinExec createMergeInnerJoin(TaskAttemptContext context, JoinNode plan, + private CommonJoinExec createMergeInnerJoin(TaskAttemptContext context, JoinNode plan, PhysicalExec leftExec, PhysicalExec rightExec) throws IOException { SortSpec[][] sortSpecs = PlannerUtil.getSortKeysFromJoinQual( plan.getJoinQual(), leftExec.getSchema(), rightExec.getSchema()); @@ -455,7 +470,7 @@ private MergeJoinExec createMergeInnerJoin(TaskAttemptContext context, JoinNode return new MergeJoinExec(context, plan, outerSort, innerSort, sortSpecs[0], sortSpecs[1]); } - private PhysicalExec createLeftOuterJoinPlan(TaskAttemptContext context, JoinNode plan, + private CommonJoinExec createLeftOuterJoinPlan(TaskAttemptContext context, JoinNode plan, PhysicalExec leftExec, PhysicalExec rightExec) throws IOException { Enforcer enforcer = context.getEnforcer(); EnforceProperty property = getAlgorithmEnforceProperty(enforcer, plan); @@ -479,7 +494,7 @@ private PhysicalExec createLeftOuterJoinPlan(TaskAttemptContext context, JoinNod } } - private PhysicalExec createBestLeftOuterJoinPlan(TaskAttemptContext context, JoinNode plan, + private CommonJoinExec createBestLeftOuterJoinPlan(TaskAttemptContext context, JoinNode plan, PhysicalExec leftExec, PhysicalExec rightExec) throws IOException { String [] rightLineage = PlannerUtil.getRelationLineage(plan.getRightChild()); long rightTableVolume = estimateSizeRecursive(context, rightLineage); @@ -505,7 +520,7 @@ private PhysicalExec createBestLeftOuterJoinPlan(TaskAttemptContext context, Joi } } - private PhysicalExec createBestRightJoinPlan(TaskAttemptContext context, JoinNode plan, + private CommonJoinExec createBestRightJoinPlan(TaskAttemptContext context, JoinNode plan, PhysicalExec leftExec, PhysicalExec rightExec) throws IOException { //if the left operand is small enough => implement it as a left outer hash join with exchanged operators (note: // blocking, but merge join is blocking as well) @@ -529,7 +544,7 @@ private PhysicalExec createBestRightJoinPlan(TaskAttemptContext context, JoinNod } } - private PhysicalExec createRightOuterMergeJoinPlan(TaskAttemptContext context, JoinNode plan, + private CommonJoinExec createRightOuterMergeJoinPlan(TaskAttemptContext context, JoinNode plan, PhysicalExec leftExec, PhysicalExec rightExec) throws IOException { //the left operand is too large, so opt for merge join implementation LOG.info("Right Outer Join (" + plan.getPID() +") chooses [Merge Join]."); @@ -551,7 +566,7 @@ private PhysicalExec createRightOuterMergeJoinPlan(TaskAttemptContext context, J return new RightOuterMergeJoinExec(context, plan, outerSort2, innerSort2, sortSpecs2[0], sortSpecs2[1]); } - private PhysicalExec createRightOuterJoinPlan(TaskAttemptContext context, JoinNode plan, + private CommonJoinExec createRightOuterJoinPlan(TaskAttemptContext context, JoinNode plan, PhysicalExec leftExec, PhysicalExec rightExec) throws IOException { Enforcer enforcer = context.getEnforcer(); EnforceProperty property = getAlgorithmEnforceProperty(enforcer, plan); @@ -573,7 +588,7 @@ private PhysicalExec createRightOuterJoinPlan(TaskAttemptContext context, JoinNo } } - private PhysicalExec createFullOuterJoinPlan(TaskAttemptContext context, JoinNode plan, + private CommonJoinExec createFullOuterJoinPlan(TaskAttemptContext context, JoinNode plan, PhysicalExec leftExec, PhysicalExec rightExec) throws IOException { Enforcer enforcer = context.getEnforcer(); EnforceProperty property = getAlgorithmEnforceProperty(enforcer, plan); @@ -596,7 +611,7 @@ private PhysicalExec createFullOuterJoinPlan(TaskAttemptContext context, JoinNod } } - private HashFullOuterJoinExec createFullOuterHashJoinPlan(TaskAttemptContext context, JoinNode plan, + private CommonJoinExec createFullOuterHashJoinPlan(TaskAttemptContext context, JoinNode plan, PhysicalExec leftExec, PhysicalExec rightExec) throws IOException { String [] leftLineage = PlannerUtil.getRelationLineage(plan.getLeftChild()); @@ -642,7 +657,7 @@ private MergeFullOuterJoinExec createFullOuterMergeJoinPlan(TaskAttemptContext c return new MergeFullOuterJoinExec(context, plan, outerSort3, innerSort3, sortSpecs3[0], sortSpecs3[1]); } - private PhysicalExec createBestFullOuterJoinPlan(TaskAttemptContext context, JoinNode plan, + private CommonJoinExec createBestFullOuterJoinPlan(TaskAttemptContext context, JoinNode plan, PhysicalExec leftExec, PhysicalExec rightExec) throws IOException { String [] leftLineage = PlannerUtil.getRelationLineage(plan.getLeftChild()); String [] rightLineage = PlannerUtil.getRelationLineage(plan.getRightChild()); @@ -659,7 +674,7 @@ private PhysicalExec createBestFullOuterJoinPlan(TaskAttemptContext context, Joi /** * Left semi join means that the left side is the IN side table, and the right side is the FROM side table. */ - private PhysicalExec createLeftSemiJoinPlan(TaskAttemptContext context, JoinNode plan, + private CommonJoinExec createLeftSemiJoinPlan(TaskAttemptContext context, JoinNode plan, PhysicalExec leftExec, PhysicalExec rightExec) throws IOException { Enforcer enforcer = context.getEnforcer(); EnforceProperty property = getAlgorithmEnforceProperty(enforcer, plan); @@ -684,7 +699,7 @@ private PhysicalExec createLeftSemiJoinPlan(TaskAttemptContext context, JoinNode /** * Left semi join means that the left side is the FROM side table, and the right side is the IN side table. */ - private PhysicalExec createRightSemiJoinPlan(TaskAttemptContext context, JoinNode plan, + private CommonJoinExec createRightSemiJoinPlan(TaskAttemptContext context, JoinNode plan, PhysicalExec leftExec, PhysicalExec rightExec) throws IOException { Enforcer enforcer = context.getEnforcer(); EnforceProperty property = getAlgorithmEnforceProperty(enforcer, plan); @@ -709,7 +724,7 @@ private PhysicalExec createRightSemiJoinPlan(TaskAttemptContext context, JoinNod /** * Left semi join means that the left side is the FROM side table, and the right side is the IN side table. */ - private PhysicalExec createLeftAntiJoinPlan(TaskAttemptContext context, JoinNode plan, + private CommonJoinExec createLeftAntiJoinPlan(TaskAttemptContext context, JoinNode plan, PhysicalExec leftExec, PhysicalExec rightExec) throws IOException { Enforcer enforcer = context.getEnforcer(); EnforceProperty property = getAlgorithmEnforceProperty(enforcer, plan); @@ -734,7 +749,7 @@ private PhysicalExec createLeftAntiJoinPlan(TaskAttemptContext context, JoinNode /** * Left semi join means that the left side is the FROM side table, and the right side is the IN side table. */ - private PhysicalExec createRightAntiJoinPlan(TaskAttemptContext context, JoinNode plan, + private CommonJoinExec createRightAntiJoinPlan(TaskAttemptContext context, JoinNode plan, PhysicalExec leftExec, PhysicalExec rightExec) throws IOException { Enforcer enforcer = context.getEnforcer(); EnforceProperty property = getAlgorithmEnforceProperty(enforcer, plan); diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/BNLJoinExec.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/BNLJoinExec.java index 6e1a553039..89534d6c62 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/BNLJoinExec.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/BNLJoinExec.java @@ -18,11 +18,11 @@ package org.apache.tajo.engine.planner.physical; +import com.google.common.base.Predicate; +import com.google.common.collect.Iterators; import org.apache.tajo.plan.util.PlannerUtil; import org.apache.tajo.plan.logical.JoinNode; -import org.apache.tajo.storage.FrameTuple; import org.apache.tajo.storage.Tuple; -import org.apache.tajo.storage.VTuple; import org.apache.tajo.worker.TaskAttemptContext; import java.io.IOException; @@ -32,163 +32,125 @@ public class BNLJoinExec extends CommonJoinExec { - private List leftTupleSlots; - private List rightTupleSlots; - private Iterator leftIterator; - private Iterator rightIterator; - - private boolean leftEnd; - private boolean rightEnd; - - // temporal tuples and states for nested loop join - private FrameTuple frameTuple; - private Tuple leftTuple = null; - private Tuple outputTuple = null; - private Tuple rightNext = null; + private final BufferedTuples leftTuples; + private final BufferedTuples rightTuples; private final static int TUPLE_SLOT_SIZE = 10000; + private Predicate predicate; + public BNLJoinExec(final TaskAttemptContext context, final JoinNode plan, final PhysicalExec leftExec, PhysicalExec rightExec) { super(context, plan, leftExec, rightExec); - this.leftTupleSlots = new ArrayList(TUPLE_SLOT_SIZE); - this.rightTupleSlots = new ArrayList(TUPLE_SLOT_SIZE); - this.leftIterator = leftTupleSlots.iterator(); - this.rightIterator = rightTupleSlots.iterator(); - this.rightEnd = false; - this.leftEnd = false; + this.leftTuples = new BufferedTuples(leftExec, leftPredicate, TUPLE_SLOT_SIZE); + this.rightTuples = new BufferedTuples(rightExec, rightPredicate, TUPLE_SLOT_SIZE); // for projection if (!plan.hasTargets()) { plan.setTargets(PlannerUtil.schemaToTargets(outSchema)); } + } - // for join - frameTuple = new FrameTuple(); - outputTuple = new VTuple(outSchema.size()); + @SuppressWarnings("unchecked") + public void init() throws IOException { + super.init(); + predicate = mergePredicates(toPredicate(equiQual), toPredicate(joinQual)); } public Tuple next() throws IOException { - - if (leftTupleSlots.isEmpty()) { - for (int k = 0; k < TUPLE_SLOT_SIZE; k++) { - Tuple t = leftChild.next(); - if (t == null) { - leftEnd = true; - break; + while (!context.isStopped()) { + if (!frameTuple.isSetLeft()) { + if (!leftTuples.hasNext()) { + return null; } - leftTupleSlots.add(t); + frameTuple.setLeft(leftTuples.next()); } - leftIterator = leftTupleSlots.iterator(); - leftTuple = leftIterator.next(); - } - - if (rightTupleSlots.isEmpty()) { - for (int k = 0; k < TUPLE_SLOT_SIZE; k++) { - Tuple t = rightChild.next(); - if (t == null) { - rightEnd = true; - break; + while (rightTuples.hasNext()) { + frameTuple.setRight(rightTuples.next()); + if (predicate == null || predicate.apply(frameTuple)) { + projector.eval(frameTuple, outTuple); + return outTuple; } - rightTupleSlots.add(t); } - rightIterator = rightTupleSlots.iterator(); + rightTuples.rescan(); + frameTuple.clear(); } + return null; + } + + private static class BufferedTuples implements Iterator { + + private final List buffer; + private final PhysicalExec source; + private final Predicate predicate; + + private transient boolean eof; + private transient Iterator buffered = Iterators.emptyIterator(); - if((rightNext = rightChild.next()) == null){ - rightEnd = true; + private BufferedTuples(PhysicalExec source, Predicate predicate, int max) { + this.buffer = new ArrayList(max); + this.source = source; + this.predicate = predicate; } - while (!context.isStopped()) { - if (!rightIterator.hasNext()) { // if leftIterator ended - if (leftIterator.hasNext()) { // if rightTupleslot remains - leftTuple = leftIterator.next(); - rightIterator = rightTupleSlots.iterator(); - } else { - if (rightEnd) { - rightChild.rescan(); - rightEnd = false; - - if (leftEnd) { - return null; - } - leftTupleSlots.clear(); - for (int k = 0; k < TUPLE_SLOT_SIZE; k++) { - Tuple t = leftChild.next(); - if (t == null) { - leftEnd = true; - break; - } - leftTupleSlots.add(t); - } - if (leftTupleSlots.isEmpty()) { - return null; - } - leftIterator = leftTupleSlots.iterator(); - leftTuple = leftIterator.next(); - - } else { - leftIterator = leftTupleSlots.iterator(); - leftTuple = leftIterator.next(); - } - - rightTupleSlots.clear(); - if (rightNext != null) { - rightTupleSlots.add(rightNext); - for (int k = 1; k < TUPLE_SLOT_SIZE; k++) { // fill right - Tuple t = rightChild.next(); - if (t == null) { - rightEnd = true; - break; - } - rightTupleSlots.add(t); - } - } else { - for (int k = 0; k < TUPLE_SLOT_SIZE; k++) { // fill right - Tuple t = rightChild.next(); - if (t == null) { - rightEnd = true; - break; - } - rightTupleSlots.add(t); - } - } - - if ((rightNext = rightChild.next()) == null) { - rightEnd = true; - } - rightIterator = rightTupleSlots.iterator(); + @Override + public boolean hasNext() { + if (buffered.hasNext()) { + return true; + } + try { + buffered = fillBuffer(); + } catch (Exception e) { + throw new RuntimeException(e); + } + return buffered.hasNext(); + } + @Override + public Tuple next() { return buffered.next(); } + @Override + public void remove() { throw new UnsupportedOperationException("remove"); } + + private Iterator fillBuffer() throws IOException { + buffer.clear(); + while (!eof && buffer.size() < TUPLE_SLOT_SIZE) { + Tuple t = source.next(); + if (isValid(t)) { + buffer.add(t); } + eof = t == null; } + return buffer.iterator(); + } - frameTuple.set(leftTuple, rightIterator.next()); - if (!hasJoinQual || joinQual.eval(frameTuple).isTrue()) { - projector.eval(frameTuple, outputTuple); - return outputTuple; - } + private boolean isValid(Tuple t) { + return t != null && (predicate == null || predicate.apply(t)); + } + + public void rescan() throws IOException { + eof = false; + buffer.clear(); + buffered = Iterators.emptyIterator(); + source.rescan(); + } + + public void close() throws IOException { + eof = true; + buffer.clear(); + buffered = Iterators.emptyIterator(); } - return null; } @Override public void rescan() throws IOException { super.rescan(); - rightEnd = false; - rightTupleSlots.clear(); - leftTupleSlots.clear(); - rightIterator = rightTupleSlots.iterator(); - leftIterator = leftTupleSlots.iterator(); + leftTuples.rescan(); + rightTuples.rescan(); } @Override public void close() throws IOException { super.close(); - - rightTupleSlots.clear(); - leftTupleSlots.clear(); - rightTupleSlots = null; - leftTupleSlots = null; - rightIterator = null; - leftIterator = null; + leftTuples.close(); + rightTuples.close(); } } diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/CommonHashJoinExec.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/CommonHashJoinExec.java index df96fb5841..8c1dfc9c39 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/CommonHashJoinExec.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/CommonHashJoinExec.java @@ -18,6 +18,9 @@ package org.apache.tajo.engine.planner.physical; +import com.google.common.base.Predicate; +import com.google.common.collect.Iterables; +import com.google.common.collect.Iterators; import org.apache.tajo.catalog.Column; import org.apache.tajo.catalog.statistics.TableStats; import org.apache.tajo.engine.utils.CacheHolder; @@ -35,6 +38,7 @@ import java.util.Iterator; import java.util.List; import java.util.Map; +import java.util.SortedMap; public abstract class CommonHashJoinExec extends CommonJoinExec { @@ -56,12 +60,14 @@ public abstract class CommonHashJoinExec extends CommonJoinExec { protected boolean finished; + protected Predicate predicate; + + @SuppressWarnings("unchecked") public CommonHashJoinExec(TaskAttemptContext context, JoinNode plan, PhysicalExec outer, PhysicalExec inner) { super(context, plan, outer, inner); - // HashJoin only can manage equi join key pairs. - this.joinKeyPairs = PlannerUtil.getJoinKeyPairs(joinQual, outer.getSchema(), - inner.getSchema(), false); + joinKeyPairs = PlannerUtil.getJoinKeyPairs(isThetaJoin() ? thetaQual : equiQual, + outer.getSchema(), inner.getSchema(), true); leftKeyList = new int[joinKeyPairs.size()]; rightKeyList = new int[joinKeyPairs.size()]; @@ -80,6 +86,17 @@ public CommonHashJoinExec(TaskAttemptContext context, JoinNode plan, PhysicalExe keyTuple = new VTuple(leftKeyList.length); } + @Override + public void init() throws IOException { + super.init(); + predicate = toFramePredicate(joinQual); + } + + @Override + public boolean isHashJoin() { + return true; + } + protected void loadRightToHashTable() throws IOException { ScanExec scanExec = PhysicalPlanUtil.findExecutor(rightChild, ScanExec.class); if (scanExec.canBroadcast()) { @@ -141,6 +158,31 @@ protected Tuple toKey(final Tuple outerTuple) { return keyTuple; } + protected Iterator filter(Tuple leftTuple) { + Iterator rightTuples; + Tuple leftKey = toKey(leftTuple); + if (thetaOp != null) { + rightTuples = filterRights(thetaOp.filter((SortedMap) tupleSlots, leftKey)); + } else { + rightTuples = filterRight(tupleSlots.get(leftKey)); + } + return filter(predicate, rightTuples); + } + + protected Iterator filterRights(Iterable rightTuples) { + List> iterables = new ArrayList>(); + for (T rightTuple : rightTuples) { + iterables.add(toTuples(rightTuple)); + } + return filterRight(Iterables.concat(iterables)); + } + + protected Iterator filterRight(T rightTuples) { + return rightTuples == null ? Iterators.emptyIterator() : filterRight(toTuples(rightTuples)); + } + + protected abstract Iterable toTuples(T value); + @Override public void rescan() throws IOException { super.rescan(); diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/CommonJoinExec.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/CommonJoinExec.java index a743407b32..9afa2443d8 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/CommonJoinExec.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/CommonJoinExec.java @@ -19,6 +19,7 @@ package org.apache.tajo.engine.planner.physical; import com.google.common.base.Predicate; +import com.google.common.base.Predicates; import com.google.common.collect.Iterators; import com.google.common.collect.Lists; import org.apache.tajo.catalog.Column; @@ -29,6 +30,7 @@ import org.apache.tajo.plan.expr.BinaryEval; import org.apache.tajo.plan.expr.EvalNode; import org.apache.tajo.plan.expr.EvalTreeUtil; +import org.apache.tajo.plan.expr.EvalType; import org.apache.tajo.plan.logical.JoinNode; import org.apache.tajo.storage.FrameTuple; import org.apache.tajo.storage.NullTuple; @@ -39,28 +41,33 @@ import java.io.IOException; import java.util.Arrays; import java.util.Iterator; -import java.util.LinkedHashSet; import java.util.List; +import java.util.Set; // common exec for all join execs public abstract class CommonJoinExec extends BinaryPhysicalExec { // from logical plan protected JoinNode plan; - protected final boolean hasJoinQual; - protected EvalNode joinQual; // ex) a.id = b.id - protected EvalNode leftJoinFilter; // ex) a > 10 - protected EvalNode rightJoinFilter; // ex) b > 5 + protected EvalNode equiQual; // ex) a.id = b.id + protected EvalNode joinQual; // ex) a.id = b.id + protected EvalNode thetaQual; // ex) a.id > b.id + protected EvalNode leftFilter; // ex) a > 10 + protected EvalNode rightFilter; // ex) b > 5 + + protected final Predicate leftPredicate; + protected final Predicate rightPredicate; protected final Schema leftSchema; protected final Schema rightSchema; + protected final ThetaOperation thetaOp; protected final FrameTuple frameTuple; protected final Tuple outTuple; // projection - protected Projector projector; + protected final Projector projector; public CommonJoinExec(TaskAttemptContext context, JoinNode plan, PhysicalExec outer, PhysicalExec inner) { super(context, SchemaUtil.merge(outer.getSchema(), inner.getSchema()), @@ -70,11 +77,16 @@ public CommonJoinExec(TaskAttemptContext context, JoinNode plan, PhysicalExec ou this.rightSchema = inner.getSchema(); if (plan.hasJoinQual()) { EvalNode[] extracted = extractJoinConditions(plan.getJoinQual(), leftSchema, rightSchema); - joinQual = extracted[0]; - leftJoinFilter = extracted[1]; - rightJoinFilter = extracted[2]; + equiQual = extracted[0]; + joinQual = extracted[1]; + leftFilter = extracted[2]; + rightFilter = extracted[3]; + thetaQual = extracted[4]; } - this.hasJoinQual = joinQual != null; + + this.thetaOp = thetaQual == null ? null : ThetaOperation.valueOf(thetaQual); + this.leftPredicate = toPredicate(leftFilter); + this.rightPredicate = toPredicate(rightFilter); // for projection this.projector = new Projector(context, inSchema, outSchema, plan.getTargets()); @@ -84,39 +96,73 @@ public CommonJoinExec(TaskAttemptContext context, JoinNode plan, PhysicalExec ou this.outTuple = new VTuple(outSchema.size()); } + // non-equi theta join + public boolean isThetaJoin() { + return thetaOp != null; + } + + public boolean isHashJoin() { + return false; + } + private EvalNode[] extractJoinConditions(EvalNode joinQual, Schema leftSchema, Schema rightSchema) { + List equiQuals = Lists.newArrayList(); List joinQuals = Lists.newArrayList(); List leftFilters = Lists.newArrayList(); List rightFilters = Lists.newArrayList(); + EvalNode thetaQual = null; + for (EvalNode eachQual : AlgebraicUtil.toConjunctiveNormalFormArray(joinQual)) { - if (EvalTreeUtil.isJoinQual(null, leftSchema, rightSchema, eachQual, true)) { - joinQuals.add(eachQual); - continue; - } if (!(eachQual instanceof BinaryEval)) { - continue; + continue; //todo 'between', etc. } BinaryEval binaryEval = (BinaryEval)eachQual; - LinkedHashSet leftColumns = EvalTreeUtil.findUniqueColumns(binaryEval.getLeftExpr()); - LinkedHashSet rightColumns = EvalTreeUtil.findUniqueColumns(binaryEval.getRightExpr()); + Set leftColumns = EvalTreeUtil.findUniqueColumns(binaryEval.getLeftExpr()); + Set rightColumns = EvalTreeUtil.findUniqueColumns(binaryEval.getRightExpr()); boolean leftInLeft = leftSchema.containsAny(leftColumns); boolean rightInLeft = leftSchema.containsAny(rightColumns); boolean leftInRight = rightSchema.containsAny(leftColumns); boolean rightInRight = rightSchema.containsAny(rightColumns); - if ((leftInLeft || leftInRight) && (rightInLeft || rightInRight)) { -// throw new IllegalStateException("Invalid filter " + binaryEval.toString()); - continue; // todo this happens sometimes. why? + + boolean columnsFromLeft = leftInLeft || rightInLeft; + boolean columnsFromRight = leftInRight || rightInRight; + if (!columnsFromLeft && !columnsFromRight) { + continue; // todo constant expression : this should be done in logical phase + } + if (columnsFromLeft ^ columnsFromRight) { + if (columnsFromLeft) { + leftFilters.add(eachQual); + } else { + rightFilters.add(eachQual); + } + continue; + } + if ((leftInLeft && rightInLeft) || (leftInRight && rightInRight)) { + continue; // todo not allowed yet : this should be checked in logical phase + } + if (eachQual.getType() == EvalType.EQUAL) { + equiQuals.add(eachQual); + continue; + } + if (leftInLeft) { + binaryEval = AlgebraicUtil.commutate(binaryEval); } - if (leftInLeft || rightInLeft) { - leftFilters.add(eachQual); - } else if (leftInRight || rightInRight) { - rightFilters.add(eachQual); + if (!isHashJoin() || thetaQual != null) { + joinQuals.add(binaryEval); + } else { + thetaQual = binaryEval; } } + if (thetaQual != null && !equiQuals.isEmpty()) { + joinQuals.add(thetaQual); // use equi-join + thetaQual = null; + } return new EvalNode[] { + equiQuals.isEmpty() ? null : AlgebraicUtil.createSingletonExprFromCNF(equiQuals), joinQuals.isEmpty() ? null : AlgebraicUtil.createSingletonExprFromCNF(joinQuals), leftFilters.isEmpty() ? null : AlgebraicUtil.createSingletonExprFromCNF(leftFilters), - rightFilters.isEmpty() ? null : AlgebraicUtil.createSingletonExprFromCNF(rightFilters) + rightFilters.isEmpty() ? null : AlgebraicUtil.createSingletonExprFromCNF(rightFilters), + thetaQual }; } @@ -124,63 +170,104 @@ public JoinNode getPlan() { return plan; } - protected boolean leftFiltered(Tuple left) { - return leftJoinFilter != null && !leftJoinFilter.eval(left).asBool(); + protected final boolean leftFiltered(Tuple left) { + return leftPredicate != null && !leftPredicate.apply(left); } - protected boolean rightFiltered(Tuple right) { - return rightJoinFilter != null && !rightJoinFilter.eval(right).asBool(); + protected final boolean rightFiltered(Tuple right) { + return rightPredicate != null && !rightPredicate.apply(right); } - protected Iterator rightFiltered(Iterable rightTuples) { - if (rightTuples == null) { - return Iterators.emptyIterator(); - } - if (rightJoinFilter == null) { - return rightTuples.iterator(); - } - return Iterators.filter(rightTuples.iterator(), new Predicate() { + protected final Iterator filterLeft(Iterable iterable) { + return filter(leftPredicate, iterable.iterator()); + } + + protected final Iterator filterRight(Iterable iterable) { + return filter(rightPredicate, iterable.iterator()); + } + + protected final Iterator filter(Predicate predicate, Iterator tuples) { + return predicate == null || !tuples.hasNext() ? tuples : Iterators.filter(tuples, predicate); + } + + protected Predicate toPredicate(final EvalNode filter) { + return filter == null ? null : new TuplePredicate(filter); + } + + protected Predicate toFramePredicate(final EvalNode filter) { + return filter == null ? null : new TuplePredicate(filter) { @Override - public boolean apply(Tuple input) { - return rightJoinFilter.eval(input).asBool(); + public boolean apply(Tuple input) { return super.apply(frameTuple.setRight(input)); } + }; + } + + protected final Predicate mergePredicates(Predicate... predicates) { + Predicate current = null; + for (Predicate predicate : predicates) { + if (current == null) { + current = predicate; + } else if (predicate != null) { + current = Predicates.and(current, predicate); } - }); + } + return current; + } + + private static class TuplePredicate implements Predicate { + private final EvalNode filter; + private TuplePredicate(EvalNode filter) { this.filter = filter; } + public boolean apply(Tuple input) { return filter.eval(input).asBool(); } } protected Iterator nullIterator(int length) { return Arrays.asList(NullTuple.create(length)).iterator(); } + @Override + protected void compile() { + super.compile(); + if (joinQual != null) { + joinQual = context.getPrecompiledEval(inSchema, joinQual); + } + // compile filters? + } + @Override public void init() throws IOException { super.init(); - if (hasJoinQual) { + if (equiQual != null) { + equiQual.bind(context.getEvalContext(), inSchema); + } + if (joinQual != null) { joinQual.bind(context.getEvalContext(), inSchema); } - if (leftJoinFilter != null) { - leftJoinFilter.bind(leftSchema); + if (thetaQual != null) { + thetaQual.bind(context.getEvalContext(), inSchema); } - if (rightJoinFilter != null) { - rightJoinFilter.bind(rightSchema); + if (leftFilter != null) { + leftFilter.bind(context.getEvalContext(), leftSchema); + } + if (rightFilter != null) { + rightFilter.bind(context.getEvalContext(), rightSchema); } } @Override - protected void compile() { - if (hasJoinQual) { - joinQual = context.getPrecompiledEval(inSchema, joinQual); - } - // compile filters? + public void rescan() throws IOException { + super.rescan(); + frameTuple.clear(); } @Override public void close() throws IOException { super.close(); + frameTuple.clear(); plan = null; + equiQual = null; joinQual = null; - leftJoinFilter = null; - rightJoinFilter = null; - projector = null; + thetaQual = null; + leftFilter = null; + rightFilter = null; } @Override diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/HashFullOuterJoinExec.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/HashFullOuterJoinExec.java index e2b0a866cd..eefbb55db4 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/HashFullOuterJoinExec.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/HashFullOuterJoinExec.java @@ -104,18 +104,8 @@ public Tuple next() throws IOException { continue; } // getting corresponding right - Pair> hashed = tupleSlots.get(toKey(leftTuple)); - if (hashed == null) { - iterator = nullIterator(rightNumCols); - continue; - } - Iterator rightTuples = rightFiltered(hashed.getSecond()); - if (!rightTuples.hasNext()) { - iterator = nullIterator(rightNumCols); - continue; - } - iterator = rightTuples; - hashed.setFirst(true); // match found + Iterator filtered = filter(leftTuple); + iterator = !filtered.hasNext() ? nullIterator(rightNumCols) : filtered; } return null; @@ -124,7 +114,9 @@ public Tuple next() throws IOException { @Override protected Map>> convert(Map> hashed, boolean fromCache) throws IOException { - Map>> tuples = new HashMap>>(hashed.size()); + Map>> tuples = + isThetaJoin() ? new TreeMap>>() : + new HashMap>>(hashed.size()); for (Map.Entry> entry : hashed.entrySet()) { // flag: initially false (whether this join key had at least one match on the counter part) tuples.put(entry.getKey(), new Pair>(false, entry.getValue())); @@ -132,6 +124,31 @@ protected Map>> convert(Map> return tuples; } + @Override + protected Iterable toTuples(Pair> value) { + return value.getSecond(); + } + + @Override + protected Iterator filterRight(Pair> rightTuples) { + Iterator iterator = super.filterRight(rightTuples); + if (iterator.hasNext()) { + rightTuples.setFirst(true); // has match + } + return iterator; + } + + @Override + protected Iterator filterRights(Iterable>> rightTuples) { + Iterator iterator = super.filterRights(rightTuples); + if (iterator.hasNext()) { + for (Pair> pair : rightTuples) { + pair.setFirst(true); // has match + } + } + return iterator; + } + @Override public void rescan() throws IOException { super.rescan(); diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/HashJoinExec.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/HashJoinExec.java index a4215fafb4..25f9f6b8b8 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/HashJoinExec.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/HashJoinExec.java @@ -35,9 +35,17 @@ public HashJoinExec(TaskAttemptContext context, JoinNode plan, PhysicalExec left @Override protected Map> convert(Map> hashed, boolean fromCache) throws IOException { + if (isThetaJoin()) { + return new TreeMap>(hashed); + } return fromCache ? new HashMap>(hashed) : hashed; } + @Override + protected Iterable toTuples(List value) { + return value; + } + @Override public Tuple next() throws IOException { if (first) { @@ -59,18 +67,12 @@ public Tuple next() throws IOException { frameTuple.setLeft(leftTuple); // getting corresponding right - Iterable hashed = getRights(toKey(leftTuple)); - Iterator rightTuples = rightFiltered(hashed); - if (rightTuples.hasNext()) { - iterator = rightTuples; + Iterator filtered = filter(leftTuple); + if (filtered.hasNext()) { + iterator = filtered; } } return null; } - - private Iterable getRights(Tuple key) { - return tupleSlots.get(key); - } - } diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/HashLeftAntiJoinExec.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/HashLeftAntiJoinExec.java index 8239270580..18238d39fa 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/HashLeftAntiJoinExec.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/HashLeftAntiJoinExec.java @@ -23,6 +23,7 @@ import org.apache.tajo.storage.Tuple; import java.io.IOException; +import java.util.Iterator; import java.util.List; /** @@ -70,8 +71,8 @@ public Tuple next() throws IOException { frameTuple.setLeft(leftTuple); // Try to find a hash bucket in in-memory hash table - List hashed = tupleSlots.get(toKey(leftTuple)); - if (hashed == null || !rightFiltered(hashed).hasNext()) { + Iterator filtered = filter(leftTuple); + if (!filtered.hasNext()) { iterator = nullIterator(0); } } diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/HashLeftOuterJoinExec.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/HashLeftOuterJoinExec.java index 8613eacb7b..3cb1736440 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/HashLeftOuterJoinExec.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/HashLeftOuterJoinExec.java @@ -61,15 +61,8 @@ public Tuple next() throws IOException { } // getting corresponding right - List hashed = tupleSlots.get(toKey(leftTuple)); - Iterator rightTuples = rightFiltered(hashed); - if (!rightTuples.hasNext()) { - //this left tuple doesn't have a match on the right.But full outer join => we should keep it anyway - //output a tuple with the nulls padded rightTuple - iterator = nullIterator(rightNumCols); - continue; - } - iterator = rightTuples; + Iterator filtered = filter(leftTuple); + iterator = filtered.hasNext() ? filtered : nullIterator(rightNumCols); } return null; diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/HashLeftSemiJoinExec.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/HashLeftSemiJoinExec.java index 41e842a533..4730a741df 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/HashLeftSemiJoinExec.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/HashLeftSemiJoinExec.java @@ -23,6 +23,7 @@ import org.apache.tajo.storage.Tuple; import java.io.IOException; +import java.util.Iterator; import java.util.List; /** @@ -72,8 +73,8 @@ public Tuple next() throws IOException { frameTuple.setLeft(leftTuple); // Try to find a hash bucket in in-memory hash table - List hashed = tupleSlots.get(toKey(leftTuple)); - if (hashed != null && rightFiltered(hashed).hasNext()) { + Iterator filtered = filter(leftTuple); + if (filtered.hasNext()) { // if found, it gets a hash bucket from the hash table. iterator = nullIterator(0); } diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/MergeFullOuterJoinExec.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/MergeFullOuterJoinExec.java index 13b73c3fa4..37a1fbdf14 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/MergeFullOuterJoinExec.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/MergeFullOuterJoinExec.java @@ -22,7 +22,6 @@ import org.apache.tajo.catalog.SortSpec; import org.apache.tajo.engine.utils.TupleUtil; import org.apache.tajo.plan.logical.JoinNode; -import org.apache.tajo.storage.FrameTuple; import org.apache.tajo.storage.Tuple; import org.apache.tajo.storage.TupleComparator; import org.apache.tajo.storage.VTuple; @@ -35,11 +34,8 @@ public class MergeFullOuterJoinExec extends CommonJoinExec { - // temporal tuples and states for nested loop join - private FrameTuple frameTuple; private Tuple leftTuple = null; private Tuple rightTuple = null; - private Tuple outTuple = null; private Tuple leftNext = null; private List leftTupleSlots; @@ -75,10 +71,6 @@ public MergeFullOuterJoinExec(TaskAttemptContext context, JoinNode plan, Physica this.tupleComparator = PhysicalPlanUtil.getComparatorsFromJoinQual( plan.getJoinQual(), leftChild.getSchema(), rightChild.getSchema()); - // for join - frameTuple = new FrameTuple(); - outTuple = new VTuple(outSchema.size()); - leftNumCols = leftChild.getSchema().size(); rightNumCols = rightChild.getSchema().size(); } @@ -269,9 +261,10 @@ public Tuple next() throws IOException { Tuple aTuple = new VTuple(rightTupleSlots.get(posRightTupleSlots)); posRightTupleSlots = posRightTupleSlots + 1; frameTuple.set(leftNext, aTuple); - joinQual.eval(frameTuple); - projector.eval(frameTuple, outTuple); - return outTuple; + if (joinQual == null || joinQual.eval(frameTuple).asBool()) { + projector.eval(frameTuple, outTuple); + return outTuple; + } } else { // right (inner) slots reached end and should be rewind if there are still tuples in the outer slots if(posLeftTupleSlots <= (leftTupleSlots.size()-1)) { @@ -283,9 +276,10 @@ public Tuple next() throws IOException { posLeftTupleSlots = posLeftTupleSlots + 1; frameTuple.set(leftNext, aTuple); - joinQual.eval(frameTuple); - projector.eval(frameTuple, outTuple); - return outTuple; + if (joinQual == null || joinQual.eval(frameTuple).asBool()) { + projector.eval(frameTuple, outTuple); + return outTuple; + } } } } // the second if end false diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/MergeJoinExec.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/MergeJoinExec.java index bf9b4cdacd..c769b4d628 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/MergeJoinExec.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/MergeJoinExec.java @@ -21,10 +21,8 @@ import com.google.common.base.Preconditions; import org.apache.tajo.catalog.SortSpec; import org.apache.tajo.plan.logical.JoinNode; -import org.apache.tajo.storage.FrameTuple; import org.apache.tajo.storage.Tuple; import org.apache.tajo.storage.TupleComparator; -import org.apache.tajo.storage.VTuple; import org.apache.tajo.worker.TaskAttemptContext; import java.io.IOException; @@ -35,10 +33,8 @@ public class MergeJoinExec extends CommonJoinExec { // temporal tuples and states for nested loop join - private FrameTuple frameTuple; private Tuple outerTuple = null; private Tuple innerTuple = null; - private Tuple outTuple = null; private Tuple outerNext = null; private List outerTupleSlots; @@ -71,10 +67,6 @@ public MergeJoinExec(TaskAttemptContext context, JoinNode plan, PhysicalExec out plan.getJoinQual(), outer.getSchema(), inner.getSchema()); this.outerIterator = outerTupleSlots.iterator(); this.innerIterator = innerTupleSlots.iterator(); - - // for join - frameTuple = new FrameTuple(); - outTuple = new VTuple(outSchema.size()); } public Tuple next() throws IOException { @@ -143,7 +135,7 @@ public Tuple next() throws IOException { frameTuple.set(outerNext, innerIterator.next()); - if (joinQual.eval(frameTuple).isTrue()) { + if (joinQual == null || joinQual.eval(frameTuple).isTrue()) { projector.eval(frameTuple, outTuple); return outTuple; } diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/NLJoinExec.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/NLJoinExec.java index 964a5232f8..7df6e36f59 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/NLJoinExec.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/NLJoinExec.java @@ -18,10 +18,9 @@ package org.apache.tajo.engine.planner.physical; +import com.google.common.base.Predicate; import org.apache.tajo.plan.logical.JoinNode; -import org.apache.tajo.storage.FrameTuple; import org.apache.tajo.storage.Tuple; -import org.apache.tajo.storage.VTuple; import org.apache.tajo.worker.TaskAttemptContext; import java.io.IOException; @@ -30,18 +29,22 @@ public class NLJoinExec extends CommonJoinExec { // temporal tuples and states for nested loop join private boolean needNewOuter; - private FrameTuple frameTuple; private Tuple outerTuple = null; private Tuple innerTuple = null; - private Tuple outTuple = null; + + private Predicate predicate; public NLJoinExec(TaskAttemptContext context, JoinNode plan, PhysicalExec outer, PhysicalExec inner) { super(context, plan, outer, inner); // for join needNewOuter = true; - frameTuple = new FrameTuple(); - outTuple = new VTuple(outSchema.size()); + } + + @SuppressWarnings("unchecked") + public void init() throws IOException { + super.init(); + predicate = mergePredicates(toPredicate(equiQual), toPredicate(joinQual)); } public Tuple next() throws IOException { @@ -62,12 +65,7 @@ public Tuple next() throws IOException { } frameTuple.set(outerTuple, innerTuple); - if (hasJoinQual) { - if (joinQual.eval(frameTuple).isTrue()) { - projector.eval(frameTuple, outTuple); - return outTuple; - } - } else { + if (predicate == null || predicate.apply(frameTuple)) { projector.eval(frameTuple, outTuple); return outTuple; } diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/RightOuterMergeJoinExec.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/RightOuterMergeJoinExec.java index fd825b1f45..c84cc6f792 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/RightOuterMergeJoinExec.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/RightOuterMergeJoinExec.java @@ -294,7 +294,7 @@ public Tuple next() throws IOException { posRightTupleSlots = posRightTupleSlots + 1; frameTuple.set(nextLeft, aTuple); - if (joinQual.eval(frameTuple).asBool()) { + if (joinQual == null || joinQual.eval(frameTuple).asBool()) { projector.eval(frameTuple, outTuple); return outTuple; } else { @@ -316,7 +316,7 @@ public Tuple next() throws IOException { frameTuple.set(nextLeft, aTuple); - if (joinQual.eval(frameTuple).asBool()) { + if (joinQual == null || joinQual.eval(frameTuple).asBool()) { projector.eval(frameTuple, outTuple); return outTuple; } else { diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/SelectionExec.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/SelectionExec.java index 7f5bbe96d9..f82f114967 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/SelectionExec.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/SelectionExec.java @@ -18,6 +18,7 @@ package org.apache.tajo.engine.planner.physical; +import org.apache.tajo.catalog.Schema; import org.apache.tajo.engine.codegen.CompilationError; import org.apache.tajo.plan.expr.EvalNode; import org.apache.tajo.plan.logical.SelectionNode; @@ -32,8 +33,14 @@ public class SelectionExec extends UnaryPhysicalExec { public SelectionExec(TaskAttemptContext context, SelectionNode plan, PhysicalExec child) { - super(context, plan.getInSchema(), plan.getOutSchema(), child); - this.qual = plan.getQual(); + this(context, plan.getInSchema(), plan.getOutSchema(), plan.getQual(), child); + } + + public SelectionExec(TaskAttemptContext context, + Schema input, Schema output, EvalNode qual, + PhysicalExec child) { + super(context, input, output, child); + this.qual = qual; } @Override diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/ThetaOperation.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/ThetaOperation.java new file mode 100644 index 0000000000..d71ab86dad --- /dev/null +++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/ThetaOperation.java @@ -0,0 +1,83 @@ +/** + * 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.tajo.engine.planner.physical; + +import com.google.common.base.Predicate; +import com.google.common.collect.Iterables; +import org.apache.tajo.plan.expr.EvalNode; +import org.apache.tajo.storage.Tuple; + +import java.util.Arrays; +import java.util.SortedMap; + +@SuppressWarnings("unchecked") +public enum ThetaOperation { + LE { + @Override + Iterable filter(SortedMap sortedMap, Tuple tuple) { + Iterable iterable = sortedMap.headMap(tuple).values(); + T equal = sortedMap.get(tuple); + return equal == null ? iterable : Iterables.concat(iterable, Arrays.asList(equal)); + } + }, + LT { + @Override + Iterable filter(SortedMap sortedMap, Tuple tuple) { + return sortedMap.headMap(tuple).values(); + } + }, + GE { + @Override + Iterable filter(SortedMap sortedMap, Tuple tuple) { + return sortedMap.tailMap(tuple).values(); + } + }, + GT { + @Override + Iterable filter(SortedMap sortedMap, Tuple tuple) { + Iterable iterable = sortedMap.tailMap(tuple).values(); + final T equal = sortedMap.get(tuple); + return equal == null ? iterable : Iterables.filter(iterable, + new Predicate() { public boolean apply(T input) { return input != equal; } } + ); + } + }, + NE { + @Override + Iterable filter(SortedMap sortedMap, Tuple tuple) { + Iterable iterable = sortedMap.values(); + final T equal = sortedMap.get(tuple); + return equal == null ? iterable : Iterables.filter(iterable, + new Predicate() { public boolean apply(T input) { return input != equal; } } + ); + } + }; + abstract Iterable filter(SortedMap sortedMap, Tuple tuple); + + public static ThetaOperation valueOf(EvalNode eval) { + switch (eval.getType()) { + case NOT_EQUAL: return NE; + case LTH: return LT; + case LEQ: return LE; + case GTH: return GT; + case GEQ: return GE; + default: throw new IllegalArgumentException("Unsupported operation " + eval.getType()); + } + } +} diff --git a/tajo-core/src/test/java/org/apache/tajo/QueryTestCaseBase.java b/tajo-core/src/test/java/org/apache/tajo/QueryTestCaseBase.java index 22c3640d30..a25fe2a47e 100644 --- a/tajo-core/src/test/java/org/apache/tajo/QueryTestCaseBase.java +++ b/tajo-core/src/test/java/org/apache/tajo/QueryTestCaseBase.java @@ -346,6 +346,7 @@ protected void starting(Description description) { protected static @interface SimpleTest { String[] queries(); String[] cleanupTables() default {}; + boolean withExplain() default false; } protected void runSimpleTests() throws Exception { @@ -358,15 +359,20 @@ protected void runSimpleTests() throws Exception { String[] queries = annotation.queries(); try { for (int i = 0; i < queries.length; i++) { + String prefix = ""; + if (annotation.withExplain()) { + ResultSet result = client.executeQueryAndGetResult("explain " + queries[i]); + prefix = resultSetToString(result); + } ResultSet result = client.executeQueryAndGetResult(queries[i]); Path resultPath = StorageUtil.concatPath( currentResultPath, methodName + "." + String.valueOf(i + 1) + ".result"); if (currentResultFS.exists(resultPath)) { assertEquals("Result Verification for: " + (i+1) + "th test", - FileUtil.readTextFromStream(currentResultFS.open(resultPath)), resultSetToString(result).trim()); + FileUtil.readTextFromStream(currentResultFS.open(resultPath)), prefix + resultSetToString(result)); } else if (!isNull(result)) { // If there is no result file expected, create gold files for new tests. - FileUtil.writeTextToStream(resultSetToString(result).trim(), currentResultFS.create(resultPath)); + FileUtil.writeTextToStream( prefix + resultSetToString(result), currentResultFS.create(resultPath)); LOG.info("New test output for " + current.getDisplayName() + " is written to " + resultPath); // should be copied to src directory } diff --git a/tajo-core/src/test/java/org/apache/tajo/TestHashThetaJoin.java b/tajo-core/src/test/java/org/apache/tajo/TestHashThetaJoin.java new file mode 100644 index 0000000000..60da847299 --- /dev/null +++ b/tajo-core/src/test/java/org/apache/tajo/TestHashThetaJoin.java @@ -0,0 +1,97 @@ +/** + * 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 WHERE + * limitations under the License. + */ + +package org.apache.tajo; + +import org.apache.tajo.conf.TajoConf; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +@Category(IntegrationTest.class) +public class TestHashThetaJoin extends QueryTestCaseBase { + + public TestHashThetaJoin() { + super(TajoConstants.DEFAULT_DATABASE_NAME); + + testingCluster.setAllTajoDaemonConfValue(TajoConf.ConfVars.$TEST_BROADCAST_JOIN_ENABLED.varname, + TajoConf.ConfVars.$TEST_BROADCAST_JOIN_ENABLED.defaultVal); + testingCluster.setAllTajoDaemonConfValue(TajoConf.ConfVars.$DIST_QUERY_BROADCAST_JOIN_THRESHOLD.varname, + TajoConf.ConfVars.$DIST_QUERY_BROADCAST_JOIN_THRESHOLD.defaultVal); + + testingCluster.setAllTajoDaemonConfValue( + TajoConf.ConfVars.$EXECUTOR_HASH_JOIN_SIZE_THRESHOLD.varname, + TajoConf.ConfVars.$EXECUTOR_HASH_JOIN_SIZE_THRESHOLD.defaultVal); + + testingCluster.setAllTajoDaemonConfValue(TajoConf.ConfVars.$EXECUTOR_HASH_JOIN_SIZE_THRESHOLD.varname, + TajoConf.ConfVars.$EXECUTOR_HASH_JOIN_SIZE_THRESHOLD.defaultVal); + testingCluster.setAllTajoDaemonConfValue(TajoConf.ConfVars.$EXECUTOR_GROUPBY_INMEMORY_HASH_THRESHOLD.varname, + TajoConf.ConfVars.$EXECUTOR_GROUPBY_INMEMORY_HASH_THRESHOLD.defaultVal); + + testingCluster.setAllTajoDaemonConfValue( + TajoConf.ConfVars.$EXECUTOR_HASH_JOIN_SIZE_THRESHOLD.varname, String.valueOf(256 * 1048576)); + testingCluster.setAllTajoDaemonConfValue(TajoConf.ConfVars.$EXECUTOR_HASH_JOIN_SIZE_THRESHOLD.varname, + String.valueOf(256 * 1048576)); + testingCluster.setAllTajoDaemonConfValue(TajoConf.ConfVars.$EXECUTOR_GROUPBY_INMEMORY_HASH_THRESHOLD.varname, + String.valueOf(256 * 1048576)); + } + + @Test + @SimpleTest(withExplain = true, queries = { + "select n_nationkey, r_regionkey from nation join region on n_nationkey >= r_regionkey WHERE n_nationkey < 7;", + "select n_nationkey, r_regionkey from nation join region on n_nationkey <= r_regionkey WHERE n_nationkey < 7;", + "select n_nationkey, r_regionkey from nation join region on n_nationkey > r_regionkey WHERE n_nationkey < 7;", + "select n_nationkey, r_regionkey from nation join region on n_nationkey < r_regionkey WHERE n_nationkey < 7;", + "select n_nationkey, r_regionkey from nation join region on n_nationkey != r_regionkey WHERE n_nationkey < 7;"}) + public final void testThetaJoinInner() throws Exception { + runSimpleTests(); + } + + @Test + @SimpleTest(withExplain = true, queries = { + "select n_nationkey, r_regionkey from nation left join region on n_nationkey >= r_regionkey WHERE n_nationkey < 7;", + "select n_nationkey, r_regionkey from nation left join region on n_nationkey <= r_regionkey WHERE n_nationkey < 7;", + "select n_nationkey, r_regionkey from nation left join region on n_nationkey > r_regionkey WHERE n_nationkey < 7;", + "select n_nationkey, r_regionkey from nation left join region on n_nationkey < r_regionkey WHERE n_nationkey < 7;", + "select n_nationkey, r_regionkey from nation left join region on n_nationkey != r_regionkey WHERE n_nationkey < 7;"}) + public final void testThetaJoinLeftOuter() throws Exception { + runSimpleTests(); + } + + @Test + @SimpleTest(withExplain = true, queries = { + "select n_nationkey, r_regionkey from nation right join region on n_nationkey >= r_regionkey WHERE n_nationkey < 7;", + "select n_nationkey, r_regionkey from nation right join region on n_nationkey <= r_regionkey WHERE n_nationkey < 7;", + "select n_nationkey, r_regionkey from nation right join region on n_nationkey > r_regionkey WHERE n_nationkey < 7;", + "select n_nationkey, r_regionkey from nation right join region on n_nationkey < r_regionkey WHERE n_nationkey < 7;", + "select n_nationkey, r_regionkey from nation right join region on n_nationkey != r_regionkey WHERE n_nationkey < 7;"}) + public final void testThetaJoinRightOuter() throws Exception { + runSimpleTests(); + } + + @Test + @SimpleTest(withExplain = true, queries = { + "select n_nationkey, r_regionkey from nation full join region on n_nationkey >= r_regionkey WHERE n_nationkey < 7;", + "select n_nationkey, r_regionkey from nation full join region on n_nationkey <= r_regionkey WHERE n_nationkey < 7;", + "select n_nationkey, r_regionkey from nation full join region on n_nationkey > r_regionkey WHERE n_nationkey < 7;", + "select n_nationkey, r_regionkey from nation full join region on n_nationkey < r_regionkey WHERE n_nationkey < 7;", + "select n_nationkey, r_regionkey from nation full join region on n_nationkey != r_regionkey WHERE n_nationkey < 7;"}) + public final void testThetaJoinFullOuter() throws Exception { + runSimpleTests(); + } + +} diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/planner/TestLogicalPlanner.java b/tajo-core/src/test/java/org/apache/tajo/engine/planner/TestLogicalPlanner.java index 0b59bc7a35..fd8d721f72 100644 --- a/tajo-core/src/test/java/org/apache/tajo/engine/planner/TestLogicalPlanner.java +++ b/tajo-core/src/test/java/org/apache/tajo/engine/planner/TestLogicalPlanner.java @@ -647,7 +647,7 @@ public final void testJoinWithMultipleJoinQual4() throws IOException, PlanningEx /* following code is commented because theta join is not supported yet * TODO It SHOULD be restored after TAJO-742 is resolved. */ - //joinQualMap.put(joinQual, Boolean.FALSE); + //joinQualMap.put(equiQual, Boolean.FALSE); LogicalNode[] nodes = PlannerUtil.findAllNodes(node, NodeType.JOIN); for(LogicalNode eachNode : nodes) { diff --git a/tajo-core/src/test/resources/queries/TestJoinQuery/testCrossJoinWithThetaJoinConditionInWhere.sql b/tajo-core/src/test/resources/queries/TestJoinQuery/testCrossJoinWithThetaJoinConditionInWhere.sql index 4e20e16baa..dc77826a98 100644 --- a/tajo-core/src/test/resources/queries/TestJoinQuery/testCrossJoinWithThetaJoinConditionInWhere.sql +++ b/tajo-core/src/test/resources/queries/TestJoinQuery/testCrossJoinWithThetaJoinConditionInWhere.sql @@ -1,2 +1,2 @@ select a.r_name as a_name, b.r_name as b_name from region a, region b -where a_name < b_name; \ No newline at end of file +where a_name < b_name order by a_name, b_name; \ No newline at end of file diff --git a/tajo-core/src/test/resources/results/TestHashThetaJoin/testThetaJoinFullOuter.1.result b/tajo-core/src/test/resources/results/TestHashThetaJoin/testThetaJoinFullOuter.1.result new file mode 100644 index 0000000000..ea4a606814 --- /dev/null +++ b/tajo-core/src/test/resources/results/TestHashThetaJoin/testThetaJoinFullOuter.1.result @@ -0,0 +1,44 @@ +explain +------------------------------- +SELECTION(3) + => Search Cond: default.nation.n_nationkey (INT4) < 7 + JOIN(6)(FULL_OUTER) + => Join Cond: default.nation.n_nationkey (INT4) >= default.region.r_regionkey (INT4) + => target list: default.nation.n_nationkey (INT4), default.region.r_regionkey (INT4) + => out schema: {(2) default.nation.n_nationkey (INT4), default.region.r_regionkey (INT4)} + => in schema: {(2) default.nation.n_nationkey (INT4), default.region.r_regionkey (INT4)} + SCAN(1) on default.region + => target list: default.region.r_regionkey (INT4) + => out schema: {(1) default.region.r_regionkey (INT4)} + => in schema: {(3) default.region.r_regionkey (INT4), default.region.r_name (TEXT), default.region.r_comment (TEXT)} + SCAN(0) on default.nation + => target list: default.nation.n_nationkey (INT4) + => out schema: {(1) default.nation.n_nationkey (INT4)} + => in schema: {(4) default.nation.n_nationkey (INT4), default.nation.n_name (TEXT), default.nation.n_regionkey (INT4), default.nation.n_comment (TEXT)} +n_nationkey,r_regionkey +------------------------------- +0,0 +1,0 +1,1 +2,0 +2,1 +2,2 +3,0 +3,1 +3,2 +3,3 +4,0 +4,1 +4,2 +4,3 +4,4 +5,0 +5,1 +5,2 +5,3 +5,4 +6,0 +6,1 +6,2 +6,3 +6,4 diff --git a/tajo-core/src/test/resources/results/TestHashThetaJoin/testThetaJoinFullOuter.2.result b/tajo-core/src/test/resources/results/TestHashThetaJoin/testThetaJoinFullOuter.2.result new file mode 100644 index 0000000000..dd6a6adbd5 --- /dev/null +++ b/tajo-core/src/test/resources/results/TestHashThetaJoin/testThetaJoinFullOuter.2.result @@ -0,0 +1,36 @@ +explain +------------------------------- +SELECTION(3) + => Search Cond: default.nation.n_nationkey (INT4) < 7 + JOIN(6)(FULL_OUTER) + => Join Cond: default.nation.n_nationkey (INT4) <= default.region.r_regionkey (INT4) + => target list: default.nation.n_nationkey (INT4), default.region.r_regionkey (INT4) + => out schema: {(2) default.nation.n_nationkey (INT4), default.region.r_regionkey (INT4)} + => in schema: {(2) default.nation.n_nationkey (INT4), default.region.r_regionkey (INT4)} + SCAN(1) on default.region + => target list: default.region.r_regionkey (INT4) + => out schema: {(1) default.region.r_regionkey (INT4)} + => in schema: {(3) default.region.r_regionkey (INT4), default.region.r_name (TEXT), default.region.r_comment (TEXT)} + SCAN(0) on default.nation + => target list: default.nation.n_nationkey (INT4) + => out schema: {(1) default.nation.n_nationkey (INT4)} + => in schema: {(4) default.nation.n_nationkey (INT4), default.nation.n_name (TEXT), default.nation.n_regionkey (INT4), default.nation.n_comment (TEXT)} +n_nationkey,r_regionkey +------------------------------- +0,0 +0,1 +0,2 +0,3 +0,4 +1,1 +1,2 +1,3 +1,4 +2,2 +2,3 +2,4 +3,3 +3,4 +4,4 +5,null +6,null diff --git a/tajo-core/src/test/resources/results/TestHashThetaJoin/testThetaJoinFullOuter.3.result b/tajo-core/src/test/resources/results/TestHashThetaJoin/testThetaJoinFullOuter.3.result new file mode 100644 index 0000000000..9aa9e5859e --- /dev/null +++ b/tajo-core/src/test/resources/results/TestHashThetaJoin/testThetaJoinFullOuter.3.result @@ -0,0 +1,40 @@ +explain +------------------------------- +SELECTION(3) + => Search Cond: default.nation.n_nationkey (INT4) < 7 + JOIN(6)(FULL_OUTER) + => Join Cond: default.nation.n_nationkey (INT4) > default.region.r_regionkey (INT4) + => target list: default.nation.n_nationkey (INT4), default.region.r_regionkey (INT4) + => out schema: {(2) default.nation.n_nationkey (INT4), default.region.r_regionkey (INT4)} + => in schema: {(2) default.nation.n_nationkey (INT4), default.region.r_regionkey (INT4)} + SCAN(1) on default.region + => target list: default.region.r_regionkey (INT4) + => out schema: {(1) default.region.r_regionkey (INT4)} + => in schema: {(3) default.region.r_regionkey (INT4), default.region.r_name (TEXT), default.region.r_comment (TEXT)} + SCAN(0) on default.nation + => target list: default.nation.n_nationkey (INT4) + => out schema: {(1) default.nation.n_nationkey (INT4)} + => in schema: {(4) default.nation.n_nationkey (INT4), default.nation.n_name (TEXT), default.nation.n_regionkey (INT4), default.nation.n_comment (TEXT)} +n_nationkey,r_regionkey +------------------------------- +0,null +1,0 +2,0 +2,1 +3,0 +3,1 +3,2 +4,0 +4,1 +4,2 +4,3 +5,0 +5,1 +5,2 +5,3 +5,4 +6,0 +6,1 +6,2 +6,3 +6,4 diff --git a/tajo-core/src/test/resources/results/TestHashThetaJoin/testThetaJoinFullOuter.4.result b/tajo-core/src/test/resources/results/TestHashThetaJoin/testThetaJoinFullOuter.4.result new file mode 100644 index 0000000000..0f0e18a43a --- /dev/null +++ b/tajo-core/src/test/resources/results/TestHashThetaJoin/testThetaJoinFullOuter.4.result @@ -0,0 +1,32 @@ +explain +------------------------------- +SELECTION(3) + => Search Cond: default.nation.n_nationkey (INT4) < 7 + JOIN(6)(FULL_OUTER) + => Join Cond: default.nation.n_nationkey (INT4) < default.region.r_regionkey (INT4) + => target list: default.nation.n_nationkey (INT4), default.region.r_regionkey (INT4) + => out schema: {(2) default.nation.n_nationkey (INT4), default.region.r_regionkey (INT4)} + => in schema: {(2) default.nation.n_nationkey (INT4), default.region.r_regionkey (INT4)} + SCAN(1) on default.region + => target list: default.region.r_regionkey (INT4) + => out schema: {(1) default.region.r_regionkey (INT4)} + => in schema: {(3) default.region.r_regionkey (INT4), default.region.r_name (TEXT), default.region.r_comment (TEXT)} + SCAN(0) on default.nation + => target list: default.nation.n_nationkey (INT4) + => out schema: {(1) default.nation.n_nationkey (INT4)} + => in schema: {(4) default.nation.n_nationkey (INT4), default.nation.n_name (TEXT), default.nation.n_regionkey (INT4), default.nation.n_comment (TEXT)} +n_nationkey,r_regionkey +------------------------------- +0,1 +0,2 +0,3 +0,4 +1,2 +1,3 +1,4 +2,3 +2,4 +3,4 +4,null +5,null +6,null diff --git a/tajo-core/src/test/resources/results/TestHashThetaJoin/testThetaJoinFullOuter.5.result b/tajo-core/src/test/resources/results/TestHashThetaJoin/testThetaJoinFullOuter.5.result new file mode 100644 index 0000000000..6a03961f24 --- /dev/null +++ b/tajo-core/src/test/resources/results/TestHashThetaJoin/testThetaJoinFullOuter.5.result @@ -0,0 +1,49 @@ +explain +------------------------------- +SELECTION(3) + => Search Cond: default.nation.n_nationkey (INT4) < 7 + JOIN(6)(FULL_OUTER) + => Join Cond: default.nation.n_nationkey (INT4) <> default.region.r_regionkey (INT4) + => target list: default.nation.n_nationkey (INT4), default.region.r_regionkey (INT4) + => out schema: {(2) default.nation.n_nationkey (INT4), default.region.r_regionkey (INT4)} + => in schema: {(2) default.nation.n_nationkey (INT4), default.region.r_regionkey (INT4)} + SCAN(1) on default.region + => target list: default.region.r_regionkey (INT4) + => out schema: {(1) default.region.r_regionkey (INT4)} + => in schema: {(3) default.region.r_regionkey (INT4), default.region.r_name (TEXT), default.region.r_comment (TEXT)} + SCAN(0) on default.nation + => target list: default.nation.n_nationkey (INT4) + => out schema: {(1) default.nation.n_nationkey (INT4)} + => in schema: {(4) default.nation.n_nationkey (INT4), default.nation.n_name (TEXT), default.nation.n_regionkey (INT4), default.nation.n_comment (TEXT)} +n_nationkey,r_regionkey +------------------------------- +0,1 +0,2 +0,3 +0,4 +1,0 +1,2 +1,3 +1,4 +2,0 +2,1 +2,3 +2,4 +3,0 +3,1 +3,2 +3,4 +4,0 +4,1 +4,2 +4,3 +5,0 +5,1 +5,2 +5,3 +5,4 +6,0 +6,1 +6,2 +6,3 +6,4 diff --git a/tajo-core/src/test/resources/results/TestHashThetaJoin/testThetaJoinInner.1.result b/tajo-core/src/test/resources/results/TestHashThetaJoin/testThetaJoinInner.1.result new file mode 100644 index 0000000000..220b05adae --- /dev/null +++ b/tajo-core/src/test/resources/results/TestHashThetaJoin/testThetaJoinInner.1.result @@ -0,0 +1,43 @@ +explain +------------------------------- +JOIN(6)(INNER) + => Join Cond: default.nation.n_nationkey (INT4) >= default.region.r_regionkey (INT4) + => target list: default.nation.n_nationkey (INT4), default.region.r_regionkey (INT4) + => out schema: {(2) default.nation.n_nationkey (INT4), default.region.r_regionkey (INT4)} + => in schema: {(2) default.nation.n_nationkey (INT4), default.region.r_regionkey (INT4)} + SCAN(1) on default.region + => target list: default.region.r_regionkey (INT4) + => out schema: {(1) default.region.r_regionkey (INT4)} + => in schema: {(3) default.region.r_regionkey (INT4), default.region.r_name (TEXT), default.region.r_comment (TEXT)} + SCAN(0) on default.nation + => filter: default.nation.n_nationkey (INT4) < 7 + => target list: default.nation.n_nationkey (INT4) + => out schema: {(1) default.nation.n_nationkey (INT4)} + => in schema: {(4) default.nation.n_nationkey (INT4), default.nation.n_name (TEXT), default.nation.n_regionkey (INT4), default.nation.n_comment (TEXT)} +n_nationkey,r_regionkey +------------------------------- +0,0 +1,0 +1,1 +2,0 +2,1 +2,2 +3,0 +3,1 +3,2 +3,3 +4,0 +4,1 +4,2 +4,3 +4,4 +5,0 +5,1 +5,2 +5,3 +5,4 +6,0 +6,1 +6,2 +6,3 +6,4 diff --git a/tajo-core/src/test/resources/results/TestHashThetaJoin/testThetaJoinInner.2.result b/tajo-core/src/test/resources/results/TestHashThetaJoin/testThetaJoinInner.2.result new file mode 100644 index 0000000000..e903ef65e8 --- /dev/null +++ b/tajo-core/src/test/resources/results/TestHashThetaJoin/testThetaJoinInner.2.result @@ -0,0 +1,33 @@ +explain +------------------------------- +JOIN(6)(INNER) + => Join Cond: default.nation.n_nationkey (INT4) <= default.region.r_regionkey (INT4) + => target list: default.nation.n_nationkey (INT4), default.region.r_regionkey (INT4) + => out schema: {(2) default.nation.n_nationkey (INT4), default.region.r_regionkey (INT4)} + => in schema: {(2) default.nation.n_nationkey (INT4), default.region.r_regionkey (INT4)} + SCAN(1) on default.region + => target list: default.region.r_regionkey (INT4) + => out schema: {(1) default.region.r_regionkey (INT4)} + => in schema: {(3) default.region.r_regionkey (INT4), default.region.r_name (TEXT), default.region.r_comment (TEXT)} + SCAN(0) on default.nation + => filter: default.nation.n_nationkey (INT4) < 7 + => target list: default.nation.n_nationkey (INT4) + => out schema: {(1) default.nation.n_nationkey (INT4)} + => in schema: {(4) default.nation.n_nationkey (INT4), default.nation.n_name (TEXT), default.nation.n_regionkey (INT4), default.nation.n_comment (TEXT)} +n_nationkey,r_regionkey +------------------------------- +0,0 +0,1 +0,2 +0,3 +0,4 +1,1 +1,2 +1,3 +1,4 +2,2 +2,3 +2,4 +3,3 +3,4 +4,4 diff --git a/tajo-core/src/test/resources/results/TestHashThetaJoin/testThetaJoinInner.3.result b/tajo-core/src/test/resources/results/TestHashThetaJoin/testThetaJoinInner.3.result new file mode 100644 index 0000000000..e212f4253a --- /dev/null +++ b/tajo-core/src/test/resources/results/TestHashThetaJoin/testThetaJoinInner.3.result @@ -0,0 +1,38 @@ +explain +------------------------------- +JOIN(6)(INNER) + => Join Cond: default.nation.n_nationkey (INT4) > default.region.r_regionkey (INT4) + => target list: default.nation.n_nationkey (INT4), default.region.r_regionkey (INT4) + => out schema: {(2) default.nation.n_nationkey (INT4), default.region.r_regionkey (INT4)} + => in schema: {(2) default.nation.n_nationkey (INT4), default.region.r_regionkey (INT4)} + SCAN(1) on default.region + => target list: default.region.r_regionkey (INT4) + => out schema: {(1) default.region.r_regionkey (INT4)} + => in schema: {(3) default.region.r_regionkey (INT4), default.region.r_name (TEXT), default.region.r_comment (TEXT)} + SCAN(0) on default.nation + => filter: default.nation.n_nationkey (INT4) < 7 + => target list: default.nation.n_nationkey (INT4) + => out schema: {(1) default.nation.n_nationkey (INT4)} + => in schema: {(4) default.nation.n_nationkey (INT4), default.nation.n_name (TEXT), default.nation.n_regionkey (INT4), default.nation.n_comment (TEXT)} +n_nationkey,r_regionkey +------------------------------- +1,0 +2,0 +2,1 +3,0 +3,1 +3,2 +4,0 +4,1 +4,2 +4,3 +5,0 +5,1 +5,2 +5,3 +5,4 +6,0 +6,1 +6,2 +6,3 +6,4 diff --git a/tajo-core/src/test/resources/results/TestHashThetaJoin/testThetaJoinInner.4.result b/tajo-core/src/test/resources/results/TestHashThetaJoin/testThetaJoinInner.4.result new file mode 100644 index 0000000000..da1a11d7e8 --- /dev/null +++ b/tajo-core/src/test/resources/results/TestHashThetaJoin/testThetaJoinInner.4.result @@ -0,0 +1,28 @@ +explain +------------------------------- +JOIN(6)(INNER) + => Join Cond: default.nation.n_nationkey (INT4) < default.region.r_regionkey (INT4) + => target list: default.nation.n_nationkey (INT4), default.region.r_regionkey (INT4) + => out schema: {(2) default.nation.n_nationkey (INT4), default.region.r_regionkey (INT4)} + => in schema: {(2) default.nation.n_nationkey (INT4), default.region.r_regionkey (INT4)} + SCAN(1) on default.region + => target list: default.region.r_regionkey (INT4) + => out schema: {(1) default.region.r_regionkey (INT4)} + => in schema: {(3) default.region.r_regionkey (INT4), default.region.r_name (TEXT), default.region.r_comment (TEXT)} + SCAN(0) on default.nation + => filter: default.nation.n_nationkey (INT4) < 7 + => target list: default.nation.n_nationkey (INT4) + => out schema: {(1) default.nation.n_nationkey (INT4)} + => in schema: {(4) default.nation.n_nationkey (INT4), default.nation.n_name (TEXT), default.nation.n_regionkey (INT4), default.nation.n_comment (TEXT)} +n_nationkey,r_regionkey +------------------------------- +0,1 +0,2 +0,3 +0,4 +1,2 +1,3 +1,4 +2,3 +2,4 +3,4 diff --git a/tajo-core/src/test/resources/results/TestHashThetaJoin/testThetaJoinInner.5.result b/tajo-core/src/test/resources/results/TestHashThetaJoin/testThetaJoinInner.5.result new file mode 100644 index 0000000000..0b5882461f --- /dev/null +++ b/tajo-core/src/test/resources/results/TestHashThetaJoin/testThetaJoinInner.5.result @@ -0,0 +1,48 @@ +explain +------------------------------- +JOIN(6)(INNER) + => Join Cond: default.nation.n_nationkey (INT4) <> default.region.r_regionkey (INT4) + => target list: default.nation.n_nationkey (INT4), default.region.r_regionkey (INT4) + => out schema: {(2) default.nation.n_nationkey (INT4), default.region.r_regionkey (INT4)} + => in schema: {(2) default.nation.n_nationkey (INT4), default.region.r_regionkey (INT4)} + SCAN(1) on default.region + => target list: default.region.r_regionkey (INT4) + => out schema: {(1) default.region.r_regionkey (INT4)} + => in schema: {(3) default.region.r_regionkey (INT4), default.region.r_name (TEXT), default.region.r_comment (TEXT)} + SCAN(0) on default.nation + => filter: default.nation.n_nationkey (INT4) < 7 + => target list: default.nation.n_nationkey (INT4) + => out schema: {(1) default.nation.n_nationkey (INT4)} + => in schema: {(4) default.nation.n_nationkey (INT4), default.nation.n_name (TEXT), default.nation.n_regionkey (INT4), default.nation.n_comment (TEXT)} +n_nationkey,r_regionkey +------------------------------- +0,1 +0,2 +0,3 +0,4 +1,0 +1,2 +1,3 +1,4 +2,0 +2,1 +2,3 +2,4 +3,0 +3,1 +3,2 +3,4 +4,0 +4,1 +4,2 +4,3 +5,0 +5,1 +5,2 +5,3 +5,4 +6,0 +6,1 +6,2 +6,3 +6,4 diff --git a/tajo-core/src/test/resources/results/TestHashThetaJoin/testThetaJoinLeftOuter.1.result b/tajo-core/src/test/resources/results/TestHashThetaJoin/testThetaJoinLeftOuter.1.result new file mode 100644 index 0000000000..796ab73b2c --- /dev/null +++ b/tajo-core/src/test/resources/results/TestHashThetaJoin/testThetaJoinLeftOuter.1.result @@ -0,0 +1,43 @@ +explain +------------------------------- +JOIN(6)(LEFT_OUTER) + => Join Cond: default.nation.n_nationkey (INT4) >= default.region.r_regionkey (INT4) + => target list: default.nation.n_nationkey (INT4), default.region.r_regionkey (INT4) + => out schema: {(2) default.nation.n_nationkey (INT4), default.region.r_regionkey (INT4)} + => in schema: {(2) default.nation.n_nationkey (INT4), default.region.r_regionkey (INT4)} + SCAN(1) on default.region + => target list: default.region.r_regionkey (INT4) + => out schema: {(1) default.region.r_regionkey (INT4)} + => in schema: {(3) default.region.r_regionkey (INT4), default.region.r_name (TEXT), default.region.r_comment (TEXT)} + SCAN(0) on default.nation + => filter: default.nation.n_nationkey (INT4) < 7 + => target list: default.nation.n_nationkey (INT4) + => out schema: {(1) default.nation.n_nationkey (INT4)} + => in schema: {(4) default.nation.n_nationkey (INT4), default.nation.n_name (TEXT), default.nation.n_regionkey (INT4), default.nation.n_comment (TEXT)} +n_nationkey,r_regionkey +------------------------------- +0,0 +1,0 +1,1 +2,0 +2,1 +2,2 +3,0 +3,1 +3,2 +3,3 +4,0 +4,1 +4,2 +4,3 +4,4 +5,0 +5,1 +5,2 +5,3 +5,4 +6,0 +6,1 +6,2 +6,3 +6,4 diff --git a/tajo-core/src/test/resources/results/TestHashThetaJoin/testThetaJoinLeftOuter.2.result b/tajo-core/src/test/resources/results/TestHashThetaJoin/testThetaJoinLeftOuter.2.result new file mode 100644 index 0000000000..fc6b1e3858 --- /dev/null +++ b/tajo-core/src/test/resources/results/TestHashThetaJoin/testThetaJoinLeftOuter.2.result @@ -0,0 +1,35 @@ +explain +------------------------------- +JOIN(6)(LEFT_OUTER) + => Join Cond: default.nation.n_nationkey (INT4) <= default.region.r_regionkey (INT4) + => target list: default.nation.n_nationkey (INT4), default.region.r_regionkey (INT4) + => out schema: {(2) default.nation.n_nationkey (INT4), default.region.r_regionkey (INT4)} + => in schema: {(2) default.nation.n_nationkey (INT4), default.region.r_regionkey (INT4)} + SCAN(1) on default.region + => target list: default.region.r_regionkey (INT4) + => out schema: {(1) default.region.r_regionkey (INT4)} + => in schema: {(3) default.region.r_regionkey (INT4), default.region.r_name (TEXT), default.region.r_comment (TEXT)} + SCAN(0) on default.nation + => filter: default.nation.n_nationkey (INT4) < 7 + => target list: default.nation.n_nationkey (INT4) + => out schema: {(1) default.nation.n_nationkey (INT4)} + => in schema: {(4) default.nation.n_nationkey (INT4), default.nation.n_name (TEXT), default.nation.n_regionkey (INT4), default.nation.n_comment (TEXT)} +n_nationkey,r_regionkey +------------------------------- +0,0 +0,1 +0,2 +0,3 +0,4 +1,1 +1,2 +1,3 +1,4 +2,2 +2,3 +2,4 +3,3 +3,4 +4,4 +5,null +6,null diff --git a/tajo-core/src/test/resources/results/TestHashThetaJoin/testThetaJoinLeftOuter.3.result b/tajo-core/src/test/resources/results/TestHashThetaJoin/testThetaJoinLeftOuter.3.result new file mode 100644 index 0000000000..97879ac802 --- /dev/null +++ b/tajo-core/src/test/resources/results/TestHashThetaJoin/testThetaJoinLeftOuter.3.result @@ -0,0 +1,39 @@ +explain +------------------------------- +JOIN(6)(LEFT_OUTER) + => Join Cond: default.nation.n_nationkey (INT4) > default.region.r_regionkey (INT4) + => target list: default.nation.n_nationkey (INT4), default.region.r_regionkey (INT4) + => out schema: {(2) default.nation.n_nationkey (INT4), default.region.r_regionkey (INT4)} + => in schema: {(2) default.nation.n_nationkey (INT4), default.region.r_regionkey (INT4)} + SCAN(1) on default.region + => target list: default.region.r_regionkey (INT4) + => out schema: {(1) default.region.r_regionkey (INT4)} + => in schema: {(3) default.region.r_regionkey (INT4), default.region.r_name (TEXT), default.region.r_comment (TEXT)} + SCAN(0) on default.nation + => filter: default.nation.n_nationkey (INT4) < 7 + => target list: default.nation.n_nationkey (INT4) + => out schema: {(1) default.nation.n_nationkey (INT4)} + => in schema: {(4) default.nation.n_nationkey (INT4), default.nation.n_name (TEXT), default.nation.n_regionkey (INT4), default.nation.n_comment (TEXT)} +n_nationkey,r_regionkey +------------------------------- +0,null +1,0 +2,0 +2,1 +3,0 +3,1 +3,2 +4,0 +4,1 +4,2 +4,3 +5,0 +5,1 +5,2 +5,3 +5,4 +6,0 +6,1 +6,2 +6,3 +6,4 diff --git a/tajo-core/src/test/resources/results/TestHashThetaJoin/testThetaJoinLeftOuter.4.result b/tajo-core/src/test/resources/results/TestHashThetaJoin/testThetaJoinLeftOuter.4.result new file mode 100644 index 0000000000..f8f44a151b --- /dev/null +++ b/tajo-core/src/test/resources/results/TestHashThetaJoin/testThetaJoinLeftOuter.4.result @@ -0,0 +1,31 @@ +explain +------------------------------- +JOIN(6)(LEFT_OUTER) + => Join Cond: default.nation.n_nationkey (INT4) < default.region.r_regionkey (INT4) + => target list: default.nation.n_nationkey (INT4), default.region.r_regionkey (INT4) + => out schema: {(2) default.nation.n_nationkey (INT4), default.region.r_regionkey (INT4)} + => in schema: {(2) default.nation.n_nationkey (INT4), default.region.r_regionkey (INT4)} + SCAN(1) on default.region + => target list: default.region.r_regionkey (INT4) + => out schema: {(1) default.region.r_regionkey (INT4)} + => in schema: {(3) default.region.r_regionkey (INT4), default.region.r_name (TEXT), default.region.r_comment (TEXT)} + SCAN(0) on default.nation + => filter: default.nation.n_nationkey (INT4) < 7 + => target list: default.nation.n_nationkey (INT4) + => out schema: {(1) default.nation.n_nationkey (INT4)} + => in schema: {(4) default.nation.n_nationkey (INT4), default.nation.n_name (TEXT), default.nation.n_regionkey (INT4), default.nation.n_comment (TEXT)} +n_nationkey,r_regionkey +------------------------------- +0,1 +0,2 +0,3 +0,4 +1,2 +1,3 +1,4 +2,3 +2,4 +3,4 +4,null +5,null +6,null diff --git a/tajo-core/src/test/resources/results/TestHashThetaJoin/testThetaJoinLeftOuter.5.result b/tajo-core/src/test/resources/results/TestHashThetaJoin/testThetaJoinLeftOuter.5.result new file mode 100644 index 0000000000..cb7527666d --- /dev/null +++ b/tajo-core/src/test/resources/results/TestHashThetaJoin/testThetaJoinLeftOuter.5.result @@ -0,0 +1,48 @@ +explain +------------------------------- +JOIN(6)(LEFT_OUTER) + => Join Cond: default.nation.n_nationkey (INT4) <> default.region.r_regionkey (INT4) + => target list: default.nation.n_nationkey (INT4), default.region.r_regionkey (INT4) + => out schema: {(2) default.nation.n_nationkey (INT4), default.region.r_regionkey (INT4)} + => in schema: {(2) default.nation.n_nationkey (INT4), default.region.r_regionkey (INT4)} + SCAN(1) on default.region + => target list: default.region.r_regionkey (INT4) + => out schema: {(1) default.region.r_regionkey (INT4)} + => in schema: {(3) default.region.r_regionkey (INT4), default.region.r_name (TEXT), default.region.r_comment (TEXT)} + SCAN(0) on default.nation + => filter: default.nation.n_nationkey (INT4) < 7 + => target list: default.nation.n_nationkey (INT4) + => out schema: {(1) default.nation.n_nationkey (INT4)} + => in schema: {(4) default.nation.n_nationkey (INT4), default.nation.n_name (TEXT), default.nation.n_regionkey (INT4), default.nation.n_comment (TEXT)} +n_nationkey,r_regionkey +------------------------------- +0,1 +0,2 +0,3 +0,4 +1,0 +1,2 +1,3 +1,4 +2,0 +2,1 +2,3 +2,4 +3,0 +3,1 +3,2 +3,4 +4,0 +4,1 +4,2 +4,3 +5,0 +5,1 +5,2 +5,3 +5,4 +6,0 +6,1 +6,2 +6,3 +6,4 diff --git a/tajo-core/src/test/resources/results/TestHashThetaJoin/testThetaJoinRightOuter.1.result b/tajo-core/src/test/resources/results/TestHashThetaJoin/testThetaJoinRightOuter.1.result new file mode 100644 index 0000000000..6353b1e316 --- /dev/null +++ b/tajo-core/src/test/resources/results/TestHashThetaJoin/testThetaJoinRightOuter.1.result @@ -0,0 +1,44 @@ +explain +------------------------------- +SELECTION(3) + => Search Cond: default.nation.n_nationkey (INT4) < 7 + JOIN(6)(RIGHT_OUTER) + => Join Cond: default.nation.n_nationkey (INT4) >= default.region.r_regionkey (INT4) + => target list: default.nation.n_nationkey (INT4), default.region.r_regionkey (INT4) + => out schema: {(2) default.nation.n_nationkey (INT4), default.region.r_regionkey (INT4)} + => in schema: {(2) default.nation.n_nationkey (INT4), default.region.r_regionkey (INT4)} + SCAN(1) on default.region + => target list: default.region.r_regionkey (INT4) + => out schema: {(1) default.region.r_regionkey (INT4)} + => in schema: {(3) default.region.r_regionkey (INT4), default.region.r_name (TEXT), default.region.r_comment (TEXT)} + SCAN(0) on default.nation + => target list: default.nation.n_nationkey (INT4) + => out schema: {(1) default.nation.n_nationkey (INT4)} + => in schema: {(4) default.nation.n_nationkey (INT4), default.nation.n_name (TEXT), default.nation.n_regionkey (INT4), default.nation.n_comment (TEXT)} +n_nationkey,r_regionkey +------------------------------- +0,0 +1,0 +2,0 +3,0 +4,0 +5,0 +6,0 +1,1 +2,1 +3,1 +4,1 +5,1 +6,1 +2,2 +3,2 +4,2 +5,2 +6,2 +3,3 +4,3 +5,3 +6,3 +4,4 +5,4 +6,4 diff --git a/tajo-core/src/test/resources/results/TestHashThetaJoin/testThetaJoinRightOuter.2.result b/tajo-core/src/test/resources/results/TestHashThetaJoin/testThetaJoinRightOuter.2.result new file mode 100644 index 0000000000..64c95d0d90 --- /dev/null +++ b/tajo-core/src/test/resources/results/TestHashThetaJoin/testThetaJoinRightOuter.2.result @@ -0,0 +1,34 @@ +explain +------------------------------- +SELECTION(3) + => Search Cond: default.nation.n_nationkey (INT4) < 7 + JOIN(6)(RIGHT_OUTER) + => Join Cond: default.nation.n_nationkey (INT4) <= default.region.r_regionkey (INT4) + => target list: default.nation.n_nationkey (INT4), default.region.r_regionkey (INT4) + => out schema: {(2) default.nation.n_nationkey (INT4), default.region.r_regionkey (INT4)} + => in schema: {(2) default.nation.n_nationkey (INT4), default.region.r_regionkey (INT4)} + SCAN(1) on default.region + => target list: default.region.r_regionkey (INT4) + => out schema: {(1) default.region.r_regionkey (INT4)} + => in schema: {(3) default.region.r_regionkey (INT4), default.region.r_name (TEXT), default.region.r_comment (TEXT)} + SCAN(0) on default.nation + => target list: default.nation.n_nationkey (INT4) + => out schema: {(1) default.nation.n_nationkey (INT4)} + => in schema: {(4) default.nation.n_nationkey (INT4), default.nation.n_name (TEXT), default.nation.n_regionkey (INT4), default.nation.n_comment (TEXT)} +n_nationkey,r_regionkey +------------------------------- +0,0 +0,1 +1,1 +0,2 +1,2 +2,2 +0,3 +1,3 +2,3 +3,3 +0,4 +1,4 +2,4 +3,4 +4,4 diff --git a/tajo-core/src/test/resources/results/TestHashThetaJoin/testThetaJoinRightOuter.3.result b/tajo-core/src/test/resources/results/TestHashThetaJoin/testThetaJoinRightOuter.3.result new file mode 100644 index 0000000000..d78b787634 --- /dev/null +++ b/tajo-core/src/test/resources/results/TestHashThetaJoin/testThetaJoinRightOuter.3.result @@ -0,0 +1,39 @@ +explain +------------------------------- +SELECTION(3) + => Search Cond: default.nation.n_nationkey (INT4) < 7 + JOIN(6)(RIGHT_OUTER) + => Join Cond: default.nation.n_nationkey (INT4) > default.region.r_regionkey (INT4) + => target list: default.nation.n_nationkey (INT4), default.region.r_regionkey (INT4) + => out schema: {(2) default.nation.n_nationkey (INT4), default.region.r_regionkey (INT4)} + => in schema: {(2) default.nation.n_nationkey (INT4), default.region.r_regionkey (INT4)} + SCAN(1) on default.region + => target list: default.region.r_regionkey (INT4) + => out schema: {(1) default.region.r_regionkey (INT4)} + => in schema: {(3) default.region.r_regionkey (INT4), default.region.r_name (TEXT), default.region.r_comment (TEXT)} + SCAN(0) on default.nation + => target list: default.nation.n_nationkey (INT4) + => out schema: {(1) default.nation.n_nationkey (INT4)} + => in schema: {(4) default.nation.n_nationkey (INT4), default.nation.n_name (TEXT), default.nation.n_regionkey (INT4), default.nation.n_comment (TEXT)} +n_nationkey,r_regionkey +------------------------------- +1,0 +2,0 +3,0 +4,0 +5,0 +6,0 +2,1 +3,1 +4,1 +5,1 +6,1 +3,2 +4,2 +5,2 +6,2 +4,3 +5,3 +6,3 +5,4 +6,4 diff --git a/tajo-core/src/test/resources/results/TestHashThetaJoin/testThetaJoinRightOuter.4.result b/tajo-core/src/test/resources/results/TestHashThetaJoin/testThetaJoinRightOuter.4.result new file mode 100644 index 0000000000..83ea9c5640 --- /dev/null +++ b/tajo-core/src/test/resources/results/TestHashThetaJoin/testThetaJoinRightOuter.4.result @@ -0,0 +1,29 @@ +explain +------------------------------- +SELECTION(3) + => Search Cond: default.nation.n_nationkey (INT4) < 7 + JOIN(6)(RIGHT_OUTER) + => Join Cond: default.nation.n_nationkey (INT4) < default.region.r_regionkey (INT4) + => target list: default.nation.n_nationkey (INT4), default.region.r_regionkey (INT4) + => out schema: {(2) default.nation.n_nationkey (INT4), default.region.r_regionkey (INT4)} + => in schema: {(2) default.nation.n_nationkey (INT4), default.region.r_regionkey (INT4)} + SCAN(1) on default.region + => target list: default.region.r_regionkey (INT4) + => out schema: {(1) default.region.r_regionkey (INT4)} + => in schema: {(3) default.region.r_regionkey (INT4), default.region.r_name (TEXT), default.region.r_comment (TEXT)} + SCAN(0) on default.nation + => target list: default.nation.n_nationkey (INT4) + => out schema: {(1) default.nation.n_nationkey (INT4)} + => in schema: {(4) default.nation.n_nationkey (INT4), default.nation.n_name (TEXT), default.nation.n_regionkey (INT4), default.nation.n_comment (TEXT)} +n_nationkey,r_regionkey +------------------------------- +0,1 +0,2 +1,2 +0,3 +1,3 +2,3 +0,4 +1,4 +2,4 +3,4 diff --git a/tajo-core/src/test/resources/results/TestHashThetaJoin/testThetaJoinRightOuter.5.result b/tajo-core/src/test/resources/results/TestHashThetaJoin/testThetaJoinRightOuter.5.result new file mode 100644 index 0000000000..43a73d65c0 --- /dev/null +++ b/tajo-core/src/test/resources/results/TestHashThetaJoin/testThetaJoinRightOuter.5.result @@ -0,0 +1,49 @@ +explain +------------------------------- +SELECTION(3) + => Search Cond: default.nation.n_nationkey (INT4) < 7 + JOIN(6)(RIGHT_OUTER) + => Join Cond: default.nation.n_nationkey (INT4) <> default.region.r_regionkey (INT4) + => target list: default.nation.n_nationkey (INT4), default.region.r_regionkey (INT4) + => out schema: {(2) default.nation.n_nationkey (INT4), default.region.r_regionkey (INT4)} + => in schema: {(2) default.nation.n_nationkey (INT4), default.region.r_regionkey (INT4)} + SCAN(1) on default.region + => target list: default.region.r_regionkey (INT4) + => out schema: {(1) default.region.r_regionkey (INT4)} + => in schema: {(3) default.region.r_regionkey (INT4), default.region.r_name (TEXT), default.region.r_comment (TEXT)} + SCAN(0) on default.nation + => target list: default.nation.n_nationkey (INT4) + => out schema: {(1) default.nation.n_nationkey (INT4)} + => in schema: {(4) default.nation.n_nationkey (INT4), default.nation.n_name (TEXT), default.nation.n_regionkey (INT4), default.nation.n_comment (TEXT)} +n_nationkey,r_regionkey +------------------------------- +1,0 +2,0 +3,0 +4,0 +5,0 +6,0 +0,1 +2,1 +3,1 +4,1 +5,1 +6,1 +0,2 +1,2 +3,2 +4,2 +5,2 +6,2 +0,3 +1,3 +2,3 +4,3 +5,3 +6,3 +0,4 +1,4 +2,4 +3,4 +5,4 +6,4 diff --git a/tajo-core/src/test/resources/results/TestSelectQuery/testExplainSelectPhysical.1.result b/tajo-core/src/test/resources/results/TestSelectQuery/testExplainSelectPhysical.1.result index 00696397d4..70d3ff7971 100644 --- a/tajo-core/src/test/resources/results/TestSelectQuery/testExplainSelectPhysical.1.result +++ b/tajo-core/src/test/resources/results/TestSelectQuery/testExplainSelectPhysical.1.result @@ -23,4 +23,4 @@ SCAN(0) on default.lineitem ======================================================= Block Id: eb_0000000000000_0000_000002 [TERMINAL] -======================================================= \ No newline at end of file +======================================================= diff --git a/tajo-core/src/test/resources/results/TestSelectQuery/testExplainSelectPhysical.2.result b/tajo-core/src/test/resources/results/TestSelectQuery/testExplainSelectPhysical.2.result index 7946c5bf83..248406c4a3 100644 --- a/tajo-core/src/test/resources/results/TestSelectQuery/testExplainSelectPhysical.2.result +++ b/tajo-core/src/test/resources/results/TestSelectQuery/testExplainSelectPhysical.2.result @@ -85,4 +85,4 @@ SORT(3) ======================================================= Block Id: eb_0000000000000_0000_000005 [TERMINAL] -======================================================= \ No newline at end of file +======================================================= diff --git a/tajo-core/src/test/resources/results/TestSelectQuery/testExplainSelectPhysical.3.result b/tajo-core/src/test/resources/results/TestSelectQuery/testExplainSelectPhysical.3.result index c4e8c2cff5..ccec4a4abc 100644 --- a/tajo-core/src/test/resources/results/TestSelectQuery/testExplainSelectPhysical.3.result +++ b/tajo-core/src/test/resources/results/TestSelectQuery/testExplainSelectPhysical.3.result @@ -86,4 +86,4 @@ HAVING(2) (?sum_2 (INT8) = 6) ======================================================= Block Id: eb_0000000000000_0000_000004 [TERMINAL] -======================================================= \ No newline at end of file +======================================================= diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/LogicalPlanner.java b/tajo-plan/src/main/java/org/apache/tajo/plan/LogicalPlanner.java index d1c1a150e5..09e4788981 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/LogicalPlanner.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/LogicalPlanner.java @@ -1988,10 +1988,6 @@ public static boolean isEvaluatableJoinQual(QueryBlock block, EvalNode evalNode, if (checkIfBeEvaluatedAtJoin(block, evalNode, node, isTopMostJoin)) { - if (isNonEquiThetaJoinQual(block, node, evalNode)) { - return false; - } - if (PlannerUtil.isOuterJoin(node.getJoinType())) { /* * For outer joins, only predicates which are specified at the on clause can be evaluated during processing join. @@ -2006,7 +2002,7 @@ public static boolean isEvaluatableJoinQual(QueryBlock block, EvalNode evalNode, * Only join predicates should be evaluated at join if the join type is inner or cross. (TAJO-1445) */ if (!EvalTreeUtil.isJoinQual(block, node.getLeftChild().getOutSchema(), node.getRightChild().getOutSchema(), - evalNode, false)) { + evalNode, true)) { return false; } } @@ -2017,18 +2013,6 @@ public static boolean isEvaluatableJoinQual(QueryBlock block, EvalNode evalNode, return false; } - public static boolean isNonEquiThetaJoinQual(final LogicalPlan.QueryBlock block, - final JoinNode joinNode, - final EvalNode evalNode) { - if (EvalTreeUtil.isJoinQual(block, joinNode.getLeftChild().getOutSchema(), - joinNode.getRightChild().getOutSchema(), evalNode, true) && - evalNode.getType() != EvalType.EQUAL) { - return true; - } else { - return false; - } - } - public static boolean checkIfBeEvaluatedAtJoin(QueryBlock block, EvalNode evalNode, JoinNode node, boolean isTopMostJoin) { Set columnRefs = EvalTreeUtil.findUniqueColumns(evalNode); diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/AlgebraicUtil.java b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/AlgebraicUtil.java index c6b7354e04..9f30a9f610 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/AlgebraicUtil.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/AlgebraicUtil.java @@ -286,6 +286,7 @@ public static BinaryEval commutate(BinaryEval inputExpr) { case AND: case OR: case EQUAL: + case NOT_EQUAL: case PLUS: case MINUS: case MULTIPLY: // these types can be commutated w/o any change diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/EvalTreeUtil.java b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/EvalTreeUtil.java index 1fa2fe090d..4b6847e3c2 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/EvalTreeUtil.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/EvalTreeUtil.java @@ -18,6 +18,7 @@ package org.apache.tajo.plan.expr; +import com.google.common.base.Predicate; import com.google.common.collect.Maps; import com.google.common.collect.Sets; import org.apache.tajo.algebra.ColumnReferenceExpr; diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/rewrite/rules/FilterPushDownRule.java b/tajo-plan/src/main/java/org/apache/tajo/plan/rewrite/rules/FilterPushDownRule.java index dc6b8ef7db..07435c285e 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/rewrite/rules/FilterPushDownRule.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/rewrite/rules/FilterPushDownRule.java @@ -245,6 +245,11 @@ private static Set extractNonPushableJoinQuals(final LogicalPlan plan, Set nonPushableQuals = TUtil.newHashSet(); // TODO: non-equi theta join quals must not be pushed until TAJO-742 is resolved. nonPushableQuals.addAll(extractNonEquiThetaJoinQuals(wherePredicates, block, joinNode)); + for (EvalNode joinQual : onPredicates) { + if (isNonEquiThetaJoinQual(block, joinNode, joinQual)) { + nonPushableQuals.add(joinQual); + } + } // for outer joins if (PlannerUtil.isOuterJoin(joinNode.getJoinType())) { diff --git a/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/FrameTuple.java b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/FrameTuple.java index a5561ed7cf..302606bf9a 100644 --- a/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/FrameTuple.java +++ b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/FrameTuple.java @@ -64,6 +64,10 @@ public FrameTuple setRight(Tuple right) { return this; } + public boolean isSetLeft() { + return left != null; + } + @Override public int size() { return size; @@ -93,7 +97,9 @@ public boolean isNotNull(int fieldid) { @Override public void clear() { - throw new UnsupportedException(); + this.left = null; + this.right = null; + this.leftSize = 0; } @Override